Fix all inspection errors currently reported. (#6236)

* Fix all inspection errors currently reported.

TeamCity builds on master are reporting inspection errors, possibly
because there was a while where it was not running due to the Apache
migration, and there was some drift.

* Fix one more location.

* Fix tests.

* Another fix.
This commit is contained in:
Gian Merlino 2018-08-26 17:36:01 -07:00 committed by Fangjin Yang
parent ef91fdbf03
commit cb40b6d369
17 changed files with 54 additions and 69 deletions

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.RangeSet;
import com.google.common.collect.Sets;
import io.druid.TestObjectMapper;
@ -40,6 +39,7 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@ -238,7 +238,7 @@ public class DataSegmentTest
makeDataSegment("test2", "2011-02-02/2011-02-03", "a"),
};
List<DataSegment> shuffled = Lists.newArrayList(sortedOrder);
List<DataSegment> shuffled = new ArrayList<>(Arrays.asList(sortedOrder));
Collections.shuffle(shuffled);
Set<DataSegment> theSet = Sets.newTreeSet(DataSegment.bucketMonthComparator());

View File

@ -22,7 +22,6 @@ package io.druid.benchmark.query.timecompare;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.Files;
import io.druid.benchmark.datagen.BenchmarkDataGenerator;
import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
@ -96,6 +95,7 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
@ -117,7 +117,7 @@ public class TimeCompareBenchmark
@Param({"100"})
private int threshold;
protected static final Map<String, String> scriptDoubleSum = Maps.newHashMap();
protected static final Map<String, String> scriptDoubleSum = new HashMap<>();
static {
scriptDoubleSum.put("fnAggregate", "function aggregate(current, a) { return current + a }");
scriptDoubleSum.put("fnReset", "function reset() { return 0 }");
@ -427,10 +427,7 @@ public class TimeCompareBenchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void queryMultiQueryableIndexTopN(Blackhole blackhole)
{
Sequence<Result<TopNResultValue>> queryResult = topNRunner.run(
QueryPlus.wrap(topNQuery),
Maps.<String, Object>newHashMap()
);
Sequence<Result<TopNResultValue>> queryResult = topNRunner.run(QueryPlus.wrap(topNQuery), new HashMap<>());
List<Result<TopNResultValue>> results = queryResult.toList();
for (Result<TopNResultValue> result : results) {
@ -446,7 +443,7 @@ public class TimeCompareBenchmark
{
Sequence<Result<TimeseriesResultValue>> queryResult = timeseriesRunner.run(
QueryPlus.wrap(timeseriesQuery),
Maps.<String, Object>newHashMap()
new HashMap<>()
);
List<Result<TimeseriesResultValue>> results = queryResult.toList();

View File

@ -268,7 +268,7 @@ public class ParallelIndexSupervisorTask extends AbstractTask implements ChatHan
return TaskStatus.fromCode(getId(), runner.run());
}
private TaskStatus runSequential(TaskToolbox toolbox) throws Exception
private TaskStatus runSequential(TaskToolbox toolbox)
{
return new IndexTask(
getId(),

View File

@ -117,7 +117,9 @@ public class EC2AutoScalerTest
EasyMock.expect(runInstancesResult.getReservation()).andReturn(reservation).atLeastOnce();
EasyMock.replay(runInstancesResult);
EasyMock.expect(describeInstancesResult.getReservations()).andReturn(Collections.singletonList(reservation)).atLeastOnce();
EasyMock.expect(describeInstancesResult.getReservations())
.andReturn(Collections.singletonList(reservation))
.atLeastOnce();
EasyMock.replay(describeInstancesResult);
EasyMock.expect(reservation.getInstances()).andReturn(Collections.singletonList(instance)).atLeastOnce();
@ -177,12 +179,8 @@ public class EC2AutoScalerTest
Arrays.fill(chunk1, reservation);
final Reservation[] chunk2 = new Reservation[n - EC2AutoScaler.MAX_AWS_FILTER_VALUES];
Arrays.fill(chunk2, reservation);
EasyMock.expect(describeInstancesResult.getReservations()).andReturn(
Lists.newArrayList(chunk1)
);
EasyMock.expect(describeInstancesResult.getReservations()).andReturn(
Lists.newArrayList(chunk2)
);
EasyMock.expect(describeInstancesResult.getReservations()).andReturn(Arrays.asList(chunk1));
EasyMock.expect(describeInstancesResult.getReservations()).andReturn(Arrays.asList(chunk2));
EasyMock.replay(describeInstancesResult);
EasyMock.expect(reservation.getInstances()).andReturn(Collections.singletonList(instance)).times(n);
@ -234,12 +232,8 @@ public class EC2AutoScalerTest
Arrays.fill(chunk1, reservation);
final Reservation[] chunk2 = new Reservation[n - EC2AutoScaler.MAX_AWS_FILTER_VALUES];
Arrays.fill(chunk2, reservation);
EasyMock.expect(describeInstancesResult.getReservations()).andReturn(
Lists.newArrayList(chunk1)
);
EasyMock.expect(describeInstancesResult.getReservations()).andReturn(
Lists.newArrayList(chunk2)
);
EasyMock.expect(describeInstancesResult.getReservations()).andReturn(Arrays.asList(chunk1));
EasyMock.expect(describeInstancesResult.getReservations()).andReturn(Arrays.asList(chunk2));
EasyMock.replay(describeInstancesResult);
EasyMock.expect(reservation.getInstances()).andReturn(Collections.singletonList(instance)).times(n);

View File

@ -47,7 +47,6 @@ public class TopNNumericResultBuilder implements TopNResultBuilder
private final String metricName;
private final List<PostAggregator> postAggs;
private final PriorityQueue<DimValHolder> pQueue;
private final Comparator<DimValHolder> dimValHolderComparator;
private final String[] aggFactoryNames;
private static final Comparator<Comparable> dimValueComparator = new Comparator<Comparable>()
{
@ -91,7 +90,8 @@ public class TopNNumericResultBuilder implements TopNResultBuilder
this.postAggs = AggregatorUtil.pruneDependentPostAgg(postAggs, this.metricName);
this.threshold = threshold;
this.metricComparator = comparator;
this.dimValHolderComparator = (d1, d2) -> {
final Comparator<DimValHolder> dimValHolderComparator = (d1, d2) -> {
//noinspection unchecked
int retVal = metricComparator.compare(d1.getTopNMetricVal(), d2.getTopNMetricVal());

View File

@ -22,7 +22,6 @@ package io.druid.collections.bitmap;
import com.carrotsearch.junitbenchmarks.BenchmarkOptions;
import com.carrotsearch.junitbenchmarks.BenchmarkRule;
import com.carrotsearch.junitbenchmarks.Clock;
import com.google.common.collect.Lists;
import io.druid.extendedset.intset.ImmutableConciseSet;
import org.junit.Assert;
import org.junit.Rule;
@ -36,6 +35,7 @@ import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Locale;
import java.util.Random;
@ -152,7 +152,7 @@ public class BitmapBenchmark
@BenchmarkOptions(warmupRounds = 1, benchmarkRounds = 2)
public void timeGenericConciseUnion()
{
ImmutableBitmap union = conciseFactory.union(Lists.newArrayList(genericConcise));
ImmutableBitmap union = conciseFactory.union(Arrays.asList(genericConcise));
Assert.assertEquals(unionCount, union.size());
}
@ -160,42 +160,42 @@ public class BitmapBenchmark
@BenchmarkOptions(warmupRounds = 1, benchmarkRounds = 5)
public void timeGenericConciseIntersection()
{
ImmutableBitmap intersection = conciseFactory.intersection(Lists.newArrayList(genericConcise));
ImmutableBitmap intersection = conciseFactory.intersection(Arrays.asList(genericConcise));
Assert.assertTrue(intersection.size() >= minIntersection);
}
@Test
public void timeRoaringUnion()
{
ImmutableRoaringBitmap union = BufferFastAggregation.horizontal_or(Lists.newArrayList(roaring).iterator());
ImmutableRoaringBitmap union = BufferFastAggregation.horizontal_or(Arrays.asList(roaring).iterator());
Assert.assertEquals(unionCount, union.getCardinality());
}
@Test
public void timeImmutableRoaringUnion()
{
ImmutableRoaringBitmap union = BufferFastAggregation.horizontal_or(Lists.newArrayList(immutableRoaring).iterator());
ImmutableRoaringBitmap union = BufferFastAggregation.horizontal_or(Arrays.asList(immutableRoaring).iterator());
Assert.assertEquals(unionCount, union.getCardinality());
}
@Test
public void timeOffheapRoaringUnion()
{
ImmutableRoaringBitmap union = BufferFastAggregation.horizontal_or(Lists.newArrayList(offheapRoaring).iterator());
ImmutableRoaringBitmap union = BufferFastAggregation.horizontal_or(Arrays.asList(offheapRoaring).iterator());
Assert.assertEquals(unionCount, union.getCardinality());
}
@Test
public void timeGenericRoaringUnion()
{
ImmutableBitmap union = roaringFactory.union(Lists.newArrayList(genericRoaring));
ImmutableBitmap union = roaringFactory.union(Arrays.asList(genericRoaring));
Assert.assertEquals(unionCount, union.size());
}
@Test
public void timeGenericRoaringIntersection()
{
ImmutableBitmap intersection = roaringFactory.intersection(Lists.newArrayList(genericRoaring));
ImmutableBitmap intersection = roaringFactory.intersection(Arrays.asList(genericRoaring));
Assert.assertTrue(intersection.size() >= minIntersection);
}
}

View File

@ -344,7 +344,7 @@ public class AggregationTestHelper implements Closeable
{
File segmentDir = tempFolder.newFolder();
createIndex(inputDataFile, parserJson, aggregators, segmentDir, minTimestamp, gran, maxRowCount, true);
return runQueryOnSegments(Lists.newArrayList(segmentDir), groupByQueryJson);
return runQueryOnSegments(Collections.singletonList(segmentDir), groupByQueryJson);
}
public Sequence<Row> createIndexAndRunQueryOnSegment(

View File

@ -76,21 +76,21 @@ public class StringFirstTimeseriesQueryTest
new MapBasedInputRow(
timestamp,
Lists.newArrayList(visitor_id, client_type),
ImmutableMap.<String, Object>of(visitor_id, "0", client_type, "iphone")
ImmutableMap.of(visitor_id, "0", client_type, "iphone")
)
);
index.add(
new MapBasedInputRow(
timestamp,
Lists.newArrayList(visitor_id, client_type),
ImmutableMap.<String, Object>of(visitor_id, "1", client_type, "iphone")
ImmutableMap.of(visitor_id, "1", client_type, "iphone")
)
);
index.add(
new MapBasedInputRow(
timestamp1,
Lists.newArrayList(visitor_id, client_type),
ImmutableMap.<String, Object>of(visitor_id, "0", client_type, "android")
ImmutableMap.of(visitor_id, "0", client_type, "android")
)
);
@ -114,7 +114,7 @@ public class StringFirstTimeseriesQueryTest
new Result<>(
time,
new TimeseriesResultValue(
ImmutableMap.<String, Object>of("last_client_type", new SerializablePairLongString(timestamp, "iphone"))
ImmutableMap.of("last_client_type", new SerializablePairLongString(timestamp, "iphone"))
)
)
);

View File

@ -76,21 +76,21 @@ public class StringLastTimeseriesQueryTest
new MapBasedInputRow(
timestamp,
Lists.newArrayList(visitor_id, client_type),
ImmutableMap.<String, Object>of(visitor_id, "0", client_type, "iphone")
ImmutableMap.of(visitor_id, "0", client_type, "iphone")
)
);
index.add(
new MapBasedInputRow(
timestamp,
Lists.newArrayList(visitor_id, client_type),
ImmutableMap.<String, Object>of(visitor_id, "1", client_type, "iphone")
ImmutableMap.of(visitor_id, "1", client_type, "iphone")
)
);
index.add(
new MapBasedInputRow(
timestamp1,
Lists.newArrayList(visitor_id, client_type),
ImmutableMap.<String, Object>of(visitor_id, "0", client_type, "android")
ImmutableMap.of(visitor_id, "0", client_type, "android")
)
);
@ -99,7 +99,7 @@ public class StringLastTimeseriesQueryTest
.granularity(QueryRunnerTestHelper.allGran)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.aggregators(
Lists.newArrayList(
Collections.singletonList(
new StringLastAggregatorFactory(
"last_client_type", client_type, 1024
)
@ -114,7 +114,7 @@ public class StringLastTimeseriesQueryTest
new Result<>(
time,
new TimeseriesResultValue(
ImmutableMap.<String, Object>of(
ImmutableMap.of(
"last_client_type",
new SerializablePairLongString(timestamp1, "android")
)

View File

@ -161,7 +161,7 @@ public class BoundFilterTest extends BaseFilterTest
);
assertFilterMatches(
new BoundDimFilter("dim3", null, "", false, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of()
ImmutableList.of()
);
} else {
assertFilterMatches(
@ -174,7 +174,7 @@ public class BoundFilterTest extends BaseFilterTest
);
assertFilterMatches(
new BoundDimFilter("dim3", null, "", false, true, false, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of("0", "1", "2", "3", "4", "5", "6", "7")
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
);
}
assertFilterMatches(

View File

@ -123,35 +123,35 @@ public class ColumnComparisonFilterTest extends BaseFilterTest
new ColumnComparisonDimFilter(
ImmutableList.of(DefaultDimensionSpec.of("dim6"), DefaultDimensionSpec.of("dim7"))
),
ImmutableList.<String>of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
);
if (NullHandling.replaceWithDefault()) {
assertFilterMatches(
new ColumnComparisonDimFilter(
ImmutableList.of(DefaultDimensionSpec.of("dim1"), DefaultDimensionSpec.of("dim6"))
),
ImmutableList.<String>of("0")
ImmutableList.of("0")
);
assertFilterMatches(
new ColumnComparisonDimFilter(
ImmutableList.of(DefaultDimensionSpec.of("dim2"), DefaultDimensionSpec.of("dim6"))
),
ImmutableList.<String>of("1", "2", "6", "7", "8")
ImmutableList.of("1", "2", "6", "7", "8")
);
} else {
assertFilterMatches(
new ColumnComparisonDimFilter(
ImmutableList.of(DefaultDimensionSpec.of("dim1"), DefaultDimensionSpec.of("dim6"))
),
ImmutableList.<String>of()
ImmutableList.of()
);
assertFilterMatches(
new ColumnComparisonDimFilter(
ImmutableList.of(DefaultDimensionSpec.of("dim2"), DefaultDimensionSpec.of("dim6"))
),
ImmutableList.<String>of("1", "6", "7", "8")
ImmutableList.of("1", "6", "7", "8")
);
}
}

View File

@ -192,7 +192,7 @@ public class SearchQueryFilterTest extends BaseFilterTest
assertFilterMatches(
new SearchQueryDimFilter("dim1", specForValue("ab"), changeNullFn),
ImmutableList.<String>of("4", "5")
ImmutableList.of("4", "5")
);
assertFilterMatches(new SearchQueryDimFilter("dim1", specForValue("ab"), changeNullFn), ImmutableList.of("4", "5"));

View File

@ -131,6 +131,7 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
final int maxToLoad = params.getCoordinatorDynamicConfig().getMaxSegmentsInNodeLoadingQueue();
int moved = 0, unmoved = 0;
//noinspection ForLoopThatDoesntUseLoopVariable
for (int iter = 0; (moved + unmoved) < maxSegmentsToMove; ++iter) {
final BalancerSegmentHolder segmentToMoveHolder = strategy.pickSegmentToMove(toMoveFrom);

View File

@ -33,6 +33,7 @@ import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@ -131,7 +132,7 @@ public class TierSelectorStrategyTest
tierSelectorStrategy
);
List<QueryableDruidServer> servers = Lists.newArrayList(expectedSelection);
List<QueryableDruidServer> servers = new ArrayList<>(Arrays.asList(expectedSelection));
List<DruidServerMetadata> expectedCandidates = Lists.newArrayList();
for (QueryableDruidServer server : servers) {

View File

@ -34,7 +34,6 @@ import io.druid.segment.TestHelper;
import io.druid.segment.loading.CacheTestSegmentLoader;
import io.druid.segment.loading.SegmentLoaderConfig;
import io.druid.server.SegmentManager;
import io.druid.server.initialization.ZkPathsConfig;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
import org.easymock.EasyMock;
@ -45,6 +44,7 @@ import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@ -98,19 +98,10 @@ public class SegmentLoadDropHandlerTest
throw new RuntimeException(e);
}
scheduledRunnable = Lists.newArrayList();
scheduledRunnable = new ArrayList<>();
segmentLoader = new CacheTestSegmentLoader();
segmentManager = new SegmentManager(segmentLoader);
final ZkPathsConfig zkPaths = new ZkPathsConfig()
{
@Override
public String getBase()
{
return "/druid";
}
};
segmentsAnnouncedByMe = new ConcurrentSkipListSet<>();
announceCount = new AtomicInteger(0);
@ -377,7 +368,7 @@ public class SegmentLoadDropHandlerTest
Assert.assertTrue(infoDir.exists());
File[] files = infoDir.listFiles();
List<File> sortedFiles = Lists.newArrayList(files);
List<File> sortedFiles = Arrays.asList(files);
Collections.sort(sortedFiles);
Assert.assertEquals(segments.size(), sortedFiles.size());
@ -478,7 +469,8 @@ public class SegmentLoadDropHandlerTest
Assert.assertEquals(SegmentLoadDropHandler.Status.SUCCESS, result.get(1).getStatus());
for (SegmentLoadDropHandler.DataSegmentChangeRequestAndStatus e : segmentLoadDropHandler.processBatch(batch).get()) {
for (SegmentLoadDropHandler.DataSegmentChangeRequestAndStatus e : segmentLoadDropHandler.processBatch(batch)
.get()) {
Assert.assertEquals(SegmentLoadDropHandler.Status.SUCCESS, e.getStatus());
}

View File

@ -22,7 +22,6 @@ package io.druid.server.coordinator.helper;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.guava.Comparators;
@ -37,6 +36,7 @@ import org.junit.Assert;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
@ -405,7 +405,7 @@ public class NewestSegmentFirstPolicyTest
final String version = DateTimes.nowUtc().toString();
final List<SegmentGenerateSpec> orderedSpecs = Lists.newArrayList(specs);
final List<SegmentGenerateSpec> orderedSpecs = Arrays.asList(specs);
orderedSpecs.sort((s1, s2) -> Comparators.intervalsByStartThenEnd().compare(s1.totalInterval, s2.totalInterval));
Collections.reverse(orderedSpecs);

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.DateTimes;
@ -37,6 +36,7 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
@ -192,7 +192,7 @@ public class DataSegmentTest
makeDataSegment("test2", "2011-02-02/2011-02-03", "a"),
};
List<DataSegment> shuffled = Lists.newArrayList(sortedOrder);
List<DataSegment> shuffled = new ArrayList<>(Arrays.asList(sortedOrder));
Collections.shuffle(shuffled);
Set<DataSegment> theSet = Sets.newTreeSet(DataSegment.bucketMonthComparator());