Improve `collection` related things that reusing a immutable object instead of creating a new object (#4135)

This commit is contained in:
Benedict Jin 2017-05-17 00:38:51 +08:00 committed by Gian Merlino
parent e4add598f0
commit e823085866
99 changed files with 579 additions and 573 deletions

View File

@ -23,6 +23,7 @@ import com.google.common.collect.Lists;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
public class CSVParseSpecTest public class CSVParseSpecTest
{ {
@ -41,7 +42,7 @@ public class CSVParseSpecTest
Lists.<SpatialDimensionSchema>newArrayList() Lists.<SpatialDimensionSchema>newArrayList()
), ),
",", ",",
Arrays.asList("a"), Collections.singletonList("a"),
false, false,
0 0
); );
@ -62,7 +63,7 @@ public class CSVParseSpecTest
Lists.<SpatialDimensionSchema>newArrayList() Lists.<SpatialDimensionSchema>newArrayList()
), ),
",", ",",
Arrays.asList("a"), Collections.singletonList("a"),
false, false,
0 0
); );

View File

@ -27,6 +27,7 @@ import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
public class DelimitedParseSpecTest public class DelimitedParseSpecTest
{ {
@ -37,10 +38,10 @@ public class DelimitedParseSpecTest
{ {
DelimitedParseSpec spec = new DelimitedParseSpec( DelimitedParseSpec spec = new DelimitedParseSpec(
new TimestampSpec("abc", "iso", null), new TimestampSpec("abc", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null),
"\u0001", "\u0001",
"\u0002", "\u0002",
Arrays.asList("abc"), Collections.singletonList("abc"),
false, false,
0 0
); );
@ -51,10 +52,10 @@ public class DelimitedParseSpecTest
Assert.assertEquals("abc", serde.getTimestampSpec().getTimestampColumn()); Assert.assertEquals("abc", serde.getTimestampSpec().getTimestampColumn());
Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat()); Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat());
Assert.assertEquals(Arrays.asList("abc"), serde.getColumns()); Assert.assertEquals(Collections.singletonList("abc"), serde.getColumns());
Assert.assertEquals("\u0001", serde.getDelimiter()); Assert.assertEquals("\u0001", serde.getDelimiter());
Assert.assertEquals("\u0002", serde.getListDelimiter()); Assert.assertEquals("\u0002", serde.getListDelimiter());
Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames()); Assert.assertEquals(Collections.singletonList("abc"), serde.getDimensionsSpec().getDimensionNames());
} }
@Test(expected = IllegalArgumentException.class) @Test(expected = IllegalArgumentException.class)
@ -73,7 +74,7 @@ public class DelimitedParseSpecTest
), ),
",", ",",
" ", " ",
Arrays.asList("a"), Collections.singletonList("a"),
false, false,
0 0
); );
@ -95,7 +96,7 @@ public class DelimitedParseSpecTest
), ),
",", ",",
null, null,
Arrays.asList("a"), Collections.singletonList("a"),
false, false,
0 0
); );

View File

@ -59,7 +59,7 @@ public class FileIteratingFirehoseTest
final List<Object[]> args = new ArrayList<>(); final List<Object[]> args = new ArrayList<>();
for (int numSkipHeadRows = 0; numSkipHeadRows < 3; numSkipHeadRows++) { for (int numSkipHeadRows = 0; numSkipHeadRows < 3; numSkipHeadRows++) {
for (List<String> texts : inputTexts) { for (List<String> texts : inputTexts) {
args.add(new Object[] {texts, numSkipHeadRows}); args.add(new Object[] { texts, numSkipHeadRows });
} }
} }
@ -86,26 +86,26 @@ public class FileIteratingFirehoseTest
this.inputs = texts; this.inputs = texts;
this.expectedResults = inputs.stream() this.expectedResults = inputs.stream()
.map(input -> input.split("\n")) .map(input -> input.split("\n"))
.flatMap(lines -> { .flatMap(lines -> {
final List<String> filteredLines = Arrays.asList(lines).stream() final List<String> filteredLines = Arrays.asList(lines).stream()
.filter(line -> line.length() > 0) .filter(line -> line.length() > 0)
.map(line -> line.split(",")[1]) .map(line -> line.split(",")[1])
.collect(Collectors.toList()); .collect(Collectors.toList());
final int numRealSkippedRows = Math.min(filteredLines.size(), numSkipHeaderRows); final int numRealSkippedRows = Math.min(filteredLines.size(), numSkipHeaderRows);
IntStream.range(0, numRealSkippedRows).forEach(i -> filteredLines.set(i, null)); IntStream.range(0, numRealSkippedRows).forEach(i -> filteredLines.set(i, null));
return filteredLines.stream(); return filteredLines.stream();
}) })
.collect(Collectors.toList()); .collect(Collectors.toList());
} }
@Test @Test
public void testFirehose() throws Exception public void testFirehose() throws Exception
{ {
final List<LineIterator> lineIterators = inputs.stream() final List<LineIterator> lineIterators = inputs.stream()
.map(s -> new LineIterator(new StringReader(s))) .map(s -> new LineIterator(new StringReader(s)))
.collect(Collectors.toList()); .collect(Collectors.toList());
try (final FileIteratingFirehose firehose = new FileIteratingFirehose(lineIterators.iterator(), parser)) { try (final FileIteratingFirehose firehose = new FileIteratingFirehose(lineIterators.iterator(), parser)) {
final List<String> results = Lists.newArrayList(); final List<String> results = Lists.newArrayList();

View File

@ -22,7 +22,6 @@ package io.druid.data.input.impl;
import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.TestObjectMapper; import io.druid.TestObjectMapper;
import io.druid.java.util.common.parsers.Parser; import io.druid.java.util.common.parsers.Parser;
import io.druid.js.JavaScriptConfig; import io.druid.js.JavaScriptConfig;
@ -32,7 +31,7 @@ import org.junit.Test;
import org.junit.rules.ExpectedException; import org.junit.rules.ExpectedException;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Collections;
import java.util.Map; import java.util.Map;
/** /**
@ -55,7 +54,7 @@ public class JavaScriptParseSpecTest
); );
JavaScriptParseSpec spec = new JavaScriptParseSpec( JavaScriptParseSpec spec = new JavaScriptParseSpec(
new TimestampSpec("abc", "iso", null), new TimestampSpec("abc", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null),
"abc", "abc",
JavaScriptConfig.getEnabledInstance() JavaScriptConfig.getEnabledInstance()
); );
@ -67,7 +66,7 @@ public class JavaScriptParseSpecTest
Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat()); Assert.assertEquals("iso", serde.getTimestampSpec().getTimestampFormat());
Assert.assertEquals("abc", serde.getFunction()); Assert.assertEquals("abc", serde.getFunction());
Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames()); Assert.assertEquals(Collections.singletonList("abc"), serde.getDimensionsSpec().getDimensionNames());
} }
@Test @Test
@ -76,7 +75,7 @@ public class JavaScriptParseSpecTest
final JavaScriptConfig config = JavaScriptConfig.getEnabledInstance(); final JavaScriptConfig config = JavaScriptConfig.getEnabledInstance();
JavaScriptParseSpec spec = new JavaScriptParseSpec( JavaScriptParseSpec spec = new JavaScriptParseSpec(
new TimestampSpec("abc", "iso", null), new TimestampSpec("abc", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null),
"function(str) { var parts = str.split(\"-\"); return { one: parts[0], two: parts[1] } }", "function(str) { var parts = str.split(\"-\"); return { one: parts[0], two: parts[1] } }",
config config
); );
@ -92,7 +91,7 @@ public class JavaScriptParseSpecTest
final JavaScriptConfig config = new JavaScriptConfig(false); final JavaScriptConfig config = new JavaScriptConfig(false);
JavaScriptParseSpec spec = new JavaScriptParseSpec( JavaScriptParseSpec spec = new JavaScriptParseSpec(
new TimestampSpec("abc", "iso", null), new TimestampSpec("abc", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null),
"abc", "abc",
config config
); );

View File

@ -26,6 +26,7 @@ import io.druid.java.util.common.parsers.ParseException;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
public class ParseSpecTest public class ParseSpecTest
{ {
@ -83,7 +84,7 @@ public class ParseSpecTest
null null
), ),
new DimensionsSpec( new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList("a")), DimensionsSpec.getDefaultSchemas(Collections.singletonList("a")),
Lists.newArrayList("B", "B"), Lists.newArrayList("B", "B"),
Lists.<SpatialDimensionSchema>newArrayList() Lists.<SpatialDimensionSchema>newArrayList()
), ),

View File

@ -25,7 +25,7 @@ import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Collections;
/** /**
*/ */
@ -38,9 +38,9 @@ public class RegexParseSpecTest
{ {
RegexParseSpec spec = new RegexParseSpec( RegexParseSpec spec = new RegexParseSpec(
new TimestampSpec("abc", "iso", null), new TimestampSpec("abc", "iso", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("abc")), null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null),
"\u0001", "\u0001",
Arrays.asList("abc"), Collections.singletonList("abc"),
"abc" "abc"
); );
final RegexParseSpec serde = jsonMapper.readValue( final RegexParseSpec serde = jsonMapper.readValue(
@ -52,6 +52,6 @@ public class RegexParseSpecTest
Assert.assertEquals("abc", serde.getPattern()); Assert.assertEquals("abc", serde.getPattern());
Assert.assertEquals("\u0001", serde.getListDelimiter()); Assert.assertEquals("\u0001", serde.getListDelimiter());
Assert.assertEquals(Arrays.asList("abc"), serde.getDimensionsSpec().getDimensionNames()); Assert.assertEquals(Collections.singletonList("abc"), serde.getDimensionsSpec().getDimensionNames());
} }
} }

View File

@ -32,6 +32,7 @@ import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Properties; import java.util.Properties;
/** /**
@ -46,7 +47,7 @@ public class PolyBindTest
props = new Properties(); props = new Properties();
injector = Guice.createInjector( injector = Guice.createInjector(
Iterables.concat( Iterables.concat(
Arrays.asList( Collections.singletonList(
new Module() new Module()
{ {
@Override @Override

View File

@ -95,6 +95,7 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -128,7 +129,11 @@ public class FilteredAggregatorBenchmark
private File tmpDir; private File tmpDir;
private static String JS_FN = "function(str) { return 'super-' + str; }"; private static String JS_FN = "function(str) { return 'super-' + str; }";
private static ExtractionFn JS_EXTRACTION_FN = new JavaScriptExtractionFn(JS_FN, false, JavaScriptConfig.getEnabledInstance()); private static ExtractionFn JS_EXTRACTION_FN = new JavaScriptExtractionFn(
JS_FN,
false,
JavaScriptConfig.getEnabledInstance()
);
static { static {
JSON_MAPPER = new DefaultObjectMapper(); JSON_MAPPER = new DefaultObjectMapper();
@ -169,10 +174,15 @@ public class FilteredAggregatorBenchmark
filter = new OrDimFilter( filter = new OrDimFilter(
Arrays.asList( Arrays.asList(
new BoundDimFilter("dimSequential", "-1", "-1", true, true, null, null, StringComparators.ALPHANUMERIC), new BoundDimFilter("dimSequential", "-1", "-1", true, true, null, null, StringComparators.ALPHANUMERIC),
new JavaScriptDimFilter("dimSequential", "function(x) { return false }", null, JavaScriptConfig.getEnabledInstance()), new JavaScriptDimFilter(
"dimSequential",
"function(x) { return false }",
null,
JavaScriptConfig.getEnabledInstance()
),
new RegexDimFilter("dimSequential", "X", null), new RegexDimFilter("dimSequential", "X", null),
new SearchQueryDimFilter("dimSequential", new ContainsSearchQuerySpec("X", false), null), new SearchQueryDimFilter("dimSequential", new ContainsSearchQuerySpec("X", false), null),
new InDimFilter("dimSequential", Arrays.asList("X"), null) new InDimFilter("dimSequential", Collections.singletonList("X"), null)
) )
); );
filteredMetrics = new AggregatorFactory[1]; filteredMetrics = new AggregatorFactory[1];
@ -208,7 +218,7 @@ public class FilteredAggregatorBenchmark
); );
BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic");
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(filteredMetrics[0]); queryAggs.add(filteredMetrics[0]);

View File

@ -96,7 +96,7 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Collections;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -174,7 +174,7 @@ public class GroupByTypeInterfaceBenchmark
BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic");
{ // basic.A { // basic.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory( queryAggs.add(new LongSumAggregatorFactory(
"sumLongSequential", "sumLongSequential",
@ -240,7 +240,7 @@ public class GroupByTypeInterfaceBenchmark
} }
{ // basic.nested { // basic.nested
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory( queryAggs.add(new LongSumAggregatorFactory(
"sumLongSequential", "sumLongSequential",

View File

@ -89,7 +89,7 @@ import org.openjdk.jmh.infra.Blackhole;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Collections;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -159,7 +159,7 @@ public class TopNTypeInterfaceBenchmark
BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic");
{ // basic.A { // basic.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
@ -199,7 +199,7 @@ public class TopNTypeInterfaceBenchmark
basicQueries.put("float", queryBuilderFloat); basicQueries.put("float", queryBuilderFloat);
} }
{ // basic.numericSort { // basic.numericSort
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
@ -215,7 +215,7 @@ public class TopNTypeInterfaceBenchmark
basicQueries.put("numericSort", queryBuilderA); basicQueries.put("numericSort", queryBuilderA);
} }
{ // basic.alphanumericSort { // basic.alphanumericSort
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));

View File

@ -22,7 +22,7 @@ package io.druid.benchmark.datagen;
import org.apache.commons.math3.distribution.EnumeratedDistribution; import org.apache.commons.math3.distribution.EnumeratedDistribution;
import org.apache.commons.math3.util.Pair; import org.apache.commons.math3.util.Pair;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
public class SequentialDistribution extends EnumeratedDistribution public class SequentialDistribution extends EnumeratedDistribution
@ -37,7 +37,7 @@ public class SequentialDistribution extends EnumeratedDistribution
public SequentialDistribution(Integer start, Integer end, List<Object> enumeratedValues) public SequentialDistribution(Integer start, Integer end, List<Object> enumeratedValues)
{ {
// just pass in some bogus probability mass function, we won't be using it // just pass in some bogus probability mass function, we won't be using it
super(Arrays.asList(new Pair<Object, Double>(null, 1.0))); super(Collections.singletonList(new Pair<Object, Double>(null, 1.0)));
this.start = start; this.start = start;
this.end = end; this.end = end;
this.enumeratedValues = enumeratedValues; this.enumeratedValues = enumeratedValues;

View File

@ -67,6 +67,7 @@ import org.openjdk.jmh.infra.Blackhole;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -173,7 +174,7 @@ public class IncrementalIndexReadBenchmark
new JavaScriptDimFilter("dimSequential", "function(x) { return false }", null, JavaScriptConfig.getEnabledInstance()), new JavaScriptDimFilter("dimSequential", "function(x) { return false }", null, JavaScriptConfig.getEnabledInstance()),
new RegexDimFilter("dimSequential", "X", null), new RegexDimFilter("dimSequential", "X", null),
new SearchQueryDimFilter("dimSequential", new ContainsSearchQuerySpec("X", false), null), new SearchQueryDimFilter("dimSequential", new ContainsSearchQuerySpec("X", false), null),
new InDimFilter("dimSequential", Arrays.asList("X"), null) new InDimFilter("dimSequential", Collections.singletonList("X"), null)
) )
); );

View File

@ -98,7 +98,7 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Collections;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -174,7 +174,7 @@ public class GroupByBenchmark
BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic");
{ // basic.A { // basic.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory( queryAggs.add(new LongSumAggregatorFactory(
"sumLongSequential", "sumLongSequential",
@ -202,7 +202,7 @@ public class GroupByBenchmark
} }
{ // basic.nested { // basic.nested
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory( queryAggs.add(new LongSumAggregatorFactory(
"sumLongSequential", "sumLongSequential",
@ -246,7 +246,7 @@ public class GroupByBenchmark
BenchmarkSchemaInfo simpleSchema = BenchmarkSchemas.SCHEMA_MAP.get("simple"); BenchmarkSchemaInfo simpleSchema = BenchmarkSchemas.SCHEMA_MAP.get("simple");
{ // simple.A { // simple.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(simpleSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory( queryAggs.add(new LongSumAggregatorFactory(
"rows", "rows",
@ -273,7 +273,7 @@ public class GroupByBenchmark
Map<String, GroupByQuery> simpleLongQueries = new LinkedHashMap<>(); Map<String, GroupByQuery> simpleLongQueries = new LinkedHashMap<>();
BenchmarkSchemaInfo simpleLongSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleLong"); BenchmarkSchemaInfo simpleLongSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleLong");
{ // simpleLong.A { // simpleLong.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleLongSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(simpleLongSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory( queryAggs.add(new LongSumAggregatorFactory(
"rows", "rows",
@ -300,7 +300,7 @@ public class GroupByBenchmark
Map<String, GroupByQuery> simpleFloatQueries = new LinkedHashMap<>(); Map<String, GroupByQuery> simpleFloatQueries = new LinkedHashMap<>();
BenchmarkSchemaInfo simpleFloatSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleFloat"); BenchmarkSchemaInfo simpleFloatSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleFloat");
{ // simpleFloat.A { // simpleFloat.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleFloatSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(simpleFloatSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory( queryAggs.add(new LongSumAggregatorFactory(
"rows", "rows",

View File

@ -97,7 +97,7 @@ import org.openjdk.jmh.infra.Blackhole;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Collections;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -188,7 +188,7 @@ public class SearchBenchmark
private static SearchQueryBuilder basicA(final BenchmarkSchemaInfo basicSchema) private static SearchQueryBuilder basicA(final BenchmarkSchemaInfo basicSchema)
{ {
final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
return Druids.newSearchQueryBuilder() return Druids.newSearchQueryBuilder()
.dataSource("blah") .dataSource("blah")
@ -199,7 +199,7 @@ public class SearchBenchmark
private static SearchQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) private static SearchQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema)
{ {
final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
final List<String> dimUniformFilterVals = Lists.newArrayList(); final List<String> dimUniformFilterVals = Lists.newArrayList();
int resultNum = (int) (100000 * 0.1); int resultNum = (int) (100000 * 0.1);
@ -230,7 +230,7 @@ public class SearchBenchmark
private static SearchQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema) private static SearchQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema)
{ {
final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
final List<String> dimUniformFilterVals = Lists.newArrayList(); final List<String> dimUniformFilterVals = Lists.newArrayList();
final int resultNum = (int) (100000 * 0.1); final int resultNum = (int) (100000 * 0.1);
@ -284,7 +284,7 @@ public class SearchBenchmark
private static SearchQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) private static SearchQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema)
{ {
final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
final List<String> dimUniformFilterVals = Lists.newArrayList(); final List<String> dimUniformFilterVals = Lists.newArrayList();
final int resultNum = (int) (100000 * 0.1); final int resultNum = (int) (100000 * 0.1);

View File

@ -88,6 +88,7 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
@ -156,7 +157,7 @@ public class SelectBenchmark
BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic");
{ // basic.A { // basic.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
Druids.SelectQueryBuilder queryBuilderA = Druids.SelectQueryBuilder queryBuilderA =
Druids.newSelectQueryBuilder() Druids.newSelectQueryBuilder()

View File

@ -92,7 +92,7 @@ import org.openjdk.jmh.infra.Blackhole;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Collections;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -155,7 +155,7 @@ public class TimeseriesBenchmark
BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic");
{ // basic.A { // basic.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
@ -176,7 +176,7 @@ public class TimeseriesBenchmark
basicQueries.put("A", queryA); basicQueries.put("A", queryA);
} }
{ {
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"); LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential");
@ -196,7 +196,7 @@ public class TimeseriesBenchmark
basicQueries.put("timeFilterNumeric", timeFilterQuery); basicQueries.put("timeFilterNumeric", timeFilterQuery);
} }
{ {
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"); LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential");
@ -216,7 +216,7 @@ public class TimeseriesBenchmark
basicQueries.put("timeFilterAlphanumeric", timeFilterQuery); basicQueries.put("timeFilterAlphanumeric", timeFilterQuery);
} }
{ {
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(new Interval(200000, 300000))); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(new Interval(200000, 300000)));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"); LongSumAggregatorFactory lsaf = new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential");
queryAggs.add(lsaf); queryAggs.add(lsaf);

View File

@ -89,7 +89,7 @@ import org.openjdk.jmh.infra.Blackhole;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Collections;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -156,7 +156,7 @@ public class TopNBenchmark
BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic");
{ // basic.A { // basic.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
@ -176,7 +176,7 @@ public class TopNBenchmark
basicQueries.put("A", queryBuilderA); basicQueries.put("A", queryBuilderA);
} }
{ // basic.numericSort { // basic.numericSort
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
@ -192,7 +192,7 @@ public class TopNBenchmark
basicQueries.put("numericSort", queryBuilderA); basicQueries.put("numericSort", queryBuilderA);
} }
{ // basic.alphanumericSort { // basic.alphanumericSort
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval())); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>(); List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")); queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));

View File

@ -42,7 +42,7 @@ public class WrappedConciseBitmap implements MutableBitmap
} }
/** /**
* Create a bitmap wrappign the given bitmap * Create a bitmap wrapping the given bitmap
* *
* @param conciseSet bitmap to be wrapped * @param conciseSet bitmap to be wrapped
*/ */

View File

@ -37,6 +37,7 @@ import org.junit.Test;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -367,7 +368,7 @@ public class OrderedMergeSequenceTest
) )
); );
SequenceTestHelper.testAll(finalMerged, Arrays.asList(1)); SequenceTestHelper.testAll(finalMerged, Collections.singletonList(1));
} }
@Test @Test
@ -379,6 +380,6 @@ public class OrderedMergeSequenceTest
) )
); );
SequenceTestHelper.testAll(seq1, Arrays.asList(1)); SequenceTestHelper.testAll(seq1, Collections.singletonList(1));
} }
} }

