Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 2 additions & 2 deletions api/src/main/java/org/apache/druid/guice/LifecycleScope.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;

/**
Expand All @@ -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)
{
Expand Down
6 changes: 5 additions & 1 deletion codestyle/druid-forbidden-apis.txt
Original file line number Diff line number Diff line change
Expand Up @@ -4,4 +4,8 @@ com.google.common.util.concurrent.Futures#transform(com.google.common.util.concu
com.google.common.collect.Iterators#emptyIterator() @ Use java.util.Collections#emptyIterator()
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
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
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -239,7 +239,7 @@ public void testFoldSpeed()
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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,11 +99,11 @@
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;
Expand Down Expand Up @@ -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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,14 +23,14 @@
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;

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;

Expand All @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
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;
Expand All @@ -49,12 +50,12 @@
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;
Expand Down Expand Up @@ -133,7 +134,6 @@
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;
Expand All @@ -149,10 +149,11 @@
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;
Expand All @@ -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)
Expand All @@ -189,7 +190,7 @@ public TestFirehose(final InputRowParser<Map<String, Object>> parser)
public void addRows(List<Map<String, Object>> rows)
{
synchronized (this) {
queue.addAll(rows);
rows.stream().map(Optional::ofNullable).forEach(queue::add);
notifyAll();
}
}
Expand All @@ -215,7 +216,7 @@ public boolean hasMore()
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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,9 +46,9 @@
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;
Expand Down Expand Up @@ -138,10 +138,12 @@
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;
Expand All @@ -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)
Expand All @@ -172,7 +174,7 @@ public TestFirehose(final InputRowParser<Map<String, Object>> parser)
public void addRows(List<Map<String, Object>> rows)
{
synchronized (this) {
queue.addAll(rows);
rows.stream().map(Optional::ofNullable).forEach(queue::add);
notifyAll();
}
}
Expand All @@ -198,7 +200,7 @@ public boolean hasMore()
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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,6 @@
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;
Expand Down Expand Up @@ -324,7 +323,7 @@ public DataSegment restore(DataSegment segment)
null,
new NoopTestTaskFileWriter()
);
Collection<Object[]> values = new LinkedList<>();
Collection<Object[]> values = new ArrayList<>();
for (InputRowParser parser : Arrays.<InputRowParser>asList(
ROW_PARSER,
new MapInputRowParser(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,15 +23,16 @@

import java.io.IOException;
import java.io.InputStream;
import java.util.LinkedList;
import java.util.ArrayDeque;
import java.util.Deque;

/**
*/
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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,8 +37,9 @@
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;

Expand Down Expand Up @@ -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;
}

Expand All @@ -135,10 +138,7 @@ public static Pair<List<AggregatorFactory>, List<PostAggregator>> condensedAggre
)
{

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);
Expand All @@ -152,7 +152,7 @@ public static Pair<List<AggregatorFactory>, List<PostAggregator>> condensedAggre
condensedAggs.add(aggregatorSpec);
}
}
return new Pair(condensedAggs, condensedPostAggs);
return new Pair<>(condensedAggs, condensedPostAggs);
}

public static BaseFloatColumnValueSelector makeColumnValueSelectorWithFloatDefault(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,11 +75,8 @@ public void run(
)
{
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?");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,9 @@
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;

Expand Down Expand Up @@ -247,13 +249,13 @@ public TopNQueryBuilder granularity(Granularity g)
@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;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -172,11 +172,12 @@ public void testNullPostAggregatorNames()
);

Assert.assertEquals(
new Pair(Lists.newArrayList(agg1, agg2), Collections.singletonList(postAgg2)), AggregatorUtil.condensedAggregators(
Lists.newArrayList(agg1, agg2),
Lists.newArrayList(postAgg1, postAgg2),
"postAgg"
)
new Pair<>(Lists.newArrayList(agg1, agg2), Collections.singletonList(postAgg2)),
AggregatorUtil.condensedAggregators(
Lists.newArrayList(agg1, agg2),
Lists.newArrayList(postAgg1, postAgg2),
"postAgg"
)
);

}
Expand Down
Loading