mirror of https://github.com/apache/druid.git
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:
parent
ef91fdbf03
commit
cb40b6d369
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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"))
|
||||
)
|
||||
)
|
||||
);
|
||||
|
|
|
@ -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")
|
||||
)
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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")
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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"));
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
Loading…
Reference in New Issue