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:
QiuMM 2018-10-29 20:02:43 +08:00 committed by Roman Leventov
parent 0a90b3d51a
commit 676f5e6d7f
428 changed files with 1806 additions and 1795 deletions

View File

@ -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;

View File

@ -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();
}

View File

@ -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();
}

View File

@ -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++) {

View File

@ -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();
}

View File

@ -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(

View File

@ -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();

View File

@ -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();

View File

@ -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) {

View File

@ -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();

View File

@ -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();

View File

@ -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;

View File

@ -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

View File

@ -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) {

View File

@ -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);

View File

@ -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();

View File

@ -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)

View File

@ -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);

View File

@ -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()

View File

@ -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());

View File

@ -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()) {

View File

@ -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());
}

View File

@ -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()) {

View File

@ -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,

View File

@ -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

View File

@ -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);

View File

@ -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));
}

View File

@ -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()) {

View File

@ -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)

View File

@ -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<>();
}
}
);

View File

@ -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()
{

View File

@ -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;

View File

@ -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) {

View File

@ -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);
}

View File

@ -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));
}

View File

@ -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(

View File

@ -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()

View File

@ -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>>()

View File

@ -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"),

View File

@ -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,

View File

@ -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();

View File

@ -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();

View File

@ -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,

View File

@ -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();

View File

@ -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"),

View File

@ -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);
}

View File

@ -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);
}

View File

@ -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(

View File

@ -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;

View File

@ -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()) {

View File

@ -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()) {

View File

@ -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) {

View File

@ -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();
}

View File

@ -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

View File

@ -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

View File

@ -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());

View File

@ -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

View File

@ -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();
}

View File

@ -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()
{

View File

@ -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);

View File

@ -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);

View File

@ -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(

View File

@ -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(),

View File

@ -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)

View File

@ -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();

View File

@ -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();

View File

@ -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));
}

View File

@ -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());
}

View File

@ -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())

View File

@ -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);

View File

@ -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")),

View File

@ -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());
}

View File

@ -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;
}

View File

@ -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());
}

View File

@ -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});
}

View File

@ -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});
}

View File

@ -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});
}

View File

@ -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});
}

View File

@ -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});
}

View File

@ -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});
}

View File

@ -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);

View File

@ -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();

View File

@ -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();

View File

@ -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();

View File

@ -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();

View File

@ -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))

View File

@ -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

View File

@ -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);

View File

@ -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});
}

View File

@ -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,

View File

@ -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,

View File

@ -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())) {

View File

@ -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,

View File

@ -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);

View File

@ -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(

View File

@ -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);

View File

@ -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());

View File

@ -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)
{

View File

@ -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();

View File

@ -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