mirror of https://github.com/apache/druid.git
Prohibit LinkedList (#6112)
* Prohibit LinkedList * Fix tests * Fix * Remove unused import
This commit is contained in:
parent
fd6786ac6c
commit
d50b69e6d4
|
@ -19,13 +19,13 @@
|
|||
|
||||
package org.apache.druid.guice;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.inject.Key;
|
||||
import com.google.inject.Provider;
|
||||
import com.google.inject.Scope;
|
||||
import org.apache.druid.java.util.common.lifecycle.Lifecycle;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -37,7 +37,7 @@ public class LifecycleScope implements Scope
|
|||
private final Lifecycle.Stage stage;
|
||||
|
||||
private Lifecycle lifecycle;
|
||||
private final List<Object> instances = Lists.newLinkedList();
|
||||
private final List<Object> instances = new ArrayList<>();
|
||||
|
||||
public LifecycleScope(Lifecycle.Stage stage)
|
||||
{
|
||||
|
|
|
@ -5,3 +5,7 @@ com.google.common.collect.Iterators#emptyIterator() @ Use java.util.Collections#
|
|||
com.google.common.base.Charsets @ Use java.nio.charset.StandardCharsets instead
|
||||
java.io.File#toURL() @ Use java.io.File#toURI() and java.net.URI#toURL() instead
|
||||
org.apache.commons.io.FileUtils#getTempDirectory() @ Use org.junit.rules.TemporaryFolder for tests instead
|
||||
|
||||
java.util.LinkedList @ Use ArrayList or ArrayDeque instead
|
||||
com.google.common.collect.Lists#newLinkedList() @ Use ArrayList or ArrayDeque instead
|
||||
com.google.common.collect.Lists#newLinkedList(java.lang.Iterable) @ Use ArrayList or ArrayDeque instead
|
||||
|
|
|
@ -20,12 +20,12 @@
|
|||
package org.apache.druid.query.aggregation.histogram;
|
||||
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
@ -239,7 +239,7 @@ public class ApproximateHistogramTest
|
|||
randNums[i] = (float) rand.nextGaussian();
|
||||
}
|
||||
|
||||
List<ApproximateHistogram> randHist = Lists.newLinkedList();
|
||||
List<ApproximateHistogram> randHist = new ArrayList<>();
|
||||
Iterator<ApproximateHistogram> it = Iterators.cycle(randHist);
|
||||
|
||||
for (int k = 0; k < numRand; ++k) {
|
||||
|
|
|
@ -99,11 +99,11 @@ import javax.ws.rs.core.Response;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
@ -181,8 +181,8 @@ public class IncrementalPublishingKafkaIndexTaskRunner implements KafkaIndexTask
|
|||
private final RowIngestionMeters rowIngestionMeters;
|
||||
|
||||
private final Set<String> publishingSequences = Sets.newConcurrentHashSet();
|
||||
private final List<ListenableFuture<SegmentsAndMetadata>> publishWaitList = new LinkedList<>();
|
||||
private final List<ListenableFuture<SegmentsAndMetadata>> handOffWaitList = new LinkedList<>();
|
||||
private final List<ListenableFuture<SegmentsAndMetadata>> publishWaitList = new ArrayList<>();
|
||||
private final List<ListenableFuture<SegmentsAndMetadata>> handOffWaitList = new ArrayList<>();
|
||||
|
||||
private volatile DateTime startTime;
|
||||
private volatile Status status = Status.NOT_STARTED; // this is only ever set by the task runner thread (runThread)
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.google.caliper.Param;
|
|||
import com.google.caliper.Runner;
|
||||
import com.google.caliper.SimpleBenchmark;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.hash.HashFunction;
|
||||
import com.google.common.hash.Hashing;
|
||||
import sun.misc.Unsafe;
|
||||
|
@ -31,6 +30,7 @@ import sun.misc.Unsafe;
|
|||
import java.lang.reflect.Field;
|
||||
import java.nio.Buffer;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
|
@ -41,7 +41,7 @@ public class HyperLogLogCollectorBenchmark extends SimpleBenchmark
|
|||
{
|
||||
private final HashFunction fn = Hashing.murmur3_128();
|
||||
|
||||
private final List<HyperLogLogCollector> collectors = Lists.newLinkedList();
|
||||
private final List<HyperLogLogCollector> collectors = new ArrayList<>();
|
||||
|
||||
@Param({"true"}) boolean targetIsDirect;
|
||||
@Param({"default", "random", "0"}) String alignment;
|
||||
|
|
|
@ -29,6 +29,7 @@ import com.google.common.collect.Lists;
|
|||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.client.cache.CacheConfig;
|
||||
import org.apache.druid.client.cache.CachePopulatorStats;
|
||||
import org.apache.druid.client.cache.MapCache;
|
||||
|
@ -49,12 +50,12 @@ import org.apache.druid.discovery.DruidNodeAnnouncer;
|
|||
import org.apache.druid.discovery.LookupNodeService;
|
||||
import org.apache.druid.indexer.IngestionState;
|
||||
import org.apache.druid.indexer.TaskState;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.Counters;
|
||||
import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData;
|
||||
import org.apache.druid.indexing.common.SegmentLoaderFactory;
|
||||
import org.apache.druid.indexing.common.TaskReport;
|
||||
import org.apache.druid.indexing.common.TaskReportFileWriter;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.TaskToolboxFactory;
|
||||
import org.apache.druid.indexing.common.TestUtils;
|
||||
|
@ -133,7 +134,6 @@ import org.apache.druid.timeline.DataSegment;
|
|||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||
import org.apache.druid.timeline.partition.NumberedShardSpec;
|
||||
import org.apache.druid.utils.Runnables;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.easymock.EasyMock;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Period;
|
||||
|
@ -149,10 +149,11 @@ import javax.annotation.Nullable;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.file.Files;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.LinkedList;
|
||||
import java.util.Deque;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
@ -178,7 +179,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
private static class TestFirehose implements Firehose
|
||||
{
|
||||
private final InputRowParser<Map<String, Object>> parser;
|
||||
private final List<Map<String, Object>> queue = new LinkedList<>();
|
||||
private final Deque<Optional<Map<String, Object>>> queue = new ArrayDeque<>();
|
||||
private boolean closed = false;
|
||||
|
||||
public TestFirehose(final InputRowParser<Map<String, Object>> parser)
|
||||
|
@ -189,7 +190,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
public void addRows(List<Map<String, Object>> rows)
|
||||
{
|
||||
synchronized (this) {
|
||||
queue.addAll(rows);
|
||||
rows.stream().map(Optional::ofNullable).forEach(queue::add);
|
||||
notifyAll();
|
||||
}
|
||||
}
|
||||
|
@ -215,7 +216,7 @@ public class AppenderatorDriverRealtimeIndexTaskTest
|
|||
public InputRow nextRow()
|
||||
{
|
||||
synchronized (this) {
|
||||
final InputRow row = parser.parseBatch(queue.remove(0)).get(0);
|
||||
final InputRow row = parser.parseBatch(queue.removeFirst().orElse(null)).get(0);
|
||||
if (row != null && row.getRaw(FAIL_DIM) != null) {
|
||||
throw new ParseException(FAIL_DIM);
|
||||
}
|
||||
|
|
|
@ -46,9 +46,9 @@ import org.apache.druid.discovery.DataNodeService;
|
|||
import org.apache.druid.discovery.DruidNodeAnnouncer;
|
||||
import org.apache.druid.discovery.LookupNodeService;
|
||||
import org.apache.druid.indexer.TaskState;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.Counters;
|
||||
import org.apache.druid.indexing.common.SegmentLoaderFactory;
|
||||
import org.apache.druid.indexer.TaskStatus;
|
||||
import org.apache.druid.indexing.common.TaskToolbox;
|
||||
import org.apache.druid.indexing.common.TaskToolboxFactory;
|
||||
import org.apache.druid.indexing.common.TestUtils;
|
||||
|
@ -138,10 +138,12 @@ import org.junit.rules.TemporaryFolder;
|
|||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.nio.file.Files;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Arrays;
|
||||
import java.util.LinkedList;
|
||||
import java.util.Deque;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
@ -161,7 +163,7 @@ public class RealtimeIndexTaskTest
|
|||
private static class TestFirehose implements Firehose
|
||||
{
|
||||
private final InputRowParser<Map<String, Object>> parser;
|
||||
private final List<Map<String, Object>> queue = new LinkedList<>();
|
||||
private final Deque<Optional<Map<String, Object>>> queue = new ArrayDeque<>();
|
||||
private boolean closed = false;
|
||||
|
||||
public TestFirehose(final InputRowParser<Map<String, Object>> parser)
|
||||
|
@ -172,7 +174,7 @@ public class RealtimeIndexTaskTest
|
|||
public void addRows(List<Map<String, Object>> rows)
|
||||
{
|
||||
synchronized (this) {
|
||||
queue.addAll(rows);
|
||||
rows.stream().map(Optional::ofNullable).forEach(queue::add);
|
||||
notifyAll();
|
||||
}
|
||||
}
|
||||
|
@ -198,7 +200,7 @@ public class RealtimeIndexTaskTest
|
|||
public InputRow nextRow()
|
||||
{
|
||||
synchronized (this) {
|
||||
final InputRow row = parser.parseBatch(queue.remove(0)).get(0);
|
||||
final InputRow row = parser.parseBatch(queue.removeFirst().orElse(null)).get(0);
|
||||
if (row != null && row.getRaw(FAIL_DIM) != null) {
|
||||
throw new ParseException(FAIL_DIM);
|
||||
}
|
||||
|
|
|
@ -110,7 +110,6 @@ import java.nio.file.Paths;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -324,7 +323,7 @@ public class IngestSegmentFirehoseFactoryTest
|
|||
null,
|
||||
new NoopTestTaskFileWriter()
|
||||
);
|
||||
Collection<Object[]> values = new LinkedList<>();
|
||||
Collection<Object[]> values = new ArrayList<>();
|
||||
for (InputRowParser parser : Arrays.<InputRowParser>asList(
|
||||
ROW_PARSER,
|
||||
new MapInputRowParser(
|
||||
|
|
|
@ -23,7 +23,8 @@ import org.apache.druid.java.util.common.logger.Logger;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.LinkedList;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Deque;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -31,7 +32,7 @@ public class AppendableByteArrayInputStream extends InputStream
|
|||
{
|
||||
private static final Logger log = new Logger(AppendableByteArrayInputStream.class);
|
||||
|
||||
private final LinkedList<byte[]> bytes = new LinkedList<byte[]>();
|
||||
private final Deque<byte[]> bytes = new ArrayDeque<>();
|
||||
private final SingleByteReaderDoer singleByteReaderDoer = new SingleByteReaderDoer();
|
||||
|
||||
private volatile boolean done = false;
|
||||
|
|
|
@ -37,8 +37,9 @@ import org.apache.druid.segment.FloatColumnSelector;
|
|||
import org.apache.druid.segment.LongColumnSelector;
|
||||
import org.apache.druid.segment.virtual.ExpressionSelectors;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -113,18 +114,20 @@ public class AggregatorUtil
|
|||
*/
|
||||
public static List<PostAggregator> pruneDependentPostAgg(List<PostAggregator> postAggregatorList, String postAggName)
|
||||
{
|
||||
LinkedList<PostAggregator> rv = Lists.newLinkedList();
|
||||
ArrayList<PostAggregator> rv = new ArrayList<>();
|
||||
Set<String> deps = new HashSet<>();
|
||||
deps.add(postAggName);
|
||||
// Iterate backwards to find the last calculated aggregate and add dependent aggregator as we find dependencies in reverse order
|
||||
// Iterate backwards to find the last calculated aggregate and add dependent aggregator as we find dependencies
|
||||
// in reverse order
|
||||
for (PostAggregator agg : Lists.reverse(postAggregatorList)) {
|
||||
if (deps.contains(agg.getName())) {
|
||||
rv.addFirst(agg); // add to the beginning of List
|
||||
rv.add(agg); // add to the beginning of List
|
||||
deps.remove(agg.getName());
|
||||
deps.addAll(agg.getDependentFields());
|
||||
}
|
||||
}
|
||||
|
||||
Collections.reverse(rv);
|
||||
return rv;
|
||||
}
|
||||
|
||||
|
@ -135,10 +138,7 @@ public class AggregatorUtil
|
|||
)
|
||||
{
|
||||
|
||||
List<PostAggregator> condensedPostAggs = AggregatorUtil.pruneDependentPostAgg(
|
||||
postAggList,
|
||||
metric
|
||||
);
|
||||
List<PostAggregator> condensedPostAggs = AggregatorUtil.pruneDependentPostAgg(postAggList, metric);
|
||||
// calculate dependent aggregators for these postAgg
|
||||
Set<String> dependencySet = new HashSet<>();
|
||||
dependencySet.add(metric);
|
||||
|
@ -152,7 +152,7 @@ public class AggregatorUtil
|
|||
condensedAggs.add(aggregatorSpec);
|
||||
}
|
||||
}
|
||||
return new Pair(condensedAggs, condensedPostAggs);
|
||||
return new Pair<>(condensedAggs, condensedPostAggs);
|
||||
}
|
||||
|
||||
public static BaseFloatColumnValueSelector makeColumnValueSelectorWithFloatDefault(
|
||||
|
|
|
@ -75,11 +75,8 @@ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm<int[], T
|
|||
)
|
||||
{
|
||||
final String metric = query.getTopNMetricSpec().getMetricName(query.getDimensionSpec());
|
||||
Pair<List<AggregatorFactory>, List<PostAggregator>> condensedAggPostAggPair = AggregatorUtil.condensedAggregators(
|
||||
query.getAggregatorSpecs(),
|
||||
query.getPostAggregatorSpecs(),
|
||||
metric
|
||||
);
|
||||
Pair<List<AggregatorFactory>, List<PostAggregator>> condensedAggPostAggPair =
|
||||
AggregatorUtil.condensedAggregators(query.getAggregatorSpecs(), query.getPostAggregatorSpecs(), metric);
|
||||
|
||||
if (condensedAggPostAggPair.lhs.isEmpty() && condensedAggPostAggPair.rhs.isEmpty()) {
|
||||
throw new ISE("WTF! Can't find the metric to do topN over?");
|
||||
|
|
|
@ -37,7 +37,9 @@ import org.apache.druid.segment.VirtualColumn;
|
|||
import org.apache.druid.segment.VirtualColumns;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -247,13 +249,13 @@ public class TopNQueryBuilder
|
|||
@SuppressWarnings("unchecked")
|
||||
public TopNQueryBuilder aggregators(List<? extends AggregatorFactory> a)
|
||||
{
|
||||
aggregatorSpecs = (List<AggregatorFactory>) a;
|
||||
aggregatorSpecs = new ArrayList<>(a); // defensive copy
|
||||
return this;
|
||||
}
|
||||
|
||||
public TopNQueryBuilder postAggregators(List<PostAggregator> p)
|
||||
public TopNQueryBuilder postAggregators(Collection<PostAggregator> p)
|
||||
{
|
||||
postAggregatorSpecs = p;
|
||||
postAggregatorSpecs = new ArrayList<>(p); // defensive copy
|
||||
return this;
|
||||
}
|
||||
|
||||
|
|
|
@ -172,7 +172,8 @@ public class AggregatorUtilTest
|
|||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
new Pair(Lists.newArrayList(agg1, agg2), Collections.singletonList(postAgg2)), AggregatorUtil.condensedAggregators(
|
||||
new Pair<>(Lists.newArrayList(agg1, agg2), Collections.singletonList(postAgg2)),
|
||||
AggregatorUtil.condensedAggregators(
|
||||
Lists.newArrayList(agg1, agg2),
|
||||
Lists.newArrayList(postAgg1, postAgg2),
|
||||
"postAgg"
|
||||
|
|
|
@ -54,6 +54,7 @@ import org.junit.Test;
|
|||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
|
@ -754,11 +755,11 @@ public class ScanQueryRunnerTest
|
|||
@Override
|
||||
public ScanResultValue apply(ScanResultValue input)
|
||||
{
|
||||
List mapEvents = Lists.newLinkedList();
|
||||
List events = ((List) input.getEvents());
|
||||
List<Map<String, Object>> mapEvents = new ArrayList<>();
|
||||
List<?> events = ((List<?>) input.getEvents());
|
||||
for (Object event : events) {
|
||||
Iterator compactedEventIter = ((List) event).iterator();
|
||||
Map mapEvent = new LinkedHashMap();
|
||||
Iterator<?> compactedEventIter = ((List<?>) event).iterator();
|
||||
Map<String, Object> mapEvent = new LinkedHashMap<>();
|
||||
for (String column : input.getColumns()) {
|
||||
mapEvent.put(column, compactedEventIter.next());
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.query.search;
|
|||
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
|
@ -64,6 +63,7 @@ import org.junit.Test;
|
|||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
@ -135,7 +135,7 @@ public class SearchQueryRunnerTest
|
|||
.query("a")
|
||||
.build();
|
||||
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "automotive", 93));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "mezzanine", 279));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "travel", 93));
|
||||
|
@ -178,7 +178,7 @@ public class SearchQueryRunnerTest
|
|||
}
|
||||
);
|
||||
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "automotive", 91));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "mezzanine", 273));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "travel", 91));
|
||||
|
@ -207,7 +207,7 @@ public class SearchQueryRunnerTest
|
|||
.query("e")
|
||||
.build();
|
||||
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.placementDimension, "preferred", 1209));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.placementishDimension, "e", 93));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.placementishDimension, "preferred", 1209));
|
||||
|
@ -232,7 +232,7 @@ public class SearchQueryRunnerTest
|
|||
.query("e")
|
||||
.build();
|
||||
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.placementishDimension, "e", 93));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.placementDimension, "preferred", 1209));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.placementishDimension, "preferred", 1209));
|
||||
|
@ -250,7 +250,7 @@ public class SearchQueryRunnerTest
|
|||
.query(new FragmentSearchQuerySpec(Arrays.asList("auto", "ve")))
|
||||
.build();
|
||||
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "automotive", 93));
|
||||
|
||||
checkSearchQuery(searchQuery, expectedHits);
|
||||
|
@ -259,7 +259,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchWithDimensionQuality()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "automotive", 93));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "mezzanine", 279));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "travel", 93));
|
||||
|
@ -281,7 +281,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchWithDimensionProvider()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.marketDimension, "total_market", 186));
|
||||
|
||||
checkSearchQuery(
|
||||
|
@ -299,7 +299,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchWithDimensionsQualityAndProvider()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "automotive", 93));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "mezzanine", 279));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "travel", 93));
|
||||
|
@ -327,7 +327,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchWithDimensionsPlacementAndProvider()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.marketDimension, "total_market", 186));
|
||||
|
||||
checkSearchQuery(
|
||||
|
@ -352,7 +352,7 @@ public class SearchQueryRunnerTest
|
|||
public void testSearchWithExtractionFilter1()
|
||||
{
|
||||
final String automotiveSnowman = "automotive☃";
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, automotiveSnowman, 93));
|
||||
|
||||
final LookupExtractionFn lookupExtractionFn = new LookupExtractionFn(
|
||||
|
@ -391,7 +391,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchWithSingleFilter1()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "mezzanine", 93));
|
||||
|
||||
checkSearchQuery(
|
||||
|
@ -415,7 +415,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchWithSingleFilter2()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.marketDimension, "total_market", 186));
|
||||
|
||||
checkSearchQuery(
|
||||
|
@ -434,7 +434,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchMultiAndFilter()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "automotive", 93));
|
||||
|
||||
DimFilter filter = new AndDimFilter(
|
||||
|
@ -458,7 +458,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchWithMultiOrFilter()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "automotive", 93));
|
||||
|
||||
DimFilter filter = new OrDimFilter(
|
||||
|
@ -482,7 +482,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchWithEmptyResults()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
|
||||
checkSearchQuery(
|
||||
Druids.newSearchQueryBuilder()
|
||||
|
@ -498,7 +498,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchWithFilterEmptyResults()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
|
||||
DimFilter filter = new AndDimFilter(
|
||||
new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "total_market", null),
|
||||
|
@ -521,7 +521,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchNonExistingDimension()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
|
||||
checkSearchQuery(
|
||||
Druids.newSearchQueryBuilder()
|
||||
|
@ -538,7 +538,7 @@ public class SearchQueryRunnerTest
|
|||
@Test
|
||||
public void testSearchAll()
|
||||
{
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.marketDimension, "spot", 837));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.marketDimension, "total_market", 186));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.marketDimension, "upfront", 186));
|
||||
|
@ -575,7 +575,7 @@ public class SearchQueryRunnerTest
|
|||
.sortSpec(new SearchSortSpec(StringComparators.NUMERIC))
|
||||
.build();
|
||||
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.placementishDimension, "a", 93));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "automotive", 93));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.qualityDimension, "entertainment", 93));
|
||||
|
@ -609,7 +609,7 @@ public class SearchQueryRunnerTest
|
|||
))
|
||||
.build();
|
||||
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit("__time2", "Friday", 169));
|
||||
|
||||
checkSearchQuery(searchQuery, expectedHits);
|
||||
|
@ -631,7 +631,7 @@ public class SearchQueryRunnerTest
|
|||
.query("1297123200000")
|
||||
.build();
|
||||
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(Column.TIME_COLUMN_NAME, "1297123200000", 13));
|
||||
checkSearchQuery(searchQuery, expectedHits);
|
||||
}
|
||||
|
@ -655,7 +655,7 @@ public class SearchQueryRunnerTest
|
|||
.query("1297123200000")
|
||||
.build();
|
||||
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(Column.TIME_COLUMN_NAME, "super-1297123200000", 13));
|
||||
checkSearchQuery(searchQuery, expectedHits);
|
||||
}
|
||||
|
@ -676,7 +676,7 @@ public class SearchQueryRunnerTest
|
|||
.query("100.7")
|
||||
.build();
|
||||
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "100.706057", 1));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "100.775597", 1));
|
||||
checkSearchQuery(searchQuery, expectedHits);
|
||||
|
@ -701,7 +701,7 @@ public class SearchQueryRunnerTest
|
|||
.query("100.7")
|
||||
.build();
|
||||
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "super-100.706057", 1));
|
||||
expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "super-100.775597", 1));
|
||||
checkSearchQuery(searchQuery, expectedHits);
|
||||
|
@ -751,7 +751,7 @@ public class SearchQueryRunnerTest
|
|||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
);
|
||||
QueryRunner runner = factory.createRunner(new QueryableIndexSegment("asdf", TestIndex.persistRealtimeAndLoadMMapped(index)));
|
||||
List<SearchHit> expectedHits = Lists.newLinkedList();
|
||||
List<SearchHit> expectedHits = new ArrayList<>();
|
||||
expectedHits.add(new SearchHit("table", "table", 1));
|
||||
expectedHits.add(new SearchHit("table", NullHandling.defaultStringValue(), 1));
|
||||
checkSearchQuery(searchQuery, runner, expectedHits);
|
||||
|
@ -769,7 +769,7 @@ public class SearchQueryRunnerTest
|
|||
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||
.build();
|
||||
|
||||
List<SearchHit> noHit = Lists.newLinkedList();
|
||||
List<SearchHit> noHit = new ArrayList<>();
|
||||
checkSearchQuery(searchQuery, noHit);
|
||||
}
|
||||
|
||||
|
@ -782,7 +782,7 @@ public class SearchQueryRunnerTest
|
|||
private void checkSearchQuery(Query searchQuery, QueryRunner runner, List<SearchHit> expectedResults)
|
||||
{
|
||||
Iterable<Result<SearchResultValue>> results = runner.run(QueryPlus.wrap(searchQuery), ImmutableMap.of()).toList();
|
||||
List<SearchHit> copy = Lists.newLinkedList(expectedResults);
|
||||
List<SearchHit> copy = new ArrayList<>(expectedResults);
|
||||
for (Result<SearchResultValue> result : results) {
|
||||
Assert.assertEquals(DateTimes.of("2011-01-12T00:00:00.000Z"), result.getTimestamp());
|
||||
Assert.assertTrue(result.getValue() instanceof Iterable);
|
||||
|
|
|
@ -36,9 +36,9 @@ import org.junit.Test;
|
|||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
@ -196,7 +196,7 @@ public class BenchmarkIndexibleWrites extends AbstractBenchmark
|
|||
)
|
||||
);
|
||||
final AtomicInteger index = new AtomicInteger(0);
|
||||
List<ListenableFuture<?>> futures = new LinkedList<>();
|
||||
List<ListenableFuture<?>> futures = new ArrayList<>();
|
||||
|
||||
final Integer loops = totalIndexSize / concurrentThreads;
|
||||
|
||||
|
@ -249,7 +249,7 @@ public class BenchmarkIndexibleWrites extends AbstractBenchmark
|
|||
);
|
||||
final AtomicInteger index = new AtomicInteger(0);
|
||||
final AtomicInteger queryableIndex = new AtomicInteger(0);
|
||||
List<ListenableFuture<?>> futures = new LinkedList<>();
|
||||
List<ListenableFuture<?>> futures = new ArrayList<>();
|
||||
|
||||
final Integer loops = totalIndexSize / concurrentThreads;
|
||||
|
||||
|
|
|
@ -68,7 +68,6 @@ import java.lang.reflect.InvocationTargetException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
@ -347,8 +346,8 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark
|
|||
);
|
||||
final long timestamp = System.currentTimeMillis();
|
||||
final Interval queryInterval = Intervals.of("1900-01-01T00:00:00Z/2900-01-01T00:00:00Z");
|
||||
final List<ListenableFuture<?>> indexFutures = new LinkedList<>();
|
||||
final List<ListenableFuture<?>> queryFutures = new LinkedList<>();
|
||||
final List<ListenableFuture<?>> indexFutures = new ArrayList<>();
|
||||
final List<ListenableFuture<?>> queryFutures = new ArrayList<>();
|
||||
final Segment incrementalIndexSegment = new IncrementalIndexSegment(incrementalIndex, null);
|
||||
final QueryRunnerFactory factory = new TimeseriesQueryRunnerFactory(
|
||||
new TimeseriesQueryQueryToolChest(QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
|
||||
|
|
|
@ -38,10 +38,10 @@ import org.apache.druid.query.QueryToolChest;
|
|||
import org.apache.druid.query.SegmentDescriptor;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.Map;
|
||||
|
||||
public class CachingQueryRunner<T> implements QueryRunner<T>
|
||||
|
@ -134,7 +134,7 @@ public class CachingQueryRunner<T> implements QueryRunner<T>
|
|||
}
|
||||
}
|
||||
|
||||
final Collection<ListenableFuture<?>> cacheFutures = Collections.synchronizedList(new LinkedList<>());
|
||||
final Collection<ListenableFuture<?>> cacheFutures = Collections.synchronizedList(new ArrayList<>());
|
||||
if (populateCache) {
|
||||
final Function cacheFn = strategy.prepareForSegmentLevelCache();
|
||||
return cachePopulator.wrap(base.run(queryPlus, responseContext), value -> cacheFn.apply(value), cache, key);
|
||||
|
|
|
@ -32,7 +32,7 @@ import org.apache.druid.java.util.common.guava.Sequences;
|
|||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.util.LinkedList;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.function.Function;
|
||||
|
@ -67,7 +67,7 @@ public class BackgroundCachePopulator implements CachePopulator
|
|||
final Cache.NamedKey cacheKey
|
||||
)
|
||||
{
|
||||
final List<ListenableFuture<CacheType>> cacheFutures = new LinkedList<>();
|
||||
final List<ListenableFuture<CacheType>> cacheFutures = new ArrayList<>();
|
||||
|
||||
final Sequence<T> wrappedSequence = Sequences.map(
|
||||
sequence,
|
||||
|
|
|
@ -19,12 +19,12 @@
|
|||
|
||||
package org.apache.druid.client.cache;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Ints;
|
||||
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -127,7 +127,7 @@ public class MapCache implements Cache
|
|||
}
|
||||
synchronized (clearLock) {
|
||||
Iterator<ByteBuffer> iter = baseMap.keySet().iterator();
|
||||
List<ByteBuffer> toRemove = Lists.newLinkedList();
|
||||
List<ByteBuffer> toRemove = new ArrayList<>();
|
||||
while (iter.hasNext()) {
|
||||
ByteBuffer next = iter.next();
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.core.type.TypeReference;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.druid.indexer.TaskInfo;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
|
@ -47,6 +46,7 @@ import javax.annotation.Nullable;
|
|||
import java.io.IOException;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
@ -488,20 +488,10 @@ public abstract class SQLMetadataStorageActionHandler<EntryType, StatusType, Log
|
|||
.bind("entryId", entryId)
|
||||
.map(ByteArrayMapper.FIRST)
|
||||
.fold(
|
||||
Lists.newLinkedList(),
|
||||
new Folder3<List<LogType>, byte[]>()
|
||||
{
|
||||
@Override
|
||||
public List<LogType> fold(
|
||||
List<LogType> list, byte[] bytes, FoldController control, StatementContext ctx
|
||||
) throws SQLException
|
||||
{
|
||||
new ArrayList<>(),
|
||||
(List<LogType> list, byte[] bytes, FoldController control, StatementContext ctx) -> {
|
||||
try {
|
||||
list.add(
|
||||
jsonMapper.readValue(
|
||||
bytes, logType
|
||||
)
|
||||
);
|
||||
list.add(jsonMapper.readValue(bytes, logType));
|
||||
return list;
|
||||
}
|
||||
catch (IOException e) {
|
||||
|
@ -512,7 +502,6 @@ public abstract class SQLMetadataStorageActionHandler<EntryType, StatusType, Log
|
|||
throw new SQLException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -624,7 +624,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
|
|||
synchronized (lock) {
|
||||
try {
|
||||
if (!(finished && queue.isEmpty())) {
|
||||
final List<DataSegment> segments = Lists.newLinkedList();
|
||||
final List<DataSegment> segments = new ArrayList<>();
|
||||
queue.drainTo(segments);
|
||||
try {
|
||||
announcer.announceSegments(segments);
|
||||
|
@ -656,7 +656,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler
|
|||
finished = true;
|
||||
// announce any remaining segments
|
||||
try {
|
||||
final List<DataSegment> segments = Lists.newLinkedList();
|
||||
final List<DataSegment> segments = new ArrayList<>();
|
||||
queue.drainTo(segments);
|
||||
announcer.announceSegments(segments);
|
||||
}
|
||||
|
|
|
@ -19,12 +19,12 @@
|
|||
|
||||
package org.apache.druid.client.cache;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -70,7 +70,7 @@ public class ByteCountingLRUMapTest
|
|||
Assert.assertEquals(oneByte, ByteBuffer.wrap(map.get(twoByte)));
|
||||
|
||||
Iterator<ByteBuffer> it = map.keySet().iterator();
|
||||
List<ByteBuffer> toRemove = Lists.newLinkedList();
|
||||
List<ByteBuffer> toRemove = new ArrayList<>();
|
||||
while (it.hasNext()) {
|
||||
ByteBuffer buf = it.next();
|
||||
if (buf.remaining() == 10) {
|
||||
|
|
|
@ -26,7 +26,8 @@ import org.junit.Test;
|
|||
import org.skife.jdbi.v2.Handle;
|
||||
import org.skife.jdbi.v2.tweak.HandleCallback;
|
||||
|
||||
import java.util.LinkedList;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
|
||||
public class SQLMetadataConnectorTest
|
||||
|
@ -47,7 +48,7 @@ public class SQLMetadataConnectorTest
|
|||
@Test
|
||||
public void testCreateTables()
|
||||
{
|
||||
final LinkedList<String> tables = new LinkedList<String>();
|
||||
final List<String> tables = new ArrayList<>();
|
||||
final String entryType = tablesConfig.getTaskEntryType();
|
||||
tables.add(tablesConfig.getConfigTable());
|
||||
tables.add(tablesConfig.getSegmentsTable());
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
|
@ -274,7 +273,7 @@ public class SegmentLoadDropHandlerTest
|
|||
@Test
|
||||
public void testLoadCache() throws Exception
|
||||
{
|
||||
List<DataSegment> segments = Lists.newLinkedList();
|
||||
List<DataSegment> segments = new ArrayList<>();
|
||||
for (int i = 0; i < COUNT; ++i) {
|
||||
segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-01")));
|
||||
segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-02")));
|
||||
|
@ -406,7 +405,7 @@ public class SegmentLoadDropHandlerTest
|
|||
announcer, EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), segmentManager
|
||||
);
|
||||
|
||||
List<DataSegment> segments = Lists.newLinkedList();
|
||||
List<DataSegment> segments = new ArrayList<>();
|
||||
for (int i = 0; i < COUNT; ++i) {
|
||||
segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-01")));
|
||||
segments.add(makeSegment("test" + i, "1", Intervals.of("P1d/2011-04-02")));
|
||||
|
|
Loading…
Reference in New Issue