View File

@ -40,6 +40,7 @@ import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
@ -92,11 +93,11 @@ public class CombiningSequenceTest
@Test @Test
public void testNoMergeOne() throws Exception public void testNoMergeOne() throws Exception
{ {
List<Pair<Integer, Integer>> pairs = Arrays.asList( List<Pair<Integer, Integer>> pairs = Collections.singletonList(
Pair.of(0, 1) Pair.of(0, 1)
); );
List<Pair<Integer, Integer>> expected = Arrays.asList( List<Pair<Integer, Integer>> expected = Collections.singletonList(
Pair.of(0, 1) Pair.of(0, 1)
); );
@ -151,7 +152,7 @@ public class CombiningSequenceTest
Pair.of(0, 1) Pair.of(0, 1)
); );
List<Pair<Integer, Integer>> expected = Arrays.asList( List<Pair<Integer, Integer>> expected = Collections.singletonList(
Pair.of(0, 2) Pair.of(0, 2)
); );

View File

@ -56,7 +56,7 @@ public class JodaUtilsTest
@Test @Test
public void testUmbrellaIntervalsNull() throws Exception public void testUmbrellaIntervalsNull() throws Exception
{ {
List<Interval> intervals = Arrays.asList(); List<Interval> intervals = Collections.emptyList();
Throwable thrown = null; Throwable thrown = null;
try { try {
Interval res = JodaUtils.umbrellaInterval(intervals); Interval res = JodaUtils.umbrellaInterval(intervals);

View File

@ -42,6 +42,7 @@ import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
@ -157,7 +158,7 @@ public class VersionedIntervalTimelineTest
public void testMay() throws Exception public void testMay() throws Exception
{ {
assertValues( assertValues(
Arrays.asList( Collections.singletonList(
createExpected("2011-05-01/2011-05-09", "4", 9) createExpected("2011-05-01/2011-05-09", "4", 9)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(new Interval("2011-05-01/2011-05-09"))
@ -216,7 +217,7 @@ public class VersionedIntervalTimelineTest
add(new Interval(overallStart, overallStart.plus(Days.ONE)), "2", 2); add(new Interval(overallStart, overallStart.plus(Days.ONE)), "2", 2);
assertValues( assertValues(
Arrays.asList( Collections.singletonList(
createExpected(oneHourInterval1.toString(), "2", 2) createExpected(oneHourInterval1.toString(), "2", 2)
), ),
timeline.lookup(oneHourInterval1) timeline.lookup(oneHourInterval1)
@ -380,7 +381,7 @@ public class VersionedIntervalTimelineTest
{ {
add("2011-05-01/2011-05-10", "5", 10); add("2011-05-01/2011-05-10", "5", 10);
assertValues( assertValues(
Arrays.asList( Collections.singletonList(
createExpected("2011-05-01/2011-05-09", "5", 10) createExpected("2011-05-01/2011-05-09", "5", 10)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(new Interval("2011-05-01/2011-05-09"))
@ -391,7 +392,7 @@ public class VersionedIntervalTimelineTest
timeline.remove(new Interval("2011-05-01/2011-05-10"), "5", makeSingle(10)) timeline.remove(new Interval("2011-05-01/2011-05-10"), "5", makeSingle(10))
); );
assertValues( assertValues(
Arrays.asList( Collections.singletonList(
createExpected("2011-05-01/2011-05-09", "4", 9) createExpected("2011-05-01/2011-05-09", "4", 9)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(new Interval("2011-05-01/2011-05-09"))
@ -399,7 +400,7 @@ public class VersionedIntervalTimelineTest
add("2011-05-01/2011-05-10", "5", 10); add("2011-05-01/2011-05-10", "5", 10);
assertValues( assertValues(
Arrays.asList( Collections.singletonList(
createExpected("2011-05-01/2011-05-09", "5", 10) createExpected("2011-05-01/2011-05-09", "5", 10)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(new Interval("2011-05-01/2011-05-09"))
@ -410,7 +411,7 @@ public class VersionedIntervalTimelineTest
timeline.remove(new Interval("2011-05-01/2011-05-10"), "4", makeSingle(9)) timeline.remove(new Interval("2011-05-01/2011-05-10"), "4", makeSingle(9))
); );
assertValues( assertValues(
Arrays.asList( Collections.singletonList(
createExpected("2011-05-01/2011-05-09", "5", 10) createExpected("2011-05-01/2011-05-09", "5", 10)
), ),
timeline.lookup(new Interval("2011-05-01/2011-05-09")) timeline.lookup(new Interval("2011-05-01/2011-05-09"))
@ -450,7 +451,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-01/2011-01-10", "1", 4); add("2011-01-01/2011-01-10", "1", 4);
assertValues( assertValues(
Arrays.asList( Collections.singletonList(
createExpected("2011-01-01/2011-01-10", "2", 2) createExpected("2011-01-01/2011-01-10", "2", 2)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-10")) timeline.lookup(new Interval("2011-01-01/2011-01-10"))
@ -1177,7 +1178,7 @@ public class VersionedIntervalTimelineTest
timeline.remove(new Interval("2011-01-10/2011-01-15"), "2", makeSingle(2)); timeline.remove(new Interval("2011-01-10/2011-01-15"), "2", makeSingle(2));
assertValues( assertValues(
Arrays.asList( Collections.singletonList(
createExpected("2011-01-01/2011-01-20", "1", 1) createExpected("2011-01-01/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(new Interval("2011-01-01/2011-01-20"))
@ -1218,7 +1219,7 @@ public class VersionedIntervalTimelineTest
timeline.remove(new Interval("2011-01-10/2011-01-14"), "2", makeSingle(3)); timeline.remove(new Interval("2011-01-10/2011-01-14"), "2", makeSingle(3));
assertValues( assertValues(
Arrays.asList( Collections.singletonList(
createExpected("2011-01-01/2011-01-20", "1", 1) createExpected("2011-01-01/2011-01-20", "1", 1)
), ),
@ -1258,7 +1259,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-01/2011-01-20", "1", 1); add("2011-01-01/2011-01-20", "1", 1);
assertValues( assertValues(
Arrays.asList( Collections.singletonList(
createExpected("2011-01-01/2011-01-20", "1", 1) createExpected("2011-01-01/2011-01-20", "1", 1)
), ),
timeline.lookup(new Interval("2011-01-01/2011-01-20")) timeline.lookup(new Interval("2011-01-01/2011-01-20"))

View File

@ -37,6 +37,7 @@ import io.druid.segment.DimensionSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
@ -168,7 +169,7 @@ public class DistinctCountAggregatorFactory extends AggregatorFactory
@Override @Override
public List<String> requiredFields() public List<String> requiredFields()
{ {
return Arrays.asList(fieldName); return Collections.singletonList(fieldName);
} }
@Override @Override

View File

@ -21,7 +21,6 @@ package io.druid.query.aggregation.distinctcount;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.MapBasedInputRow;
import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
@ -40,7 +39,7 @@ import io.druid.segment.incremental.OnheapIncrementalIndex;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
public class DistinctCountTimeseriesQueryTest public class DistinctCountTimeseriesQueryTest
@ -100,7 +99,7 @@ public class DistinctCountTimeseriesQueryTest
Lists.<Result<TimeseriesResultValue>>newLinkedList() Lists.<Result<TimeseriesResultValue>>newLinkedList()
); );
List<Result<TimeseriesResultValue>> expectedResults = Arrays.asList( List<Result<TimeseriesResultValue>> expectedResults = Collections.singletonList(
new Result<>( new Result<>(
time, time,
new TimeseriesResultValue( new TimeseriesResultValue(

View File

@ -44,6 +44,7 @@ import org.junit.Test;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -118,7 +119,7 @@ public class DistinctCountTopNQueryTest
Lists.<Result<TopNResultValue>>newLinkedList() Lists.<Result<TopNResultValue>>newLinkedList()
); );
List<Result<TopNResultValue>> expectedResults = Arrays.asList( List<Result<TopNResultValue>> expectedResults = Collections.singletonList(
new Result<>( new Result<>(
time, time,
new TopNResultValue( new TopNResultValue(

View File

@ -24,14 +24,14 @@ import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.inject.Binder; import com.google.inject.Binder;
import io.druid.initialization.DruidModule; import io.druid.initialization.DruidModule;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
public class OrcExtensionsModule implements DruidModule public class OrcExtensionsModule implements DruidModule
{ {
@Override @Override
public List<? extends Module> getJacksonModules() { public List<? extends Module> getJacksonModules() {
return Arrays.asList( return Collections.singletonList(
new SimpleModule("OrcInputRowParserModule") new SimpleModule("OrcInputRowParserModule")
.registerSubtypes( .registerSubtypes(
new NamedType(OrcHadoopInputRowParser.class, "orc") new NamedType(OrcHadoopInputRowParser.class, "orc")

View File

@ -25,7 +25,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.inject.Binder; import com.google.inject.Binder;
import io.druid.initialization.DruidModule; import io.druid.initialization.DruidModule;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
public class ParquetExtensionsModule implements DruidModule public class ParquetExtensionsModule implements DruidModule
@ -34,8 +34,8 @@ public class ParquetExtensionsModule implements DruidModule
@Override @Override
public List<? extends Module> getJacksonModules() public List<? extends Module> getJacksonModules()
{ {
return Arrays.asList( return Collections.singletonList(
new SimpleModule("ParuqetInputRowParserModule") new SimpleModule("ParquetInputRowParserModule")
.registerSubtypes( .registerSubtypes(
new NamedType(ParquetHadoopInputRowParser.class, "parquet") new NamedType(ParquetHadoopInputRowParser.class, "parquet")
) )

View File

@ -191,15 +191,15 @@ public class ScanQuery extends BaseQuery<ScanResultValue>
public String toString() public String toString()
{ {
return "ScanQuery{" + return "ScanQuery{" +
"dataSource='" + getDataSource() + '\'' + "dataSource='" + getDataSource() + '\'' +
", querySegmentSpec=" + getQuerySegmentSpec() + ", querySegmentSpec=" + getQuerySegmentSpec() +
", descending=" + isDescending() + ", descending=" + isDescending() +
", resultFormat='" + resultFormat + '\'' + ", resultFormat='" + resultFormat + '\'' +
", batchSize=" + batchSize + ", batchSize=" + batchSize +
", limit=" + limit + ", limit=" + limit +
", dimFilter=" + dimFilter + ", dimFilter=" + dimFilter +
", columns=" + columns + ", columns=" + columns +
'}'; '}';
} }
/** /**

View File

@ -47,6 +47,7 @@ import org.junit.runners.Parameterized;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
@ -414,8 +415,7 @@ public class ScanQueryRunnerTest
Lists.<ScanResultValue>newArrayList() Lists.<ScanResultValue>newArrayList()
); );
List<ScanResultValue> expectedResults = Arrays.asList( List<ScanResultValue> expectedResults = Collections.emptyList();
);
verify(expectedResults, populateNullColumnAtLastForQueryableIndexCase(results, "null_column")); verify(expectedResults, populateNullColumnAtLastForQueryableIndexCase(results, "null_column"));
} }

View File

@ -25,7 +25,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.inject.Binder; import com.google.inject.Binder;
import io.druid.initialization.DruidModule; import io.druid.initialization.DruidModule;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
public class ThriftExtensionsModule implements DruidModule public class ThriftExtensionsModule implements DruidModule
@ -34,7 +34,7 @@ public class ThriftExtensionsModule implements DruidModule
@Override @Override
public List<? extends Module> getJacksonModules() public List<? extends Module> getJacksonModules()
{ {
return Arrays.asList( return Collections.singletonList(
new SimpleModule("ThriftInputRowParserModule") new SimpleModule("ThriftInputRowParserModule")
.registerSubtypes( .registerSubtypes(
new NamedType(ThriftInputRowParser.class, "thrift") new NamedType(ThriftInputRowParser.class, "thrift")

View File

@ -29,6 +29,7 @@ import org.joda.time.DateTime;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.sql.Timestamp; import java.sql.Timestamp;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
@ -142,7 +143,7 @@ public class TimestampAggregatorFactory extends AggregatorFactory
@Override @Override
public List<String> requiredFields() public List<String> requiredFields()
{ {
return Arrays.asList(fieldName); return Collections.singletonList(fieldName);
} }
@Override @Override

View File

@ -24,7 +24,7 @@ import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.inject.Binder; import com.google.inject.Binder;
import io.druid.initialization.DruidModule; import io.druid.initialization.DruidModule;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
public class TimestampMinMaxModule implements DruidModule public class TimestampMinMaxModule implements DruidModule
@ -32,7 +32,7 @@ public class TimestampMinMaxModule implements DruidModule
@Override @Override
public List<? extends Module> getJacksonModules() public List<? extends Module> getJacksonModules()
{ {
return Arrays.asList( return Collections.singletonList(
new SimpleModule("TimestampMinMaxModule") new SimpleModule("TimestampMinMaxModule")
.registerSubtypes( .registerSubtypes(
new NamedType(TimestampMaxAggregatorFactory.class, "timeMax"), new NamedType(TimestampMaxAggregatorFactory.class, "timeMax"),

View File

@ -55,6 +55,7 @@ import org.junit.runners.Parameterized;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -164,8 +165,8 @@ public class MapVirtualColumnTest
"params", mapOf("key1", "value1", "key5", "value5") "params", mapOf("key1", "value1", "key5", "value5")
) )
); );
List<VirtualColumn> virtualColumns = Arrays.<VirtualColumn>asList(new MapVirtualColumn("keys", "values", "params")); List<VirtualColumn> virtualColumns = Collections.singletonList(new MapVirtualColumn("keys", "values", "params"));
SelectQuery selectQuery = builder.dimensions(Arrays.asList("dim")) SelectQuery selectQuery = builder.dimensions(Collections.singletonList("dim"))
.metrics(Arrays.asList("params.key1", "params.key3", "params.key5", "params")) .metrics(Arrays.asList("params.key1", "params.key3", "params.key5", "params"))
.virtualColumns(virtualColumns) .virtualColumns(virtualColumns)
.build(); .build();

View File

@ -55,7 +55,7 @@ public class AvroValueInputFormat extends FileInputFormat<NullWritable, GenericR
if (StringUtils.isNotBlank(schemaFilePath)) { if (StringUtils.isNotBlank(schemaFilePath)) {
log.info("Using file: %s as reader schema.", schemaFilePath); log.info("Using file: %s as reader schema.", schemaFilePath);
try (FSDataInputStream inputStream = try (FSDataInputStream inputStream =
FileSystem.get(context.getConfiguration()).open(new Path(schemaFilePath))) { FileSystem.get(context.getConfiguration()).open(new Path(schemaFilePath))) {
readerSchema = new Schema.Parser().parse(inputStream); readerSchema = new Schema.Parser().parse(inputStream);
} }
} }

View File

@ -80,14 +80,12 @@ public class SchemaRepoBasedAvroBytesDecoder<SUBJECT, ID> implements AvroBytesDe
DatumReader<GenericRecord> reader = new GenericDatumReader<>(schema); DatumReader<GenericRecord> reader = new GenericDatumReader<>(schema);
try (ByteBufferInputStream inputStream = new ByteBufferInputStream(Collections.singletonList(bytes))) { try (ByteBufferInputStream inputStream = new ByteBufferInputStream(Collections.singletonList(bytes))) {
return reader.read(null, DecoderFactory.get().binaryDecoder(inputStream, null)); return reader.read(null, DecoderFactory.get().binaryDecoder(inputStream, null));
} } catch (EOFException eof) {
catch (EOFException eof) {
// waiting for avro v1.9.0 (#AVRO-813) // waiting for avro v1.9.0 (#AVRO-813)
throw new ParseException( throw new ParseException(
eof, "Avro's unnecessary EOFException, detail: [%s]", "https://issues.apache.org/jira/browse/AVRO-813" eof, "Avro's unnecessary EOFException, detail: [%s]", "https://issues.apache.org/jira/browse/AVRO-813"
); );
} } catch (IOException e) {
catch (IOException e) {
throw new ParseException(e, "Fail to decode avro message!"); throw new ParseException(e, "Fail to decode avro message!");
} }
} }
@ -110,8 +108,8 @@ public class SchemaRepoBasedAvroBytesDecoder<SUBJECT, ID> implements AvroBytesDe
return false; return false;
} }
return !(schemaRepository != null return !(schemaRepository != null
? !schemaRepository.equals(that.schemaRepository) ? !schemaRepository.equals(that.schemaRepository)
: that.schemaRepository != null); : that.schemaRepository != null);
} }
@Override @Override

View File

@ -20,7 +20,6 @@
package io.druid.segment.loading; package io.druid.segment.loading;
import com.google.common.io.ByteStreams; import com.google.common.io.ByteStreams;
import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.CompressionUtils;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.storage.hdfs.HdfsDataSegmentPuller; import io.druid.storage.hdfs.HdfsDataSegmentPuller;
@ -165,12 +164,10 @@ public class HdfsDataSegmentPullerTest
final URI uri = URI.create(uriBase.toString() + zipPath.toString()); final URI uri = URI.create(uriBase.toString() + zipPath.toString());
try (final OutputStream outputStream = miniCluster.getFileSystem().create(zipPath)) { try (final OutputStream outputStream = miniCluster.getFileSystem().create(zipPath);
try (final OutputStream gzStream = new GZIPOutputStream(outputStream)) { final OutputStream gzStream = new GZIPOutputStream(outputStream);
try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) {
ByteStreams.copy(inputStream, gzStream); ByteStreams.copy(inputStream, gzStream);
}
}
} }
try { try {
Assert.assertFalse(outFile.exists()); Assert.assertFalse(outFile.exists());
@ -201,10 +198,9 @@ public class HdfsDataSegmentPullerTest
final URI uri = URI.create(uriBase.toString() + perTestPath.toString()); final URI uri = URI.create(uriBase.toString() + perTestPath.toString());
try (final OutputStream outputStream = miniCluster.getFileSystem().create(zipPath)) { try (final OutputStream outputStream = miniCluster.getFileSystem().create(zipPath);
try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) {
ByteStreams.copy(inputStream, outputStream); ByteStreams.copy(inputStream, outputStream);
}
} }
try { try {
Assert.assertFalse(outFile.exists()); Assert.assertFalse(outFile.exists());

View File

@ -20,7 +20,6 @@
package io.druid.segment.loading; package io.druid.segment.loading;
import com.google.common.io.ByteStreams; import com.google.common.io.ByteStreams;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import io.druid.storage.hdfs.HdfsFileTimestampVersionFinder; import io.druid.storage.hdfs.HdfsFileTimestampVersionFinder;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
@ -110,20 +109,18 @@ public class HdfsFileTimestampVersionFinderTest
{ {
final Path oldPath = new Path(perTestPath, "555test.txt"); final Path oldPath = new Path(perTestPath, "555test.txt");
Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath)); Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath));
try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath)) { try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath);
try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) {
ByteStreams.copy(inputStream, outputStream); ByteStreams.copy(inputStream, outputStream);
}
} }
Thread.sleep(10); Thread.sleep(10);
final Path newPath = new Path(perTestPath, "666test.txt"); final Path newPath = new Path(perTestPath, "666test.txt");
Assert.assertFalse(miniCluster.getFileSystem().exists(newPath)); Assert.assertFalse(miniCluster.getFileSystem().exists(newPath));
try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath)) { try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath);
try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) {
ByteStreams.copy(inputStream, outputStream); ByteStreams.copy(inputStream, outputStream);
}
} }
Assert.assertEquals(newPath.toString(), finder.getLatestVersion(oldPath.toUri(), Pattern.compile(".*")).getPath()); Assert.assertEquals(newPath.toString(), finder.getLatestVersion(oldPath.toUri(), Pattern.compile(".*")).getPath());
@ -134,20 +131,18 @@ public class HdfsFileTimestampVersionFinderTest
{ {
final Path oldPath = new Path(perTestPath, "555test.txt"); final Path oldPath = new Path(perTestPath, "555test.txt");
Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath)); Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath));
try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath)) { try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath);
try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) {
ByteStreams.copy(inputStream, outputStream); ByteStreams.copy(inputStream, outputStream);
}
} }
Thread.sleep(10); Thread.sleep(10);
final Path newPath = new Path(perTestPath, "666test.txt"); final Path newPath = new Path(perTestPath, "666test.txt");
Assert.assertFalse(miniCluster.getFileSystem().exists(newPath)); Assert.assertFalse(miniCluster.getFileSystem().exists(newPath));
try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath)) { try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath);
try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) {
ByteStreams.copy(inputStream, outputStream); ByteStreams.copy(inputStream, outputStream);
}
} }
Assert.assertEquals(newPath.toString(), finder.getLatestVersion(newPath.toUri(), Pattern.compile(".*")).getPath()); Assert.assertEquals(newPath.toString(), finder.getLatestVersion(newPath.toUri(), Pattern.compile(".*")).getPath());
@ -166,20 +161,18 @@ public class HdfsFileTimestampVersionFinderTest
{ {
final Path oldPath = new Path(perTestPath, "555test.txt"); final Path oldPath = new Path(perTestPath, "555test.txt");
Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath)); Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath));
try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath)) { try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath);
try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) {
ByteStreams.copy(inputStream, outputStream); ByteStreams.copy(inputStream, outputStream);
}
} }
Thread.sleep(10); Thread.sleep(10);
final Path newPath = new Path(perTestPath, "666test.txt"); final Path newPath = new Path(perTestPath, "666test.txt");
Assert.assertFalse(miniCluster.getFileSystem().exists(newPath)); Assert.assertFalse(miniCluster.getFileSystem().exists(newPath));
try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath)) { try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath);
try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) {
ByteStreams.copy(inputStream, outputStream); ByteStreams.copy(inputStream, outputStream);
}
} }
Assert.assertEquals( Assert.assertEquals(
@ -193,20 +186,18 @@ public class HdfsFileTimestampVersionFinderTest
{ {
final Path oldPath = new Path(perTestPath, "555test.txt"); final Path oldPath = new Path(perTestPath, "555test.txt");
Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath)); Assert.assertFalse(miniCluster.getFileSystem().exists(oldPath));
try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath)) { try (final OutputStream outputStream = miniCluster.getFileSystem().create(oldPath);
try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) {
ByteStreams.copy(inputStream, outputStream); ByteStreams.copy(inputStream, outputStream);
}
} }
Thread.sleep(10); Thread.sleep(10);
final Path newPath = new Path(perTestPath, "666test.txt2"); final Path newPath = new Path(perTestPath, "666test.txt2");
Assert.assertFalse(miniCluster.getFileSystem().exists(newPath)); Assert.assertFalse(miniCluster.getFileSystem().exists(newPath));
try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath)) { try (final OutputStream outputStream = miniCluster.getFileSystem().create(newPath);
try (final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) { final InputStream inputStream = new ByteArrayInputStream(pathByteContents)) {
ByteStreams.copy(inputStream, outputStream); ByteStreams.copy(inputStream, outputStream);
}
} }
Assert.assertEquals( Assert.assertEquals(

View File

@ -37,6 +37,7 @@ import org.apache.commons.codec.binary.Base64;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
@ -225,7 +226,7 @@ public class ApproximateHistogramAggregatorFactory extends AggregatorFactory
@Override @Override
public List<String> requiredFields() public List<String> requiredFields()
{ {
return Arrays.asList(fieldName); return Collections.singletonList(fieldName);
} }
@Override @Override

View File

@ -24,7 +24,6 @@ import com.google.common.collect.Lists;
import io.druid.data.input.Row; import io.druid.data.input.Row;
import io.druid.query.QueryRunner; import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper; import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec; import io.druid.query.dimension.DimensionSpec;
import io.druid.query.groupby.GroupByQuery; import io.druid.query.groupby.GroupByQuery;
@ -42,6 +41,7 @@ import org.junit.runners.Parameterized;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
/** /**
@ -181,13 +181,13 @@ public class ApproximateHistogramGroupByQueryTest
) )
) )
.setPostAggregatorSpecs( .setPostAggregatorSpecs(
Arrays.<PostAggregator>asList( Collections.singletonList(
new QuantilePostAggregator("quantile", "apphisto", 0.5f) new QuantilePostAggregator("quantile", "apphisto", 0.5f)
) )
) )
.build(); .build();
List<Row> expectedResults = Arrays.asList( List<Row> expectedResults = Collections.singletonList(
GroupByQueryRunnerTestHelper.createExpectedRow( GroupByQueryRunnerTestHelper.createExpectedRow(
"1970-01-01T00:00:00.000Z", "1970-01-01T00:00:00.000Z",
"marketalias", "upfront", "marketalias", "upfront",
@ -255,13 +255,13 @@ public class ApproximateHistogramGroupByQueryTest
) )
) )
.setPostAggregatorSpecs( .setPostAggregatorSpecs(
Arrays.<PostAggregator>asList( Collections.singletonList(
new QuantilePostAggregator("quantile", "quantile", 0.5f) new QuantilePostAggregator("quantile", "quantile", 0.5f)
) )
) )
.build(); .build();
List<Row> expectedResults = Arrays.asList( List<Row> expectedResults = Collections.singletonList(
GroupByQueryRunnerTestHelper.createExpectedRow( GroupByQueryRunnerTestHelper.createExpectedRow(
"1970-01-01T00:00:00.000Z", "1970-01-01T00:00:00.000Z",
"marketalias", "upfront", "marketalias", "upfront",

View File

@ -47,6 +47,7 @@ import org.junit.runners.Parameterized;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -54,7 +55,7 @@ import java.util.Map;
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
public class ApproximateHistogramTopNQueryTest public class ApproximateHistogramTopNQueryTest
{ {
@Parameterized.Parameters(name="{0}") @Parameterized.Parameters(name = "{0}")
public static Iterable<Object[]> constructorFeeder() throws IOException public static Iterable<Object[]> constructorFeeder() throws IOException
{ {
return QueryRunnerTestHelper.transformToConstructionFeeder( return QueryRunnerTestHelper.transformToConstructionFeeder(
@ -62,7 +63,10 @@ public class ApproximateHistogramTopNQueryTest
QueryRunnerTestHelper.makeQueryRunners( QueryRunnerTestHelper.makeQueryRunners(
new TopNQueryRunnerFactory( new TopNQueryRunnerFactory(
TestQueryRunners.getPool(), TestQueryRunners.getPool(),
new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()), new TopNQueryQueryToolChest(
new TopNQueryConfig(),
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER QueryRunnerTestHelper.NOOP_QUERYWATCHER
) )
), ),
@ -79,7 +83,10 @@ public class ApproximateHistogramTopNQueryTest
} }
} }
), ),
new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()), new TopNQueryQueryToolChest(
new TopNQueryConfig(),
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
),
QueryRunnerTestHelper.NOOP_QUERYWATCHER QueryRunnerTestHelper.NOOP_QUERYWATCHER
) )
) )
@ -136,109 +143,109 @@ public class ApproximateHistogramTopNQueryTest
) )
.build(); .build();
List<Result<TopNResultValue>> expectedResults = Arrays.asList( List<Result<TopNResultValue>> expectedResults = Collections.singletonList(
new Result<TopNResultValue>( new Result<TopNResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"), new DateTime("2011-01-12T00:00:00.000Z"),
new TopNResultValue( new TopNResultValue(
Arrays.<Map<String, Object>>asList( Arrays.<Map<String, Object>>asList(
ImmutableMap.<String, Object>builder() ImmutableMap.<String, Object>builder()
.put(QueryRunnerTestHelper.marketDimension, "total_market") .put(QueryRunnerTestHelper.marketDimension, "total_market")
.put("rows", 186L) .put("rows", 186L)
.put("index", 215679.82879638672D) .put("index", 215679.82879638672D)
.put("addRowsIndexConstant", 215866.82879638672D) .put("addRowsIndexConstant", 215866.82879638672D)
.put(QueryRunnerTestHelper.dependentPostAggMetric, 216053.82879638672D) .put(QueryRunnerTestHelper.dependentPostAggMetric, 216053.82879638672D)
.put("uniques", QueryRunnerTestHelper.UNIQUES_2) .put("uniques", QueryRunnerTestHelper.UNIQUES_2)
.put("maxIndex", 1743.9217529296875D) .put("maxIndex", 1743.9217529296875D)
.put("minIndex", 792.3260498046875D) .put("minIndex", 792.3260498046875D)
.put("quantile", 1085.6775f) .put("quantile", 1085.6775f)
.put( .put(
"apphisto", "apphisto",
new Histogram( new Histogram(
new float[]{ new float[]{
554.4271240234375f, 554.4271240234375f,
792.3260498046875f, 792.3260498046875f,
1030.2249755859375f, 1030.2249755859375f,
1268.1239013671875f, 1268.1239013671875f,
1506.0228271484375f, 1506.0228271484375f,
1743.9217529296875f 1743.9217529296875f
}, },
new double[]{ new double[]{
0.0D, 0.0D,
39.42073059082031D, 39.42073059082031D,
103.29110717773438D, 103.29110717773438D,
34.93659591674805D, 34.93659591674805D,
8.351564407348633D 8.351564407348633D
} }
) )
) )
.build(), .build(),
ImmutableMap.<String, Object>builder() ImmutableMap.<String, Object>builder()
.put(QueryRunnerTestHelper.marketDimension, "upfront") .put(QueryRunnerTestHelper.marketDimension, "upfront")
.put("rows", 186L) .put("rows", 186L)
.put("index", 192046.1060180664D) .put("index", 192046.1060180664D)
.put("addRowsIndexConstant", 192233.1060180664D) .put("addRowsIndexConstant", 192233.1060180664D)
.put(QueryRunnerTestHelper.dependentPostAggMetric, 192420.1060180664D) .put(QueryRunnerTestHelper.dependentPostAggMetric, 192420.1060180664D)
.put("uniques", QueryRunnerTestHelper.UNIQUES_2) .put("uniques", QueryRunnerTestHelper.UNIQUES_2)
.put("maxIndex", 1870.06103515625D) .put("maxIndex", 1870.06103515625D)
.put("minIndex", 545.9906005859375D) .put("minIndex", 545.9906005859375D)
.put("quantile", 880.9881f) .put("quantile", 880.9881f)
.put( .put(
"apphisto", "apphisto",
new Histogram( new Histogram(
new float[]{ new float[]{
214.97299194335938f, 214.97299194335938f,
545.9906005859375f, 545.9906005859375f,
877.0081787109375f, 877.0081787109375f,
1208.0257568359375f, 1208.0257568359375f,
1539.0433349609375f, 1539.0433349609375f,
1870.06103515625f 1870.06103515625f
}, },
new double[]{ new double[]{
0.0D, 0.0D,
67.53287506103516D, 67.53287506103516D,
72.22068786621094D, 72.22068786621094D,
31.984678268432617D, 31.984678268432617D,
14.261756896972656D 14.261756896972656D
} }
) )
) )
.build(), .build(),
ImmutableMap.<String, Object>builder() ImmutableMap.<String, Object>builder()
.put(QueryRunnerTestHelper.marketDimension, "spot") .put(QueryRunnerTestHelper.marketDimension, "spot")
.put("rows", 837L) .put("rows", 837L)
.put("index", 95606.57232284546D) .put("index", 95606.57232284546D)
.put("addRowsIndexConstant", 96444.57232284546D) .put("addRowsIndexConstant", 96444.57232284546D)
.put(QueryRunnerTestHelper.dependentPostAggMetric, 97282.57232284546D) .put(QueryRunnerTestHelper.dependentPostAggMetric, 97282.57232284546D)
.put("uniques", QueryRunnerTestHelper.UNIQUES_9) .put("uniques", QueryRunnerTestHelper.UNIQUES_9)
.put("maxIndex", 277.2735290527344D) .put("maxIndex", 277.2735290527344D)
.put("minIndex", 59.02102279663086D) .put("minIndex", 59.02102279663086D)
.put("quantile", 101.78856f) .put("quantile", 101.78856f)
.put( .put(
"apphisto", "apphisto",
new Histogram( new Histogram(
new float[]{ new float[]{
4.457897186279297f, 4.457897186279297f,
59.02102279663086f, 59.02102279663086f,
113.58415222167969f, 113.58415222167969f,
168.14727783203125f, 168.14727783203125f,
222.7104034423828f, 222.7104034423828f,
277.2735290527344f 277.2735290527344f
}, },
new double[]{ new double[]{
0.0D, 0.0D,
462.4309997558594D, 462.4309997558594D,
357.5404968261719D, 357.5404968261719D,
15.022850036621094D, 15.022850036621094D,
2.0056631565093994D 2.0056631565093994D
} }
) )
) )
.build() .build()
) )
) )
) )
); );
HashMap<String,Object> context = new HashMap<String, Object>(); HashMap<String, Object> context = new HashMap<String, Object>();
TestHelper.assertExpectedResults(expectedResults, runner.run(query, context)); TestHelper.assertExpectedResults(expectedResults, runner.run(query, context));
} }

View File

@ -214,7 +214,7 @@ public class URIExtractionNamespaceCacheFactoryTest
{ {
if (cacheManagerCreatorsIt.hasNext()) { if (cacheManagerCreatorsIt.hasNext()) {
Function<Lifecycle, NamespaceExtractionCacheManager> cacheManagerCreator = cacheManagerCreatorsIt.next(); Function<Lifecycle, NamespaceExtractionCacheManager> cacheManagerCreator = cacheManagerCreatorsIt.next();
return new Object[]{ compressions[0], compressions[1], cacheManagerCreator }; return new Object[]{compressions[0], compressions[1], cacheManagerCreator};
} else { } else {
cacheManagerCreatorsIt = cacheManagerCreators.iterator(); cacheManagerCreatorsIt = cacheManagerCreators.iterator();
compressions = compressionIt.next(); compressions = compressionIt.next();
@ -275,19 +275,18 @@ public class URIExtractionNamespaceCacheFactoryTest
Assert.assertTrue(tmpFileParent.isDirectory()); Assert.assertTrue(tmpFileParent.isDirectory());
tmpFile = Files.createTempFile(tmpFileParent.toPath(), "druidTestURIExtractionNS", suffix).toFile(); tmpFile = Files.createTempFile(tmpFileParent.toPath(), "druidTestURIExtractionNS", suffix).toFile();
final ObjectMapper mapper = new DefaultObjectMapper(); final ObjectMapper mapper = new DefaultObjectMapper();
try (OutputStream ostream = outStreamSupplier.apply(tmpFile)) { try (OutputStream ostream = outStreamSupplier.apply(tmpFile);
try (OutputStreamWriter out = new OutputStreamWriter(ostream)) { OutputStreamWriter out = new OutputStreamWriter(ostream)) {
out.write(mapper.writeValueAsString(ImmutableMap.<String, String>of( out.write(mapper.writeValueAsString(ImmutableMap.<String, String>of(
"boo", "boo",
"bar", "bar",
"foo", "foo",
"bar", "bar",
"", "",
"MissingValue", "MissingValue",
"emptyString", "emptyString",
"" ""
))); )));
}
} }
populator = new URIExtractionNamespaceCacheFactory(FINDERS); populator = new URIExtractionNamespaceCacheFactory(FINDERS);
namespace = new URIExtractionNamespace( namespace = new URIExtractionNamespace(

View File

@ -37,6 +37,7 @@ import org.apache.commons.codec.binary.Base64;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
@ -218,7 +219,7 @@ public class VarianceAggregatorFactory extends AggregatorFactory
@Override @Override
public List<String> requiredFields() public List<String> requiredFields()
{ {
return Arrays.asList(fieldName); return Collections.singletonList(fieldName);
} }
@Override @Override

View File

@ -775,10 +775,7 @@ public class IndexGeneratorJob implements Jobby
FileUtils.deleteDirectory(file); FileUtils.deleteDirectory(file);
} }
} }
catch (ExecutionException e) { catch (ExecutionException | TimeoutException e) {
throw Throwables.propagate(e);
}
catch (TimeoutException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
} }
finally { finally {

View File

@ -46,6 +46,7 @@ import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -245,14 +246,14 @@ public class IndexGeneratorCombinerTest
InputRow capturedRow1 = InputRowSerde.fromBytes(captureVal1.getValue().getBytes(), aggregators); InputRow capturedRow1 = InputRowSerde.fromBytes(captureVal1.getValue().getBytes(), aggregators);
Assert.assertEquals(Arrays.asList("host", "keywords"), capturedRow1.getDimensions()); Assert.assertEquals(Arrays.asList("host", "keywords"), capturedRow1.getDimensions());
Assert.assertEquals(Arrays.asList("host1"), capturedRow1.getDimension("host")); Assert.assertEquals(Collections.singletonList("host1"), capturedRow1.getDimension("host"));
Assert.assertEquals(Arrays.asList("bar", "foo"), capturedRow1.getDimension("keywords")); Assert.assertEquals(Arrays.asList("bar", "foo"), capturedRow1.getDimension("keywords"));
Assert.assertEquals(10, capturedRow1.getLongMetric("visited_sum")); Assert.assertEquals(10, capturedRow1.getLongMetric("visited_sum"));
Assert.assertEquals(1.0, (Double)HyperUniquesAggregatorFactory.estimateCardinality(capturedRow1.getRaw("unique_hosts")), 0.001); Assert.assertEquals(1.0, (Double)HyperUniquesAggregatorFactory.estimateCardinality(capturedRow1.getRaw("unique_hosts")), 0.001);
InputRow capturedRow2 = InputRowSerde.fromBytes(captureVal2.getValue().getBytes(), aggregators); InputRow capturedRow2 = InputRowSerde.fromBytes(captureVal2.getValue().getBytes(), aggregators);
Assert.assertEquals(Arrays.asList("host", "keywords"), capturedRow2.getDimensions()); Assert.assertEquals(Arrays.asList("host", "keywords"), capturedRow2.getDimensions());
Assert.assertEquals(Arrays.asList("host2"), capturedRow2.getDimension("host")); Assert.assertEquals(Collections.singletonList("host2"), capturedRow2.getDimension("host"));
Assert.assertEquals(Arrays.asList("bar", "foo"), capturedRow2.getDimension("keywords")); Assert.assertEquals(Arrays.asList("bar", "foo"), capturedRow2.getDimension("keywords"));
Assert.assertEquals(5, capturedRow2.getLongMetric("visited_sum")); Assert.assertEquals(5, capturedRow2.getLongMetric("visited_sum"));
Assert.assertEquals(1.0, (Double)HyperUniquesAggregatorFactory.estimateCardinality(capturedRow2.getRaw("unique_hosts")), 0.001); Assert.assertEquals(1.0, (Double)HyperUniquesAggregatorFactory.estimateCardinality(capturedRow2.getRaw("unique_hosts")), 0.001);

View File

@ -86,13 +86,14 @@ public class UtilsCompressionTest
tmpFolder.delete(); tmpFolder.delete();
} }
@Test public void testExistsCompressedFile() throws IOException @Test
public void testExistsCompressedFile() throws IOException
{ {
boolean expected = Utils.exists(mockJobContext,defaultFileSystem,tmpPathWithoutExtension); boolean expected = Utils.exists(mockJobContext, defaultFileSystem, tmpPathWithoutExtension);
Assert.assertTrue("Should be true since file is created", expected); Assert.assertTrue("Should be true since file is created", expected);
tmpFolder.delete(); tmpFolder.delete();
expected = Utils.exists(mockJobContext,defaultFileSystem,tmpPathWithoutExtension); expected = Utils.exists(mockJobContext, defaultFileSystem, tmpPathWithoutExtension);
Assert.assertFalse("Should be false since file is deleted",expected); Assert.assertFalse("Should be false since file is deleted", expected);
} }
@Test @Test
@ -100,11 +101,11 @@ public class UtilsCompressionTest
{ {
boolean overwrite = true; boolean overwrite = true;
OutputStream outStream = codec.createOutputStream(defaultFileSystem.create(tmpPathWithExtension, overwrite)); OutputStream outStream = codec.createOutputStream(defaultFileSystem.create(tmpPathWithExtension, overwrite));
writeStingToOutputStream(DUMMY_STRING,outStream); writeStingToOutputStream(DUMMY_STRING, outStream);
InputStream inStream = Utils.openInputStream(mockJobContext, tmpPathWithoutExtension); InputStream inStream = Utils.openInputStream(mockJobContext, tmpPathWithoutExtension);
Assert.assertNotNull("Input stream should not be Null",inStream); Assert.assertNotNull("Input stream should not be Null", inStream);
String actual = new String(ByteStreams.toByteArray(inStream), StandardCharsets.UTF_8.toString()); String actual = new String(ByteStreams.toByteArray(inStream), StandardCharsets.UTF_8.toString());
Assert.assertEquals("Strings not matching", DUMMY_STRING,actual); Assert.assertEquals("Strings not matching", DUMMY_STRING, actual);
inStream.close(); inStream.close();
} }
@ -112,12 +113,12 @@ public class UtilsCompressionTest
public void testCompressedMakePathAndOutputStream() throws IOException public void testCompressedMakePathAndOutputStream() throws IOException
{ {
boolean overwrite = true; boolean overwrite = true;
OutputStream outStream = Utils.makePathAndOutputStream(mockJobContext,tmpPathWithoutExtension, overwrite); OutputStream outStream = Utils.makePathAndOutputStream(mockJobContext, tmpPathWithoutExtension, overwrite);
Assert.assertNotNull("Output stream should not be null",outStream); Assert.assertNotNull("Output stream should not be null", outStream);
writeStingToOutputStream(DUMMY_STRING,outStream); writeStingToOutputStream(DUMMY_STRING, outStream);
InputStream inStream = codec.createInputStream(defaultFileSystem.open(tmpPathWithExtension)); InputStream inStream = codec.createInputStream(defaultFileSystem.open(tmpPathWithExtension));
String actual = new String(ByteStreams.toByteArray(inStream), StandardCharsets.UTF_8.toString()); String actual = new String(ByteStreams.toByteArray(inStream), StandardCharsets.UTF_8.toString());
Assert.assertEquals("Strings not matching", DUMMY_STRING,actual); Assert.assertEquals("Strings not matching", DUMMY_STRING, actual);
inStream.close(); inStream.close();
} }

View File

@ -22,9 +22,7 @@ package io.druid.indexer;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import com.google.common.io.ByteStreams; import com.google.common.io.ByteStreams;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import org.apache.commons.io.FileUtils; import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@ -67,7 +65,8 @@ public class UtilsTest
private static class CreateValueFromKey implements Function private static class CreateValueFromKey implements Function
{ {
@Override public Object apply(Object input) @Override
public Object apply(Object input)
{ {
return input.toString() + DUMMY_STRING; return input.toString() + DUMMY_STRING;
} }
@ -82,7 +81,7 @@ public class UtilsTest
EasyMock.replay(mockJobContext); EasyMock.replay(mockJobContext);
setOfKeys = new HashSet(); setOfKeys = new HashSet();
setOfKeys.addAll(new ArrayList<>(Arrays.asList("key1","key2","key3"))); setOfKeys.addAll(new ArrayList<>(Arrays.asList("key1", "key2", "key3")));
expectedMap = (Map<String, Object>) Maps.asMap(setOfKeys, new CreateValueFromKey()); expectedMap = (Map<String, Object>) Maps.asMap(setOfKeys, new CreateValueFromKey());
tmpFile = tmpFolder.newFile(TMP_FILE_NAME); tmpFile = tmpFolder.newFile(TMP_FILE_NAME);
@ -99,20 +98,20 @@ public class UtilsTest
@Test @Test
public void testExistsPlainFile() throws IOException public void testExistsPlainFile() throws IOException
{ {
boolean expected = Utils.exists(mockJobContext,defaultFileSystem,tmpPath); boolean expected = Utils.exists(mockJobContext, defaultFileSystem, tmpPath);
Assert.assertTrue("Should be true since file is created",expected); Assert.assertTrue("Should be true since file is created", expected);
tmpFolder.delete(); tmpFolder.delete();
expected = Utils.exists(mockJobContext,defaultFileSystem,tmpPath); expected = Utils.exists(mockJobContext, defaultFileSystem, tmpPath);
Assert.assertFalse("Should be false since file is deleted",expected); Assert.assertFalse("Should be false since file is deleted", expected);
EasyMock.verify(mockJobContext); EasyMock.verify(mockJobContext);
} }
@Test @Test
public void testPlainStoreThenGetStats() throws IOException public void testPlainStoreThenGetStats() throws IOException
{ {
Utils.storeStats(mockJobContext, tmpPath,expectedMap); Utils.storeStats(mockJobContext, tmpPath, expectedMap);
Map actualMap = Utils.getStats(mockJobContext, tmpPath); Map actualMap = Utils.getStats(mockJobContext, tmpPath);
Assert.assertThat(actualMap,Is.is(actualMap)); Assert.assertThat(actualMap, Is.is(actualMap));
EasyMock.verify(mockJobContext); EasyMock.verify(mockJobContext);
} }
@ -120,7 +119,7 @@ public class UtilsTest
public void testExceptionInMakePathAndOutputStream() throws IOException public void testExceptionInMakePathAndOutputStream() throws IOException
{ {
boolean overwrite = false; boolean overwrite = false;
Utils.makePathAndOutputStream(mockJobContext,tmpPath,overwrite); Utils.makePathAndOutputStream(mockJobContext, tmpPath, overwrite);
} }
@Test @Test

View File

@ -26,7 +26,7 @@ import io.druid.segment.loading.SegmentLoaderLocalCacheManager;
import io.druid.segment.loading.StorageLocationConfig; import io.druid.segment.loading.StorageLocationConfig;
import java.io.File; import java.io.File;
import java.util.Arrays; import java.util.Collections;
/** /**
*/ */
@ -45,7 +45,7 @@ public class SegmentLoaderFactory
public SegmentLoader manufacturate(File storageDir) public SegmentLoader manufacturate(File storageDir)
{ {
return loader.withConfig( return loader.withConfig(
new SegmentLoaderConfig().withLocations(Arrays.asList(new StorageLocationConfig().setPath(storageDir))) new SegmentLoaderConfig().withLocations(Collections.singletonList(new StorageLocationConfig().setPath(storageDir)))
); );
} }
} }

View File

@ -19,11 +19,13 @@
package io.druid.indexing.common.actions; package io.druid.indexing.common.actions;
import static org.easymock.EasyMock.anyObject; import com.fasterxml.jackson.core.JsonProcessingException;
import static org.easymock.EasyMock.createMock; import com.fasterxml.jackson.databind.ObjectMapper;
import static org.easymock.EasyMock.expect; import com.google.common.util.concurrent.Futures;
import static org.easymock.EasyMock.replay; import com.metamx.http.client.HttpClient;
import com.metamx.http.client.Request;
import com.metamx.http.client.response.StatusResponseHandler;
import com.metamx.http.client.response.StatusResponseHolder;
import io.druid.client.selector.Server; import io.druid.client.selector.Server;
import io.druid.curator.discovery.ServerDiscoverySelector; import io.druid.curator.discovery.ServerDiscoverySelector;
import io.druid.indexing.common.RetryPolicyConfig; import io.druid.indexing.common.RetryPolicyConfig;
@ -32,13 +34,6 @@ import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.task.NoopTask; import io.druid.indexing.common.task.NoopTask;
import io.druid.indexing.common.task.Task; import io.druid.indexing.common.task.Task;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.jboss.netty.handler.codec.http.HttpResponseStatus;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -46,13 +41,16 @@ import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import com.fasterxml.jackson.core.JsonProcessingException; import java.io.IOException;
import com.fasterxml.jackson.databind.ObjectMapper; import java.util.Collections;
import com.google.common.util.concurrent.Futures; import java.util.HashMap;
import com.metamx.http.client.HttpClient; import java.util.List;
import com.metamx.http.client.Request; import java.util.Map;
import com.metamx.http.client.response.StatusResponseHandler;
import com.metamx.http.client.response.StatusResponseHolder; import static org.easymock.EasyMock.anyObject;
import static org.easymock.EasyMock.createMock;
import static org.easymock.EasyMock.expect;
import static org.easymock.EasyMock.replay;
public class RemoteTaskActionClientTest public class RemoteTaskActionClientTest
{ {
@ -99,7 +97,12 @@ public class RemoteTaskActionClientTest
long now = System.currentTimeMillis(); long now = System.currentTimeMillis();
result = Arrays.asList(new TaskLock("groupId", "dataSource", new Interval(now - 30 * 1000, now), "version")); result = Collections.singletonList(new TaskLock(
"groupId",
"dataSource",
new Interval(now - 30 * 1000, now),
"version"
));
} }
@Test @Test

View File

@ -207,7 +207,7 @@ public class IndexTaskTest
null, null,
new ArbitraryGranularitySpec( new ArbitraryGranularitySpec(
Granularities.MINUTE, Granularities.MINUTE,
Arrays.asList(new Interval("2014/2015")) Collections.singletonList(new Interval("2014/2015"))
), ),
10, 10,
null, null,
@ -244,7 +244,7 @@ public class IndexTaskTest
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.HOUR, Granularities.HOUR,
Granularities.HOUR, Granularities.HOUR,
Arrays.asList(new Interval("2015-03-01T08:00:00Z/2015-03-01T09:00:00Z")) Collections.singletonList(new Interval("2015-03-01T08:00:00Z/2015-03-01T09:00:00Z"))
), ),
50, 50,
null, null,
@ -497,7 +497,7 @@ public class IndexTaskTest
public <RetType> RetType submit(TaskAction<RetType> taskAction) throws IOException public <RetType> RetType submit(TaskAction<RetType> taskAction) throws IOException
{ {
if (taskAction instanceof LockListAction) { if (taskAction instanceof LockListAction) {
return (RetType) Arrays.asList( return (RetType) Collections.singletonList(
new TaskLock( new TaskLock(
"", "", null, new DateTime().toString() "", "", null, new DateTime().toString()
) )

View File

@ -128,7 +128,7 @@ public class EC2AutoScalerTest
Assert.assertEquals(created.getNodeIds().size(), 1); Assert.assertEquals(created.getNodeIds().size(), 1);
Assert.assertEquals("theInstance", created.getNodeIds().get(0)); Assert.assertEquals("theInstance", created.getNodeIds().get(0));
AutoScalingData deleted = autoScaler.terminate(Arrays.asList("dummyIP")); AutoScalingData deleted = autoScaler.terminate(Collections.singletonList("dummyIP"));
Assert.assertEquals(deleted.getNodeIds().size(), 1); Assert.assertEquals(deleted.getNodeIds().size(), 1);
Assert.assertEquals(INSTANCE_ID, deleted.getNodeIds().get(0)); Assert.assertEquals(INSTANCE_ID, deleted.getNodeIds().get(0));
@ -185,7 +185,7 @@ public class EC2AutoScalerTest
); );
EasyMock.replay(describeInstancesResult); EasyMock.replay(describeInstancesResult);
EasyMock.expect(reservation.getInstances()).andReturn(Arrays.asList(instance)).times(n); EasyMock.expect(reservation.getInstances()).andReturn(Collections.singletonList(instance)).times(n);
EasyMock.replay(reservation); EasyMock.replay(reservation);
List<String> ids = autoScaler.ipToIdLookup(ips); List<String> ids = autoScaler.ipToIdLookup(ips);

View File

@ -46,7 +46,7 @@ import java.nio.ByteBuffer;
import java.nio.channels.Channels; import java.nio.channels.Channels;
import java.nio.channels.GatheringByteChannel; import java.nio.channels.GatheringByteChannel;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -134,7 +134,7 @@ public class FileSmoosher implements Closeable
public void add(String name, ByteBuffer bufferToAdd) throws IOException public void add(String name, ByteBuffer bufferToAdd) throws IOException
{ {
add(name, Arrays.asList(bufferToAdd)); add(name, Collections.singletonList(bufferToAdd));
} }
public void add(String name, List<ByteBuffer> bufferToAdd) throws IOException public void add(String name, List<ByteBuffer> bufferToAdd) throws IOException

View File

@ -24,7 +24,7 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Sequences;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -55,7 +55,7 @@ public class BySegmentQueryRunner<T> implements QueryRunner<T>
final Sequence<T> baseSequence = base.run(queryPlus, responseContext); final Sequence<T> baseSequence = base.run(queryPlus, responseContext);
final List<T> results = Sequences.toList(baseSequence, Lists.<T>newArrayList()); final List<T> results = Sequences.toList(baseSequence, Lists.<T>newArrayList());
return Sequences.simple( return Sequences.simple(
Arrays.asList( Collections.singletonList(
(T) new Result<BySegmentResultValueClass<T>>( (T) new Result<BySegmentResultValueClass<T>>(
timestamp, timestamp,
new BySegmentResultValueClass<T>( new BySegmentResultValueClass<T>(

View File

@ -133,11 +133,9 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
} }
return retVal; return retVal;
} } catch (QueryInterruptedException e) {
catch (QueryInterruptedException e) {
throw Throwables.propagate(e); throw Throwables.propagate(e);
} } catch (Exception e) {
catch (Exception e) {
log.error(e, "Exception with one of the sequences!"); log.error(e, "Exception with one of the sequences!");
throw Throwables.propagate(e); throw Throwables.propagate(e);
} }
@ -156,24 +154,20 @@ public class ChainedExecutionQueryRunner<T> implements QueryRunner<T>
return new MergeIterable<>( return new MergeIterable<>(
ordering.nullsFirst(), ordering.nullsFirst(),
QueryContexts.hasTimeout(query) ? QueryContexts.hasTimeout(query) ?
futures.get(QueryContexts.getTimeout(query), TimeUnit.MILLISECONDS) : futures.get(QueryContexts.getTimeout(query), TimeUnit.MILLISECONDS) :
futures.get() futures.get()
).iterator(); ).iterator();
} } catch (InterruptedException e) {
catch (InterruptedException e) {
log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId()); log.warn(e, "Query interrupted, cancelling pending results, query id [%s]", query.getId());
futures.cancel(true); futures.cancel(true);
throw new QueryInterruptedException(e); throw new QueryInterruptedException(e);
} } catch (CancellationException e) {
catch (CancellationException e) {
throw new QueryInterruptedException(e); throw new QueryInterruptedException(e);
} } catch (TimeoutException e) {
catch (TimeoutException e) {
log.info("Query timeout, cancelling pending results for query id [%s]", query.getId()); log.info("Query timeout, cancelling pending results for query id [%s]", query.getId());
futures.cancel(true); futures.cancel(true);
throw new QueryInterruptedException(e); throw new QueryInterruptedException(e);
} } catch (ExecutionException e) {
catch (ExecutionException e) {
throw Throwables.propagate(e.getCause()); throw Throwables.propagate(e.getCause());
} }
} }

View File

@ -31,7 +31,7 @@ import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.joda.time.Period; import org.joda.time.Period;
import java.util.Arrays; import java.util.Collections;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -113,7 +113,8 @@ public class IntervalChunkingQueryRunner<T> implements QueryRunner<T>
), ),
executor, queryWatcher executor, queryWatcher
).run( ).run(
queryPlus.withQuerySegmentSpec(new MultipleIntervalSegmentSpec(Arrays.asList(singleInterval))), queryPlus.withQuerySegmentSpec(
new MultipleIntervalSegmentSpec(Collections.singletonList(singleInterval))),
responseContext responseContext
); );
} }

View File

@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.annotation.JsonTypeName;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
@JsonTypeName("table") @JsonTypeName("table")
@ -45,7 +45,7 @@ public class TableDataSource implements DataSource
@Override @Override
public List<String> getNames() public List<String> getNames()
{ {
return Arrays.asList(name); return Collections.singletonList(name);
} }
public String toString() { return name; } public String toString() { return name; }

View File

@ -35,12 +35,10 @@ import org.joda.time.Period;
import java.util.Arrays; import java.util.Arrays;
import java.util.Map; import java.util.Map;
/** /**
* TimewarpOperator is an example post-processing operator that maps current time * TimewarpOperator is an example post-processing operator that maps current time
* to the latest period ending withing the specified data interval and truncates * to the latest period ending withing the specified data interval and truncates
* the query interval to discard data that would be mapped to the future. * the query interval to discard data that would be mapped to the future.
*
*/ */
public class TimewarpOperator<T> implements PostProcessingOperator<T> public class TimewarpOperator<T> implements PostProcessingOperator<T>
{ {
@ -49,12 +47,11 @@ public class TimewarpOperator<T> implements PostProcessingOperator<T>
private final long originMillis; private final long originMillis;
/** /**
*
* @param dataInterval interval containing the actual data * @param dataInterval interval containing the actual data
* @param period time will be offset by a multiple of the given period * @param period time will be offset by a multiple of the given period
* until there is at least a full period ending within the data interval * until there is at least a full period ending within the data interval
* @param origin origin to be used to align time periods * @param origin origin to be used to align time periods
* (e.g. to determine on what day of the week a weekly period starts) * (e.g. to determine on what day of the week a weekly period starts)
*/ */
@JsonCreator @JsonCreator
public TimewarpOperator( public TimewarpOperator(
@ -69,7 +66,6 @@ public class TimewarpOperator<T> implements PostProcessingOperator<T>
this.periodMillis = period.toStandardDuration().getMillis(); this.periodMillis = period.toStandardDuration().getMillis();
} }
@Override @Override
public QueryRunner<T> postProcess(QueryRunner<T> baseQueryRunner) public QueryRunner<T> postProcess(QueryRunner<T> baseQueryRunner)
{ {
@ -107,9 +103,10 @@ public class TimewarpOperator<T> implements PostProcessingOperator<T>
TimeBoundaryResultValue boundary = (TimeBoundaryResultValue) value; TimeBoundaryResultValue boundary = (TimeBoundaryResultValue) value;
DateTime minTime = null; DateTime minTime = null;
try{ try {
minTime = boundary.getMinTime(); minTime = boundary.getMinTime();
} catch(IllegalArgumentException e) {} } catch (IllegalArgumentException e) {
}
final DateTime maxTime = boundary.getMaxTime(); final DateTime maxTime = boundary.getMaxTime();
@ -138,6 +135,7 @@ public class TimewarpOperator<T> implements PostProcessingOperator<T>
* Map time t into the last `period` ending within `dataInterval` * Map time t into the last `period` ending within `dataInterval`
* *
* @param t the current time to be mapped into `dataInterval` * @param t the current time to be mapped into `dataInterval`
*
* @return the offset between the mapped time and time t * @return the offset between the mapped time and time t
*/ */
protected long computeOffset(final long t) protected long computeOffset(final long t)
@ -145,14 +143,15 @@ public class TimewarpOperator<T> implements PostProcessingOperator<T>
// start is the beginning of the last period ending within dataInterval // start is the beginning of the last period ending within dataInterval
long start = dataInterval.getEndMillis() - periodMillis; long start = dataInterval.getEndMillis() - periodMillis;
long startOffset = start % periodMillis - originMillis % periodMillis; long startOffset = start % periodMillis - originMillis % periodMillis;
if(startOffset < 0) { if (startOffset < 0) {
startOffset += periodMillis; startOffset += periodMillis;
}; }
;
start -= startOffset; start -= startOffset;
// tOffset is the offset time t within the last period // tOffset is the offset time t within the last period
long tOffset = t % periodMillis - originMillis % periodMillis; long tOffset = t % periodMillis - originMillis % periodMillis;
if(tOffset < 0) { if (tOffset < 0) {
tOffset += periodMillis; tOffset += periodMillis;
} }
tOffset += start; tOffset += start;

View File

@ -30,6 +30,7 @@ import io.druid.segment.FloatColumnSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
@ -156,7 +157,7 @@ public class DoubleMaxAggregatorFactory extends AggregatorFactory
@Override @Override
public List<String> requiredFields() public List<String> requiredFields()
{ {
return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
} }
@Override @Override

View File

@ -30,6 +30,7 @@ import io.druid.segment.FloatColumnSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
@ -156,7 +157,7 @@ public class DoubleMinAggregatorFactory extends AggregatorFactory
@Override @Override
public List<String> requiredFields() public List<String> requiredFields()
{ {
return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
} }
@Override @Override

View File

@ -30,6 +30,7 @@ import io.druid.segment.FloatColumnSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
@ -156,7 +157,7 @@ public class DoubleSumAggregatorFactory extends AggregatorFactory
@Override @Override
public List<String> requiredFields() public List<String> requiredFields()
{ {
return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
} }
@Override @Override

View File

@ -30,6 +30,7 @@ import io.druid.segment.LongColumnSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
@ -152,7 +153,7 @@ public class LongMaxAggregatorFactory extends AggregatorFactory
@Override @Override
public List<String> requiredFields() public List<String> requiredFields()
{ {
return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
} }
@Override @Override

View File

@ -30,6 +30,7 @@ import io.druid.segment.LongColumnSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
@ -152,7 +153,7 @@ public class LongMinAggregatorFactory extends AggregatorFactory
@Override @Override
public List<String> requiredFields() public List<String> requiredFields()
{ {
return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
} }
@Override @Override

View File

@ -30,6 +30,7 @@ import io.druid.segment.LongColumnSelector;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
@ -152,7 +153,7 @@ public class LongSumAggregatorFactory extends AggregatorFactory
@Override @Override
public List<String> requiredFields() public List<String> requiredFields()
{ {
return fieldName != null ? Arrays.asList(fieldName) : Parser.findRequiredBindings(expression); return fieldName != null ? Collections.singletonList(fieldName) : Parser.findRequiredBindings(expression);
} }
@Override @Override

View File

@ -37,6 +37,7 @@ import org.apache.commons.codec.binary.Base64;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Objects; import java.util.Objects;
@ -193,7 +194,7 @@ public class HyperUniquesAggregatorFactory extends AggregatorFactory
@Override @Override
public List<String> requiredFields() public List<String> requiredFields()
{ {
return Arrays.asList(fieldName); return Collections.singletonList(fieldName);
} }
@JsonProperty @JsonProperty

View File

@ -34,7 +34,6 @@ import io.druid.query.spec.QuerySegmentSpec;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -57,7 +56,7 @@ public class DataSourceMetadataQuery extends BaseQuery<Result<DataSourceMetadata
super( super(
dataSource, dataSource,
(querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Collections.singletonList(MY_Y2K_INTERVAL)) (querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Collections.singletonList(MY_Y2K_INTERVAL))
: querySegmentSpec, : querySegmentSpec,
false, false,
context context
); );
@ -102,10 +101,12 @@ public class DataSourceMetadataQuery extends BaseQuery<Result<DataSourceMetadata
public Iterable<Result<DataSourceMetadataResultValue>> buildResult(DateTime timestamp, DateTime maxIngestedEventTime) public Iterable<Result<DataSourceMetadataResultValue>> buildResult(DateTime timestamp, DateTime maxIngestedEventTime)
{ {
return Arrays.asList(new Result<>(timestamp, new DataSourceMetadataResultValue(maxIngestedEventTime))); return Collections.singletonList(new Result<>(timestamp, new DataSourceMetadataResultValue(maxIngestedEventTime)));
} }
public Iterable<Result<DataSourceMetadataResultValue>> mergeResults(List<Result<DataSourceMetadataResultValue>> results) public Iterable<Result<DataSourceMetadataResultValue>> mergeResults(
List<Result<DataSourceMetadataResultValue>> results
)
{ {
if (results == null || results.isEmpty()) { if (results == null || results.isEmpty()) {
return Lists.newArrayList(); return Lists.newArrayList();
@ -126,10 +127,10 @@ public class DataSourceMetadataQuery extends BaseQuery<Result<DataSourceMetadata
public String toString() public String toString()
{ {
return "DataSourceMetadataQuery{" + return "DataSourceMetadataQuery{" +
"dataSource='" + getDataSource() + '\'' + "dataSource='" + getDataSource() + '\'' +
", querySegmentSpec=" + getQuerySegmentSpec() + ", querySegmentSpec=" + getQuerySegmentSpec() +
", duration=" + getDuration() + ", duration=" + getDuration() +
'}'; '}';
} }
} }

View File

@ -735,17 +735,17 @@ public class GroupByQuery extends BaseQuery<Row>
public String toString() public String toString()
{ {
return "GroupByQuery{" + return "GroupByQuery{" +
"dataSource='" + getDataSource() + '\'' + "dataSource='" + getDataSource() + '\'' +
", querySegmentSpec=" + getQuerySegmentSpec() + ", querySegmentSpec=" + getQuerySegmentSpec() +
", virtualColumns=" + virtualColumns + ", virtualColumns=" + virtualColumns +
", limitSpec=" + limitSpec + ", limitSpec=" + limitSpec +
", dimFilter=" + dimFilter + ", dimFilter=" + dimFilter +
", granularity=" + granularity + ", granularity=" + granularity +
", dimensions=" + dimensions + ", dimensions=" + dimensions +
", aggregatorSpecs=" + aggregatorSpecs + ", aggregatorSpecs=" + aggregatorSpecs +
", postAggregatorSpecs=" + postAggregatorSpecs + ", postAggregatorSpecs=" + postAggregatorSpecs +
", havingSpec=" + havingSpec + ", havingSpec=" + havingSpec +
'}'; '}';
} }
@Override @Override
@ -762,13 +762,13 @@ public class GroupByQuery extends BaseQuery<Row>
} }
final GroupByQuery that = (GroupByQuery) o; final GroupByQuery that = (GroupByQuery) o;
return Objects.equals(virtualColumns, that.virtualColumns) && return Objects.equals(virtualColumns, that.virtualColumns) &&
Objects.equals(limitSpec, that.limitSpec) && Objects.equals(limitSpec, that.limitSpec) &&
Objects.equals(havingSpec, that.havingSpec) && Objects.equals(havingSpec, that.havingSpec) &&
Objects.equals(dimFilter, that.dimFilter) && Objects.equals(dimFilter, that.dimFilter) &&
Objects.equals(granularity, that.granularity) && Objects.equals(granularity, that.granularity) &&
Objects.equals(dimensions, that.dimensions) && Objects.equals(dimensions, that.dimensions) &&
Objects.equals(aggregatorSpecs, that.aggregatorSpecs) && Objects.equals(aggregatorSpecs, that.aggregatorSpecs) &&
Objects.equals(postAggregatorSpecs, that.postAggregatorSpecs); Objects.equals(postAggregatorSpecs, that.postAggregatorSpecs);
} }
@Override @Override

View File

@ -26,7 +26,7 @@ import io.druid.data.input.Row;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Collections;
/** /**
* The "=" operator in a "having" clause. This is similar to SQL's "having aggregation = value", * The "=" operator in a "having" clause. This is similar to SQL's "having aggregation = value",
@ -71,7 +71,7 @@ public class EqualToHavingSpec extends BaseHavingSpec
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] aggBytes = StringUtils.toUtf8(aggregationName); final byte[] aggBytes = StringUtils.toUtf8(aggregationName);
final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); final byte[] valBytes = Bytes.toArray(Collections.singletonList(value));
return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length)
.put(CACHE_KEY) .put(CACHE_KEY)
.put(aggBytes) .put(aggBytes)

View File

@ -21,12 +21,10 @@ package io.druid.query.groupby.having;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.primitives.Bytes;
import io.druid.data.input.Row; import io.druid.data.input.Row;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays;
/** /**
* The "&gt;" operator in a "having" clause. This is similar to SQL's "having aggregation &gt; value", * The "&gt;" operator in a "having" clause. This is similar to SQL's "having aggregation &gt; value",
@ -71,12 +69,12 @@ public class GreaterThanHavingSpec extends BaseHavingSpec
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] aggBytes = StringUtils.toUtf8(aggregationName); final byte[] aggBytes = StringUtils.toUtf8(aggregationName);
final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); final byte[] valBytes = new byte[] { value.byteValue() };
return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length)
.put(CACHE_KEY) .put(CACHE_KEY)
.put(aggBytes) .put(aggBytes)
.put(valBytes) .put(valBytes)
.array(); .array();
} }
/** /**

View File

@ -25,7 +25,7 @@ import io.druid.data.input.Row;
import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.StringUtils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Collections;
/** /**
* The "&lt;" operator in a "having" clause. This is similar to SQL's "having aggregation &lt; value", * The "&lt;" operator in a "having" clause. This is similar to SQL's "having aggregation &lt; value",
@ -69,7 +69,7 @@ public class LessThanHavingSpec extends BaseHavingSpec
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] aggBytes = StringUtils.toUtf8(aggregationName); final byte[] aggBytes = StringUtils.toUtf8(aggregationName);
final byte[] valBytes = Bytes.toArray(Arrays.asList(value)); final byte[] valBytes = Bytes.toArray(Collections.singletonList(value));
return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length) return ByteBuffer.allocate(1 + aggBytes.length + valBytes.length)
.put(CACHE_KEY) .put(CACHE_KEY)
.put(aggBytes) .put(aggBytes)

View File

@ -51,7 +51,7 @@ import io.druid.segment.Segment;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.CancellationException; import java.util.concurrent.CancellationException;
@ -110,7 +110,8 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
columns.put(columnName, column); columns.put(columnName, column);
} }
} }
List<Interval> retIntervals = query.analyzingInterval() ? Arrays.asList(segment.getDataInterval()) : null; List<Interval> retIntervals = query.analyzingInterval() ?
Collections.singletonList(segment.getDataInterval()) : null;
final Map<String, AggregatorFactory> aggregators; final Map<String, AggregatorFactory> aggregators;
Metadata metadata = null; Metadata metadata = null;
@ -162,7 +163,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
} }
return Sequences.simple( return Sequences.simple(
Arrays.asList( Collections.singletonList(
new SegmentAnalysis( new SegmentAnalysis(
segment.getIdentifier(), segment.getIdentifier(),
retIntervals, retIntervals,
@ -229,7 +230,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
future.cancel(true); future.cancel(true);
throw new QueryInterruptedException(e); throw new QueryInterruptedException(e);
} }
catch(CancellationException e) { catch (CancellationException e) {
throw new QueryInterruptedException(e); throw new QueryInterruptedException(e);
} }
catch (TimeoutException e) { catch (TimeoutException e) {

View File

@ -37,7 +37,7 @@ import io.druid.query.spec.QuerySegmentSpec;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Collections;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -50,7 +50,7 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
* Prepend 0xFF before the analysisTypes as a separator to avoid * Prepend 0xFF before the analysisTypes as a separator to avoid
* any potential confusion with string values. * any potential confusion with string values.
*/ */
public static final byte[] ANALYSIS_TYPES_CACHE_PREFIX = new byte[]{(byte) 0xFF}; public static final byte[] ANALYSIS_TYPES_CACHE_PREFIX = new byte[] { (byte) 0xFF };
public enum AnalysisType public enum AnalysisType
{ {
@ -78,7 +78,7 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
return new byte[]{(byte) this.ordinal()}; return new byte[] { (byte) this.ordinal() };
} }
} }
@ -112,8 +112,8 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
{ {
super( super(
dataSource, dataSource,
(querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Arrays.asList(DEFAULT_INTERVAL)) (querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Collections.singletonList(DEFAULT_INTERVAL))
: querySegmentSpec, : querySegmentSpec,
false, false,
context context
); );
@ -230,7 +230,6 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
return bytes.array(); return bytes.array();
} }
@Override @Override
public Query<SegmentAnalysis> withOverriddenContext(Map<String, Object> contextOverride) public Query<SegmentAnalysis> withOverriddenContext(Map<String, Object> contextOverride)
{ {
@ -259,14 +258,14 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
public String toString() public String toString()
{ {
return "SegmentMetadataQuery{" + return "SegmentMetadataQuery{" +
"dataSource='" + getDataSource() + '\'' + "dataSource='" + getDataSource() + '\'' +
", querySegmentSpec=" + getQuerySegmentSpec() + ", querySegmentSpec=" + getQuerySegmentSpec() +
", toInclude=" + toInclude + ", toInclude=" + toInclude +
", merge=" + merge + ", merge=" + merge +
", usingDefaultInterval=" + usingDefaultInterval + ", usingDefaultInterval=" + usingDefaultInterval +
", analysisTypes=" + analysisTypes + ", analysisTypes=" + analysisTypes +
", lenientAggregatorMerge=" + lenientAggregatorMerge + ", lenientAggregatorMerge=" + lenientAggregatorMerge +
'}'; '}';
} }
@Override @Override
@ -283,10 +282,10 @@ public class SegmentMetadataQuery extends BaseQuery<SegmentAnalysis>
} }
SegmentMetadataQuery that = (SegmentMetadataQuery) o; SegmentMetadataQuery that = (SegmentMetadataQuery) o;
return merge == that.merge && return merge == that.merge &&
usingDefaultInterval == that.usingDefaultInterval && usingDefaultInterval == that.usingDefaultInterval &&
lenientAggregatorMerge == that.lenientAggregatorMerge && lenientAggregatorMerge == that.lenientAggregatorMerge &&
Objects.equals(toInclude, that.toInclude) && Objects.equals(toInclude, that.toInclude) &&
Objects.equals(analysisTypes, that.analysisTypes); Objects.equals(analysisTypes, that.analysisTypes);
} }
@Override @Override

View File

@ -162,14 +162,14 @@ public class SearchQuery extends BaseQuery<Result<SearchResultValue>>
public String toString() public String toString()
{ {
return "SearchQuery{" + return "SearchQuery{" +
"dataSource='" + getDataSource() + '\'' + "dataSource='" + getDataSource() + '\'' +
", dimFilter=" + dimFilter + ", dimFilter=" + dimFilter +
", granularity='" + granularity + '\'' + ", granularity='" + granularity + '\'' +
", dimensions=" + dimensions + ", dimensions=" + dimensions +
", querySpec=" + querySpec + ", querySpec=" + querySpec +
", querySegmentSpec=" + getQuerySegmentSpec() + ", querySegmentSpec=" + getQuerySegmentSpec() +
", limit=" + limit + ", limit=" + limit +
'}'; '}';
} }
@Override @Override

View File

@ -176,16 +176,16 @@ public class SelectQuery extends BaseQuery<Result<SelectResultValue>>
public String toString() public String toString()
{ {
return "SelectQuery{" + return "SelectQuery{" +
"dataSource='" + getDataSource() + '\'' + "dataSource='" + getDataSource() + '\'' +
", querySegmentSpec=" + getQuerySegmentSpec() + ", querySegmentSpec=" + getQuerySegmentSpec() +
", descending=" + isDescending() + ", descending=" + isDescending() +
", dimFilter=" + dimFilter + ", dimFilter=" + dimFilter +
", granularity=" + granularity + ", granularity=" + granularity +
", dimensions=" + dimensions + ", dimensions=" + dimensions +
", metrics=" + metrics + ", metrics=" + metrics +
", virtualColumns=" + virtualColumns + ", virtualColumns=" + virtualColumns +
", pagingSpec=" + pagingSpec + ", pagingSpec=" + pagingSpec +
'}'; '}';
} }
@Override @Override

View File

@ -26,6 +26,7 @@ import io.druid.java.util.common.IAE;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -39,7 +40,7 @@ public class LegacySegmentSpec extends MultipleIntervalSegmentSpec
if (intervals instanceof String) { if (intervals instanceof String) {
intervalStringList = Arrays.asList((((String) intervals).split(","))); intervalStringList = Arrays.asList((((String) intervals).split(",")));
} else if (intervals instanceof Interval) { } else if (intervals instanceof Interval) {
intervalStringList = Arrays.asList(intervals.toString()); intervalStringList = Collections.singletonList(intervals.toString());
} else if (intervals instanceof Map) { } else if (intervals instanceof Map) {
intervalStringList = (List) ((Map) intervals).get("intervals"); intervalStringList = (List) ((Map) intervals).get("intervals");
} else if (intervals instanceof List) { } else if (intervals instanceof List) {

View File

@ -21,7 +21,7 @@ package io.druid.query.spec;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
/** /**
@ -35,7 +35,7 @@ public class QuerySegmentSpecs
public static QuerySegmentSpec create(Interval interval) public static QuerySegmentSpec create(Interval interval)
{ {
return create(Arrays.asList(interval)); return create(Collections.singletonList(interval));
} }
public static QuerySegmentSpec create(List<Interval> intervals) public static QuerySegmentSpec create(List<Interval> intervals)

View File

@ -25,7 +25,7 @@ import io.druid.query.QuerySegmentWalker;
import io.druid.query.SegmentDescriptor; import io.druid.query.SegmentDescriptor;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
/** /**
@ -43,13 +43,13 @@ public class SpecificSegmentSpec implements QuerySegmentSpec
@Override @Override
public List<Interval> getIntervals() public List<Interval> getIntervals()
{ {
return Arrays.asList(descriptor.getInterval()); return Collections.singletonList(descriptor.getInterval());
} }
@Override @Override
public <T> QueryRunner<T> lookup(Query<T> query, QuerySegmentWalker walker) public <T> QueryRunner<T> lookup(Query<T> query, QuerySegmentWalker walker)
{ {
return walker.getQueryRunnerForSegments(query, Arrays.asList(descriptor)); return walker.getQueryRunnerForSegments(query, Collections.singletonList(descriptor));
} }
public SegmentDescriptor getDescriptor() { return descriptor; } public SegmentDescriptor getDescriptor() { return descriptor; }

View File

@ -37,7 +37,7 @@ import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -68,8 +68,8 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
{ {
super( super(
dataSource, dataSource,
(querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Arrays.asList(MY_Y2K_INTERVAL)) (querySegmentSpec == null) ? new MultipleIntervalSegmentSpec(Collections.singletonList(MY_Y2K_INTERVAL))
: querySegmentSpec, : querySegmentSpec,
false, false,
context context
); );
@ -79,7 +79,8 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
} }
@Override @Override
public boolean hasFilters() { public boolean hasFilters()
{
return dimFilter != null; return dimFilter != null;
} }
@ -123,15 +124,15 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final byte[] filterBytes = dimFilter == null ? new byte[]{} : dimFilter.getCacheKey(); final byte[] filterBytes = dimFilter == null ? new byte[] {} : dimFilter.getCacheKey();
final byte[] boundBytes = StringUtils.toUtf8(bound); final byte[] boundBytes = StringUtils.toUtf8(bound);
final byte delimiter = (byte) 0xff; final byte delimiter = (byte) 0xff;
return ByteBuffer.allocate(2 + boundBytes.length + filterBytes.length) return ByteBuffer.allocate(2 + boundBytes.length + filterBytes.length)
.put(CACHE_TYPE_ID) .put(CACHE_TYPE_ID)
.put(boundBytes) .put(boundBytes)
.put(delimiter) .put(delimiter)
.put(filterBytes) .put(filterBytes)
.array(); .array();
} }
public Iterable<Result<TimeBoundaryResultValue>> buildResult(DateTime timestamp, DateTime min, DateTime max) public Iterable<Result<TimeBoundaryResultValue>> buildResult(DateTime timestamp, DateTime min, DateTime max)
@ -208,12 +209,12 @@ public class TimeBoundaryQuery extends BaseQuery<Result<TimeBoundaryResultValue>
public String toString() public String toString()
{ {
return "TimeBoundaryQuery{" + return "TimeBoundaryQuery{" +
"dataSource='" + getDataSource() + '\'' + "dataSource='" + getDataSource() + '\'' +
", querySegmentSpec=" + getQuerySegmentSpec() + ", querySegmentSpec=" + getQuerySegmentSpec() +
", duration=" + getDuration() + ", duration=" + getDuration() +
", bound=" + bound + ", bound=" + bound +
", dimFilter=" + dimFilter + ", dimFilter=" + dimFilter +
'}'; '}';
} }
@Override @Override

View File

@ -160,16 +160,16 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
public String toString() public String toString()
{ {
return "TimeseriesQuery{" + return "TimeseriesQuery{" +
"dataSource='" + getDataSource() + '\'' + "dataSource='" + getDataSource() + '\'' +
", querySegmentSpec=" + getQuerySegmentSpec() + ", querySegmentSpec=" + getQuerySegmentSpec() +
", descending=" + isDescending() + ", descending=" + isDescending() +
", virtualColumns=" + virtualColumns + ", virtualColumns=" + virtualColumns +
", dimFilter=" + dimFilter + ", dimFilter=" + dimFilter +
", granularity='" + granularity + '\'' + ", granularity='" + granularity + '\'' +
", aggregatorSpecs=" + aggregatorSpecs + ", aggregatorSpecs=" + aggregatorSpecs +
", postAggregatorSpecs=" + postAggregatorSpecs + ", postAggregatorSpecs=" + postAggregatorSpecs +
", context=" + getContext() + ", context=" + getContext() +
'}'; '}';
} }
@Override @Override
@ -186,10 +186,10 @@ public class TimeseriesQuery extends BaseQuery<Result<TimeseriesResultValue>>
} }
final TimeseriesQuery that = (TimeseriesQuery) o; final TimeseriesQuery that = (TimeseriesQuery) o;
return Objects.equals(virtualColumns, that.virtualColumns) && return Objects.equals(virtualColumns, that.virtualColumns) &&
Objects.equals(dimFilter, that.dimFilter) && Objects.equals(dimFilter, that.dimFilter) &&
Objects.equals(granularity, that.granularity) && Objects.equals(granularity, that.granularity) &&
Objects.equals(aggregatorSpecs, that.aggregatorSpecs) && Objects.equals(aggregatorSpecs, that.aggregatorSpecs) &&
Objects.equals(postAggregatorSpecs, that.postAggregatorSpecs); Objects.equals(postAggregatorSpecs, that.postAggregatorSpecs);
} }
@Override @Override

View File

@ -83,8 +83,8 @@ public class TopNQuery extends BaseQuery<Result<TopNResultValue>>
this.postAggregatorSpecs = Queries.prepareAggregations( this.postAggregatorSpecs = Queries.prepareAggregations(
this.aggregatorSpecs, this.aggregatorSpecs,
postAggregatorSpecs == null postAggregatorSpecs == null
? ImmutableList.<PostAggregator>of() ? ImmutableList.<PostAggregator>of()
: postAggregatorSpecs : postAggregatorSpecs
); );
Preconditions.checkNotNull(dimensionSpec, "dimensionSpec can't be null"); Preconditions.checkNotNull(dimensionSpec, "dimensionSpec can't be null");
@ -213,17 +213,17 @@ public class TopNQuery extends BaseQuery<Result<TopNResultValue>>
public String toString() public String toString()
{ {
return "TopNQuery{" + return "TopNQuery{" +
"dataSource='" + getDataSource() + '\'' + "dataSource='" + getDataSource() + '\'' +
", dimensionSpec=" + dimensionSpec + ", dimensionSpec=" + dimensionSpec +
", topNMetricSpec=" + topNMetricSpec + ", topNMetricSpec=" + topNMetricSpec +
", threshold=" + threshold + ", threshold=" + threshold +
", querySegmentSpec=" + getQuerySegmentSpec() + ", querySegmentSpec=" + getQuerySegmentSpec() +
", virtualColumns=" + virtualColumns + ", virtualColumns=" + virtualColumns +
", dimFilter=" + dimFilter + ", dimFilter=" + dimFilter +
", granularity='" + granularity + '\'' + ", granularity='" + granularity + '\'' +
", aggregatorSpecs=" + aggregatorSpecs + ", aggregatorSpecs=" + aggregatorSpecs +
", postAggregatorSpecs=" + postAggregatorSpecs + ", postAggregatorSpecs=" + postAggregatorSpecs +
'}'; '}';
} }
@Override @Override
@ -240,13 +240,13 @@ public class TopNQuery extends BaseQuery<Result<TopNResultValue>>
} }
final TopNQuery topNQuery = (TopNQuery) o; final TopNQuery topNQuery = (TopNQuery) o;
return threshold == topNQuery.threshold && return threshold == topNQuery.threshold &&
Objects.equals(virtualColumns, topNQuery.virtualColumns) && Objects.equals(virtualColumns, topNQuery.virtualColumns) &&
Objects.equals(dimensionSpec, topNQuery.dimensionSpec) && Objects.equals(dimensionSpec, topNQuery.dimensionSpec) &&
Objects.equals(topNMetricSpec, topNQuery.topNMetricSpec) && Objects.equals(topNMetricSpec, topNQuery.topNMetricSpec) &&
Objects.equals(dimFilter, topNQuery.dimFilter) && Objects.equals(dimFilter, topNQuery.dimFilter) &&
Objects.equals(granularity, topNQuery.granularity) && Objects.equals(granularity, topNQuery.granularity) &&
Objects.equals(aggregatorSpecs, topNQuery.aggregatorSpecs) && Objects.equals(aggregatorSpecs, topNQuery.aggregatorSpecs) &&
Objects.equals(postAggregatorSpecs, topNQuery.postAggregatorSpecs); Objects.equals(postAggregatorSpecs, topNQuery.postAggregatorSpecs);
} }
@Override @Override

View File

@ -94,6 +94,7 @@ import java.nio.ByteBuffer;
import java.nio.ByteOrder; import java.nio.ByteOrder;
import java.util.AbstractList; import java.util.AbstractList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator; import java.util.Iterator;
import java.util.LinkedHashSet; import java.util.LinkedHashSet;
import java.util.List; import java.util.List;
@ -605,14 +606,14 @@ public class IndexIO
); );
bitmaps = GenericIndexed.fromIterable( bitmaps = GenericIndexed.fromIterable(
Iterables.concat(Arrays.asList(theNullSet), bitmaps), Iterables.concat(Collections.singletonList(theNullSet), bitmaps),
bitmapSerdeFactory.getObjectStrategy() bitmapSerdeFactory.getObjectStrategy()
); );
} else { } else {
bumpedDictionary = false; bumpedDictionary = false;
bitmaps = GenericIndexed.fromIterable( bitmaps = GenericIndexed.fromIterable(
Iterables.concat( Iterables.concat(
Arrays.asList( Collections.singletonList(
bitmapFactory bitmapFactory
.union(Arrays.asList(theNullSet, bitmaps.get(0))) .union(Arrays.asList(theNullSet, bitmaps.get(0)))
), ),

View File

@ -86,6 +86,7 @@ import java.nio.IntBuffer;
import java.nio.channels.FileChannel; import java.nio.channels.FileChannel;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -101,7 +102,7 @@ public class IndexMerger
{ {
private static final Logger log = new Logger(IndexMerger.class); private static final Logger log = new Logger(IndexMerger.class);
protected static final ListIndexed EMPTY_STR_DIM_VAL = new ListIndexed<>(Arrays.asList(""), String.class); protected static final ListIndexed EMPTY_STR_DIM_VAL = new ListIndexed<>(Collections.singletonList(""), String.class);
protected static final SerializerUtils serializerUtils = new SerializerUtils(); protected static final SerializerUtils serializerUtils = new SerializerUtils();
protected static final int INVALID_ROW = -1; protected static final int INVALID_ROW = -1;
protected static final Splitter SPLITTER = Splitter.on(","); protected static final Splitter SPLITTER = Splitter.on(",");

View File

@ -64,14 +64,14 @@ import java.io.IOException;
import java.nio.IntBuffer; import java.nio.IntBuffer;
import java.nio.MappedByteBuffer; import java.nio.MappedByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
public class StringDimensionMergerV9 implements DimensionMergerV9<int[]> public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
{ {
private static final Logger log = new Logger(StringDimensionMergerV9.class); private static final Logger log = new Logger(StringDimensionMergerV9.class);
protected static final ListIndexed EMPTY_STR_DIM_VAL = new ListIndexed<>(Arrays.asList(""), String.class); protected static final ListIndexed EMPTY_STR_DIM_VAL = new ListIndexed<>(Collections.singletonList(""), String.class);
protected static final int[] EMPTY_STR_DIM_ARRAY = new int[]{0}; protected static final int[] EMPTY_STR_DIM_ARRAY = new int[]{0};
protected static final Splitter SPLITTER = Splitter.on(","); protected static final Splitter SPLITTER = Splitter.on(",");

View File

@ -23,7 +23,6 @@ import com.google.common.io.ByteSink;
import com.google.common.io.ByteStreams; import com.google.common.io.ByteStreams;
import com.google.common.io.CountingOutputStream; import com.google.common.io.CountingOutputStream;
import com.google.common.primitives.Ints; import com.google.common.primitives.Ints;
import io.druid.collections.ResourceHolder; import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidResourceHolder; import io.druid.collections.StupidResourceHolder;
import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.FileSmoosher;
@ -62,7 +61,7 @@ public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer
{ {
this.ioPeon = ioPeon; this.ioPeon = ioPeon;
this.sizePer = writer.getBlockSize(CompressedPools.BUFFER_SIZE); this.sizePer = writer.getBlockSize(CompressedPools.BUFFER_SIZE);
this.flattener = new GenericIndexedWriter<ResourceHolder<ByteBuffer>>( this.flattener = new GenericIndexedWriter<>(
ioPeon, ioPeon,
filenameBase, filenameBase,
VSizeCompressedObjectStrategy.getBufferForOrder( VSizeCompressedObjectStrategy.getBufferForOrder(

View File

@ -75,14 +75,14 @@ public class CompressedIntsIndexedWriter extends SingleValueIndexedIntsWriter
public CompressedIntsIndexedWriter( public CompressedIntsIndexedWriter(
final int chunkFactor, final int chunkFactor,
final CompressedObjectStrategy.CompressionStrategy compression, final CompressedObjectStrategy.CompressionStrategy compression,
GenericIndexedWriter<ResourceHolder<IntBuffer>> flattner GenericIndexedWriter<ResourceHolder<IntBuffer>> flattener
) )
{ {
this.chunkFactor = chunkFactor; this.chunkFactor = chunkFactor;
this.compression = compression; this.compression = compression;
this.endBuffer = IntBuffer.allocate(chunkFactor); this.endBuffer = IntBuffer.allocate(chunkFactor);
this.numInserted = 0; this.numInserted = 0;
this.flattener = flattner; this.flattener = flattener;
} }
@Override @Override

View File

@ -357,7 +357,7 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier<Inde
/** /**
* Returns the value at the given index in the current decompression buffer * Returns the value at the given index in the current decompression buffer
* *
* @param index index of the value in the curent buffer * @param index index of the value in the current buffer
* *
* @return the value at the given index * @return the value at the given index
*/ */

View File

@ -391,7 +391,6 @@ public class GenericIndexedWriter<T> implements Closeable
} }
writeHeaderLong(smoosher, headerFile, bagSizePower, buffer); writeHeaderLong(smoosher, headerFile, bagSizePower, buffer);
} }
} }
public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
@ -403,7 +402,8 @@ public class GenericIndexedWriter<T> implements Closeable
} }
} }
private void writeHeaderLong(FileSmoosher smoosher, RandomAccessFile headerFile, int bagSizePower, byte[] buffer) throws IOException private void writeHeaderLong(FileSmoosher smoosher, RandomAccessFile headerFile, int bagSizePower, byte[] buffer)
throws IOException
{ {
ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES).order(ByteOrder.nativeOrder()); ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES).order(ByteOrder.nativeOrder());

View File

@ -30,7 +30,7 @@ public class IndexedIntsIterator extends AbstractIntIterator
private final IndexedInts baseInts; private final IndexedInts baseInts;
private final int size; private final int size;
int currIndex = 0; private int currIndex = 0;
public IndexedIntsIterator( public IndexedIntsIterator(
IndexedInts baseInts IndexedInts baseInts
@ -48,7 +48,8 @@ public class IndexedIntsIterator extends AbstractIntIterator
} }
@Override @Override
public int nextInt() { public int nextInt()
{
return baseInts.get(currIndex++); return baseInts.get(currIndex++);
} }

View File

@ -116,16 +116,11 @@ public class VSizeIndexedWriter extends MultiValueIndexedIntsWriter implements C
numBytesWritten < Integer.MAX_VALUE, "Wrote[%s] bytes, which is too many.", numBytesWritten numBytesWritten < Integer.MAX_VALUE, "Wrote[%s] bytes, which is too many.", numBytesWritten
); );
OutputStream metaOut = ioPeon.makeOutputStream(metaFileName); try (OutputStream metaOut = ioPeon.makeOutputStream(metaFileName)) {
try {
metaOut.write(new byte[]{VERSION, numBytesForMax}); metaOut.write(new byte[]{VERSION, numBytesForMax});
metaOut.write(Ints.toByteArray((int) numBytesWritten + 4)); metaOut.write(Ints.toByteArray((int) numBytesWritten + 4));
metaOut.write(Ints.toByteArray(numWritten)); metaOut.write(Ints.toByteArray(numWritten));
} }
finally {
metaOut.close();
}
} }
public InputSupplier<InputStream> combineStreams() public InputSupplier<InputStream> combineStreams()
@ -133,8 +128,8 @@ public class VSizeIndexedWriter extends MultiValueIndexedIntsWriter implements C
return ByteStreams.join( return ByteStreams.join(
Iterables.transform( Iterables.transform(
Arrays.asList(metaFileName, headerFileName, valuesFileName), Arrays.asList(metaFileName, headerFileName, valuesFileName),
new Function<String,InputSupplier<InputStream>>() { new Function<String, InputSupplier<InputStream>>()
{
@Override @Override
public InputSupplier<InputStream> apply(final String input) public InputSupplier<InputStream> apply(final String input)
{ {

View File

@ -70,6 +70,7 @@ import java.io.Closeable;
import java.lang.reflect.Array; import java.lang.reflect.Array;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator; import java.util.Comparator;
import java.util.Deque; import java.util.Deque;
import java.util.Iterator; import java.util.Iterator;
@ -868,9 +869,9 @@ public abstract class IncrementalIndex<AggregatorType> implements Iterable<Row>,
public Object apply(@Nullable Object input) public Object apply(@Nullable Object input)
{ {
if (input == null || Array.getLength(input) == 0) { if (input == null || Array.getLength(input) == 0) {
return Arrays.asList("null"); return Collections.singletonList("null");
} }
return Arrays.asList(input); return Collections.singletonList(input);
} }
} }
) + '}'; ) + '}';

View File

@ -36,7 +36,7 @@ import io.druid.java.util.common.ISE;
import io.druid.java.util.common.parsers.ParseException; import io.druid.java.util.common.parsers.ParseException;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -189,7 +189,7 @@ public class SpatialDimensionRowTransformer implements Function<InputRow, InputR
} }
if (spatialDimVals.size() == spatialDim.getDims().size()) { if (spatialDimVals.size() == spatialDim.getDims().size()) {
spatialLookup.put(spatialDimName, Arrays.asList(JOINER.join(spatialDimVals))); spatialLookup.put(spatialDimName, Collections.singletonList(JOINER.join(spatialDimVals)));
finalDims.add(spatialDimName); finalDims.add(spatialDimName);
} }
} }

View File

@ -247,7 +247,6 @@ public class GroupByQueryRunnerFailureTest
@Test(timeout = 10000, expected = InsufficientResourcesException.class) @Test(timeout = 10000, expected = InsufficientResourcesException.class)
public void testInsufficientResourcesOnBroker() throws IOException public void testInsufficientResourcesOnBroker() throws IOException
{ {
final ReferenceCountingResourceHolder<List<ByteBuffer>> holder = mergeBufferPool.takeBatch(1, 10);
final GroupByQuery query = GroupByQuery final GroupByQuery query = GroupByQuery
.builder() .builder()
.setDataSource( .setDataSource(
@ -267,10 +266,8 @@ public class GroupByQueryRunnerFailureTest
.setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 500)) .setContext(ImmutableMap.of(QueryContexts.TIMEOUT_KEY, 500))
.build(); .build();
try { try (ReferenceCountingResourceHolder<List<ByteBuffer>> holder = mergeBufferPool.takeBatch(1, 10)) {
GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); GroupByQueryRunnerTestHelper.runQuery(factory, runner, query);
} finally {
holder.close();
} }
} }
} }

View File

@ -48,7 +48,7 @@ import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -154,7 +154,7 @@ public class SpecificSegmentQueryRunnerTest
public Sequence run(QueryPlus queryPlus, Map responseContext) public Sequence run(QueryPlus queryPlus, Map responseContext)
{ {
return Sequences.withEffect( return Sequences.withEffect(
Sequences.simple(Arrays.asList(value)), Sequences.simple(Collections.singletonList(value)),
new Runnable() new Runnable()
{ {
@Override @Override

View File

@ -201,7 +201,6 @@ public class IndexMergerV9CompatibilityTest
finally { finally {
if (index != null) { if (index != null) {
index.close(); index.close();
;
} }
if (outDir != null) { if (outDir != null) {

View File

@ -115,10 +115,9 @@ public class CompressedVSizeIndexedV3WriterTest
private void checkSerializedSizeAndData(int offsetChunkFactor, int valueChunkFactor) throws Exception private void checkSerializedSizeAndData(int offsetChunkFactor, int valueChunkFactor) throws Exception
{ {
FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory()); FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory());
final IOPeon ioPeon = new TmpFileIOPeon();
final IndexedMultivalue<IndexedInts> indexedMultivalue; final IndexedMultivalue<IndexedInts> indexedMultivalue;
try { try (IOPeon ioPeon = new TmpFileIOPeon()) {
int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0; int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0;
CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter( CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter(
ioPeon, "offset", offsetChunkFactor, byteOrder, compressionStrategy ioPeon, "offset", offsetChunkFactor, byteOrder, compressionStrategy
@ -170,9 +169,6 @@ public class CompressedVSizeIndexedV3WriterTest
} }
CloseQuietly.close(indexedMultivalue); CloseQuietly.close(indexedMultivalue);
} }
finally {
ioPeon.close();
}
} }
int getMaxValue(final List<int[]> vals) int getMaxValue(final List<int[]> vals)
@ -245,10 +241,9 @@ public class CompressedVSizeIndexedV3WriterTest
offsetChunkFactor offsetChunkFactor
)).toFile(); )).toFile();
FileSmoosher smoosher = new FileSmoosher(tmpDirectory); FileSmoosher smoosher = new FileSmoosher(tmpDirectory);
final IOPeon ioPeon = new TmpFileIOPeon();
int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0; int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0;
try { try (IOPeon ioPeon = new TmpFileIOPeon()) {
CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter( CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter(
offsetChunkFactor, offsetChunkFactor,
compressionStrategy, compressionStrategy,
@ -316,9 +311,6 @@ public class CompressedVSizeIndexedV3WriterTest
CloseQuietly.close(indexedMultivalue); CloseQuietly.close(indexedMultivalue);
mapper.close(); mapper.close();
} }
finally {
ioPeon.close();
}
} }
@Test @Test

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.inject.Binder; import com.google.inject.Binder;
import com.google.inject.Provides; import com.google.inject.Provides;
import io.druid.guice.annotations.Self; import io.druid.guice.annotations.Self;
import io.druid.initialization.DruidModule; import io.druid.initialization.DruidModule;
import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; import io.druid.java.util.common.concurrent.ScheduledExecutorFactory;
@ -36,7 +35,7 @@ import io.druid.timeline.partition.LinearShardSpec;
import io.druid.timeline.partition.NumberedShardSpec; import io.druid.timeline.partition.NumberedShardSpec;
import io.druid.timeline.partition.SingleDimensionShardSpec; import io.druid.timeline.partition.SingleDimensionShardSpec;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
/** /**
@ -61,7 +60,7 @@ public class ServerModule implements DruidModule
@Override @Override
public List<? extends com.fasterxml.jackson.databind.Module> getJacksonModules() public List<? extends com.fasterxml.jackson.databind.Module> getJacksonModules()
{ {
return Arrays.asList( return Collections.singletonList(
new SimpleModule() new SimpleModule()
.registerSubtypes( .registerSubtypes(
new NamedType(SingleDimensionShardSpec.class, "single"), new NamedType(SingleDimensionShardSpec.class, "single"),

View File

@ -68,6 +68,7 @@ import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
@ -381,11 +382,11 @@ public class ServerManager implements QuerySegmentWalker
final PartitionChunk<ReferenceCountingSegment> chunk = entry.getChunk(input.getPartitionNumber()); final PartitionChunk<ReferenceCountingSegment> chunk = entry.getChunk(input.getPartitionNumber());
if (chunk == null) { if (chunk == null) {
return Arrays.<QueryRunner<T>>asList(new ReportTimelineMissingSegmentQueryRunner<T>(input)); return Collections.singletonList(new ReportTimelineMissingSegmentQueryRunner<T>(input));
} }
final ReferenceCountingSegment adapter = chunk.getObject(); final ReferenceCountingSegment adapter = chunk.getObject();
return Arrays.asList( return Collections.singletonList(
buildAndDecorateQueryRunner(factory, toolChest, adapter, input, cpuTimeAccumulator) buildAndDecorateQueryRunner(factory, toolChest, adapter, input, cpuTimeAccumulator)
); );
} }

View File

@ -305,7 +305,7 @@ public class ZkCoordinator implements DataSegmentChangeHandler
} }
/** /**
* Load a single segment. If the segment is loaded succesfully, this function simply returns. Otherwise it will * Load a single segment. If the segment is loaded successfully, this function simply returns. Otherwise it will
* throw a SegmentLoadingException * throw a SegmentLoadingException
* *
* @throws SegmentLoadingException * @throws SegmentLoadingException
@ -606,7 +606,7 @@ public class ZkCoordinator implements DataSegmentChangeHandler
throw new SegmentLoadingException(e, "Failed to announce segments[%s]", queue); throw new SegmentLoadingException(e, "Failed to announce segments[%s]", queue);
} }
// get any exception that may have been thrown in background annoucing // get any exception that may have been thrown in background announcing
try { try {
// check in case intervalMillis is <= 0 // check in case intervalMillis is <= 0
if (startedAnnouncing != null) { if (startedAnnouncing != null) {

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.EmittingLogger;
import io.druid.java.util.common.ISE; import io.druid.java.util.common.ISE;
import io.druid.java.util.common.concurrent.ScheduledExecutors; import io.druid.java.util.common.concurrent.ScheduledExecutors;
import io.druid.server.coordination.DataSegmentChangeRequest; import io.druid.server.coordination.DataSegmentChangeRequest;
@ -38,8 +37,8 @@ import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.WatchedEvent;
import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.data.Stat;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
@ -155,7 +154,7 @@ public class LoadQueuePeon
log.info("Asking server peon[%s] to load segment[%s]", basePath, segment.getIdentifier()); log.info("Asking server peon[%s] to load segment[%s]", basePath, segment.getIdentifier());
queuedSize.addAndGet(segment.getSize()); queuedSize.addAndGet(segment.getSize());
segmentsToLoad.put(segment, new SegmentHolder(segment, LOAD, Arrays.asList(callback))); segmentsToLoad.put(segment, new SegmentHolder(segment, LOAD, Collections.singletonList(callback)));
} }
public void dropSegment( public void dropSegment(
@ -184,7 +183,7 @@ public class LoadQueuePeon
} }
log.info("Asking server peon[%s] to drop segment[%s]", basePath, segment.getIdentifier()); log.info("Asking server peon[%s] to drop segment[%s]", basePath, segment.getIdentifier());
segmentsToDrop.put(segment, new SegmentHolder(segment, DROP, Arrays.asList(callback))); segmentsToDrop.put(segment, new SegmentHolder(segment, DROP, Collections.singletonList(callback)));
} }
private void processSegmentChangeRequest() { private void processSegmentChangeRequest() {

View File

@ -45,11 +45,11 @@ public class CostBalancerStrategyBenchmark extends AbstractBenchmark
return Arrays.asList( return Arrays.asList(
(CostBalancerStrategy[]) Arrays.asList( (CostBalancerStrategy[]) Arrays.asList(
new CostBalancerStrategy(MoreExecutors.listeningDecorator( new CostBalancerStrategy(MoreExecutors.listeningDecorator(
Executors.newFixedThreadPool(1))) Executors.newFixedThreadPool(1)))
).toArray(), ).toArray(),
(CostBalancerStrategy[]) Arrays.asList( (CostBalancerStrategy[]) Arrays.asList(
new CostBalancerStrategy(MoreExecutors.listeningDecorator( new CostBalancerStrategy(MoreExecutors.listeningDecorator(
Executors.newFixedThreadPool(4))) Executors.newFixedThreadPool(4)))
).toArray() ).toArray()
); );
} }
@ -71,7 +71,8 @@ public class CostBalancerStrategyBenchmark extends AbstractBenchmark
} }
@AfterClass @AfterClass
public static void tearDown(){ public static void tearDown()
{
serverHolderList = null; serverHolderList = null;
} }

View File

@ -38,7 +38,7 @@ import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays; import java.util.Collections;
import java.util.List; import java.util.List;
public class DruidCoordinatorCleanupOvershadowedTest public class DruidCoordinatorCleanupOvershadowedTest
@ -63,6 +63,7 @@ public class DruidCoordinatorCleanupOvershadowedTest
.interval(new Interval(start, start.plusHours(1))) .interval(new Interval(start, start.plusHours(1)))
.version("2") .version("2")
.build(); .build();
@Test @Test
public void testRun() public void testRun()
{ {
@ -70,14 +71,16 @@ public class DruidCoordinatorCleanupOvershadowedTest
availableSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); availableSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2);
druidCluster = new DruidCluster( druidCluster = new DruidCluster(
ImmutableMap.of("normal", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create(Arrays.asList( ImmutableMap.of("normal", MinMaxPriorityQueue.orderedBy(Ordering.natural().reverse()).create(
new ServerHolder(druidServer, mockPeon Collections.singletonList(new ServerHolder(druidServer, mockPeon))
))))); )));
EasyMock.expect(druidServer.getDataSources()) EasyMock.expect(druidServer.getDataSources())
.andReturn(ImmutableList.of(druidDataSource)) .andReturn(ImmutableList.of(druidDataSource))
.anyTimes(); .anyTimes();
EasyMock.expect(druidDataSource.getSegments()).andReturn(ImmutableSet.<DataSegment>of(segmentV1, segmentV2)).anyTimes(); EasyMock.expect(druidDataSource.getSegments())
.andReturn(ImmutableSet.<DataSegment>of(segmentV1, segmentV2))
.anyTimes();
EasyMock.expect(druidDataSource.getName()).andReturn("test").anyTimes(); EasyMock.expect(druidDataSource.getName()).andReturn("test").anyTimes();
coordinator.removeSegment(segmentV1); coordinator.removeSegment(segmentV1);
coordinator.removeSegment(segmentV0); coordinator.removeSegment(segmentV0);