Skip to content

Commit

Permalink
Resolve some unchecked/rawtypes warnings
Browse files Browse the repository at this point in the history
  • Loading branch information
findepi committed Feb 17, 2020
1 parent 4bebb49 commit 8380483
Show file tree
Hide file tree
Showing 26 changed files with 50 additions and 32 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,8 @@ public List<PropertyMetadata<?>> getTableProperties()
List.class,
ImmutableList.of(),
false,
value -> ImmutableList.copyOf(((List<String>) value).stream()
value -> ImmutableList.copyOf(((List<?>) value).stream()
.map(String.class::cast)
.map(name -> name.toLowerCase(ENGLISH))
.collect(toList())),
List.class::cast),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,7 @@ public void createTable(ConnectorSession session, ConnectorTableMetadata tableMe
@Override
public Optional<ConnectorNewTableLayout> getNewTableLayout(ConnectorSession connectorSession, ConnectorTableMetadata tableMetadata)
{
@SuppressWarnings("unchecked")
List<String> distributeColumns = (List<String>) tableMetadata.getProperties().get(DISTRIBUTED_ON);
if (distributeColumns.isEmpty()) {
return Optional.empty();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,6 @@
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;

import static com.google.common.base.Preconditions.checkArgument;
import static io.prestosql.client.ClientStandardTypes.ARRAY;
Expand Down Expand Up @@ -99,14 +98,14 @@ private static Object fixValue(ClientTypeSignature signature, Object value)
ClientTypeSignature keySignature = signature.getArgumentsAsTypeSignatures().get(0);
ClientTypeSignature valueSignature = signature.getArgumentsAsTypeSignatures().get(1);
Map<Object, Object> fixedValue = new HashMap<>();
for (Map.Entry<?, ?> entry : (Set<Map.Entry<?, ?>>) Map.class.cast(value).entrySet()) {
for (Map.Entry<?, ?> entry : ((Map<?, ?>) value).entrySet()) {
fixedValue.put(fixValue(keySignature, entry.getKey()), fixValue(valueSignature, entry.getValue()));
}
return fixedValue;
}
if (signature.getRawType().equals(ROW)) {
Map<String, Object> fixedValue = new LinkedHashMap<>();
List<Object> listValue = List.class.cast(value);
List<?> listValue = ((List<?>) value);
checkArgument(listValue.size() == signature.getArguments().size(), "Mismatched data values and row type");
for (int i = 0; i < listValue.size(); i++) {
ClientTypeSignatureParameter parameter = signature.getArguments().get(i);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ public void decode(SearchHit hit, Supplier<Object> getter, BlockBuilder output)
}
else if (data instanceof List) {
BlockBuilder array = output.beginBlockEntry();
((List) data).forEach(element -> elementDecoder.decode(hit, () -> element, array));
((List<?>) data).forEach(element -> elementDecoder.decode(hit, () -> element, array));
output.closeEntry();
}
else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -303,7 +303,7 @@ private ListenableFuture<?> loadPartition(HivePartitionMetadata partition)
String partitionName = partition.getHivePartition().getPartitionId();
Properties schema = getPartitionSchema(table, partition.getPartition());
List<HivePartitionKey> partitionKeys = getPartitionKeys(table, partition.getPartition());
TupleDomain<HiveColumnHandle> effectivePredicate = (TupleDomain<HiveColumnHandle>) compactEffectivePredicate;
TupleDomain<HiveColumnHandle> effectivePredicate = compactEffectivePredicate.transform(HiveColumnHandle.class::cast);

Path path = new Path(getPartitionLocation(table, partition.getPartition()));
Configuration configuration = hdfsEnvironment.getConfiguration(hdfsContext, path);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ public class PrestoResultSet
this.columnInfoList = getColumnInfo(columns);
this.resultSetMetaData = new PrestoResultSetMetaData(columnInfoList);

this.results = new AsyncIterator(flatten(new ResultsPageIterator(client, progressCallback, warningsManager), maxRows), client, Thread.currentThread());
this.results = new AsyncIterator<>(flatten(new ResultsPageIterator(client, progressCallback, warningsManager), maxRows), client, Thread.currentThread());
}

public String getQueryId()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,6 @@ public KafkaConsumer<byte[], byte[]> create()
properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
properties.put(RECEIVE_BUFFER_CONFIG, Long.toString(kafkaBufferSize.toBytes()));
properties.put(ENABLE_AUTO_COMMIT_CONFIG, Boolean.toString(false));
return new KafkaConsumer(properties);
return new KafkaConsumer<>(properties);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -173,7 +173,9 @@ public static PartitionDesign getPartitionDesign(Map<String, Object> tableProper
{
requireNonNull(tableProperties);

@SuppressWarnings("unchecked")
List<String> hashColumns = (List<String>) tableProperties.get(PARTITION_BY_HASH_COLUMNS);
@SuppressWarnings("unchecked")
List<String> hashColumns2 = (List<String>) tableProperties.get(PARTITION_BY_HASH_COLUMNS_2);

PartitionDesign design = new PartitionDesign();
Expand All @@ -191,6 +193,7 @@ else if (!hashColumns2.isEmpty()) {
throw new PrestoException(GENERIC_USER_ERROR, "Table property " + PARTITION_BY_HASH_COLUMNS_2 + " is only allowed if there is also " + PARTITION_BY_HASH_COLUMNS);
}

@SuppressWarnings("unchecked")
List<String> rangeColumns = (List<String>) tableProperties.get(PARTITION_BY_RANGE_COLUMNS);
if (!rangeColumns.isEmpty()) {
RangePartitionDefinition range = new RangePartitionDefinition();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,9 @@ public PlanNodeStatsEstimate calculateStats(PlanNode node, StatsProvider sourceS

private static <T extends PlanNode> Optional<PlanNodeStatsEstimate> calculateStats(Rule<T> rule, PlanNode node, StatsProvider sourceStats, Lookup lookup, Session session, TypeProvider types)
{
return rule.calculate((T) node, sourceStats, lookup, session, types);
@SuppressWarnings("unchecked")
T typedNode = (T) node;
return rule.calculate(typedNode, sourceStats, lookup, session, types);
}

public interface Rule<T extends PlanNode>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -104,7 +104,7 @@ private PagesIndex(
this.valueAddresses = new LongArrayList(expectedPositions);
this.eagerCompact = eagerCompact;

//noinspection rawtypes
//noinspection unchecked
channels = (ObjectArrayList<Block>[]) new ObjectArrayList[types.size()];
for (int i = 0; i < channels.length; i++) {
channels[i] = ObjectArrayList.wrap(new Block[1024], 0);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,7 @@ public PartitionedLookupSourceFactory(List<Type> types, List<Type> outputTypes,
this.outputTypes = ImmutableList.copyOf(requireNonNull(outputTypes, "outputTypes is null"));
this.hashChannelTypes = ImmutableList.copyOf(hashChannelTypes);
checkArgument(partitionCount > 0);
//noinspection unchecked
this.partitions = (Supplier<LookupSource>[]) new Supplier<?>[partitionCount];
this.outer = outer;
spilledLookupSource = new SpilledLookupSource(outputTypes.size());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -280,7 +280,7 @@ public TestingPrestoServer(
if (coordinator) {
dispatchManager = injector.getInstance(DispatchManager.class);
queryManager = (SqlQueryManager) injector.getInstance(QueryManager.class);
resourceGroupManager = Optional.of(injector.getInstance(InternalResourceGroupManager.class));
resourceGroupManager = Optional.of((InternalResourceGroupManager<?>) injector.getInstance(InternalResourceGroupManager.class));
sessionPropertyDefaults = injector.getInstance(SessionPropertyDefaults.class);
nodePartitioningManager = injector.getInstance(NodePartitioningManager.class);
clusterMemoryManager = injector.getInstance(ClusterMemoryManager.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -382,7 +382,8 @@ public Expression visitValues(ValuesNode node, Void context)
return domainTranslator.toPredicate(TupleDomain.withColumnDomains(domains.build()).simplify());
}

private Iterable<Expression> pullNullableConjunctsThroughOuterJoin(List<Expression> conjuncts, Collection<Symbol> outputSymbols, Predicate<Symbol>... nullSymbolScopes)
@SafeVarargs
private final Iterable<Expression> pullNullableConjunctsThroughOuterJoin(List<Expression> conjuncts, Collection<Symbol> outputSymbols, Predicate<Symbol>... nullSymbolScopes)
{
// Conjuncts without any symbol dependencies cannot be applied to the effective predicate (e.g. FALSE literal)
return conjuncts.stream()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1268,6 +1268,7 @@ private static String formatFrame(WindowNode.Frame frame)
return builder.toString();
}

@SafeVarargs
private static String formatHash(Optional<Symbol>... hashes)
{
List<Symbol> symbols = stream(hashes)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -309,7 +309,7 @@ else if (TIMESTAMP_WITH_TIME_ZONE.equals(type)) {
type.writeLong(blockBuilder, packDateTimeWithZone(millisUtc, timeZoneKey));
}
else if (type instanceof ArrayType) {
List<Object> list = (List<Object>) value;
List<?> list = (List<?>) value;
Type elementType = ((ArrayType) type).getElementType();
BlockBuilder arrayBlockBuilder = blockBuilder.beginBlockEntry();
for (Object element : list) {
Expand All @@ -318,18 +318,18 @@ else if (type instanceof ArrayType) {
blockBuilder.closeEntry();
}
else if (type instanceof MapType) {
Map<Object, Object> map = (Map<Object, Object>) value;
Map<?, ?> map = (Map<?, ?>) value;
Type keyType = ((MapType) type).getKeyType();
Type valueType = ((MapType) type).getValueType();
BlockBuilder mapBlockBuilder = blockBuilder.beginBlockEntry();
for (Entry<Object, Object> entry : map.entrySet()) {
for (Entry<?, ?> entry : map.entrySet()) {
writeValue(keyType, mapBlockBuilder, entry.getKey());
writeValue(valueType, mapBlockBuilder, entry.getValue());
}
blockBuilder.closeEntry();
}
else if (type instanceof RowType) {
List<Object> row = (List<Object>) value;
List<?> row = (List<?>) value;
List<Type> fieldTypes = type.getTypeParameters();
BlockBuilder rowBlockBuilder = blockBuilder.beginBlockEntry();
for (int field = 0; field < row.size(); field++) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
public class MaterializedRow
{
private final int precision;
private final List<Object> values;
private final List<?> values;

public MaterializedRow(int precision, Object... values)
{
Expand All @@ -48,7 +48,7 @@ public MaterializedRow(int precision, List<Object> values)
checkArgument(precision > 0, "Need at least one digit of precision");
this.precision = precision;

this.values = (List<Object>) processValue(precision, values);
this.values = (List<?>) processValue(precision, values);
}

private static Object processValue(int precision, Object value)
Expand All @@ -66,7 +66,7 @@ private static Object processValue(int precision, Object value)
}
if (value instanceof Map) {
Map<Object, Object> map = new HashMap<>();
for (Entry<Object, Object> entry : ((Map<Object, Object>) value).entrySet()) {
for (Entry<?, ?> entry : ((Map<?, ?>) value).entrySet()) {
map.put(processValue(precision, entry.getKey()), processValue(precision, entry.getValue()));
}
return map;
Expand Down Expand Up @@ -112,7 +112,7 @@ private static Object processField(Object value)
}
if (value instanceof Map) {
Map<Object, Object> map = new HashMap<>();
for (Entry<Object, Object> entry : ((Map<Object, Object>) value).entrySet()) {
for (Entry<?, ?> entry : ((Map<?, ?>) value).entrySet()) {
map.put(processField(entry.getKey()), processField(entry.getValue()));
}
return map;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,7 @@ private static Slice[] flattenMapEntries(Slice[][] mapEntries)

public static <T> T[] alternatingNullValues(T[] objects)
{
@SuppressWarnings("unchecked")
T[] objectsWithNulls = (T[]) Array.newInstance(objects.getClass().getComponentType(), objects.length * 2 + 1);
for (int i = 0; i < objects.length; i++) {
objectsWithNulls[i * 2] = null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,13 +145,13 @@ else if (fieldValue instanceof String) {
protected <T> void assertPositionValue(Block block, int position, T expectedValue)
{
if (expectedValue instanceof List) {
assertValue(block, position, (List<Object>) expectedValue);
assertValue(block, position, (List<?>) expectedValue);
return;
}
super.assertPositionValue(block, position, expectedValue);
}

private void assertValue(Block rowBlock, int position, List<Object> row)
private void assertValue(Block rowBlock, int position, List<?> row)
{
// null rows are handled by assertPositionValue
requireNonNull(row, "row is null");
Expand Down Expand Up @@ -181,6 +181,7 @@ else if (fieldValue instanceof String) {

private List<Object>[] generateTestRows(List<Type> fieldTypes, int numRows)
{
@SuppressWarnings("unchecked")
List<Object>[] testRows = new List[numRows];
for (int i = 0; i < numRows; i++) {
List<Object> testRow = new ArrayList<>(fieldTypes.size());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,7 @@ private void writeRow(List<Object> testRow, BlockBuilder rowBlockBuilder)
// copied & modifed from TestRowBlock
private List<Object>[] generateTestRows(List<Type> fieldTypes, int numRows)
{
@SuppressWarnings("unchecked")
List<Object>[] testRows = new List[numRows];
for (int i = 0; i < numRows; i++) {
List<Object> testRow = new ArrayList<>(fieldTypes.size());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,8 @@ private void testSimpleSpiller(Spiller spiller)
testSpiller(TYPES, spiller, firstSpill, secondSpill);
}

private void testSpiller(List<Type> types, Spiller spiller, List<Page>... spills)
@SafeVarargs
private final void testSpiller(List<Type> types, Spiller spiller, List<Page>... spills)
throws ExecutionException, InterruptedException
{
long spilledBytesBefore = spillerStats.getTotalSpilledBytes();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -458,9 +458,10 @@ private static Set<Expression> equalityAsSet(Expression expression)
return ImmutableSet.of(comparisonExpression.getLeft(), comparisonExpression.getRight());
}

@SafeVarargs
private static <E> Set<E> set(E... elements)
{
return setCopy(Arrays.asList(elements));
return ImmutableSet.copyOf(elements);
}

private static <E> Set<E> setCopy(Iterable<E> elements)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,7 +87,8 @@ public final Stream<String> replaceAll(List<Parameter>... parametersLists)
return queries.stream();
}

public void replaceAll(Consumer<String> queryConsumer, List<Parameter>... parametersLists)
@SafeVarargs
public final void replaceAll(Consumer<String> queryConsumer, List<Parameter>... parametersLists)
{
requireNonNull(queryConsumer, "queryConsumer is null");
replaceAll(queryTemplate, queryConsumer, ImmutableList.copyOf(parametersLists));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -314,7 +314,7 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(C
return Optional.empty();
}

return Optional.of(new ConstraintApplicationResult(
return Optional.of(new ConstraintApplicationResult<>(
new RaptorTableHandle(table.getSchemaName(),
table.getTableName(),
table.getTableId(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -876,9 +876,11 @@ public void testGroupByNanArray()
MaterializedResult actual = computeActual("SELECT a FROM (VALUES (ARRAY[nan(), 2e0, 3e0]), (ARRAY[nan(), 2e0, 3e0])) t(a) GROUP BY a");
List<MaterializedRow> actualRows = actual.getMaterializedRows();
assertEquals(actualRows.size(), 1);
assertTrue(Double.isNaN(((List<Double>) actualRows.get(0).getField(0)).get(0)));
assertEquals(((List<Double>) actualRows.get(0).getField(0)).get(1), 2.0);
assertEquals(((List<Double>) actualRows.get(0).getField(0)).get(2), 3.0);
@SuppressWarnings("unchecked")
List<Double> value = (List<Double>) actualRows.get(0).getField(0);
assertTrue(Double.isNaN(value.get(0)));
assertEquals(value.get(1), 2.0);
assertEquals(value.get(2), 3.0);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4192,6 +4192,7 @@ public void testArrayShuffle()
assertEquals(rows.size(), 10);

for (MaterializedRow row : rows) {
@SuppressWarnings("unchecked")
List<Integer> actual = (List<Integer>) row.getField(0);

// check if the result is a correct permutation
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -232,21 +232,21 @@ else if (INTERVAL_YEAR_MONTH.equals(type)) {
return new SqlIntervalYearMonth(IntervalYearMonth.parseMonths(String.valueOf(value)));
}
else if (type instanceof ArrayType) {
return ((List<Object>) value).stream()
return ((List<?>) value).stream()
.map(element -> convertToRowValue(((ArrayType) type).getElementType(), element))
.collect(toList());
}
else if (type instanceof MapType) {
Map<Object, Object> result = new HashMap<>();
((Map<Object, Object>) value)
((Map<?, ?>) value)
.forEach((k, v) -> result.put(
convertToRowValue(((MapType) type).getKeyType(), k),
convertToRowValue(((MapType) type).getValueType(), v)));
return result;
}
else if (type instanceof RowType) {
List<Type> fieldTypes = type.getTypeParameters();
List<Object> fieldValues = ImmutableList.copyOf(((Map<Object, Object>) value).values());
List<Object> fieldValues = ImmutableList.copyOf(((Map<?, ?>) value).values());
return dataToRow(fieldTypes).apply(fieldValues);
}
else if (type instanceof DecimalType) {
Expand Down

0 comments on commit 8380483

Please sign in to comment.