mirror of https://github.com/apache/druid.git
Prohibit some guava collection APIs and use JDK collection APIs directly (#6511)
* Prohibit some guava collection APIs and use JDK APIs directly * reset files that changed by accident * sort codestyle/druid-forbidden-apis.txt alphabetically
This commit is contained in:
parent
0a90b3d51a
commit
676f5e6d7f
|
@ -21,7 +21,6 @@ package org.apache.druid.benchmark;
|
|||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.segment.data.ColumnarInts;
|
||||
import org.apache.druid.segment.data.ColumnarMultiInts;
|
||||
|
@ -45,6 +44,7 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.channels.WritableByteChannel;
|
||||
import java.util.ArrayList;
|
||||
import java.util.BitSet;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
@ -73,7 +73,7 @@ public class CompressedVSizeColumnarMultiIntsBenchmark
|
|||
public void setup() throws IOException
|
||||
{
|
||||
Random rand = ThreadLocalRandom.current();
|
||||
List<int[]> rows = Lists.newArrayList();
|
||||
List<int[]> rows = new ArrayList<>();
|
||||
final int bound = 1 << bytes;
|
||||
for (int i = 0; i < 0x100000; i++) {
|
||||
int count = rand.nextInt(valuesPerRowBound) + 1;
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.benchmark;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
|
@ -91,6 +90,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -243,7 +243,7 @@ public class FilteredAggregatorBenchmark
|
|||
toolChest
|
||||
);
|
||||
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
|
||||
return queryResult.toList();
|
||||
}
|
||||
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
|||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
|
@ -94,6 +93,7 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -451,7 +451,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
toolChest
|
||||
);
|
||||
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
|
||||
return queryResult.toList();
|
||||
}
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.benchmark;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.primitives.Ints;
|
||||
import org.apache.druid.java.util.common.guava.MergeSequence;
|
||||
|
@ -64,7 +63,7 @@ public class MergeSequenceBenchmark
|
|||
public void setup()
|
||||
{
|
||||
Random rand = ThreadLocalRandom.current();
|
||||
sequences = Lists.newArrayList();
|
||||
sequences = new ArrayList<>();
|
||||
for (int i = 0; i < count; i++) {
|
||||
int[] sequence = new int[sequenceLength];
|
||||
for (int j = 0; j < sequenceLength; j++) {
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.benchmark;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo;
|
||||
|
@ -86,6 +85,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;
|
||||
|
@ -319,7 +319,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
toolChest
|
||||
);
|
||||
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
|
||||
return queryResult.toList();
|
||||
}
|
||||
|
||||
|
|
|
@ -26,8 +26,6 @@ import com.google.common.base.Suppliers;
|
|||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
|
@ -103,6 +101,7 @@ import java.io.IOException;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -547,7 +546,7 @@ public class GroupByBenchmark
|
|||
toolChest
|
||||
);
|
||||
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
|
||||
return queryResult.toList();
|
||||
}
|
||||
|
||||
|
@ -600,7 +599,7 @@ public class GroupByBenchmark
|
|||
(QueryToolChest) toolChest
|
||||
);
|
||||
|
||||
Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
|
||||
Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
|
||||
List<Row> results = queryResult.toList();
|
||||
|
||||
for (Row result : results) {
|
||||
|
@ -624,7 +623,7 @@ public class GroupByBenchmark
|
|||
final GroupByQuery spillingQuery = query.withOverriddenContext(
|
||||
ImmutableMap.of("bufferGrouperMaxSize", 4000)
|
||||
);
|
||||
Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(spillingQuery), Maps.newHashMap());
|
||||
Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(spillingQuery), new HashMap<>());
|
||||
List<Row> results = queryResult.toList();
|
||||
|
||||
for (Row result : results) {
|
||||
|
@ -651,7 +650,7 @@ public class GroupByBenchmark
|
|||
(QueryToolChest) toolChest
|
||||
);
|
||||
|
||||
Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
|
||||
Sequence<Row> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
|
||||
List<Row> results = queryResult.toList();
|
||||
|
||||
for (Row result : results) {
|
||||
|
@ -661,7 +660,7 @@ public class GroupByBenchmark
|
|||
|
||||
private List<QueryRunner<Row>> makeMultiRunners()
|
||||
{
|
||||
List<QueryRunner<Row>> runners = Lists.newArrayList();
|
||||
List<QueryRunner<Row>> runners = new ArrayList<>();
|
||||
for (int i = 0; i < numSegments; i++) {
|
||||
String segmentName = "qIndex" + i;
|
||||
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
|
@ -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;
|
||||
|
@ -199,21 +199,21 @@ public class SearchBenchmark
|
|||
{
|
||||
final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
|
||||
|
||||
final List<String> dimUniformFilterVals = Lists.newArrayList();
|
||||
final List<String> dimUniformFilterVals = new ArrayList<>();
|
||||
int resultNum = (int) (100000 * 0.1);
|
||||
int step = 100000 / resultNum;
|
||||
for (int i = 1; i < 100001 && dimUniformFilterVals.size() < resultNum; i += step) {
|
||||
dimUniformFilterVals.add(String.valueOf(i));
|
||||
}
|
||||
|
||||
List<String> dimHyperUniqueFilterVals = Lists.newArrayList();
|
||||
List<String> dimHyperUniqueFilterVals = new ArrayList<>();
|
||||
resultNum = (int) (100000 * 0.1);
|
||||
step = 100000 / resultNum;
|
||||
for (int i = 0; i < 100001 && dimHyperUniqueFilterVals.size() < resultNum; i += step) {
|
||||
dimHyperUniqueFilterVals.add(String.valueOf(i));
|
||||
}
|
||||
|
||||
final List<DimFilter> dimFilters = Lists.newArrayList();
|
||||
final List<DimFilter> dimFilters = new ArrayList<>();
|
||||
dimFilters.add(new InDimFilter("dimUniform", dimUniformFilterVals, null));
|
||||
dimFilters.add(new InDimFilter("dimHyperUnique", dimHyperUniqueFilterVals, null));
|
||||
|
||||
|
@ -230,7 +230,7 @@ public class SearchBenchmark
|
|||
{
|
||||
final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
|
||||
|
||||
final List<String> dimUniformFilterVals = Lists.newArrayList();
|
||||
final List<String> dimUniformFilterVals = new ArrayList<>();
|
||||
final int resultNum = (int) (100000 * 0.1);
|
||||
final int step = 100000 / resultNum;
|
||||
for (int i = 1; i < 100001 && dimUniformFilterVals.size() < resultNum; i += step) {
|
||||
|
@ -238,7 +238,7 @@ public class SearchBenchmark
|
|||
}
|
||||
|
||||
final String dimName = "dimUniform";
|
||||
final List<DimFilter> dimFilters = Lists.newArrayList();
|
||||
final List<DimFilter> dimFilters = new ArrayList<>();
|
||||
dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, IdentityExtractionFn.getInstance()));
|
||||
dimFilters.add(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance()));
|
||||
dimFilters.add(new BoundDimFilter(dimName, "100", "10000", true, true, true, new DimExtractionFn()
|
||||
|
@ -284,7 +284,7 @@ public class SearchBenchmark
|
|||
{
|
||||
final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
|
||||
|
||||
final List<String> dimUniformFilterVals = Lists.newArrayList();
|
||||
final List<String> dimUniformFilterVals = new ArrayList<>();
|
||||
final int resultNum = (int) (100000 * 0.1);
|
||||
final int step = 100000 / resultNum;
|
||||
for (int i = 1; i < 100001 && dimUniformFilterVals.size() < resultNum; i += step) {
|
||||
|
@ -292,7 +292,7 @@ public class SearchBenchmark
|
|||
}
|
||||
|
||||
final String dimName = "dimUniform";
|
||||
final List<DimFilter> dimFilters = Lists.newArrayList();
|
||||
final List<DimFilter> dimFilters = new ArrayList<>();
|
||||
dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, null));
|
||||
dimFilters.add(new SelectorDimFilter(dimName, "3", null));
|
||||
dimFilters.add(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null));
|
||||
|
@ -402,7 +402,7 @@ public class SearchBenchmark
|
|||
toolChest
|
||||
);
|
||||
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
|
||||
return queryResult.toList();
|
||||
}
|
||||
|
||||
|
@ -448,7 +448,7 @@ public class SearchBenchmark
|
|||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
List<QueryRunner<Row>> singleSegmentRunners = Lists.newArrayList();
|
||||
List<QueryRunner<Row>> singleSegmentRunners = new ArrayList<>();
|
||||
QueryToolChest toolChest = factory.getToolchest();
|
||||
for (int i = 0; i < numSegments; i++) {
|
||||
String segmentName = "qIndex" + i;
|
||||
|
@ -469,7 +469,7 @@ public class SearchBenchmark
|
|||
|
||||
Sequence<Result<SearchResultValue>> queryResult = theRunner.run(
|
||||
QueryPlus.wrap(query),
|
||||
Maps.newHashMap()
|
||||
new HashMap<>()
|
||||
);
|
||||
List<Result<SearchResultValue>> results = queryResult.toList();
|
||||
|
||||
|
|
|
@ -22,8 +22,6 @@ package org.apache.druid.benchmark.query;
|
|||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
|
@ -266,7 +264,7 @@ public class SelectBenchmark
|
|||
toolChest
|
||||
);
|
||||
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
|
||||
return queryResult.toList();
|
||||
}
|
||||
|
||||
|
@ -353,7 +351,7 @@ public class SelectBenchmark
|
|||
SelectQuery queryCopy = query.withPagingSpec(PagingSpec.newSpec(pagingThreshold));
|
||||
|
||||
String segmentName;
|
||||
List<QueryRunner<Result<SelectResultValue>>> singleSegmentRunners = Lists.newArrayList();
|
||||
List<QueryRunner<Result<SelectResultValue>>> singleSegmentRunners = new ArrayList<>();
|
||||
QueryToolChest toolChest = factory.getToolchest();
|
||||
for (int i = 0; i < numSegments; i++) {
|
||||
segmentName = "qIndex" + i;
|
||||
|
@ -375,7 +373,7 @@ public class SelectBenchmark
|
|||
|
||||
boolean done = false;
|
||||
while (!done) {
|
||||
Sequence<Result<SelectResultValue>> queryResult = theRunner.run(QueryPlus.wrap(queryCopy), Maps.newHashMap());
|
||||
Sequence<Result<SelectResultValue>> queryResult = theRunner.run(QueryPlus.wrap(queryCopy), new HashMap<>());
|
||||
List<Result<SelectResultValue>> results = queryResult.toList();
|
||||
|
||||
SelectResultValue result = results.get(0).getValue();
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.benchmark.query;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo;
|
||||
|
@ -66,6 +65,7 @@ import org.openjdk.jmh.annotations.Warmup;
|
|||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -170,7 +170,7 @@ public class SqlBenchmark
|
|||
@OutputTimeUnit(TimeUnit.MILLISECONDS)
|
||||
public void queryNative(Blackhole blackhole)
|
||||
{
|
||||
final Sequence<Row> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, Maps.newHashMap());
|
||||
final Sequence<Row> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, new HashMap<>());
|
||||
final List<Row> resultList = resultSequence.toList();
|
||||
|
||||
for (Row row : resultList) {
|
||||
|
|
|
@ -20,8 +20,6 @@
|
|||
package org.apache.druid.benchmark.query;
|
||||
|
||||
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 org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
|
@ -91,6 +89,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;
|
||||
|
@ -326,7 +325,7 @@ public class TimeseriesBenchmark
|
|||
toolChest
|
||||
);
|
||||
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
|
||||
return queryResult.toList();
|
||||
}
|
||||
|
||||
|
@ -389,7 +388,7 @@ public class TimeseriesBenchmark
|
|||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
List<QueryRunner<Result<TimeseriesResultValue>>> singleSegmentRunners = Lists.newArrayList();
|
||||
List<QueryRunner<Result<TimeseriesResultValue>>> singleSegmentRunners = new ArrayList<>();
|
||||
QueryToolChest toolChest = factory.getToolchest();
|
||||
for (int i = 0; i < numSegments; i++) {
|
||||
String segmentName = "qIndex" + i;
|
||||
|
@ -410,7 +409,7 @@ public class TimeseriesBenchmark
|
|||
|
||||
Sequence<Result<TimeseriesResultValue>> queryResult = theRunner.run(
|
||||
QueryPlus.wrap(query),
|
||||
Maps.newHashMap()
|
||||
new HashMap<>()
|
||||
);
|
||||
List<Result<TimeseriesResultValue>> results = queryResult.toList();
|
||||
|
||||
|
|
|
@ -20,8 +20,6 @@
|
|||
package org.apache.druid.benchmark.query;
|
||||
|
||||
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 org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
|
@ -88,6 +86,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;
|
||||
|
@ -307,7 +306,7 @@ public class TopNBenchmark
|
|||
toolChest
|
||||
);
|
||||
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), Maps.newHashMap());
|
||||
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
|
||||
return queryResult.toList();
|
||||
}
|
||||
|
||||
|
@ -350,7 +349,7 @@ public class TopNBenchmark
|
|||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void queryMultiQueryableIndex(Blackhole blackhole)
|
||||
{
|
||||
List<QueryRunner<Result<TopNResultValue>>> singleSegmentRunners = Lists.newArrayList();
|
||||
List<QueryRunner<Result<TopNResultValue>>> singleSegmentRunners = new ArrayList<>();
|
||||
QueryToolChest toolChest = factory.getToolchest();
|
||||
for (int i = 0; i < numSegments; i++) {
|
||||
String segmentName = "qIndex" + i;
|
||||
|
@ -371,7 +370,7 @@ public class TopNBenchmark
|
|||
|
||||
Sequence<Result<TopNResultValue>> queryResult = theRunner.run(
|
||||
QueryPlus.wrap(query),
|
||||
Maps.newHashMap()
|
||||
new HashMap<>()
|
||||
);
|
||||
List<Result<TopNResultValue>> results = queryResult.toList();
|
||||
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.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.io.Files;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
|
@ -353,7 +352,7 @@ public class TimeCompareBenchmark
|
|||
qIndexes.add(qIndex);
|
||||
}
|
||||
|
||||
List<QueryRunner<Result<TopNResultValue>>> singleSegmentRunners = Lists.newArrayList();
|
||||
List<QueryRunner<Result<TopNResultValue>>> singleSegmentRunners = new ArrayList<>();
|
||||
QueryToolChest toolChest = topNFactory.getToolchest();
|
||||
for (int i = 0; i < numSegments; i++) {
|
||||
String segmentName = "qIndex" + i;
|
||||
|
@ -379,7 +378,7 @@ public class TimeCompareBenchmark
|
|||
)
|
||||
);
|
||||
|
||||
List<QueryRunner<Result<TimeseriesResultValue>>> singleSegmentRunnersT = Lists.newArrayList();
|
||||
List<QueryRunner<Result<TimeseriesResultValue>>> singleSegmentRunnersT = new ArrayList<>();
|
||||
QueryToolChest toolChestT = timeseriesFactory.getToolchest();
|
||||
for (int i = 0; i < numSegments; i++) {
|
||||
String segmentName = "qIndex" + i;
|
||||
|
|
|
@ -1,14 +1,27 @@
|
|||
com.google.common.collect.MapMaker @ Create java.util.concurrent.ConcurrentHashMap directly
|
||||
com.google.common.collect.Maps#newConcurrentMap() @ Create java.util.concurrent.ConcurrentHashMap directly
|
||||
com.google.common.util.concurrent.Futures#transform(com.google.common.util.concurrent.ListenableFuture, com.google.common.util.concurrent.AsyncFunction) @ Use org.apache.druid.java.util.common.concurrent.ListenableFutures#transformAsync
|
||||
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
|
||||
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
|
||||
java.util.Random#<init>() @ Use ThreadLocalRandom.current() or the constructor with a seed (the latter in tests only!)
|
||||
com.fasterxml.jackson.databind.ObjectMapper#reader(com.fasterxml.jackson.core.type.TypeReference) @ Use ObjectMapper#readerFor instead
|
||||
com.fasterxml.jackson.databind.ObjectMapper#reader(com.fasterxml.jackson.databind.JavaType) @ Use ObjectMapper#readerFor instead
|
||||
com.fasterxml.jackson.databind.ObjectMapper#reader(java.lang.Class) @ Use ObjectMapper#readerFor instead
|
||||
com.fasterxml.jackson.databind.ObjectMapper#reader(com.fasterxml.jackson.core.type.TypeReference) @ Use ObjectMapper#readerFor instead
|
||||
com.google.common.base.Charsets @ Use java.nio.charset.StandardCharsets instead
|
||||
com.google.common.collect.Iterators#emptyIterator() @ Use java.util.Collections#emptyIterator()
|
||||
com.google.common.collect.Lists#newArrayList() @ Create java.util.ArrayList directly
|
||||
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
|
||||
com.google.common.collect.MapMaker @ Create java.util.concurrent.ConcurrentHashMap directly
|
||||
com.google.common.collect.Maps#newConcurrentMap() @ Create java.util.concurrent.ConcurrentHashMap directly
|
||||
com.google.common.collect.Maps#newHashMap() @ Create java.util.HashMap directly
|
||||
com.google.common.collect.Maps#newHashMap(java.util.Map) @ Create java.util.HashMap directly
|
||||
com.google.common.collect.Maps#newTreeMap() @ Create java.util.TreeMap directly
|
||||
com.google.common.collect.Maps#newTreeMap(java.util.Comparator) @ Create java.util.TreeMap directly
|
||||
com.google.common.collect.Maps#newTreeMap(java.util.SortedMap) @ Create java.util.TreeMap directly
|
||||
com.google.common.collect.Queues#newArrayDeque() @ Create java.util.ArrayDeque directly
|
||||
com.google.common.collect.Queues#newConcurrentLinkedQueue() @ Create java.util.concurrent.ConcurrentLinkedQueue directly
|
||||
com.google.common.collect.Queues#newLinkedBlockingQueue() @ Create java.util.concurrent.LinkedBlockingQueue directly
|
||||
com.google.common.collect.Sets#newHashSet() @ Create java.util.HashSet directly
|
||||
com.google.common.collect.Sets#newLinkedHashSet() @ Create java.util.LinkedHashSet directly
|
||||
com.google.common.collect.Sets#newTreeSet() @ Create java.util.TreeSet directly
|
||||
com.google.common.collect.Sets#newTreeSet(java.util.Comparator) @ Create java.util.TreeSet directly
|
||||
com.google.common.util.concurrent.Futures#transform(com.google.common.util.concurrent.ListenableFuture, com.google.common.util.concurrent.AsyncFunction) @ Use org.apache.druid.java.util.common.concurrent.ListenableFutures#transformAsync
|
||||
java.io.File#toURL() @ Use java.io.File#toURI() and java.net.URI#toURL() instead
|
||||
java.util.LinkedList @ Use ArrayList or ArrayDeque instead
|
||||
java.util.Random#<init>() @ Use ThreadLocalRandom.current() or the constructor with a seed (the latter in tests only!)
|
||||
org.apache.commons.io.FileUtils#getTempDirectory() @ Use org.junit.rules.TemporaryFolder for tests instead
|
|
@ -21,7 +21,6 @@ package org.apache.druid.data.input;
|
|||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSortedSet;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Longs;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -32,6 +31,7 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
|
@ -47,7 +47,7 @@ public class Rows
|
|||
*/
|
||||
public static List<Object> toGroupKey(long timeStamp, InputRow inputRow)
|
||||
{
|
||||
final Map<String, Set<String>> dims = Maps.newTreeMap();
|
||||
final Map<String, Set<String>> dims = new TreeMap<>();
|
||||
for (final String dim : inputRow.getDimensions()) {
|
||||
final Set<String> dimValues = ImmutableSortedSet.copyOf(inputRow.getDimension(dim));
|
||||
if (dimValues.size() > 0) {
|
||||
|
|
|
@ -32,7 +32,9 @@ import org.apache.druid.guice.annotations.PublicApi;
|
|||
import org.apache.druid.java.util.common.parsers.ParserUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -75,15 +77,15 @@ public class DimensionsSpec
|
|||
)
|
||||
{
|
||||
this.dimensions = dimensions == null
|
||||
? Lists.newArrayList()
|
||||
? new ArrayList<>()
|
||||
: Lists.newArrayList(dimensions);
|
||||
|
||||
this.dimensionExclusions = (dimensionExclusions == null)
|
||||
? Sets.newHashSet()
|
||||
? new HashSet<>()
|
||||
: Sets.newHashSet(dimensionExclusions);
|
||||
|
||||
List<SpatialDimensionSchema> spatialDims = (spatialDimensions == null)
|
||||
? Lists.newArrayList()
|
||||
? new ArrayList<>()
|
||||
: spatialDimensions;
|
||||
|
||||
verify(spatialDims);
|
||||
|
|
|
@ -29,8 +29,6 @@ import com.google.common.base.Function;
|
|||
import com.google.common.base.Strings;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
import com.google.inject.ProvisionException;
|
||||
import com.google.inject.spi.Message;
|
||||
|
@ -43,6 +41,7 @@ import javax.validation.Path;
|
|||
import javax.validation.Validator;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -76,7 +75,7 @@ public class JsonConfigurator
|
|||
// Make it end with a period so we only include properties with sub-object thingies.
|
||||
final String propertyBase = propertyPrefix.endsWith(".") ? propertyPrefix : propertyPrefix + ".";
|
||||
|
||||
Map<String, Object> jsonMap = Maps.newHashMap();
|
||||
Map<String, Object> jsonMap = new HashMap<>();
|
||||
for (String prop : props.stringPropertyNames()) {
|
||||
if (prop.startsWith(propertyBase)) {
|
||||
final String propValue = props.getProperty(prop);
|
||||
|
@ -109,7 +108,7 @@ public class JsonConfigurator
|
|||
|
||||
final Set<ConstraintViolation<T>> violations = validator.validate(config);
|
||||
if (!violations.isEmpty()) {
|
||||
List<String> messages = Lists.newArrayList();
|
||||
List<String> messages = new ArrayList<>();
|
||||
|
||||
for (ConstraintViolation<T> violation : violations) {
|
||||
StringBuilder path = new StringBuilder();
|
||||
|
|
|
@ -25,13 +25,14 @@ import com.google.inject.Injector;
|
|||
import com.google.inject.Key;
|
||||
import com.google.inject.Provider;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ListProvider<T> implements Provider<List<T>>
|
||||
{
|
||||
private final List<Key<? extends T>> itemsToLoad = Lists.newArrayList();
|
||||
private final List<Key<? extends T>> itemsToLoad = new ArrayList<>();
|
||||
private Injector injector;
|
||||
|
||||
public ListProvider<T> add(Class<? extends T> clazz)
|
||||
|
|
|
@ -19,8 +19,7 @@
|
|||
|
||||
package org.apache.druid.indexer;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class TaskMetricsUtils
|
||||
|
@ -37,7 +36,7 @@ public class TaskMetricsUtils
|
|||
long rowsThrownAway
|
||||
)
|
||||
{
|
||||
Map<String, Object> metricsMap = Maps.newHashMap();
|
||||
Map<String, Object> metricsMap = new HashMap<>();
|
||||
metricsMap.put(ROWS_PROCESSED, rowsProcessed);
|
||||
metricsMap.put(ROWS_PROCESSED_WITH_ERRORS, rowsProcessedWithErrors);
|
||||
metricsMap.put(ROWS_UNPARSEABLE, rowsUnparseable);
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.java.util.common;
|
|||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.io.ByteSource;
|
||||
import com.google.common.io.Files;
|
||||
|
||||
|
@ -34,6 +33,7 @@ import java.io.OutputStream;
|
|||
import java.nio.MappedByteBuffer;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.nio.file.StandardCopyOption;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.UUID;
|
||||
|
@ -91,7 +91,7 @@ public class FileUtils
|
|||
*/
|
||||
public static class FileCopyResult
|
||||
{
|
||||
private final Collection<File> files = Lists.newArrayList();
|
||||
private final Collection<File> files = new ArrayList<>();
|
||||
private long size = 0L;
|
||||
|
||||
public Collection<File> getFiles()
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package org.apache.druid.java.util.common;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.druid.java.util.common.guava.Comparators;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
@ -28,6 +26,7 @@ import org.joda.time.Interval;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -39,21 +38,21 @@ public class JodaUtils
|
|||
|
||||
public static ArrayList<Interval> condenseIntervals(Iterable<Interval> intervals)
|
||||
{
|
||||
ArrayList<Interval> retVal = Lists.newArrayList();
|
||||
ArrayList<Interval> retVal = new ArrayList<>();
|
||||
|
||||
final SortedSet<Interval> sortedIntervals;
|
||||
|
||||
if (intervals instanceof SortedSet) {
|
||||
sortedIntervals = (SortedSet<Interval>) intervals;
|
||||
} else {
|
||||
sortedIntervals = Sets.newTreeSet(Comparators.intervalsByStartThenEnd());
|
||||
sortedIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd());
|
||||
for (Interval interval : intervals) {
|
||||
sortedIntervals.add(interval);
|
||||
}
|
||||
}
|
||||
|
||||
if (sortedIntervals.isEmpty()) {
|
||||
return Lists.newArrayList();
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
Iterator<Interval> intervalsIter = sortedIntervals.iterator();
|
||||
|
@ -82,8 +81,8 @@ public class JodaUtils
|
|||
|
||||
public static Interval umbrellaInterval(Iterable<Interval> intervals)
|
||||
{
|
||||
ArrayList<DateTime> startDates = Lists.newArrayList();
|
||||
ArrayList<DateTime> endDates = Lists.newArrayList();
|
||||
ArrayList<DateTime> startDates = new ArrayList<>();
|
||||
ArrayList<DateTime> endDates = new ArrayList<>();
|
||||
|
||||
for (Interval interval : intervals) {
|
||||
startDates.add(interval.getStart());
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.java.util.common.granularity;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.primitives.Longs;
|
||||
import org.apache.druid.java.util.common.Cacheable;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
|
@ -31,6 +30,7 @@ import org.joda.time.DateTimeZone;
|
|||
import org.joda.time.Interval;
|
||||
import org.joda.time.format.DateTimeFormatter;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.NoSuchElementException;
|
||||
|
@ -78,7 +78,7 @@ public abstract class Granularity implements Cacheable
|
|||
|
||||
public static List<Granularity> granularitiesFinerThan(final Granularity gran0)
|
||||
{
|
||||
final List<Granularity> retVal = Lists.newArrayList();
|
||||
final List<Granularity> retVal = new ArrayList<>();
|
||||
final DateTime origin = (gran0 instanceof PeriodGranularity) ? ((PeriodGranularity) gran0).getOrigin() : null;
|
||||
final DateTimeZone tz = (gran0 instanceof PeriodGranularity) ? ((PeriodGranularity) gran0).getTimeZone() : null;
|
||||
for (GranularityType gran : GranularityType.values()) {
|
||||
|
|
|
@ -19,8 +19,7 @@
|
|||
|
||||
package org.apache.druid.java.util.common.guava;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -44,7 +43,7 @@ public class MergeIterable<T> implements Iterable<T>
|
|||
@Override
|
||||
public Iterator<T> iterator()
|
||||
{
|
||||
List<Iterator<T>> iterators = Lists.newArrayList();
|
||||
List<Iterator<T>> iterators = new ArrayList<>();
|
||||
for (Iterable<T> baseIterable : baseIterables) {
|
||||
iterators.add(baseIterable.iterator());
|
||||
}
|
||||
|
|
|
@ -21,8 +21,6 @@ package org.apache.druid.java.util.common.io.smoosh;
|
|||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Ints;
|
||||
import org.apache.druid.java.util.common.FileUtils;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
|
@ -50,6 +48,7 @@ import java.util.ArrayList;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
/**
|
||||
* A class that concatenates files together into configurable sized chunks,
|
||||
|
@ -77,12 +76,12 @@ public class FileSmoosher implements Closeable
|
|||
private final File baseDir;
|
||||
private final int maxChunkSize;
|
||||
|
||||
private final List<File> outFiles = Lists.newArrayList();
|
||||
private final Map<String, Metadata> internalFiles = Maps.newTreeMap();
|
||||
private final List<File> outFiles = new ArrayList<>();
|
||||
private final Map<String, Metadata> internalFiles = new TreeMap<>();
|
||||
// list of files completed writing content using delegated smooshedWriter.
|
||||
private List<File> completedFiles = Lists.newArrayList();
|
||||
private List<File> completedFiles = new ArrayList<>();
|
||||
// list of files in process writing content using delegated smooshedWriter.
|
||||
private List<File> filesInProcess = Lists.newArrayList();
|
||||
private List<File> filesInProcess = new ArrayList<>();
|
||||
|
||||
private Outer currOut = null;
|
||||
private boolean writerCurrentlyInUse = false;
|
||||
|
@ -250,7 +249,7 @@ public class FileSmoosher implements Closeable
|
|||
{
|
||||
// Get processed elements from the stack and write.
|
||||
List<File> fileToProcess = new ArrayList<>(completedFiles);
|
||||
completedFiles = Lists.newArrayList();
|
||||
completedFiles = new ArrayList<>();
|
||||
for (File file : fileToProcess) {
|
||||
add(file);
|
||||
if (!file.delete()) {
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.java.util.common.io.smoosh;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Closeables;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.druid.java.util.common.ByteBufferUtils;
|
||||
|
@ -35,9 +34,11 @@ import java.io.InputStreamReader;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.nio.MappedByteBuffer;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
||||
/**
|
||||
* Class that works in conjunction with FileSmoosher. This class knows how to map in a set of files smooshed
|
||||
|
@ -72,7 +73,7 @@ public class SmooshedFileMapper implements Closeable
|
|||
outFiles.add(FileSmoosher.makeChunkFile(baseDir, i));
|
||||
}
|
||||
|
||||
Map<String, Metadata> internalFiles = Maps.newTreeMap();
|
||||
Map<String, Metadata> internalFiles = new TreeMap<>();
|
||||
while ((line = in.readLine()) != null) {
|
||||
splits = line.split(",");
|
||||
|
||||
|
@ -94,7 +95,7 @@ public class SmooshedFileMapper implements Closeable
|
|||
|
||||
private final List<File> outFiles;
|
||||
private final Map<String, Metadata> internalFiles;
|
||||
private final List<MappedByteBuffer> buffersList = Lists.newArrayList();
|
||||
private final List<MappedByteBuffer> buffersList = new ArrayList<>();
|
||||
|
||||
private SmooshedFileMapper(
|
||||
List<File> outFiles,
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import java.nio.charset.CharsetEncoder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.List;
|
||||
|
@ -90,7 +91,7 @@ public class JSONToLowerParser implements Parser<String, Object>
|
|||
}
|
||||
this.exclude = exclude != null
|
||||
? Sets.newHashSet(Iterables.transform(exclude, StringUtils::toLowerCase))
|
||||
: Sets.newHashSet();
|
||||
: new HashSet<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.java.util.common.parsers;
|
|||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.druid.common.config.NullHandling;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
@ -29,6 +28,7 @@ import org.joda.time.DateTimeZone;
|
|||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TimeZone;
|
||||
|
@ -79,8 +79,8 @@ public class ParserUtils
|
|||
|
||||
public static Set<String> findDuplicates(Iterable<String> fieldNames)
|
||||
{
|
||||
Set<String> duplicates = Sets.newHashSet();
|
||||
Set<String> uniqueNames = Sets.newHashSet();
|
||||
Set<String> duplicates = new HashSet<>();
|
||||
Set<String> uniqueNames = new HashSet<>();
|
||||
|
||||
for (String fieldName : fieldNames) {
|
||||
String next = StringUtils.toLowerCase(fieldName);
|
||||
|
|
|
@ -27,6 +27,7 @@ import com.google.common.collect.Iterables;
|
|||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.collect.Utils;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Matcher;
|
||||
|
@ -93,7 +94,7 @@ public class RegexParser implements Parser<String, Object>
|
|||
throw new ParseException("Incorrect Regex: %s . No match found.", pattern);
|
||||
}
|
||||
|
||||
List<String> values = Lists.newArrayList();
|
||||
List<String> values = new ArrayList<>();
|
||||
for (int i = 1; i <= matcher.groupCount(); i++) {
|
||||
values.add(matcher.group(i));
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.java.util.emitter.core;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -51,7 +50,7 @@ public class Emitters
|
|||
Lifecycle lifecycle
|
||||
)
|
||||
{
|
||||
Map<String, Object> jsonified = Maps.newHashMap();
|
||||
Map<String, Object> jsonified = new HashMap<>();
|
||||
if (props.getProperty(LOG_EMITTER_PROP) != null) {
|
||||
jsonified = makeLoggingMap(props);
|
||||
jsonified.put("type", "logging");
|
||||
|
@ -75,7 +74,7 @@ public class Emitters
|
|||
|
||||
static Map<String, Object> makeHttpMap(Properties props)
|
||||
{
|
||||
Map<String, Object> httpMap = Maps.newHashMap();
|
||||
Map<String, Object> httpMap = new HashMap<>();
|
||||
|
||||
final String urlProperty = "org.apache.druid.java.util.emitter.http.url";
|
||||
|
||||
|
@ -133,7 +132,7 @@ public class Emitters
|
|||
// Package-visible for unit tests
|
||||
static Map<String, Object> makeLoggingMap(Properties props)
|
||||
{
|
||||
Map<String, Object> loggingMap = Maps.newHashMap();
|
||||
Map<String, Object> loggingMap = new HashMap<>();
|
||||
|
||||
loggingMap.put(
|
||||
"loggerClass", props.getProperty("org.apache.druid.java.util.emitter.logging.class", LoggingEmitter.class.getName())
|
||||
|
@ -146,7 +145,7 @@ public class Emitters
|
|||
|
||||
static Map<String, Object> makeCustomFactoryMap(Properties props)
|
||||
{
|
||||
Map<String, Object> factoryMap = Maps.newHashMap();
|
||||
Map<String, Object> factoryMap = new HashMap<>();
|
||||
String prefix = "org.apache.druid.java.util.emitter.";
|
||||
|
||||
for (Map.Entry<Object, Object> entry : props.entrySet()) {
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.joda.time.DateTime;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -130,7 +131,7 @@ public class ServiceMetricEvent implements Event
|
|||
|
||||
public static class Builder
|
||||
{
|
||||
private final Map<String, Object> userDims = Maps.newTreeMap();
|
||||
private final Map<String, Object> userDims = new TreeMap<>();
|
||||
private String feed = "metrics";
|
||||
|
||||
public Builder setFeed(String feed)
|
||||
|
|
|
@ -20,8 +20,6 @@
|
|||
package org.apache.druid.java.util.http.client;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Multimap;
|
||||
import com.google.common.collect.Multimaps;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -35,8 +33,10 @@ import org.jboss.netty.handler.codec.http.HttpMethod;
|
|||
import java.net.URL;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -49,13 +49,13 @@ public class Request
|
|||
private final HttpMethod method;
|
||||
private final URL url;
|
||||
private final Multimap<String, String> headers = Multimaps.newListMultimap(
|
||||
Maps.newHashMap(),
|
||||
new HashMap<>(),
|
||||
new Supplier<List<String>>()
|
||||
{
|
||||
@Override
|
||||
public List<String> get()
|
||||
{
|
||||
return Lists.newArrayList();
|
||||
return new ArrayList<>();
|
||||
}
|
||||
}
|
||||
);
|
||||
|
|
|
@ -24,8 +24,6 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.antlr.v4.runtime.ANTLRInputStream;
|
||||
import org.antlr.v4.runtime.CommonTokenStream;
|
||||
import org.antlr.v4.runtime.tree.ParseTree;
|
||||
|
@ -37,6 +35,8 @@ import org.apache.druid.math.expr.antlr.ExprParser;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.lang.reflect.Modifier;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -47,7 +47,7 @@ public class Parser
|
|||
private static final Map<String, Function> FUNCTIONS;
|
||||
|
||||
static {
|
||||
Map<String, Function> functionMap = Maps.newHashMap();
|
||||
Map<String, Function> functionMap = new HashMap<>();
|
||||
for (Class clazz : Function.class.getClasses()) {
|
||||
if (!Modifier.isAbstract(clazz.getModifiers()) && Function.class.isAssignableFrom(clazz)) {
|
||||
try {
|
||||
|
@ -132,7 +132,7 @@ public class Parser
|
|||
|
||||
public static List<String> findRequiredBindings(Expr expr)
|
||||
{
|
||||
final Set<String> found = Sets.newLinkedHashSet();
|
||||
final Set<String> found = new LinkedHashSet<>();
|
||||
expr.visit(
|
||||
new Expr.Visitor()
|
||||
{
|
||||
|
|
|
@ -21,9 +21,9 @@ package org.apache.druid.metadata;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
|
@ -39,9 +39,9 @@ public class MetadataStorageTablesConfig
|
|||
|
||||
private static final String DEFAULT_BASE = "druid";
|
||||
|
||||
private final Map<String, String> entryTables = Maps.newHashMap();
|
||||
private final Map<String, String> logTables = Maps.newHashMap();
|
||||
private final Map<String, String> lockTables = Maps.newHashMap();
|
||||
private final Map<String, String> entryTables = new HashMap<>();
|
||||
private final Map<String, String> logTables = new HashMap<>();
|
||||
private final Map<String, String> lockTables = new HashMap<>();
|
||||
|
||||
@JsonProperty("base")
|
||||
private final String base;
|
||||
|
|
|
@ -23,10 +23,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.java.util.common.guava.Comparators;
|
||||
|
@ -37,6 +34,8 @@ import org.joda.time.Interval;
|
|||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.IdentityHashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -75,7 +74,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
|
|||
final NavigableMap<Interval, TimelineEntry> incompletePartitionsTimeline = new TreeMap<Interval, TimelineEntry>(
|
||||
Comparators.intervalsByStartThenEnd()
|
||||
);
|
||||
private final Map<Interval, TreeMap<VersionType, TimelineEntry>> allTimelineEntries = Maps.newHashMap();
|
||||
private final Map<Interval, TreeMap<VersionType, TimelineEntry>> allTimelineEntries = new HashMap<>();
|
||||
|
||||
private final Comparator<? super VersionType> versionComparator;
|
||||
|
||||
|
@ -311,11 +310,11 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
|
|||
{
|
||||
try {
|
||||
lock.readLock().lock();
|
||||
Set<TimelineObjectHolder<VersionType, ObjectType>> retVal = Sets.newHashSet();
|
||||
Set<TimelineObjectHolder<VersionType, ObjectType>> retVal = new HashSet<>();
|
||||
|
||||
Map<Interval, Map<VersionType, TimelineEntry>> overShadowed = Maps.newHashMap();
|
||||
Map<Interval, Map<VersionType, TimelineEntry>> overShadowed = new HashMap<>();
|
||||
for (Map.Entry<Interval, TreeMap<VersionType, TimelineEntry>> versionEntry : allTimelineEntries.entrySet()) {
|
||||
Map<VersionType, TimelineEntry> versionCopy = Maps.newHashMap();
|
||||
Map<VersionType, TimelineEntry> versionCopy = new HashMap<>();
|
||||
versionCopy.putAll(versionEntry.getValue());
|
||||
overShadowed.put(versionEntry.getKey(), versionCopy);
|
||||
}
|
||||
|
@ -529,7 +528,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
|
|||
boolean incompleteOk
|
||||
)
|
||||
{
|
||||
List<Interval> intervalsToRemove = Lists.newArrayList();
|
||||
List<Interval> intervalsToRemove = new ArrayList<>();
|
||||
TimelineEntry removed = timeline.get(interval);
|
||||
|
||||
if (removed == null) {
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.timeline.partition;
|
|||
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -38,13 +37,13 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
|
|||
|
||||
public PartitionHolder(PartitionChunk<T> initialChunk)
|
||||
{
|
||||
this.holderSet = Sets.newTreeSet();
|
||||
this.holderSet = new TreeSet<>();
|
||||
add(initialChunk);
|
||||
}
|
||||
|
||||
public PartitionHolder(List<PartitionChunk<T>> initialChunks)
|
||||
{
|
||||
this.holderSet = Sets.newTreeSet();
|
||||
this.holderSet = new TreeSet<>();
|
||||
for (PartitionChunk<T> chunk : initialChunks) {
|
||||
add(chunk);
|
||||
}
|
||||
|
@ -52,7 +51,7 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
|
|||
|
||||
public PartitionHolder(PartitionHolder partitionHolder)
|
||||
{
|
||||
this.holderSet = Sets.newTreeSet();
|
||||
this.holderSet = new TreeSet<>();
|
||||
this.holderSet.addAll(partitionHolder.holderSet);
|
||||
}
|
||||
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.collections;
|
||||
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.concurrent.Execs;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
@ -28,6 +27,7 @@ import org.junit.Rule;
|
|||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
@ -153,7 +153,7 @@ public class BlockingPoolTest
|
|||
@Override
|
||||
public List<ReferenceCountingResourceHolder<Integer>> call()
|
||||
{
|
||||
List<ReferenceCountingResourceHolder<Integer>> result = Lists.newArrayList();
|
||||
List<ReferenceCountingResourceHolder<Integer>> result = new ArrayList<>();
|
||||
for (int i = 0; i < limit1; i++) {
|
||||
result.add(pool.take(10));
|
||||
}
|
||||
|
@ -167,7 +167,7 @@ public class BlockingPoolTest
|
|||
@Override
|
||||
public List<ReferenceCountingResourceHolder<Integer>> call()
|
||||
{
|
||||
List<ReferenceCountingResourceHolder<Integer>> result = Lists.newArrayList();
|
||||
List<ReferenceCountingResourceHolder<Integer>> result = new ArrayList<>();
|
||||
for (int i = 0; i < limit2; i++) {
|
||||
result.add(pool.take(10));
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.collections;
|
||||
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.junit.Assert;
|
||||
|
@ -42,7 +41,7 @@ public class CombiningIterableTest
|
|||
Set<Integer> mergedLists = new HashSet<>();
|
||||
mergedLists.addAll(firstList);
|
||||
mergedLists.addAll(secondList);
|
||||
ArrayList<Iterable<Integer>> iterators = Lists.newArrayList();
|
||||
ArrayList<Iterable<Integer>> iterators = new ArrayList<>();
|
||||
iterators.add(firstList);
|
||||
iterators.add(secondList);
|
||||
CombiningIterable<Integer> actualIterable = CombiningIterable.createSplatted(
|
||||
|
|
|
@ -36,7 +36,7 @@ public class OrderedMergeIteratorTest
|
|||
@Test
|
||||
public void testSanity()
|
||||
{
|
||||
final ArrayList<Iterator<Integer>> iterators = Lists.newArrayList();
|
||||
final ArrayList<Iterator<Integer>> iterators = new ArrayList<>();
|
||||
iterators.add(Arrays.asList(1, 3, 5, 7, 9).iterator());
|
||||
iterators.add(Arrays.asList(2, 8).iterator());
|
||||
iterators.add(Arrays.asList(4, 6, 8).iterator());
|
||||
|
@ -52,7 +52,7 @@ public class OrderedMergeIteratorTest
|
|||
@Test
|
||||
public void testScrewsUpOnOutOfOrderBeginningOfList()
|
||||
{
|
||||
final ArrayList<Iterator<Integer>> iterators = Lists.newArrayList();
|
||||
final ArrayList<Iterator<Integer>> iterators = new ArrayList<>();
|
||||
iterators.add(Arrays.asList(1, 3, 5, 7, 9).iterator());
|
||||
iterators.add(Arrays.asList(4, 6).iterator());
|
||||
iterators.add(Arrays.asList(2, 8).iterator());
|
||||
|
@ -68,7 +68,7 @@ public class OrderedMergeIteratorTest
|
|||
@Test
|
||||
public void testScrewsUpOnOutOfOrderInList()
|
||||
{
|
||||
final ArrayList<Iterator<Integer>> iterators = Lists.newArrayList();
|
||||
final ArrayList<Iterator<Integer>> iterators = new ArrayList<>();
|
||||
iterators.add(Arrays.asList(1, 3, 5, 4, 7, 9).iterator());
|
||||
iterators.add(Arrays.asList(2, 8).iterator());
|
||||
iterators.add(Arrays.asList(4, 6).iterator());
|
||||
|
@ -86,7 +86,7 @@ public class OrderedMergeIteratorTest
|
|||
{
|
||||
final boolean[] done = new boolean[]{false, false};
|
||||
|
||||
final ArrayList<Iterator<Integer>> iterators = Lists.newArrayList();
|
||||
final ArrayList<Iterator<Integer>> iterators = new ArrayList<>();
|
||||
iterators.add(
|
||||
new IteratorShell<Integer>(Arrays.asList(1, 2, 3).iterator())
|
||||
{
|
||||
|
@ -170,7 +170,7 @@ public class OrderedMergeIteratorTest
|
|||
@Test(expected = NoSuchElementException.class)
|
||||
public void testNoElementInNext()
|
||||
{
|
||||
final ArrayList<Iterator<Integer>> iterators = Lists.newArrayList();
|
||||
final ArrayList<Iterator<Integer>> iterators = new ArrayList<>();
|
||||
OrderedMergeIterator<Integer> iter = new OrderedMergeIterator<Integer>(
|
||||
Ordering.natural(),
|
||||
iterators.iterator()
|
||||
|
@ -181,7 +181,7 @@ public class OrderedMergeIteratorTest
|
|||
@Test(expected = UnsupportedOperationException.class)
|
||||
public void testRemove()
|
||||
{
|
||||
final ArrayList<Iterator<Integer>> iterators = Lists.newArrayList();
|
||||
final ArrayList<Iterator<Integer>> iterators = new ArrayList<>();
|
||||
OrderedMergeIterator<Integer> iter = new OrderedMergeIterator<Integer>(
|
||||
Ordering.natural(),
|
||||
iterators.iterator()
|
||||
|
|
|
@ -174,7 +174,7 @@ public class OrderedMergeSequenceTest
|
|||
{
|
||||
final boolean[] done = new boolean[]{false, false};
|
||||
|
||||
final ArrayList<Sequence<Integer>> sequences = Lists.newArrayList();
|
||||
final ArrayList<Sequence<Integer>> sequences = new ArrayList<>();
|
||||
sequences.add(
|
||||
new BaseSequence<Integer, Iterator<Integer>>(
|
||||
new BaseSequence.IteratorMaker<Integer, Iterator<Integer>>()
|
||||
|
|
|
@ -19,9 +19,9 @@
|
|||
|
||||
package org.apache.druid.data.input.impl;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
||||
|
@ -39,8 +39,8 @@ public class CSVParseSpecTest
|
|||
),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b")),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList()
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>()
|
||||
),
|
||||
",",
|
||||
Collections.singletonList("a"),
|
||||
|
@ -61,8 +61,8 @@ public class CSVParseSpecTest
|
|||
),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(Arrays.asList("a,", "b")),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList()
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>()
|
||||
),
|
||||
",",
|
||||
Collections.singletonList("a"),
|
||||
|
|
|
@ -20,12 +20,12 @@
|
|||
package org.apache.druid.data.input.impl;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.TestObjectMapper;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
||||
|
@ -70,8 +70,8 @@ public class DelimitedParseSpecTest
|
|||
),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b")),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList()
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>()
|
||||
),
|
||||
",",
|
||||
" ",
|
||||
|
@ -93,8 +93,8 @@ public class DelimitedParseSpecTest
|
|||
),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(Arrays.asList("a,", "b")),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList()
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>()
|
||||
),
|
||||
",",
|
||||
null,
|
||||
|
@ -116,8 +116,8 @@ public class DelimitedParseSpecTest
|
|||
),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b")),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList()
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>()
|
||||
),
|
||||
",",
|
||||
null,
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.data.input.impl;
|
|||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.commons.io.LineIterator;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.junit.Assert;
|
||||
|
@ -112,7 +111,7 @@ public class FileIteratingFirehoseTest
|
|||
.collect(Collectors.toList());
|
||||
|
||||
try (final FileIteratingFirehose firehose = new FileIteratingFirehose(lineIterators.iterator(), parser)) {
|
||||
final List<String> results = Lists.newArrayList();
|
||||
final List<String> results = new ArrayList<>();
|
||||
|
||||
while (firehose.hasMore()) {
|
||||
final InputRow inputRow = firehose.nextRow();
|
||||
|
|
|
@ -19,11 +19,11 @@
|
|||
|
||||
package org.apache.druid.data.input.impl;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import junit.framework.Assert;
|
||||
import org.apache.druid.java.util.common.parsers.Parser;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -40,8 +40,8 @@ public class JSONLowercaseParseSpecTest
|
|||
),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(Arrays.asList("A", "B")),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList()
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>()
|
||||
)
|
||||
);
|
||||
Parser parser = spec.makeParser();
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.junit.Test;
|
|||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
|
||||
|
@ -70,8 +71,8 @@ public class ParseSpecTest
|
|||
),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "b", "a")),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList()
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>()
|
||||
),
|
||||
",",
|
||||
" ",
|
||||
|
@ -94,7 +95,7 @@ public class ParseSpecTest
|
|||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(Arrays.asList("a", "B")),
|
||||
Collections.singletonList("B"),
|
||||
Lists.newArrayList()
|
||||
new ArrayList<>()
|
||||
),
|
||||
",",
|
||||
null,
|
||||
|
@ -117,7 +118,7 @@ public class ParseSpecTest
|
|||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(Collections.singletonList("a")),
|
||||
Lists.newArrayList("B", "B"),
|
||||
Lists.newArrayList()
|
||||
new ArrayList<>()
|
||||
),
|
||||
",",
|
||||
null,
|
||||
|
@ -138,7 +139,7 @@ public class ParseSpecTest
|
|||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(Collections.singletonList("a")),
|
||||
Lists.newArrayList("B", "B"),
|
||||
Lists.newArrayList()
|
||||
new ArrayList<>()
|
||||
),
|
||||
",",
|
||||
null,
|
||||
|
|
|
@ -25,7 +25,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.impl.prefetch.JsonIterator;
|
||||
|
@ -105,7 +104,7 @@ public class SqlFirehoseTest
|
|||
.collect(Collectors.toList());
|
||||
|
||||
try (final SqlFirehose firehose = new SqlFirehose(lineIterators.iterator(), parser, closeable)) {
|
||||
final List<Object> results = Lists.newArrayList();
|
||||
final List<Object> results = new ArrayList<>();
|
||||
|
||||
while (firehose.hasMore()) {
|
||||
final InputRow inputRow = firehose.nextRow();
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.data.input.impl.prefetch;
|
|||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.io.CountingOutputStream;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.io.filefilter.TrueFileFilter;
|
||||
|
@ -74,8 +73,8 @@ public class PrefetchableTextFilesFirehoseFactoryTest
|
|||
),
|
||||
new DimensionsSpec(
|
||||
DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList()
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>()
|
||||
),
|
||||
",",
|
||||
Arrays.asList("timestamp", "a", "b"),
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.google.common.collect.Lists;
|
|||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
|
@ -44,7 +45,7 @@ public class FilteredSequenceTest
|
|||
};
|
||||
|
||||
for (int i = 0; i < 25; ++i) {
|
||||
List<Integer> vals = Lists.newArrayList();
|
||||
List<Integer> vals = new ArrayList<>();
|
||||
for (int j = 0; j < i; ++j) {
|
||||
vals.add(j);
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.google.common.collect.Lists;
|
|||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
|
||||
|
@ -43,7 +44,7 @@ public class MappedSequenceTest
|
|||
};
|
||||
|
||||
for (int i = 4; i < 5; ++i) {
|
||||
List<Integer> vals = Lists.newArrayList();
|
||||
List<Integer> vals = new ArrayList<>();
|
||||
for (int j = 0; j < i; ++j) {
|
||||
vals.add(j);
|
||||
}
|
||||
|
|
|
@ -166,8 +166,8 @@ public class LifecycleTest
|
|||
{
|
||||
Lifecycle lifecycle = new Lifecycle();
|
||||
|
||||
List<Integer> startOrder = Lists.newArrayList();
|
||||
List<Integer> stopOrder = Lists.newArrayList();
|
||||
List<Integer> startOrder = new ArrayList<>();
|
||||
List<Integer> stopOrder = new ArrayList<>();
|
||||
|
||||
lifecycle.addManagedInstance(new ObjectToBeLifecycled(0, startOrder, stopOrder));
|
||||
lifecycle.addManagedInstance(new ObjectToBeLifecycled(1, startOrder, stopOrder), Lifecycle.Stage.NORMAL);
|
||||
|
@ -198,8 +198,8 @@ public class LifecycleTest
|
|||
{
|
||||
final Lifecycle lifecycle = new Lifecycle();
|
||||
|
||||
final List<Integer> startOrder = Lists.newArrayList();
|
||||
final List<Integer> stopOrder = Lists.newArrayList();
|
||||
final List<Integer> startOrder = new ArrayList<>();
|
||||
final List<Integer> stopOrder = new ArrayList<>();
|
||||
|
||||
lifecycle.addManagedInstance(new ObjectToBeLifecycled(0, startOrder, stopOrder));
|
||||
lifecycle.addHandler(
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.druid.TestObjectMapper;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
|
@ -45,6 +44,7 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -241,7 +241,7 @@ public class DataSegmentTest
|
|||
List<DataSegment> shuffled = new ArrayList<>(Arrays.asList(sortedOrder));
|
||||
Collections.shuffle(shuffled);
|
||||
|
||||
Set<DataSegment> theSet = Sets.newTreeSet(DataSegment.bucketMonthComparator());
|
||||
Set<DataSegment> theSet = new TreeSet<>(DataSegment.bucketMonthComparator());
|
||||
theSet.addAll(shuffled);
|
||||
|
||||
int index = 0;
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.maxmind.geoip2.DatabaseReader;
|
||||
import com.maxmind.geoip2.exception.GeoIp2Exception;
|
||||
import com.maxmind.geoip2.model.Omni;
|
||||
|
@ -38,6 +37,7 @@ import java.io.IOException;
|
|||
import java.net.InetAddress;
|
||||
import java.net.URL;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Matcher;
|
||||
|
@ -88,7 +88,7 @@ class WikipediaIrcDecoder implements IrcDecoder
|
|||
)
|
||||
{
|
||||
if (namespaces == null) {
|
||||
namespaces = Maps.newHashMap();
|
||||
namespaces = new HashMap<>();
|
||||
}
|
||||
this.namespaces = namespaces;
|
||||
this.geoIpDatabase = geoIpDatabase;
|
||||
|
@ -177,8 +177,8 @@ class WikipediaIrcDecoder implements IrcDecoder
|
|||
@Override
|
||||
public InputRow decodeMessage(final DateTime timestamp, String channel, String msg)
|
||||
{
|
||||
final Map<String, String> dimensions = Maps.newHashMap();
|
||||
final Map<String, Float> metrics = Maps.newHashMap();
|
||||
final Map<String, String> dimensions = new HashMap<>();
|
||||
final Map<String, Float> metrics = new HashMap<>();
|
||||
|
||||
Matcher m = pattern.matcher(msg);
|
||||
if (!m.matches()) {
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.extendedset.intset;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import junit.framework.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
@ -97,7 +96,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
@Test
|
||||
public void testIntersection3()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
ConciseSet set2 = new ConciseSet();
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
|
@ -115,7 +114,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
@Test
|
||||
public void testIntersection4()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
ConciseSet set2 = new ConciseSet();
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
|
@ -137,7 +136,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
{
|
||||
final int[] ints1 = {33, 100000};
|
||||
final int[] ints2 = {34, 200000};
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i : ints1) {
|
||||
|
@ -158,7 +157,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
@Test
|
||||
public void testIntersection6()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 0; i < 5; i++) {
|
||||
set1.add(i);
|
||||
|
@ -197,7 +196,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
set2.add(i);
|
||||
}
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
expected.add(100);
|
||||
expected.add(500);
|
||||
for (int i = 600; i < 700; i++) {
|
||||
|
@ -224,7 +223,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
}
|
||||
set2.add(4001);
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
expected.add(100);
|
||||
expected.add(500);
|
||||
for (int i = 600; i < 700; i++) {
|
||||
|
@ -248,7 +247,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
set2.add(i);
|
||||
}
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
expected.add(2005);
|
||||
expected.add(3005);
|
||||
|
||||
|
@ -269,7 +268,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
set2.add(600);
|
||||
set2.add(4001);
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
expected.add(500);
|
||||
expected.add(600);
|
||||
|
||||
|
@ -290,7 +289,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
set2.add(i);
|
||||
}
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
expected.add(2005);
|
||||
for (int i = 2800; i < 3007; i++) {
|
||||
expected.add(i);
|
||||
|
@ -315,7 +314,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
}
|
||||
set2.add(10005);
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
expected.add(2005);
|
||||
for (int i = 2800; i < 3007; i++) {
|
||||
expected.add(i);
|
||||
|
@ -336,7 +335,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
set2.add(i);
|
||||
}
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
|
||||
verifyIntersection(expected, set1, set2);
|
||||
}
|
||||
|
@ -356,7 +355,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
set2.add(100);
|
||||
set2.add(101);
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
expected.add(0);
|
||||
expected.add(3);
|
||||
expected.add(5);
|
||||
|
@ -382,7 +381,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
set2.add(i);
|
||||
}
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
expected.add(0);
|
||||
expected.add(3);
|
||||
expected.add(5);
|
||||
|
@ -406,7 +405,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
set2.add(100);
|
||||
set2.add(101);
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
|
||||
verifyIntersection(expected, set1, set2);
|
||||
}
|
||||
|
@ -422,7 +421,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
ConciseSet set2 = new ConciseSet();
|
||||
set2.add(4001);
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
expected.add(4001);
|
||||
|
||||
verifyIntersection(expected, set1, set2);
|
||||
|
@ -441,7 +440,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
set2.add(i);
|
||||
}
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
for (int i = 32; i < 62; i++) {
|
||||
expected.add(i);
|
||||
}
|
||||
|
@ -460,7 +459,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
set2.add(i);
|
||||
}
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
expected.add(2005);
|
||||
|
||||
verifyIntersection(expected, set1, set2);
|
||||
|
@ -583,7 +582,7 @@ public class ImmutableConciseSetIntersectionTest
|
|||
|
||||
private void verifyIntersection(List<Integer> expected, List<ImmutableConciseSet> sets)
|
||||
{
|
||||
List<Integer> actual = Lists.newArrayList();
|
||||
List<Integer> actual = new ArrayList<>();
|
||||
ImmutableConciseSet set = ImmutableConciseSet.intersection(sets);
|
||||
IntSet.IntIterator itr = set.iterator();
|
||||
while (itr.hasNext()) {
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.extendedset.intset;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import junit.framework.Assert;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.junit.Test;
|
||||
|
@ -476,7 +475,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion3()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 62; i < 10001; i++) {
|
||||
set1.add(i);
|
||||
|
@ -506,7 +505,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion4()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 63; i < 1001; i++) {
|
||||
set1.add(i);
|
||||
|
@ -578,7 +577,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion6()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 0; i < 30; i++) {
|
||||
if (i != 28) {
|
||||
|
@ -612,7 +611,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion7()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 64; i < 1005; i++) {
|
||||
set1.add(i);
|
||||
|
@ -642,7 +641,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion8()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
if (i != 27) {
|
||||
|
@ -676,7 +675,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion9()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
if (!(i == 27 || i == 28)) {
|
||||
|
@ -712,7 +711,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion10()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 0; i < 1000; i += 2) {
|
||||
set1.add(i);
|
||||
|
@ -742,7 +741,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion11()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 0; i < 1000; i += 2) {
|
||||
set1.add(i);
|
||||
|
@ -802,7 +801,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion13()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
final int[] ints1 = {0};
|
||||
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
|
@ -835,7 +834,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion14()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
final int[] ints1 = {0, 100};
|
||||
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
|
@ -869,7 +868,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion15()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
final int[] ints1 = {1, 100};
|
||||
final int[] ints2 = {0};
|
||||
|
||||
|
@ -956,7 +955,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion18()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
set1.add(i);
|
||||
|
@ -985,7 +984,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion19()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 0; i < 93; i++) {
|
||||
set1.add(i);
|
||||
|
@ -1014,7 +1013,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testUnion20()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
ConciseSet set1 = new ConciseSet();
|
||||
for (int i = 0; i < 5; i++) {
|
||||
set1.add(i);
|
||||
|
@ -1064,7 +1063,7 @@ public class ImmutableConciseSetTest
|
|||
ImmutableConciseSet.newImmutableFromMutable(set2)
|
||||
);
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
for (int i = 0; i < 93; i++) {
|
||||
expected.add(i);
|
||||
}
|
||||
|
@ -1090,7 +1089,7 @@ public class ImmutableConciseSetTest
|
|||
ImmutableConciseSet.newImmutableFromMutable(set2)
|
||||
);
|
||||
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
for (int i = 0; i < 32; i++) {
|
||||
expected.add(i);
|
||||
}
|
||||
|
@ -1130,7 +1129,7 @@ public class ImmutableConciseSetTest
|
|||
|
||||
private void verifyUnion(List<Integer> expected, List<ImmutableConciseSet> sets)
|
||||
{
|
||||
List<Integer> actual = Lists.newArrayList();
|
||||
List<Integer> actual = new ArrayList<>();
|
||||
ImmutableConciseSet set = ImmutableConciseSet.union(sets);
|
||||
IntSet.IntIterator itr = set.iterator();
|
||||
while (itr.hasNext()) {
|
||||
|
@ -1146,7 +1145,7 @@ public class ImmutableConciseSetTest
|
|||
public void testComplement1()
|
||||
{
|
||||
final int[] ints = {1, 100};
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
|
||||
ConciseSet set = new ConciseSet();
|
||||
for (int i : ints) {
|
||||
|
@ -1170,7 +1169,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testComplement2()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
|
||||
ConciseSet set = new ConciseSet();
|
||||
for (int i = 0; i < 15; i++) {
|
||||
|
@ -1188,7 +1187,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testComplement3()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
final int length = 21;
|
||||
|
||||
ConciseSet set = new ConciseSet();
|
||||
|
@ -1210,7 +1209,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testComplement4()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
final int length = 41;
|
||||
|
||||
ConciseSet set = new ConciseSet();
|
||||
|
@ -1232,7 +1231,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testComplement5()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
final int length = 1001;
|
||||
|
||||
ConciseSet set = new ConciseSet();
|
||||
|
@ -1254,7 +1253,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testComplement6()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
final int length = 1001;
|
||||
|
||||
ConciseSet set = new ConciseSet();
|
||||
|
@ -1278,7 +1277,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testComplement7()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
final int length = 37;
|
||||
|
||||
ConciseSet set = new ConciseSet();
|
||||
|
@ -1298,7 +1297,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testComplement8()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
final int length = 32;
|
||||
|
||||
ConciseSet set = new ConciseSet();
|
||||
|
@ -1362,7 +1361,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testComplement10()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
final int length = 93;
|
||||
|
||||
for (int i = 0; i < length; i++) {
|
||||
|
@ -1380,7 +1379,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testComplement11()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
int length = 18930;
|
||||
for (int i = 0; i < 500; i++) {
|
||||
expected.add(i);
|
||||
|
@ -1404,7 +1403,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testComplement12()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
int length = 10;
|
||||
for (int i = 0; i < 10; i++) {
|
||||
expected.add(i);
|
||||
|
@ -1421,7 +1420,7 @@ public class ImmutableConciseSetTest
|
|||
@Test
|
||||
public void testComplement13()
|
||||
{
|
||||
List<Integer> expected = Lists.newArrayList();
|
||||
List<Integer> expected = new ArrayList<>();
|
||||
int length = 10;
|
||||
for (int i = 0; i < length; i++) {
|
||||
expected.add(i);
|
||||
|
@ -1433,7 +1432,7 @@ public class ImmutableConciseSetTest
|
|||
|
||||
private void verifyComplement(List<Integer> expected, ImmutableConciseSet set, int endIndex)
|
||||
{
|
||||
List<Integer> actual = Lists.newArrayList();
|
||||
List<Integer> actual = new ArrayList<>();
|
||||
|
||||
ImmutableConciseSet res;
|
||||
if (endIndex == NO_COMPLEMENT_LENGTH) {
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.emitter.ambari.metrics;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import junitparams.JUnitParamsRunner;
|
||||
import junitparams.Parameters;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
|
@ -39,6 +38,7 @@ import java.io.File;
|
|||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.HashMap;
|
||||
|
||||
|
||||
@RunWith(JUnitParamsRunner.class)
|
||||
|
@ -60,7 +60,7 @@ public class WhiteListBasedDruidToTimelineEventConverterTest
|
|||
EasyMock.expect(event.getHost()).andReturn(hostname).anyTimes();
|
||||
EasyMock.expect(event.getService()).andReturn(serviceName).anyTimes();
|
||||
EasyMock.expect(event.getCreatedTime()).andReturn(createdTime).anyTimes();
|
||||
EasyMock.expect(event.getUserDims()).andReturn(Maps.newHashMap()).anyTimes();
|
||||
EasyMock.expect(event.getUserDims()).andReturn(new HashMap<>()).anyTimes();
|
||||
EasyMock.expect(event.getValue()).andReturn(10).anyTimes();
|
||||
EasyMock.expect(event.getFeed()).andReturn("metrics").anyTimes();
|
||||
}
|
||||
|
|
|
@ -21,8 +21,6 @@ package org.apache.druid.storage.azure;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import com.microsoft.azure.storage.StorageException;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
|
@ -41,6 +39,8 @@ import org.junit.rules.TemporaryFolder;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.easymock.EasyMock.expectLastCall;
|
||||
|
@ -107,9 +107,9 @@ public class AzureDataSegmentPusherTest extends EasyMockSupport
|
|||
"foo",
|
||||
Intervals.of("2015/2016"),
|
||||
"0",
|
||||
Maps.newHashMap(),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList(),
|
||||
new HashMap<>(),
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>(),
|
||||
NoneShardSpec.instance(),
|
||||
0,
|
||||
size
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package org.apache.druid.storage.cloudfiles;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
|
@ -35,6 +33,8 @@ import org.junit.Test;
|
|||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -75,9 +75,9 @@ public class CloudFilesDataSegmentPusherTest
|
|||
"foo",
|
||||
Intervals.of("2015/2016"),
|
||||
"0",
|
||||
Maps.newHashMap(),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList(),
|
||||
new HashMap<>(),
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>(),
|
||||
NoneShardSpec.instance(),
|
||||
0,
|
||||
size
|
||||
|
|
|
@ -25,7 +25,6 @@ import com.google.api.services.storage.Storage;
|
|||
import com.google.api.services.storage.model.Objects;
|
||||
import com.google.api.services.storage.model.StorageObject;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.segment.loading.DataSegmentFinder;
|
||||
|
@ -36,6 +35,7 @@ import java.io.ByteArrayInputStream;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -63,7 +63,7 @@ public class GoogleDataSegmentFinder implements DataSegmentFinder
|
|||
@Override
|
||||
public Set<DataSegment> findSegments(String workingDirPath, boolean updateDescriptor) throws SegmentLoadingException
|
||||
{
|
||||
final Set<DataSegment> segments = Sets.newHashSet();
|
||||
final Set<DataSegment> segments = new HashSet<>();
|
||||
|
||||
try {
|
||||
Storage.Objects.List listObjects = storage.list(config.getBucket());
|
||||
|
|
|
@ -21,8 +21,6 @@ package org.apache.druid.storage.google;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
|
@ -37,6 +35,8 @@ import org.junit.Test;
|
|||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
|
||||
import static org.easymock.EasyMock.expectLastCall;
|
||||
|
||||
|
@ -78,9 +78,9 @@ public class GoogleDataSegmentPusherTest extends EasyMockSupport
|
|||
"foo",
|
||||
Intervals.of("2015/2016"),
|
||||
"0",
|
||||
Maps.newHashMap(),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList(),
|
||||
new HashMap<>(),
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>(),
|
||||
new NoneShardSpec(),
|
||||
0,
|
||||
size
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.emitter.graphite;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import junitparams.JUnitParamsRunner;
|
||||
import junitparams.Parameters;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
|
@ -38,6 +37,7 @@ import java.io.File;
|
|||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.HashMap;
|
||||
|
||||
|
||||
@RunWith(JUnitParamsRunner.class)
|
||||
|
@ -65,7 +65,7 @@ public class WhiteListBasedConverterTest
|
|||
EasyMock.expect(event.getHost()).andReturn(hostname).anyTimes();
|
||||
EasyMock.expect(event.getService()).andReturn(serviceName).anyTimes();
|
||||
EasyMock.expect(event.getCreatedTime()).andReturn(createdTime).anyTimes();
|
||||
EasyMock.expect(event.getUserDims()).andReturn(Maps.newHashMap()).anyTimes();
|
||||
EasyMock.expect(event.getUserDims()).andReturn(new HashMap<>()).anyTimes();
|
||||
EasyMock.expect(event.getValue()).andReturn(10).anyTimes();
|
||||
}
|
||||
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.firehose.kafka;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Closeables;
|
||||
import org.apache.druid.data.input.ByteBufferInputRowParser;
|
||||
import org.apache.druid.data.input.Committer;
|
||||
|
@ -38,6 +37,7 @@ import java.io.Closeable;
|
|||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -123,7 +123,7 @@ public class KafkaEightSimpleConsumerFirehoseFactory implements
|
|||
|
||||
private Map<Integer, Long> loadOffsetFromPreviousMetaData(Object lastCommit)
|
||||
{
|
||||
Map<Integer, Long> offsetMap = Maps.newHashMap();
|
||||
Map<Integer, Long> offsetMap = new HashMap<>();
|
||||
if (lastCommit == null) {
|
||||
return offsetMap;
|
||||
}
|
||||
|
@ -183,7 +183,7 @@ public class KafkaEightSimpleConsumerFirehoseFactory implements
|
|||
private volatile Iterator<InputRow> nextIterator = Collections.emptyIterator();
|
||||
|
||||
{
|
||||
lastOffsetPartitions = Maps.newHashMap();
|
||||
lastOffsetPartitions = new HashMap<>();
|
||||
lastOffsetPartitions.putAll(lastOffsets);
|
||||
}
|
||||
|
||||
|
@ -250,7 +250,7 @@ public class KafkaEightSimpleConsumerFirehoseFactory implements
|
|||
@Override
|
||||
public Committer makeCommitter()
|
||||
{
|
||||
final Map<Integer, Long> offsets = Maps.newHashMap(lastOffsetPartitions);
|
||||
final Map<Integer, Long> offsets = new HashMap<>(lastOffsetPartitions);
|
||||
|
||||
return new Committer()
|
||||
{
|
||||
|
|
|
@ -22,10 +22,8 @@ package org.apache.druid.indexing.materializedview;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.MapDifference;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
|
@ -55,10 +53,14 @@ import org.joda.time.Interval;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public class MaterializedViewSupervisor implements Supervisor
|
||||
|
@ -79,8 +81,8 @@ public class MaterializedViewSupervisor implements Supervisor
|
|||
private final String supervisorId;
|
||||
private final int maxTaskCount;
|
||||
private final long minDataLagMs;
|
||||
private final Map<Interval, HadoopIndexTask> runningTasks = Maps.newHashMap();
|
||||
private final Map<Interval, String> runningVersion = Maps.newHashMap();
|
||||
private final Map<Interval, HadoopIndexTask> runningTasks = new HashMap<>();
|
||||
private final Map<Interval, String> runningVersion = new HashMap<>();
|
||||
// taskLock is used to synchronize runningTask and runningVersion
|
||||
private final Object taskLock = new Object();
|
||||
// stateLock is used to synchronize materializedViewSupervisor's status
|
||||
|
@ -90,7 +92,7 @@ public class MaterializedViewSupervisor implements Supervisor
|
|||
private ListeningScheduledExecutorService exec = null;
|
||||
// In the missing intervals, baseDataSource has data but derivedDataSource does not, which means
|
||||
// data in these intervals of derivedDataSource needs to be rebuilt.
|
||||
private Set<Interval> missInterval = Sets.newHashSet();
|
||||
private Set<Interval> missInterval = new HashSet<>();
|
||||
|
||||
public MaterializedViewSupervisor(
|
||||
TaskMaster taskMaster,
|
||||
|
@ -325,13 +327,11 @@ public class MaterializedViewSupervisor implements Supervisor
|
|||
// use max created_date of base segments as the version of derivative segments
|
||||
Map<Interval, String> maxCreatedDate = baseSegmentsSnapshot.lhs;
|
||||
Map<Interval, String> derivativeVersion = derivativeSegmentsSnapshot.lhs;
|
||||
SortedMap<Interval, String> sortedToBuildInterval = Maps.newTreeMap(
|
||||
Comparators.inverse(Comparators.intervalsByStartThenEnd())
|
||||
);
|
||||
SortedMap<Interval, String> sortedToBuildInterval = new TreeMap<>(Comparators.inverse(Comparators.intervalsByStartThenEnd()));
|
||||
// find the intervals to drop and to build
|
||||
MapDifference<Interval, String> difference = Maps.difference(maxCreatedDate, derivativeVersion);
|
||||
Map<Interval, String> toBuildInterval = Maps.newHashMap(difference.entriesOnlyOnLeft());
|
||||
Map<Interval, String> toDropInterval = Maps.newHashMap(difference.entriesOnlyOnRight());
|
||||
Map<Interval, String> toBuildInterval = new HashMap<>(difference.entriesOnlyOnLeft());
|
||||
Map<Interval, String> toDropInterval = new HashMap<>(difference.entriesOnlyOnRight());
|
||||
// if some intervals are in running tasks and the versions are the same, remove it from toBuildInterval
|
||||
// if some intervals are in running tasks, but the versions are different, stop the task.
|
||||
for (Interval interval : runningVersion.keySet()) {
|
||||
|
@ -390,12 +390,12 @@ public class MaterializedViewSupervisor implements Supervisor
|
|||
List<DataSegment> snapshot
|
||||
)
|
||||
{
|
||||
Map<Interval, String> versions = Maps.newHashMap();
|
||||
Map<Interval, List<DataSegment>> segments = Maps.newHashMap();
|
||||
Map<Interval, String> versions = new HashMap<>();
|
||||
Map<Interval, List<DataSegment>> segments = new HashMap<>();
|
||||
for (DataSegment segment : snapshot) {
|
||||
Interval interval = segment.getInterval();
|
||||
versions.put(interval, segment.getVersion());
|
||||
segments.putIfAbsent(interval, Lists.newArrayList());
|
||||
segments.putIfAbsent(interval, new ArrayList<>());
|
||||
segments.get(interval).add(segment);
|
||||
}
|
||||
return new Pair<>(versions, segments);
|
||||
|
@ -410,8 +410,8 @@ public class MaterializedViewSupervisor implements Supervisor
|
|||
.map(DataSegment::getInterval)
|
||||
.max(Comparators.intervalsByStartThenEnd())
|
||||
.get();
|
||||
Map<Interval, String> maxCreatedDate = Maps.newHashMap();
|
||||
Map<Interval, List<DataSegment>> segments = Maps.newHashMap();
|
||||
Map<Interval, String> maxCreatedDate = new HashMap<>();
|
||||
Map<Interval, List<DataSegment>> segments = new HashMap<>();
|
||||
for (Pair<DataSegment, String> entry : snapshot) {
|
||||
DataSegment segment = entry.lhs;
|
||||
String createDate = entry.rhs;
|
||||
|
@ -426,7 +426,7 @@ public class MaterializedViewSupervisor implements Supervisor
|
|||
DateTimes.of(maxCreatedDate.getOrDefault(interval, DateTimes.MIN.toString()))
|
||||
).toString()
|
||||
);
|
||||
segments.putIfAbsent(interval, Lists.newArrayList());
|
||||
segments.putIfAbsent(interval, new ArrayList<>());
|
||||
segments.get(interval).add(segment);
|
||||
}
|
||||
return new Pair<>(maxCreatedDate, segments);
|
||||
|
|
|
@ -25,8 +25,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.commons.codec.digest.DigestUtils;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
|
@ -54,6 +52,8 @@ import org.apache.druid.server.security.AuthorizerMapper;
|
|||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -131,7 +131,7 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec
|
|||
this.hadoopCoordinates = hadoopCoordinates;
|
||||
this.hadoopDependencyCoordinates = hadoopDependencyCoordinates;
|
||||
this.classpathPrefix = classpathPrefix;
|
||||
this.context = context == null ? Maps.newHashMap() : context;
|
||||
this.context = context == null ? new HashMap<>() : context;
|
||||
this.objectMapper = objectMapper;
|
||||
this.taskMaster = taskMaster;
|
||||
this.taskStorage = taskStorage;
|
||||
|
@ -143,11 +143,11 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec
|
|||
this.config = config;
|
||||
this.suspended = suspended != null ? suspended : false;
|
||||
|
||||
this.metrics = Sets.newHashSet();
|
||||
this.metrics = new HashSet<>();
|
||||
for (AggregatorFactory aggregatorFactory : aggregators) {
|
||||
metrics.add(aggregatorFactory.getName());
|
||||
}
|
||||
this.dimensions = Sets.newHashSet();
|
||||
this.dimensions = new HashSet<>();
|
||||
for (DimensionSchema schema : dimensionsSpec.getDimensions()) {
|
||||
dimensions.add(schema.getName());
|
||||
}
|
||||
|
@ -158,10 +158,10 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec
|
|||
String taskId = StringUtils.format("%s_%s_%s", TASK_PREFIX, dataSourceName, DateTimes.nowUtc());
|
||||
|
||||
// generate parser
|
||||
Map<String, Object> parseSpec = Maps.newHashMap();
|
||||
Map<String, Object> parseSpec = new HashMap<>();
|
||||
parseSpec.put("format", "timeAndDims");
|
||||
parseSpec.put("dimensionsSpec", dimensionsSpec);
|
||||
Map<String, Object> parser = Maps.newHashMap();
|
||||
Map<String, Object> parser = new HashMap<>();
|
||||
parser.put("type", "map");
|
||||
parser.put("parseSpec", parseSpec);
|
||||
|
||||
|
@ -221,7 +221,7 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec
|
|||
);
|
||||
|
||||
// generate HadoopIOConfig
|
||||
Map<String, Object> inputSpec = Maps.newHashMap();
|
||||
Map<String, Object> inputSpec = new HashMap<>();
|
||||
inputSpec.put("type", "dataSource");
|
||||
inputSpec.put("ingestionSpec", datasourceIngestionSpec);
|
||||
HadoopIOConfig hadoopIOConfig = new HadoopIOConfig(inputSpec, null, null);
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
|
|||
import com.google.common.base.Optional;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.StringDimensionSchema;
|
||||
|
@ -56,6 +55,7 @@ import org.junit.rules.ExpectedException;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -156,7 +156,7 @@ public class MaterializedViewSupervisorTest
|
|||
expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
|
||||
expect(taskStorage.getActiveTasks()).andReturn(ImmutableList.of()).anyTimes();
|
||||
Pair<SortedMap<Interval, String>, Map<Interval, List<DataSegment>>> toBuildInterval = supervisor.checkSegments();
|
||||
Map<Interval, List<DataSegment>> expectedSegments = Maps.newHashMap();
|
||||
Map<Interval, List<DataSegment>> expectedSegments = new HashMap<>();
|
||||
expectedSegments.put(
|
||||
Intervals.of("2015-01-01T00Z/2015-01-02T00Z"),
|
||||
Collections.singletonList(
|
||||
|
|
|
@ -21,9 +21,6 @@ package org.apache.druid.query.materializedview;
|
|||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSortedSet;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.client.TimelineServerView;
|
||||
import org.apache.druid.query.Query;
|
||||
|
@ -35,7 +32,10 @@ import org.apache.druid.query.topn.TopNQuery;
|
|||
import org.apache.druid.timeline.TimelineObjectHolder;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -100,7 +100,7 @@ public class DataSourceOptimizer
|
|||
// get all fields which the query required
|
||||
Set<String> requiredFields = MaterializedViewUtils.getRequiredFields(query);
|
||||
|
||||
Set<DerivativeDataSource> derivativesWithRequiredFields = Sets.newHashSet();
|
||||
Set<DerivativeDataSource> derivativesWithRequiredFields = new HashSet<>();
|
||||
for (DerivativeDataSource derivativeDataSource : derivatives) {
|
||||
derivativesHitCount.putIfAbsent(derivativeDataSource.getName(), new AtomicLong(0));
|
||||
if (derivativeDataSource.getColumns().containsAll(requiredFields)) {
|
||||
|
@ -116,7 +116,7 @@ public class DataSourceOptimizer
|
|||
return Collections.singletonList(query);
|
||||
}
|
||||
|
||||
List<Query> queries = Lists.newArrayList();
|
||||
List<Query> queries = new ArrayList<>();
|
||||
List<Interval> remainingQueryIntervals = (List<Interval>) query.getIntervals();
|
||||
|
||||
for (DerivativeDataSource derivativeDataSource : ImmutableSortedSet.copyOf(derivativesWithRequiredFields)) {
|
||||
|
@ -187,10 +187,10 @@ public class DataSourceOptimizer
|
|||
finally {
|
||||
lock.writeLock().unlock();
|
||||
}
|
||||
List<DataSourceOptimizerStats> stats = Lists.newArrayList();
|
||||
List<DataSourceOptimizerStats> stats = new ArrayList<>();
|
||||
Map<String, Set<DerivativeDataSource>> baseToDerivatives = DerivativeDataSourceManager.getAllDerivatives();
|
||||
for (Map.Entry<String, Set<DerivativeDataSource>> entry : baseToDerivatives.entrySet()) {
|
||||
Map<String, Long> derivativesStat = Maps.newHashMap();
|
||||
Map<String, Long> derivativesStat = new HashMap<>();
|
||||
for (DerivativeDataSource derivative : entry.getValue()) {
|
||||
derivativesStat.put(
|
||||
derivative.getName(),
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper;
|
|||
import com.google.common.base.Supplier;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningScheduledExecutorService;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
|
@ -52,9 +51,11 @@ import org.skife.jdbi.v2.tweak.ResultSetMapper;
|
|||
import java.io.IOException;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
@ -144,7 +145,7 @@ public class DerivativeDataSourceManager
|
|||
|
||||
public static ImmutableSet<DerivativeDataSource> getDerivatives(String datasource)
|
||||
{
|
||||
return ImmutableSet.copyOf(derivativesRef.get().getOrDefault(datasource, Sets.newTreeSet()));
|
||||
return ImmutableSet.copyOf(derivativesRef.get().getOrDefault(datasource, new TreeSet<>()));
|
||||
}
|
||||
|
||||
public static ImmutableMap<String, Set<DerivativeDataSource>> getAllDerivatives()
|
||||
|
@ -199,7 +200,7 @@ public class DerivativeDataSourceManager
|
|||
|
||||
ConcurrentHashMap<String, SortedSet<DerivativeDataSource>> newDerivatives = new ConcurrentHashMap<>();
|
||||
for (DerivativeDataSource derivative : derivativeDataSources) {
|
||||
newDerivatives.putIfAbsent(derivative.getBaseDataSource(), Sets.newTreeSet());
|
||||
newDerivatives.putIfAbsent(derivative.getBaseDataSource(), new TreeSet<>());
|
||||
newDerivatives.get(derivative.getBaseDataSource()).add(derivative);
|
||||
}
|
||||
ConcurrentHashMap<String, SortedSet<DerivativeDataSource>> current;
|
||||
|
@ -227,7 +228,7 @@ public class DerivativeDataSourceManager
|
|||
{
|
||||
return connector.retryWithHandle(
|
||||
new HandleCallback<Long>() {
|
||||
Set<Interval> intervals = Sets.newHashSet();
|
||||
Set<Interval> intervals = new HashSet<>();
|
||||
long totalSize = 0;
|
||||
@Override
|
||||
public Long withHandle(Handle handle)
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.query.materializedview;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.JodaUtils;
|
||||
import org.apache.druid.query.Query;
|
||||
|
@ -31,6 +30,7 @@ import org.apache.druid.query.timeseries.TimeseriesQuery;
|
|||
import org.apache.druid.query.topn.TopNQuery;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
@ -106,7 +106,7 @@ public class MaterializedViewUtils
|
|||
}
|
||||
Iterator<Interval> it1 = JodaUtils.condenseIntervals(interval1).iterator();
|
||||
Iterator<Interval> it2 = JodaUtils.condenseIntervals(interval2).iterator();
|
||||
List<Interval> remaining = Lists.newArrayList();
|
||||
List<Interval> remaining = new ArrayList<>();
|
||||
Interval currInterval1 = it1.next();
|
||||
Interval currInterval2 = it2.next();
|
||||
long start1 = currInterval1.getStartMillis();
|
||||
|
|
|
@ -25,7 +25,6 @@ import com.google.common.base.Function;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.MapBasedInputRow;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
|
@ -50,6 +49,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
|
|||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
@ -95,7 +95,7 @@ public class OrcHadoopInputRowParser implements InputRowParser<OrcStruct>
|
|||
@Override
|
||||
public List<InputRow> parseBatch(OrcStruct input)
|
||||
{
|
||||
Map<String, Object> map = Maps.newHashMap();
|
||||
Map<String, Object> map = new HashMap<>();
|
||||
List<? extends StructField> fields = oip.getAllStructFieldRefs();
|
||||
for (StructField field : fields) {
|
||||
ObjectInspector objectInspector = field.getFieldObjectInspector();
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
|
@ -75,10 +74,12 @@ import java.io.FileInputStream;
|
|||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.TreeMap;
|
||||
import java.util.zip.ZipEntry;
|
||||
import java.util.zip.ZipInputStream;
|
||||
|
||||
|
@ -322,11 +323,11 @@ public class OrcIndexGeneratorJobTest
|
|||
Integer[][][] shardInfoForEachShard
|
||||
)
|
||||
{
|
||||
Map<Long, List<HadoopyShardSpec>> shardSpecs = Maps.newTreeMap(DateTimeComparator.getInstance());
|
||||
Map<Long, List<HadoopyShardSpec>> shardSpecs = new TreeMap<>(DateTimeComparator.getInstance());
|
||||
int shardCount = 0;
|
||||
int segmentNum = 0;
|
||||
for (Interval segmentGranularity : config.getSegmentGranularIntervals().get()) {
|
||||
List<ShardSpec> specs = Lists.newArrayList();
|
||||
List<ShardSpec> specs = new ArrayList<>();
|
||||
for (Integer[] shardInfo : shardInfoForEachShard[segmentNum++]) {
|
||||
specs.add(new HashBasedNumberedShardSpec(shardInfo[0], shardInfo[1], null, HadoopDruidIndexerConfig.JSON_MAPPER));
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.data.input.parquet;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.avro.LogicalType;
|
||||
import org.apache.avro.LogicalTypes;
|
||||
import org.apache.avro.Schema;
|
||||
|
@ -36,6 +35,7 @@ import org.apache.druid.java.util.common.DateTimes;
|
|||
import org.joda.time.DateTime;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
|
@ -57,7 +57,7 @@ public class ParquetHadoopInputRowParser implements InputRowParser<GenericRecord
|
|||
this.binaryAsString = binaryAsString == null ? false : binaryAsString;
|
||||
|
||||
List<DimensionSchema> dimensionSchema = parseSpec.getDimensionsSpec().getDimensions();
|
||||
this.dimensions = Lists.newArrayList();
|
||||
this.dimensions = new ArrayList<>();
|
||||
for (DimensionSchema dim : dimensionSchema) {
|
||||
this.dimensions.add(dim.getName());
|
||||
}
|
||||
|
|
|
@ -21,8 +21,6 @@
|
|||
package org.apache.parquet.avro;
|
||||
//CHECKSTYLE.ON: PackageName
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.druid.data.input.impl.DimensionSchema;
|
||||
|
@ -35,6 +33,8 @@ import org.apache.parquet.io.api.RecordMaterializer;
|
|||
import org.apache.parquet.schema.MessageType;
|
||||
import org.apache.parquet.schema.Type;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -62,17 +62,17 @@ public class DruidParquetReadSupport extends AvroReadSupport<GenericRecord>
|
|||
String tsField = config.getParser().getParseSpec().getTimestampSpec().getTimestampColumn();
|
||||
|
||||
List<DimensionSchema> dimensionSchema = config.getParser().getParseSpec().getDimensionsSpec().getDimensions();
|
||||
Set<String> dimensions = Sets.newHashSet();
|
||||
Set<String> dimensions = new HashSet<>();
|
||||
for (DimensionSchema dim : dimensionSchema) {
|
||||
dimensions.add(dim.getName());
|
||||
}
|
||||
|
||||
Set<String> metricsFields = Sets.newHashSet();
|
||||
Set<String> metricsFields = new HashSet<>();
|
||||
for (AggregatorFactory agg : config.getSchema().getDataSchema().getAggregators()) {
|
||||
metricsFields.addAll(agg.requiredFields());
|
||||
}
|
||||
|
||||
List<Type> partialFields = Lists.newArrayList();
|
||||
List<Type> partialFields = new ArrayList<>();
|
||||
|
||||
for (Type type : fullSchema.getFields()) {
|
||||
if (tsField.equals(type.getName())
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
*/
|
||||
package org.apache.druid.data.input.parquet;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.util.Utf8;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
|
@ -38,6 +37,7 @@ import org.junit.Test;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
@ -129,7 +129,7 @@ public class DruidParquetInputTest
|
|||
TaskAttemptContext context = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID());
|
||||
|
||||
try (RecordReader reader = inputFormat.createRecordReader(split, context)) {
|
||||
List<InputRow> records = Lists.newArrayList();
|
||||
List<InputRow> records = new ArrayList<>();
|
||||
InputRowParser parser = config.getParser();
|
||||
|
||||
reader.initialize(split, context);
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.junit.runners.Parameterized;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.sql.Timestamp;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.zip.ZipFile;
|
||||
|
||||
|
@ -62,7 +63,7 @@ public class TimestampGroupByAggregationTest
|
|||
@Parameterized.Parameters(name = "{index}: Test for {0}, config = {1}")
|
||||
public static Iterable<Object[]> constructorFeeder()
|
||||
{
|
||||
final List<Object[]> constructors = Lists.newArrayList();
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
|
||||
final List<List<Object>> partialConstructors = ImmutableList.of(
|
||||
ImmutableList.of("timeMin", "tmin", "time_min", DateTimes.of("2011-01-12T01:00:00.000Z")),
|
||||
|
|
|
@ -21,24 +21,22 @@ package org.apache.druid.query.aggregation.datasketches.hll;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.yahoo.sketches.hll.HllSketch;
|
||||
import com.yahoo.sketches.hll.TgtHllType;
|
||||
import com.yahoo.sketches.hll.Union;
|
||||
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.AggregatorUtil;
|
||||
import org.apache.druid.query.aggregation.PostAggregator;
|
||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Comparator;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
* Returns a union of a given list of sketches.
|
||||
* @author Alexander Saydakov
|
||||
|
@ -94,7 +92,7 @@ public class HllSketchUnionPostAggregator implements PostAggregator
|
|||
@Override
|
||||
public Set<String> getDependentFields()
|
||||
{
|
||||
final Set<String> dependentFields = Sets.newLinkedHashSet();
|
||||
final Set<String> dependentFields = new LinkedHashSet<>();
|
||||
for (final PostAggregator field : fields) {
|
||||
dependentFields.addAll(field.getDependentFields());
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Ordering;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Doubles;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.PostAggregator;
|
||||
|
@ -31,6 +30,7 @@ import org.apache.druid.query.aggregation.post.PostAggregatorIds;
|
|||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
public class SketchEstimatePostAggregator implements PostAggregator
|
||||
|
@ -55,7 +55,7 @@ public class SketchEstimatePostAggregator implements PostAggregator
|
|||
@Override
|
||||
public Set<String> getDependentFields()
|
||||
{
|
||||
Set<String> dependentFields = Sets.newHashSet();
|
||||
Set<String> dependentFields = new HashSet<>();
|
||||
dependentFields.addAll(field.getDependentFields());
|
||||
return dependentFields;
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.query.aggregation.datasketches.theta;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.yahoo.sketches.Util;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||
|
@ -30,6 +29,7 @@ import org.apache.druid.query.aggregation.post.PostAggregatorIds;
|
|||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.LinkedHashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -63,7 +63,7 @@ public class SketchSetPostAggregator implements PostAggregator
|
|||
@Override
|
||||
public Set<String> getDependentFields()
|
||||
{
|
||||
Set<String> dependentFields = Sets.newLinkedHashSet();
|
||||
Set<String> dependentFields = new LinkedHashSet<>();
|
||||
for (PostAggregator field : fields) {
|
||||
dependentFields.addAll(field.getDependentFields());
|
||||
}
|
||||
|
|
|
@ -19,19 +19,6 @@
|
|||
|
||||
package org.apache.druid.query.aggregation.datasketches.hll;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||
|
@ -39,6 +26,17 @@ import org.apache.druid.java.util.common.guava.Sequence;
|
|||
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
||||
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class HllSketchAggregatorTest
|
||||
|
@ -59,7 +57,7 @@ public class HllSketchAggregatorTest
|
|||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<?> constructorFeeder()
|
||||
{
|
||||
final List<Object[]> constructors = Lists.newArrayList();
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
|
||||
constructors.add(new Object[] {config});
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.query.aggregation.datasketches.quantiles;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
|
@ -40,6 +39,7 @@ import org.junit.runners.Parameterized;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -68,7 +68,7 @@ public class DoublesSketchAggregatorTest
|
|||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<?> constructorFeeder()
|
||||
{
|
||||
final List<Object[]> constructors = Lists.newArrayList();
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
|
||||
constructors.add(new Object[]{config});
|
||||
}
|
||||
|
|
|
@ -53,6 +53,7 @@ import org.junit.runners.Parameterized;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Comparator;
|
||||
|
@ -82,7 +83,7 @@ public class SketchAggregationTest
|
|||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<?> constructorFeeder()
|
||||
{
|
||||
final List<Object[]> constructors = Lists.newArrayList();
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
|
||||
constructors.add(new Object[]{config});
|
||||
}
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.query.aggregation.datasketches.theta;
|
|||
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.io.Files;
|
||||
import org.apache.druid.data.input.MapBasedRow;
|
||||
import org.apache.druid.data.input.Row;
|
||||
|
@ -48,6 +47,7 @@ import org.junit.runners.Parameterized;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -73,7 +73,7 @@ public class SketchAggregationWithSimpleDataTest
|
|||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<?> constructorFeeder()
|
||||
{
|
||||
final List<Object[]> constructors = Lists.newArrayList();
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
|
||||
constructors.add(new Object[]{config});
|
||||
}
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.junit.runners.Parameterized;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -76,7 +77,7 @@ public class OldApiSketchAggregationTest
|
|||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<?> constructorFeeder()
|
||||
{
|
||||
final List<Object[]> constructors = Lists.newArrayList();
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
|
||||
constructors.add(new Object[]{config});
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.druid.query.aggregation.datasketches.tuple;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.yahoo.sketches.quantiles.DoublesSketch;
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
|
@ -38,6 +37,7 @@ import org.junit.runners.Parameterized;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -61,7 +61,7 @@ public class ArrayOfDoublesSketchAggregationTest
|
|||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<?> constructorFeeder()
|
||||
{
|
||||
final List<Object[]> constructors = Lists.newArrayList();
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
|
||||
constructors.add(new Object[] {config});
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.security.basic;
|
|||
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -41,6 +40,7 @@ import java.security.NoSuchAlgorithmException;
|
|||
import java.security.SecureRandom;
|
||||
import java.security.spec.InvalidKeySpecException;
|
||||
import java.util.Base64;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
public class BasicAuthUtils
|
||||
|
@ -150,7 +150,7 @@ public class BasicAuthUtils
|
|||
{
|
||||
Map<String, BasicAuthenticatorUser> userMap;
|
||||
if (userMapBytes == null) {
|
||||
userMap = Maps.newHashMap();
|
||||
userMap = new HashMap<>();
|
||||
} else {
|
||||
try {
|
||||
userMap = objectMapper.readValue(userMapBytes, AUTHENTICATOR_USER_MAP_TYPE_REFERENCE);
|
||||
|
@ -182,7 +182,7 @@ public class BasicAuthUtils
|
|||
{
|
||||
Map<String, BasicAuthorizerUser> userMap;
|
||||
if (userMapBytes == null) {
|
||||
userMap = Maps.newHashMap();
|
||||
userMap = new HashMap<>();
|
||||
} else {
|
||||
try {
|
||||
userMap = objectMapper.readValue(userMapBytes, BasicAuthUtils.AUTHORIZER_USER_MAP_TYPE_REFERENCE);
|
||||
|
@ -211,7 +211,7 @@ public class BasicAuthUtils
|
|||
{
|
||||
Map<String, BasicAuthorizerRole> roleMap;
|
||||
if (roleMapBytes == null) {
|
||||
roleMap = Maps.newHashMap();
|
||||
roleMap = new HashMap<>();
|
||||
} else {
|
||||
try {
|
||||
roleMap = objectMapper.readValue(roleMapBytes, BasicAuthUtils.AUTHORIZER_ROLE_MAP_TYPE_REFERENCE);
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.security.basic.authentication.endpoint;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.guice.annotations.Smile;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -54,7 +53,7 @@ public class CoordinatorBasicAuthenticatorResourceHandler implements BasicAuthen
|
|||
this.storageUpdater = storageUpdater;
|
||||
this.objectMapper = objectMapper;
|
||||
|
||||
this.authenticatorMap = Maps.newHashMap();
|
||||
this.authenticatorMap = new HashMap<>();
|
||||
for (Map.Entry<String, Authenticator> authenticatorEntry : authenticatorMapper.getAuthenticatorMap().entrySet()) {
|
||||
final String authenticatorName = authenticatorEntry.getKey();
|
||||
final Authenticator authenticator = authenticatorEntry.getValue();
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.security.basic.authentication.endpoint;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
|
@ -50,7 +49,7 @@ public class DefaultBasicAuthenticatorResourceHandler implements BasicAuthentica
|
|||
{
|
||||
this.cacheManager = cacheManager;
|
||||
|
||||
this.authenticatorMap = Maps.newHashMap();
|
||||
this.authenticatorMap = new HashMap<>();
|
||||
for (Map.Entry<String, Authenticator> authenticatorEntry : authenticatorMapper.getAuthenticatorMap().entrySet()) {
|
||||
final String authenticatorName = authenticatorEntry.getKey();
|
||||
final Authenticator authenticator = authenticatorEntry.getValue();
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.druid.security.basic.authorization.endpoint;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.guice.annotations.Smile;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -64,7 +63,7 @@ public class CoordinatorBasicAuthorizerResourceHandler implements BasicAuthorize
|
|||
this.storageUpdater = storageUpdater;
|
||||
this.objectMapper = objectMapper;
|
||||
|
||||
this.authorizerMap = Maps.newHashMap();
|
||||
this.authorizerMap = new HashMap<>();
|
||||
for (Map.Entry<String, Authorizer> authorizerEntry : authorizerMapper.getAuthorizerMap().entrySet()) {
|
||||
final String authorizerName = authorizerEntry.getKey();
|
||||
final Authorizer authorizer = authorizerEntry.getValue();
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.security.basic.authorization.endpoint;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
|
@ -51,7 +50,7 @@ public class DefaultBasicAuthorizerResourceHandler implements BasicAuthorizerRes
|
|||
{
|
||||
this.cacheManager = cacheManager;
|
||||
|
||||
this.authorizerMap = Maps.newHashMap();
|
||||
this.authorizerMap = new HashMap<>();
|
||||
for (Map.Entry<String, Authorizer> authorizerEntry : authorizerMapper.getAuthorizerMap().entrySet()) {
|
||||
final String authorizerName = authorizerEntry.getKey();
|
||||
final Authorizer authorizer = authorizerEntry.getValue();
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.druid.metadata.MetadataStorageTablesConfig;
|
||||
import org.apache.druid.metadata.TestDerbyConnector;
|
||||
import org.apache.druid.security.basic.BasicAuthCommonCacheConfig;
|
||||
|
@ -46,6 +45,7 @@ import org.junit.Rule;
|
|||
import org.junit.Test;
|
||||
import org.junit.rules.ExpectedException;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -121,7 +121,7 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdaterTest
|
|||
public void testCreateDeleteUser()
|
||||
{
|
||||
updater.createUser(AUTHORIZER_NAME, "druid");
|
||||
Map<String, BasicAuthorizerUser> expectedUserMap = Maps.newHashMap(BASE_USER_MAP);
|
||||
Map<String, BasicAuthorizerUser> expectedUserMap = new HashMap<>(BASE_USER_MAP);
|
||||
expectedUserMap.put("druid", new BasicAuthorizerUser("druid", ImmutableSet.of()));
|
||||
Map<String, BasicAuthorizerUser> actualUserMap = BasicAuthUtils.deserializeAuthorizerUserMap(
|
||||
objectMapper,
|
||||
|
@ -160,7 +160,7 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdaterTest
|
|||
public void testCreateDeleteRole()
|
||||
{
|
||||
updater.createRole(AUTHORIZER_NAME, "druid");
|
||||
Map<String, BasicAuthorizerRole> expectedRoleMap = Maps.newHashMap(BASE_ROLE_MAP);
|
||||
Map<String, BasicAuthorizerRole> expectedRoleMap = new HashMap<>(BASE_ROLE_MAP);
|
||||
expectedRoleMap.put("druid", new BasicAuthorizerRole("druid", ImmutableList.of()));
|
||||
Map<String, BasicAuthorizerRole> actualRoleMap = BasicAuthUtils.deserializeAuthorizerRoleMap(
|
||||
objectMapper,
|
||||
|
@ -202,10 +202,10 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdaterTest
|
|||
updater.createRole(AUTHORIZER_NAME, "druidRole");
|
||||
updater.assignRole(AUTHORIZER_NAME, "druid", "druidRole");
|
||||
|
||||
Map<String, BasicAuthorizerUser> expectedUserMap = Maps.newHashMap(BASE_USER_MAP);
|
||||
Map<String, BasicAuthorizerUser> expectedUserMap = new HashMap<>(BASE_USER_MAP);
|
||||
expectedUserMap.put("druid", new BasicAuthorizerUser("druid", ImmutableSet.of("druidRole")));
|
||||
|
||||
Map<String, BasicAuthorizerRole> expectedRoleMap = Maps.newHashMap(BASE_ROLE_MAP);
|
||||
Map<String, BasicAuthorizerRole> expectedRoleMap = new HashMap<>(BASE_ROLE_MAP);
|
||||
expectedRoleMap.put("druidRole", new BasicAuthorizerRole("druidRole", ImmutableList.of()));
|
||||
|
||||
Map<String, BasicAuthorizerUser> actualUserMap = BasicAuthUtils.deserializeAuthorizerUserMap(
|
||||
|
@ -270,10 +270,10 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdaterTest
|
|||
updater.createUser(AUTHORIZER_NAME, "druid");
|
||||
updater.createRole(AUTHORIZER_NAME, "druidRole");
|
||||
|
||||
Map<String, BasicAuthorizerUser> expectedUserMap = Maps.newHashMap(BASE_USER_MAP);
|
||||
Map<String, BasicAuthorizerUser> expectedUserMap = new HashMap<>(BASE_USER_MAP);
|
||||
expectedUserMap.put("druid", new BasicAuthorizerUser("druid", ImmutableSet.of()));
|
||||
|
||||
Map<String, BasicAuthorizerRole> expectedRoleMap = Maps.newHashMap(BASE_ROLE_MAP);
|
||||
Map<String, BasicAuthorizerRole> expectedRoleMap = new HashMap<>(BASE_ROLE_MAP);
|
||||
expectedRoleMap.put("druidRole", new BasicAuthorizerRole("druidRole", ImmutableList.of()));
|
||||
|
||||
Map<String, BasicAuthorizerUser> actualUserMap = BasicAuthUtils.deserializeAuthorizerUserMap(
|
||||
|
@ -309,10 +309,10 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdaterTest
|
|||
|
||||
updater.setPermissions(AUTHORIZER_NAME, "druidRole", permsToAdd);
|
||||
|
||||
Map<String, BasicAuthorizerUser> expectedUserMap = Maps.newHashMap(BASE_USER_MAP);
|
||||
Map<String, BasicAuthorizerUser> expectedUserMap = new HashMap<>(BASE_USER_MAP);
|
||||
expectedUserMap.put("druid", new BasicAuthorizerUser("druid", ImmutableSet.of("druidRole")));
|
||||
|
||||
Map<String, BasicAuthorizerRole> expectedRoleMap = Maps.newHashMap(BASE_ROLE_MAP);
|
||||
Map<String, BasicAuthorizerRole> expectedRoleMap = new HashMap<>(BASE_ROLE_MAP);
|
||||
expectedRoleMap.put(
|
||||
"druidRole",
|
||||
new BasicAuthorizerRole("druidRole", BasicAuthorizerPermission.makePermissionList(permsToAdd))
|
||||
|
|
|
@ -32,8 +32,6 @@ import com.fasterxml.jackson.databind.module.SimpleModule;
|
|||
import com.fasterxml.jackson.databind.ser.std.ToStringSerializer;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.druid.indexer.Bucket;
|
||||
import org.apache.druid.indexer.HadoopDruidIndexerConfig;
|
||||
|
@ -67,7 +65,9 @@ import org.junit.rules.TemporaryFolder;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -150,9 +150,9 @@ public class HdfsDataSegmentPusherTest
|
|||
"foo",
|
||||
Intervals.of("2015/2016"),
|
||||
"0",
|
||||
Maps.newHashMap(),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList(),
|
||||
new HashMap<>(),
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>(),
|
||||
NoneShardSpec.instance(),
|
||||
0,
|
||||
size
|
||||
|
@ -195,9 +195,9 @@ public class HdfsDataSegmentPusherTest
|
|||
"foo",
|
||||
Intervals.of("2015/2016"),
|
||||
"0",
|
||||
Maps.newHashMap(),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList(),
|
||||
new HashMap<>(),
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>(),
|
||||
new NumberedShardSpec(i, i),
|
||||
0,
|
||||
size
|
||||
|
@ -301,9 +301,9 @@ public class HdfsDataSegmentPusherTest
|
|||
"foo",
|
||||
Intervals.of("2015/2016"),
|
||||
"0",
|
||||
Maps.newHashMap(),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList(),
|
||||
new HashMap<>(),
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>(),
|
||||
NoneShardSpec.instance(),
|
||||
0,
|
||||
size
|
||||
|
|
|
@ -21,14 +21,13 @@ package org.apache.druid.query.aggregation.histogram;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonValue;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Floats;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -1378,7 +1377,7 @@ public class ApproximateHistogram
|
|||
} else {
|
||||
byte approxCount = (byte) (-1 * count);
|
||||
|
||||
Map<Float, Long> approx = Maps.newHashMap();
|
||||
Map<Float, Long> approx = new HashMap<>();
|
||||
|
||||
for (int i = 0; i < approxCount; ++i) {
|
||||
final float value = buf.getFloat();
|
||||
|
@ -1394,7 +1393,7 @@ public class ApproximateHistogram
|
|||
|
||||
byte exactCount = buf.get();
|
||||
|
||||
Map<Float, Long> exact = Maps.newHashMap();
|
||||
Map<Float, Long> exact = new HashMap<>();
|
||||
|
||||
for (int i = 0; i < exactCount; ++i) {
|
||||
final float value = buf.getFloat();
|
||||
|
@ -1407,7 +1406,7 @@ public class ApproximateHistogram
|
|||
|
||||
int binCount = exact.size() + approx.size();
|
||||
|
||||
List<Float> pos = Lists.newArrayList();
|
||||
List<Float> pos = new ArrayList<>();
|
||||
pos.addAll(exact.keySet());
|
||||
pos.addAll(approx.keySet());
|
||||
Collections.sort(pos);
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.junit.runner.RunWith;
|
|||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -63,7 +64,7 @@ public class ApproximateHistogramAggregationTest
|
|||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<?> constructorFeeder()
|
||||
{
|
||||
final List<Object[]> constructors = Lists.newArrayList();
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) {
|
||||
constructors.add(new Object[]{config});
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.query.aggregation.histogram;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.druid.data.input.Row;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -43,6 +42,7 @@ import org.junit.runner.RunWith;
|
|||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
|
@ -111,7 +111,7 @@ public class ApproximateHistogramGroupByQueryTest
|
|||
v1Config.setMaxIntermediateRows(10000);
|
||||
v1SingleThreadedConfig.setMaxIntermediateRows(10000);
|
||||
|
||||
final List<Object[]> constructors = Lists.newArrayList();
|
||||
final List<Object[]> constructors = new ArrayList<>();
|
||||
final List<GroupByQueryConfig> configs = ImmutableList.of(
|
||||
v1Config,
|
||||
v1SingleThreadedConfig,
|
||||
|
|
|
@ -29,7 +29,6 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Longs;
|
||||
import com.google.common.util.concurrent.FutureCallback;
|
||||
|
@ -927,7 +926,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner implements KafkaIndexTask
|
|||
|
||||
private Map<String, Object> getTaskCompletionUnparseableEvents()
|
||||
{
|
||||
Map<String, Object> unparseableEventsMap = Maps.newHashMap();
|
||||
Map<String, Object> unparseableEventsMap = new HashMap<>();
|
||||
List<String> buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions(
|
||||
savedParseExceptions
|
||||
);
|
||||
|
@ -939,7 +938,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner implements KafkaIndexTask
|
|||
|
||||
private Map<String, Object> getTaskCompletionRowStats()
|
||||
{
|
||||
Map<String, Object> metrics = Maps.newHashMap();
|
||||
Map<String, Object> metrics = new HashMap<>();
|
||||
metrics.put(
|
||||
RowIngestionMeters.BUILD_SEGMENTS,
|
||||
rowIngestionMeters.getTotals()
|
||||
|
@ -974,7 +973,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner implements KafkaIndexTask
|
|||
private Set<Integer> assignPartitionsAndSeekToNext(KafkaConsumer consumer, String topic)
|
||||
{
|
||||
// Initialize consumer assignment.
|
||||
final Set<Integer> assignment = Sets.newHashSet();
|
||||
final Set<Integer> assignment = new HashSet<>();
|
||||
for (Map.Entry<Integer, Long> entry : nextOffsets.entrySet()) {
|
||||
final long endOffset = endOffsets.get(entry.getKey());
|
||||
if (entry.getValue() < endOffset) {
|
||||
|
@ -1042,7 +1041,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner implements KafkaIndexTask
|
|||
TaskToolbox taskToolbox
|
||||
) throws InterruptedException, IOException
|
||||
{
|
||||
final Map<TopicPartition, Long> resetPartitions = Maps.newHashMap();
|
||||
final Map<TopicPartition, Long> resetPartitions = new HashMap<>();
|
||||
boolean doReset = false;
|
||||
if (tuningConfig.isResetOffsetAutomatically()) {
|
||||
for (Map.Entry<TopicPartition, Long> outOfRangePartition : outOfRangePartitions.entrySet()) {
|
||||
|
@ -1121,7 +1120,7 @@ public class IncrementalPublishingKafkaIndexTaskRunner implements KafkaIndexTask
|
|||
private void sendResetRequestAndWait(Map<TopicPartition, Long> outOfRangePartitions, TaskToolbox taskToolbox)
|
||||
throws IOException
|
||||
{
|
||||
Map<Integer, Long> partitionOffsetMap = Maps.newHashMap();
|
||||
Map<Integer, Long> partitionOffsetMap = new HashMap<>();
|
||||
for (Map.Entry<TopicPartition, Long> outOfRangePartition : outOfRangePartitions.entrySet()) {
|
||||
partitionOffsetMap.put(outOfRangePartition.getKey().partition(), outOfRangePartition.getValue());
|
||||
}
|
||||
|
@ -1292,9 +1291,9 @@ public class IncrementalPublishingKafkaIndexTaskRunner implements KafkaIndexTask
|
|||
)
|
||||
{
|
||||
authorizationCheck(req, Action.READ);
|
||||
Map<String, Object> returnMap = Maps.newHashMap();
|
||||
Map<String, Object> totalsMap = Maps.newHashMap();
|
||||
Map<String, Object> averagesMap = Maps.newHashMap();
|
||||
Map<String, Object> returnMap = new HashMap<>();
|
||||
Map<String, Object> totalsMap = new HashMap<>();
|
||||
Map<String, Object> averagesMap = new HashMap<>();
|
||||
|
||||
totalsMap.put(
|
||||
RowIngestionMeters.BUILD_SEGMENTS,
|
||||
|
|
|
@ -21,10 +21,10 @@ package org.apache.druid.indexing.kafka;
|
|||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
|
@ -77,7 +77,7 @@ public class KafkaDataSourceMetadata implements DataSourceMetadata
|
|||
|
||||
if (that.getKafkaPartitions().getTopic().equals(kafkaPartitions.getTopic())) {
|
||||
// Same topic, merge offsets.
|
||||
final Map<Integer, Long> newMap = Maps.newHashMap();
|
||||
final Map<Integer, Long> newMap = new HashMap<>();
|
||||
|
||||
for (Map.Entry<Integer, Long> entry : kafkaPartitions.getPartitionOffsetMap().entrySet()) {
|
||||
newMap.put(entry.getKey(), entry.getValue());
|
||||
|
@ -109,7 +109,7 @@ public class KafkaDataSourceMetadata implements DataSourceMetadata
|
|||
|
||||
if (that.getKafkaPartitions().getTopic().equals(kafkaPartitions.getTopic())) {
|
||||
// Same topic, remove partitions present in "that" from "this"
|
||||
final Map<Integer, Long> newMap = Maps.newHashMap();
|
||||
final Map<Integer, Long> newMap = new HashMap<>();
|
||||
|
||||
for (Map.Entry<Integer, Long> entry : kafkaPartitions.getPartitionOffsetMap().entrySet()) {
|
||||
if (!that.getKafkaPartitions().getPartitionOffsetMap().containsKey(entry.getKey())) {
|
||||
|
|
|
@ -25,8 +25,6 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.druid.data.input.Committer;
|
||||
import org.apache.druid.data.input.InputRow;
|
||||
import org.apache.druid.data.input.impl.InputRowParser;
|
||||
|
@ -307,7 +305,7 @@ public class LegacyKafkaIndexTaskRunner implements KafkaIndexTaskRunner
|
|||
}
|
||||
|
||||
// Set up sequenceNames.
|
||||
final Map<Integer, String> sequenceNames = Maps.newHashMap();
|
||||
final Map<Integer, String> sequenceNames = new HashMap<>();
|
||||
for (Integer partitionNum : nextOffsets.keySet()) {
|
||||
sequenceNames.put(partitionNum, StringUtils.format("%s_%s", ioConfig.getBaseSequenceName(), partitionNum));
|
||||
}
|
||||
|
@ -599,7 +597,7 @@ public class LegacyKafkaIndexTaskRunner implements KafkaIndexTaskRunner
|
|||
private Set<Integer> assignPartitionsAndSeekToNext(KafkaConsumer consumer, String topic)
|
||||
{
|
||||
// Initialize consumer assignment.
|
||||
final Set<Integer> assignment = Sets.newHashSet();
|
||||
final Set<Integer> assignment = new HashSet<>();
|
||||
for (Map.Entry<Integer, Long> entry : nextOffsets.entrySet()) {
|
||||
final long endOffset = endOffsets.get(entry.getKey());
|
||||
if (entry.getValue() < endOffset) {
|
||||
|
@ -667,7 +665,7 @@ public class LegacyKafkaIndexTaskRunner implements KafkaIndexTaskRunner
|
|||
TaskToolbox taskToolbox
|
||||
) throws InterruptedException, IOException
|
||||
{
|
||||
final Map<TopicPartition, Long> resetPartitions = Maps.newHashMap();
|
||||
final Map<TopicPartition, Long> resetPartitions = new HashMap<>();
|
||||
boolean doReset = false;
|
||||
if (tuningConfig.isResetOffsetAutomatically()) {
|
||||
for (Map.Entry<TopicPartition, Long> outOfRangePartition : outOfRangePartitions.entrySet()) {
|
||||
|
@ -708,7 +706,7 @@ public class LegacyKafkaIndexTaskRunner implements KafkaIndexTaskRunner
|
|||
private void sendResetRequestAndWait(Map<TopicPartition, Long> outOfRangePartitions, TaskToolbox taskToolbox)
|
||||
throws IOException
|
||||
{
|
||||
Map<Integer, Long> partitionOffsetMap = Maps.newHashMap();
|
||||
Map<Integer, Long> partitionOffsetMap = new HashMap<>();
|
||||
for (Map.Entry<TopicPartition, Long> outOfRangePartition : outOfRangePartitions.entrySet()) {
|
||||
partitionOffsetMap.put(outOfRangePartition.getKey().partition(), outOfRangePartition.getValue());
|
||||
}
|
||||
|
@ -783,7 +781,7 @@ public class LegacyKafkaIndexTaskRunner implements KafkaIndexTaskRunner
|
|||
|
||||
private Map<String, Object> getTaskCompletionUnparseableEvents()
|
||||
{
|
||||
Map<String, Object> unparseableEventsMap = Maps.newHashMap();
|
||||
Map<String, Object> unparseableEventsMap = new HashMap<>();
|
||||
List<String> buildSegmentsParseExceptionMessages = IndexTaskUtils.getMessagesFromSavedParseExceptions(
|
||||
savedParseExceptions
|
||||
);
|
||||
|
@ -795,7 +793,7 @@ public class LegacyKafkaIndexTaskRunner implements KafkaIndexTaskRunner
|
|||
|
||||
private Map<String, Object> getTaskCompletionRowStats()
|
||||
{
|
||||
Map<String, Object> metrics = Maps.newHashMap();
|
||||
Map<String, Object> metrics = new HashMap<>();
|
||||
metrics.put(
|
||||
RowIngestionMeters.BUILD_SEGMENTS,
|
||||
rowIngestionMeters.getTotals()
|
||||
|
@ -942,9 +940,9 @@ public class LegacyKafkaIndexTaskRunner implements KafkaIndexTaskRunner
|
|||
)
|
||||
{
|
||||
authorizationCheck(req, Action.READ);
|
||||
Map<String, Object> returnMap = Maps.newHashMap();
|
||||
Map<String, Object> totalsMap = Maps.newHashMap();
|
||||
Map<String, Object> averagesMap = Maps.newHashMap();
|
||||
Map<String, Object> returnMap = new HashMap<>();
|
||||
Map<String, Object> totalsMap = new HashMap<>();
|
||||
Map<String, Object> averagesMap = new HashMap<>();
|
||||
|
||||
totalsMap.put(
|
||||
RowIngestionMeters.BUILD_SEGMENTS,
|
||||
|
|
|
@ -35,7 +35,6 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.primitives.Longs;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
|
@ -1084,8 +1083,8 @@ public class KafkaSupervisor implements Supervisor
|
|||
private void discoverTasks() throws ExecutionException, InterruptedException, TimeoutException
|
||||
{
|
||||
int taskCount = 0;
|
||||
List<String> futureTaskIds = Lists.newArrayList();
|
||||
List<ListenableFuture<Boolean>> futures = Lists.newArrayList();
|
||||
List<String> futureTaskIds = new ArrayList<>();
|
||||
List<ListenableFuture<Boolean>> futures = new ArrayList<>();
|
||||
List<Task> tasks = taskStorage.getActiveTasks();
|
||||
final Map<Integer, TaskGroup> taskGroupsToVerify = new HashMap<>();
|
||||
|
||||
|
@ -1441,8 +1440,8 @@ public class KafkaSupervisor implements Supervisor
|
|||
|
||||
private void updateTaskStatus() throws ExecutionException, InterruptedException, TimeoutException
|
||||
{
|
||||
final List<ListenableFuture<Boolean>> futures = Lists.newArrayList();
|
||||
final List<String> futureTaskIds = Lists.newArrayList();
|
||||
final List<ListenableFuture<Boolean>> futures = new ArrayList<>();
|
||||
final List<String> futureTaskIds = new ArrayList<>();
|
||||
|
||||
// update status (and startTime if unknown) of current tasks in taskGroups
|
||||
for (TaskGroup group : taskGroups.values()) {
|
||||
|
@ -1509,8 +1508,8 @@ public class KafkaSupervisor implements Supervisor
|
|||
|
||||
private void checkTaskDuration() throws InterruptedException, ExecutionException, TimeoutException
|
||||
{
|
||||
final List<ListenableFuture<Map<Integer, Long>>> futures = Lists.newArrayList();
|
||||
final List<Integer> futureGroupIds = Lists.newArrayList();
|
||||
final List<ListenableFuture<Map<Integer, Long>>> futures = new ArrayList<>();
|
||||
final List<Integer> futureGroupIds = new ArrayList<>();
|
||||
|
||||
for (Entry<Integer, TaskGroup> entry : taskGroups.entrySet()) {
|
||||
Integer groupId = entry.getKey();
|
||||
|
@ -1610,7 +1609,7 @@ public class KafkaSupervisor implements Supervisor
|
|||
}
|
||||
|
||||
// 2) Pause running tasks
|
||||
final List<ListenableFuture<Map<Integer, Long>>> pauseFutures = Lists.newArrayList();
|
||||
final List<ListenableFuture<Map<Integer, Long>>> pauseFutures = new ArrayList<>();
|
||||
final List<String> pauseTaskIds = ImmutableList.copyOf(taskGroup.taskIds());
|
||||
for (final String taskId : pauseTaskIds) {
|
||||
pauseFutures.add(taskClient.pauseAsync(taskId));
|
||||
|
@ -1646,7 +1645,7 @@ public class KafkaSupervisor implements Supervisor
|
|||
|
||||
// 4) Set the end offsets for each task to the values from step 3 and resume the tasks. All the tasks should
|
||||
// finish reading and start publishing within a short period, depending on how in sync the tasks were.
|
||||
final List<ListenableFuture<Boolean>> setEndOffsetFutures = Lists.newArrayList();
|
||||
final List<ListenableFuture<Boolean>> setEndOffsetFutures = new ArrayList<>();
|
||||
final List<String> setEndOffsetTaskIds = ImmutableList.copyOf(taskGroup.taskIds());
|
||||
|
||||
if (setEndOffsetTaskIds.isEmpty()) {
|
||||
|
@ -1711,14 +1710,14 @@ public class KafkaSupervisor implements Supervisor
|
|||
*/
|
||||
private void checkPendingCompletionTasks() throws ExecutionException, InterruptedException, TimeoutException
|
||||
{
|
||||
List<ListenableFuture<?>> futures = Lists.newArrayList();
|
||||
List<ListenableFuture<?>> futures = new ArrayList<>();
|
||||
|
||||
for (Entry<Integer, CopyOnWriteArrayList<TaskGroup>> pendingGroupList : pendingCompletionTaskGroups.entrySet()) {
|
||||
|
||||
boolean stopTasksInTaskGroup = false;
|
||||
Integer groupId = pendingGroupList.getKey();
|
||||
CopyOnWriteArrayList<TaskGroup> taskGroupList = pendingGroupList.getValue();
|
||||
List<TaskGroup> toRemove = Lists.newArrayList();
|
||||
List<TaskGroup> toRemove = new ArrayList<>();
|
||||
|
||||
for (TaskGroup group : taskGroupList) {
|
||||
boolean foundSuccess = false, entireTaskGroupFailed = false;
|
||||
|
@ -1793,7 +1792,7 @@ public class KafkaSupervisor implements Supervisor
|
|||
|
||||
private void checkCurrentTaskState() throws ExecutionException, InterruptedException, TimeoutException
|
||||
{
|
||||
List<ListenableFuture<?>> futures = Lists.newArrayList();
|
||||
List<ListenableFuture<?>> futures = new ArrayList<>();
|
||||
Iterator<Entry<Integer, TaskGroup>> iTaskGroups = taskGroups.entrySet().iterator();
|
||||
while (iTaskGroups.hasNext()) {
|
||||
Entry<Integer, TaskGroup> taskGroupEntry = iTaskGroups.next();
|
||||
|
@ -1914,7 +1913,7 @@ public class KafkaSupervisor implements Supervisor
|
|||
}
|
||||
TaskGroup group = taskGroups.get(groupId);
|
||||
|
||||
Map<String, Object> consumerProperties = Maps.newHashMap(ioConfig.getConsumerProperties());
|
||||
Map<String, Object> consumerProperties = new HashMap<>(ioConfig.getConsumerProperties());
|
||||
DateTime minimumMessageTime = taskGroups.get(groupId).minimumMessageTime.orNull();
|
||||
DateTime maximumMessageTime = taskGroups.get(groupId).maximumMessageTime.orNull();
|
||||
|
||||
|
@ -2101,7 +2100,7 @@ public class KafkaSupervisor implements Supervisor
|
|||
return Futures.immediateFuture(null);
|
||||
}
|
||||
|
||||
final List<ListenableFuture<Void>> futures = Lists.newArrayList();
|
||||
final List<ListenableFuture<Void>> futures = new ArrayList<>();
|
||||
for (Entry<String, TaskData> entry : taskGroup.tasks.entrySet()) {
|
||||
final String taskId = entry.getKey();
|
||||
final TaskData taskData = entry.getValue();
|
||||
|
@ -2184,7 +2183,7 @@ public class KafkaSupervisor implements Supervisor
|
|||
payload
|
||||
);
|
||||
|
||||
List<TaskReportData> taskReports = Lists.newArrayList();
|
||||
List<TaskReportData> taskReports = new ArrayList<>();
|
||||
|
||||
try {
|
||||
for (TaskGroup taskGroup : taskGroups.values()) {
|
||||
|
@ -2392,7 +2391,7 @@ public class KafkaSupervisor implements Supervisor
|
|||
private Map<String, Map<String, Object>> getCurrentTotalStats()
|
||||
throws InterruptedException, ExecutionException, TimeoutException
|
||||
{
|
||||
Map<String, Map<String, Object>> allStats = Maps.newHashMap();
|
||||
Map<String, Map<String, Object>> allStats = new HashMap<>();
|
||||
final List<ListenableFuture<StatsFromTaskResult>> futures = new ArrayList<>();
|
||||
final List<Pair<Integer, String>> groupAndTaskIds = new ArrayList<>();
|
||||
|
||||
|
@ -2438,7 +2437,7 @@ public class KafkaSupervisor implements Supervisor
|
|||
for (int i = 0; i < results.size(); i++) {
|
||||
StatsFromTaskResult result = results.get(i);
|
||||
if (result != null) {
|
||||
Map<String, Object> groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> Maps.newHashMap());
|
||||
Map<String, Object> groupMap = allStats.computeIfAbsent(result.getGroupId(), k -> new HashMap<>());
|
||||
groupMap.put(result.getTaskId(), result.getStats());
|
||||
} else {
|
||||
Pair<Integer, String> groupAndTaskId = groupAndTaskIds.get(i);
|
||||
|
|
|
@ -59,6 +59,7 @@ import org.junit.runners.Parameterized;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -629,8 +630,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
).times(numRequests);
|
||||
replayAll();
|
||||
|
||||
List<URL> expectedUrls = Lists.newArrayList();
|
||||
List<ListenableFuture<Boolean>> futures = Lists.newArrayList();
|
||||
List<URL> expectedUrls = new ArrayList<>();
|
||||
List<ListenableFuture<Boolean>> futures = new ArrayList<>();
|
||||
for (String testId : TEST_IDS) {
|
||||
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "stop")));
|
||||
futures.add(client.stopAsync(testId, false));
|
||||
|
@ -661,8 +662,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
).times(numRequests);
|
||||
replayAll();
|
||||
|
||||
List<URL> expectedUrls = Lists.newArrayList();
|
||||
List<ListenableFuture<Boolean>> futures = Lists.newArrayList();
|
||||
List<URL> expectedUrls = new ArrayList<>();
|
||||
List<ListenableFuture<Boolean>> futures = new ArrayList<>();
|
||||
for (String testId : TEST_IDS) {
|
||||
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "resume")));
|
||||
futures.add(client.resumeAsync(testId));
|
||||
|
@ -694,8 +695,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
).times(numRequests);
|
||||
replayAll();
|
||||
|
||||
List<URL> expectedUrls = Lists.newArrayList();
|
||||
List<ListenableFuture<Map<Integer, Long>>> futures = Lists.newArrayList();
|
||||
List<URL> expectedUrls = new ArrayList<>();
|
||||
List<ListenableFuture<Map<Integer, Long>>> futures = new ArrayList<>();
|
||||
for (String testId : TEST_IDS) {
|
||||
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "pause")));
|
||||
futures.add(client.pauseAsync(testId));
|
||||
|
@ -727,8 +728,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
).times(numRequests);
|
||||
replayAll();
|
||||
|
||||
List<URL> expectedUrls = Lists.newArrayList();
|
||||
List<ListenableFuture<KafkaIndexTask.Status>> futures = Lists.newArrayList();
|
||||
List<URL> expectedUrls = new ArrayList<>();
|
||||
List<ListenableFuture<KafkaIndexTask.Status>> futures = new ArrayList<>();
|
||||
for (String testId : TEST_IDS) {
|
||||
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "status")));
|
||||
futures.add(client.getStatusAsync(testId));
|
||||
|
@ -761,8 +762,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
).times(numRequests);
|
||||
replayAll();
|
||||
|
||||
List<URL> expectedUrls = Lists.newArrayList();
|
||||
List<ListenableFuture<DateTime>> futures = Lists.newArrayList();
|
||||
List<URL> expectedUrls = new ArrayList<>();
|
||||
List<ListenableFuture<DateTime>> futures = new ArrayList<>();
|
||||
for (String testId : TEST_IDS) {
|
||||
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "time/start")));
|
||||
futures.add(client.getStartTimeAsync(testId));
|
||||
|
@ -794,8 +795,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
).times(numRequests);
|
||||
replayAll();
|
||||
|
||||
List<URL> expectedUrls = Lists.newArrayList();
|
||||
List<ListenableFuture<Map<Integer, Long>>> futures = Lists.newArrayList();
|
||||
List<URL> expectedUrls = new ArrayList<>();
|
||||
List<ListenableFuture<Map<Integer, Long>>> futures = new ArrayList<>();
|
||||
for (String testId : TEST_IDS) {
|
||||
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "offsets/current")));
|
||||
futures.add(client.getCurrentOffsetsAsync(testId, false));
|
||||
|
@ -827,8 +828,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
).times(numRequests);
|
||||
replayAll();
|
||||
|
||||
List<URL> expectedUrls = Lists.newArrayList();
|
||||
List<ListenableFuture<Map<Integer, Long>>> futures = Lists.newArrayList();
|
||||
List<URL> expectedUrls = new ArrayList<>();
|
||||
List<ListenableFuture<Map<Integer, Long>>> futures = new ArrayList<>();
|
||||
for (String testId : TEST_IDS) {
|
||||
expectedUrls.add(new URL(StringUtils.format(URL_FORMATTER, TEST_HOST, TEST_PORT, testId, "offsets/end")));
|
||||
futures.add(client.getEndOffsetsAsync(testId));
|
||||
|
@ -860,8 +861,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
).times(numRequests);
|
||||
replayAll();
|
||||
|
||||
List<URL> expectedUrls = Lists.newArrayList();
|
||||
List<ListenableFuture<Boolean>> futures = Lists.newArrayList();
|
||||
List<URL> expectedUrls = new ArrayList<>();
|
||||
List<ListenableFuture<Boolean>> futures = new ArrayList<>();
|
||||
for (String testId : TEST_IDS) {
|
||||
expectedUrls.add(new URL(StringUtils.format(
|
||||
URL_FORMATTER,
|
||||
|
@ -899,8 +900,8 @@ public class KafkaIndexTaskClientTest extends EasyMockSupport
|
|||
).times(numRequests);
|
||||
replayAll();
|
||||
|
||||
List<URL> expectedUrls = Lists.newArrayList();
|
||||
List<ListenableFuture<Boolean>> futures = Lists.newArrayList();
|
||||
List<URL> expectedUrls = new ArrayList<>();
|
||||
List<ListenableFuture<Boolean>> futures = new ArrayList<>();
|
||||
for (String testId : TEST_IDS) {
|
||||
expectedUrls.add(
|
||||
new URL(
|
||||
|
|
|
@ -30,8 +30,6 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.io.Files;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
|
@ -162,8 +160,10 @@ import java.io.IOException;
|
|||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
@ -189,7 +189,7 @@ public class KafkaIndexTaskTest
|
|||
private static ListeningExecutorService taskExec;
|
||||
private static int topicPostfix;
|
||||
|
||||
private final List<Task> runningTasks = Lists.newArrayList();
|
||||
private final List<Task> runningTasks = new ArrayList<>();
|
||||
|
||||
private long handoffConditionTimeout = 0;
|
||||
private boolean reportParseExceptions = false;
|
||||
|
@ -210,7 +210,7 @@ public class KafkaIndexTaskTest
|
|||
private String topic;
|
||||
private List<ProducerRecord<byte[], byte[]>> records;
|
||||
private final boolean isIncrementalHandoffSupported;
|
||||
private final Set<Integer> checkpointRequestsHash = Sets.newHashSet();
|
||||
private final Set<Integer> checkpointRequestsHash = new HashSet<>();
|
||||
private File reportsFile;
|
||||
private RowIngestionMetersFactory rowIngestionMetersFactory;
|
||||
|
||||
|
@ -2234,7 +2234,7 @@ public class KafkaIndexTaskTest
|
|||
@Override
|
||||
public List<StorageLocationConfig> getLocations()
|
||||
{
|
||||
return Lists.newArrayList();
|
||||
return new ArrayList<>();
|
||||
}
|
||||
};
|
||||
toolboxFactory = new TaskToolboxFactory(
|
||||
|
@ -2335,7 +2335,7 @@ public class KafkaIndexTaskTest
|
|||
IndexIO indexIO = new TestUtils().getTestIndexIO();
|
||||
QueryableIndex index = indexIO.loadIndex(outputLocation);
|
||||
DictionaryEncodedColumn<String> theColumn = (DictionaryEncodedColumn<String>) index.getColumnHolder(column).getColumn();
|
||||
List<String> values = Lists.newArrayList();
|
||||
List<String> values = new ArrayList<>();
|
||||
for (int i = 0; i < theColumn.length(); i++) {
|
||||
int id = theColumn.getSingleValueRow(i);
|
||||
String value = theColumn.lookupName(id);
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.indexing.kafka.test;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
import kafka.server.KafkaConfig;
|
||||
import kafka.server.KafkaServer;
|
||||
|
@ -38,6 +37,7 @@ import javax.annotation.Nullable;
|
|||
import java.io.Closeable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
@ -102,7 +102,7 @@ public class TestBroker implements Closeable
|
|||
|
||||
public Map<String, String> producerProperties()
|
||||
{
|
||||
final Map<String, String> props = Maps.newHashMap();
|
||||
final Map<String, String> props = new HashMap<>();
|
||||
props.put("bootstrap.servers", StringUtils.format("localhost:%d", getPort()));
|
||||
props.put("key.serializer", ByteArraySerializer.class.getName());
|
||||
props.put("value.serializer", ByteArraySerializer.class.getName());
|
||||
|
@ -112,7 +112,7 @@ public class TestBroker implements Closeable
|
|||
|
||||
public Map<String, Object> consumerProperties()
|
||||
{
|
||||
final Map<String, Object> props = Maps.newHashMap();
|
||||
final Map<String, Object> props = new HashMap<>();
|
||||
props.put("bootstrap.servers", StringUtils.format("localhost:%d", getPort()));
|
||||
props.put("key.deserializer", ByteArrayDeserializer.class.getName());
|
||||
props.put("value.deserializer", ByteArrayDeserializer.class.getName());
|
||||
|
|
|
@ -36,7 +36,6 @@ import com.google.common.base.Throwables;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Ordering;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
|
@ -354,7 +353,7 @@ public class S3DataSegmentFinderTest
|
|||
private static class MockAmazonS3Client extends ServerSideEncryptingAmazonS3
|
||||
{
|
||||
private final File baseDir;
|
||||
private final Map<String, Map<String, ObjectMetadata>> storage = Maps.newHashMap();
|
||||
private final Map<String, Map<String, ObjectMetadata>> storage = new HashMap<>();
|
||||
|
||||
public MockAmazonS3Client(File baseDir)
|
||||
{
|
||||
|
|
|
@ -35,8 +35,6 @@ import com.amazonaws.services.s3.model.S3ObjectSummary;
|
|||
import com.amazonaws.services.s3.model.StorageClass;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.druid.java.util.common.Intervals;
|
||||
import org.apache.druid.java.util.common.MapUtils;
|
||||
import org.apache.druid.segment.loading.SegmentLoadingException;
|
||||
|
@ -46,6 +44,7 @@ import org.junit.Assert;
|
|||
import org.junit.Test;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -182,7 +181,7 @@ public class S3DataSegmentMoverTest
|
|||
|
||||
private static class MockAmazonS3Client extends ServerSideEncryptingAmazonS3
|
||||
{
|
||||
Map<String, Set<String>> storage = Maps.newHashMap();
|
||||
Map<String, Set<String>> storage = new HashMap<>();
|
||||
boolean copied = false;
|
||||
boolean deletedOld = false;
|
||||
|
||||
|
@ -271,7 +270,7 @@ public class S3DataSegmentMoverTest
|
|||
public PutObjectResult putObject(String bucketName, String key, File file)
|
||||
{
|
||||
if (!storage.containsKey(bucketName)) {
|
||||
storage.put(bucketName, Sets.newHashSet());
|
||||
storage.put(bucketName, new HashSet<>());
|
||||
}
|
||||
storage.get(bucketName).add(key);
|
||||
return new PutObjectResult();
|
||||
|
|
|
@ -27,8 +27,6 @@ import com.amazonaws.services.s3.model.Permission;
|
|||
import com.amazonaws.services.s3.model.PutObjectRequest;
|
||||
import com.amazonaws.services.s3.model.PutObjectResult;
|
||||
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 org.apache.commons.io.IOUtils;
|
||||
import org.apache.druid.jackson.DefaultObjectMapper;
|
||||
|
@ -45,6 +43,8 @@ import org.junit.rules.TemporaryFolder;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -133,9 +133,9 @@ public class S3DataSegmentPusherTest
|
|||
"foo",
|
||||
Intervals.of("2015/2016"),
|
||||
"0",
|
||||
Maps.newHashMap(),
|
||||
Lists.newArrayList(),
|
||||
Lists.newArrayList(),
|
||||
new HashMap<>(),
|
||||
new ArrayList<>(),
|
||||
new ArrayList<>(),
|
||||
NoneShardSpec.instance(),
|
||||
0,
|
||||
size
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue