From 15ccf451f9b12cbceff853dd71fac7accc258be8 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 17 May 2016 16:23:48 -0700 Subject: [PATCH] Move QueryGranularity static fields to QueryGranularities (#2980) * Move QueryGranularity static fields to QueryGranularityUtil * Fixes #2979 * Add test showing #2979 * change name to QueryGranularities --- .../IncrementalIndexAddRowsBenchmark.java | 4 +- .../DistinctCountGroupByQueryTest.java | 4 +- .../DistinctCountTimeseriesQueryTest.java | 4 +- .../DistinctCountTopNQueryTest.java | 4 +- .../theta/SketchAggregationTest.java | 10 +- .../oldapi/OldApiSketchAggregationTest.java | 6 +- .../ApproximateHistogramAggregationTest.java | 4 +- .../indexing/kafka/KafkaIndexTaskTest.java | 7 +- .../kafka/supervisor/KafkaSupervisorTest.java | 4 +- .../hadoop/DatasourceIngestionSpec.java | 3 +- .../indexer/BatchDeltaIngestionTest.java | 6 +- .../DetermineHashedPartitionsJobTest.java | 4 +- .../indexer/DeterminePartitionsJobTest.java | 4 +- .../indexer/HadoopDruidIndexerConfigTest.java | 6 +- .../indexer/IndexGeneratorCombinerTest.java | 4 +- .../druid/indexer/IndexGeneratorJobTest.java | 4 +- .../java/io/druid/indexer/JobHelperTest.java | 4 +- .../hadoop/DatasourceIngestionSpecTest.java | 6 +- .../indexer/path/DatasourcePathSpecTest.java | 4 +- .../indexer/path/GranularityPathSpecTest.java | 4 +- .../updater/HadoopConverterJobTest.java | 4 +- .../IngestSegmentFirehoseFactory.java | 4 +- .../actions/SegmentAllocateActionTest.java | 53 ++++++----- .../indexing/common/task/IndexTaskTest.java | 8 +- .../common/task/RealtimeIndexTaskTest.java | 6 +- .../indexing/common/task/TaskSerdeTest.java | 4 +- .../IngestSegmentFirehoseFactoryTest.java | 4 +- ...estSegmentFirehoseFactoryTimelineTest.java | 4 +- .../indexing/overlord/TaskLifecycleTest.java | 4 +- .../druid/granularity/QueryGranularities.java | 45 +++++++++ .../druid/granularity/QueryGranularity.java | 68 ++++--------- .../src/main/java/io/druid/query/Druids.java | 7 +- .../druid/query/metadata/SegmentAnalyzer.java | 4 +- .../druid/query/search/SearchQueryRunner.java | 4 +- .../query/search/search/SearchQuery.java | 3 +- .../io/druid/query/topn/TopNQueryBuilder.java | 3 +- .../incremental/IncrementalIndexSchema.java | 3 +- .../granularity/QueryGranularityTest.java | 95 +++++++++++-------- .../druid/query/MultiValuedDimensionTest.java | 12 +-- .../io/druid/query/QueryRunnerTestHelper.java | 5 +- ...ResultGranularTimestampComparatorTest.java | 7 +- .../HyperUniquesAggregationTest.java | 4 +- .../DataSourceMetadataQueryTest.java | 4 +- .../GroupByQueryRunnerFactoryTest.java | 6 +- .../query/groupby/GroupByQueryRunnerTest.java | 8 +- .../metadata/SegmentMetadataQueryTest.java | 4 +- .../query/search/SearchBinaryFnTest.java | 20 ++-- .../search/SearchQueryQueryToolChestTest.java | 4 +- .../select/MultiSegmentSelectQueryTest.java | 4 +- .../query/select/SelectBinaryFnTest.java | 4 +- .../spec/SpecificSegmentQueryRunnerTest.java | 6 +- .../timeseries/TimeseriesBinaryFnTest.java | 12 +-- .../TimeseriesQueryQueryToolChestTest.java | 4 +- .../TimeseriesQueryRunnerBonusTest.java | 6 +- .../timeseries/TimeseriesQueryRunnerTest.java | 9 +- .../query/topn/TopNBinaryFnBenchmark.java | 4 +- .../io/druid/query/topn/TopNBinaryFnTest.java | 16 ++-- .../topn/TopNQueryQueryToolChestTest.java | 4 +- .../druid/query/topn/TopNQueryRunnerTest.java | 11 ++- .../java/io/druid/segment/AppendTest.java | 3 +- .../java/io/druid/segment/EmptyIndexTest.java | 4 +- .../java/io/druid/segment/IndexIOTest.java | 6 +- .../io/druid/segment/IndexMergerTest.java | 26 ++--- .../IndexMergerV9CompatibilityTest.java | 4 +- .../IndexMergerV9WithSpatialIndexTest.java | 16 ++-- .../java/io/druid/segment/MetadataTest.java | 12 +-- .../io/druid/segment/SchemalessIndex.java | 8 +- .../io/druid/segment/SchemalessTestFull.java | 3 +- .../druid/segment/SchemalessTestSimple.java | 3 +- .../test/java/io/druid/segment/TestIndex.java | 4 +- .../segment/data/IncrementalIndexTest.java | 14 +-- .../druid/segment/filter/BaseFilterTest.java | 4 +- .../filter/SpatialFilterBonusTest.java | 16 ++-- .../segment/filter/SpatialFilterTest.java | 16 ++-- .../IncrementalIndexStorageAdapterTest.java | 16 ++-- .../incremental/IncrementalIndexTest.java | 4 +- .../OnheapIncrementalIndexBenchmark.java | 7 +- .../incremental/TimeAndDimsCompTest.java | 4 +- .../granularity/UniformGranularitySpec.java | 3 +- .../client/CachingClusteredClientTest.java | 3 +- .../druid/client/CachingQueryRunnerTest.java | 4 +- .../segment/indexing/DataSchemaTest.java | 10 +- .../granularity/ArbitraryGranularityTest.java | 10 +- .../granularity/UniformGranularityTest.java | 4 +- .../segment/realtime/FireDepartmentTest.java | 4 +- .../segment/realtime/RealtimeManagerTest.java | 8 +- .../appenderator/AppenderatorTest.java | 12 +-- .../appenderator/AppenderatorTester.java | 4 +- .../firehose/IngestSegmentFirehoseTest.java | 6 +- .../plumber/RealtimePlumberSchoolTest.java | 6 +- .../segment/realtime/plumber/SinkTest.java | 4 +- .../coordination/ServerManagerTest.java | 23 ++--- .../cli/validate/DruidJsonValidatorTest.java | 4 +- 93 files changed, 445 insertions(+), 403 deletions(-) create mode 100644 processing/src/main/java/io/druid/granularity/QueryGranularities.java diff --git a/benchmarks/src/main/java/io/druid/benchmark/IncrementalIndexAddRowsBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/IncrementalIndexAddRowsBenchmark.java index c9d10d25571..492fb327464 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/IncrementalIndexAddRowsBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/IncrementalIndexAddRowsBenchmark.java @@ -22,7 +22,7 @@ package io.druid.benchmark; import com.google.common.collect.ImmutableMap; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -122,7 +122,7 @@ public class IncrementalIndexAddRowsBenchmark { return new OnheapIncrementalIndex( 0, - QueryGranularity.NONE, + QueryGranularities.NONE, aggs, false, false, diff --git a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java index 2b9ece5ae67..81a2319c00e 100644 --- a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java @@ -27,7 +27,7 @@ import com.google.common.collect.Lists; import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.Row; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.aggregation.AggregatorFactory; @@ -89,7 +89,7 @@ public class DistinctCountGroupByQueryTest ); IncrementalIndex index = new OnheapIncrementalIndex( - 0, QueryGranularity.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 + 0, QueryGranularities.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 ); String visitor_id = "visitor_id"; String client_type = "client_type"; diff --git a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java index 7581c4ed1e1..a41fe5a33ef 100644 --- a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java @@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import io.druid.data.input.MapBasedInputRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; @@ -51,7 +51,7 @@ public class DistinctCountTimeseriesQueryTest TimeseriesQueryEngine engine = new TimeseriesQueryEngine(); IncrementalIndex index = new OnheapIncrementalIndex( - 0, QueryGranularity.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 + 0, QueryGranularities.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 ); String visitor_id = "visitor_id"; String client_type = "client_type"; diff --git a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java index 561a0909db1..0023c42389f 100644 --- a/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/io/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java @@ -25,7 +25,7 @@ import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedInputRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.QueryRunnerTestHelper; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; @@ -66,7 +66,7 @@ public class DistinctCountTopNQueryTest ); IncrementalIndex index = new OnheapIncrementalIndex( - 0, QueryGranularity.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 + 0, QueryGranularities.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 ); String visitor_id = "visitor_id"; String client_type = "client_type"; diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java index ad1c79be6b1..8529b98c82c 100644 --- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java +++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java @@ -30,7 +30,7 @@ import com.yahoo.sketches.theta.Sketch; import com.yahoo.sketches.theta.Sketches; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Result; import io.druid.query.aggregation.AggregationTestHelper; import io.druid.query.aggregation.AggregatorFactory; @@ -73,7 +73,7 @@ public class SketchAggregationTest readFileFromClasspathAsString("simple_test_data_record_parser.json"), readFileFromClasspathAsString("simple_test_data_aggregators.json"), 0, - QueryGranularity.NONE, + QueryGranularities.NONE, 5, readFileFromClasspathAsString("simple_test_data_group_by_query.json") ); @@ -167,7 +167,7 @@ public class SketchAggregationTest readFileFromClasspathAsString("simple_test_data_record_parser.json"), readFileFromClasspathAsString("simple_test_data_aggregators.json"), 0, - QueryGranularity.NONE, + QueryGranularities.NONE, 5000, readFileFromClasspathAsString("select_query.json") ); @@ -186,7 +186,7 @@ public class SketchAggregationTest readFileFromClasspathAsString("sketch_test_data_record_parser.json"), readFileFromClasspathAsString("sketch_test_data_aggregators.json"), 0, - QueryGranularity.NONE, + QueryGranularities.NONE, 5, readFileFromClasspathAsString("sketch_test_data_group_by_query.json") ); @@ -227,7 +227,7 @@ public class SketchAggregationTest + " }" + "]", 0, - QueryGranularity.NONE, + QueryGranularities.NONE, 5, readFileFromClasspathAsString("simple_test_data_group_by_query.json") ); diff --git a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java index b7bad2c527d..dcba329259e 100644 --- a/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java +++ b/extensions-core/datasketches/src/test/java/io/druid/query/aggregation/datasketches/theta/oldapi/OldApiSketchAggregationTest.java @@ -25,7 +25,7 @@ import com.google.common.io.Files; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.data.input.MapBasedRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregationTestHelper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; @@ -69,7 +69,7 @@ public class OldApiSketchAggregationTest readFileFromClasspathAsString("simple_test_data_record_parser.json"), readFileFromClasspathAsString("oldapi/old_simple_test_data_aggregators.json"), 0, - QueryGranularity.NONE, + QueryGranularities.NONE, 5, readFileFromClasspathAsString("oldapi/old_simple_test_data_group_by_query.json") ); @@ -101,7 +101,7 @@ public class OldApiSketchAggregationTest readFileFromClasspathAsString("sketch_test_data_record_parser.json"), readFileFromClasspathAsString("oldapi/old_sketch_test_data_aggregators.json"), 0, - QueryGranularity.NONE, + QueryGranularities.NONE, 5, readFileFromClasspathAsString("oldapi/old_sketch_test_data_group_by_query.json") ); diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java index 27b52c18d24..ec3c035d00b 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/ApproximateHistogramAggregationTest.java @@ -23,7 +23,7 @@ import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.data.input.MapBasedRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregationTestHelper; import org.junit.Assert; import org.junit.Rule; @@ -115,7 +115,7 @@ public class ApproximateHistogramAggregationTest parseSpec, metricSpec, 0, - QueryGranularity.NONE, + QueryGranularities.NONE, 50000, query ); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 6b5e1b8df48..8f51b68ddb8 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -19,7 +19,6 @@ package io.druid.indexing.kafka; -import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; @@ -56,7 +55,7 @@ import io.druid.data.input.impl.JSONPathFieldSpec; import io.druid.data.input.impl.JSONPathSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; @@ -198,7 +197,7 @@ public class KafkaIndexTaskTest Map.class ), new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularity.DAY, QueryGranularity.NONE, null), + new UniformGranularitySpec(Granularity.DAY, QueryGranularities.NONE, null), objectMapper ); } @@ -1442,7 +1441,7 @@ public class KafkaIndexTaskTest ImmutableList.of( new LongSumAggregatorFactory("rows", "rows") ) - ).granularity(QueryGranularity.ALL) + ).granularity(QueryGranularities.ALL) .intervals("0000/3000") .build(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index fd381c2cf7f..94b4adbf82b 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -35,7 +35,7 @@ import io.druid.data.input.impl.JSONPathSpec; import io.druid.data.input.impl.StringDimensionSchema; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.indexing.common.TaskLocation; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.task.RealtimeIndexTask; @@ -1014,7 +1014,7 @@ public class KafkaSupervisorTest extends EasyMockSupport new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec( Granularity.HOUR, - QueryGranularity.NONE, + QueryGranularities.NONE, ImmutableList.of() ), objectMapper diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceIngestionSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceIngestionSpec.java index e7e7bb8c1b9..ffa143f2ef1 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceIngestionSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceIngestionSpec.java @@ -25,6 +25,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import io.druid.common.utils.JodaUtils; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.filter.DimFilter; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -76,7 +77,7 @@ public class DatasourceIngestionSpec this.segments = segments; this.filter = filter; - this.granularity = granularity == null ? QueryGranularity.NONE : granularity; + this.granularity = granularity == null ? QueryGranularities.NONE : granularity; this.dimensions = dimensions; this.metrics = metrics; diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java index 36be2c0289e..a0a65191963 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java @@ -34,7 +34,7 @@ import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.indexer.hadoop.WindowedDataSegment; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; @@ -324,7 +324,7 @@ public class BatchDeltaIngestionTest ImmutableList.of("host"), ImmutableList.of("visited_sum", "unique_hosts"), null, - QueryGranularity.NONE + QueryGranularities.NONE ); List rows = Lists.newArrayList(); @@ -358,7 +358,7 @@ public class BatchDeltaIngestionTest new HyperUniquesAggregatorFactory("unique_hosts", "host2") }, new UniformGranularitySpec( - Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(INTERVAL_FULL) + Granularity.DAY, QueryGranularities.NONE, ImmutableList.of(INTERVAL_FULL) ), MAPPER ), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java index 57f3697e7cc..9bac1597955 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/DetermineHashedPartitionsJobTest.java @@ -27,7 +27,7 @@ import io.druid.data.input.impl.DelimitedParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.indexer.partitions.HashedPartitionsSpec; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -133,7 +133,7 @@ public class DetermineHashedPartitionsJobTest new AggregatorFactory[]{new DoubleSumAggregatorFactory("index", "index")}, new UniformGranularitySpec( Granularity.DAY, - QueryGranularity.NONE, + QueryGranularities.NONE, ImmutableList.of(new Interval(interval)) ), HadoopDruidIndexerConfig.JSON_MAPPER diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java index 5ee8b713275..400902c1a6c 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/DeterminePartitionsJobTest.java @@ -27,7 +27,7 @@ import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.indexer.partitions.SingleDimensionPartitionsSpec; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -236,7 +236,7 @@ public class DeterminePartitionsJobTest ), new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, new UniformGranularitySpec( - Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(new Interval(interval)) + Granularity.DAY, QueryGranularities.NONE, ImmutableList.of(new Interval(interval)) ), HadoopDruidIndexerConfig.JSON_MAPPER ), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java index 3e9bfdd5c77..3535f56b392 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -27,7 +27,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.Granularity; import io.druid.data.input.MapBasedInputRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.indexing.DataSchema; @@ -203,7 +203,7 @@ public class HadoopDruidIndexerConfigTest new AggregatorFactory[0], new UniformGranularitySpec( Granularity.MINUTE, - QueryGranularity.MINUTE, + QueryGranularities.MINUTE, ImmutableList.of(new Interval("2010-01-01/P1D")) ), jsonMapper @@ -242,7 +242,7 @@ public class HadoopDruidIndexerConfigTest ); final long timestamp = new DateTime("2010-01-01T01:00:01").getMillis(); final Bucket expectedBucket = config.getBucket(new MapBasedInputRow(timestamp, dims, values)).get(); - final long nextBucketTimestamp = QueryGranularity.MINUTE.next(QueryGranularity.MINUTE.truncate(timestamp)); + final long nextBucketTimestamp = QueryGranularities.MINUTE.next(QueryGranularities.MINUTE.truncate(timestamp)); // check that all rows having same set of dims and truncated timestamp hash to same bucket for (int i = 0; timestamp + i < nextBucketTimestamp; i++) { Assert.assertEquals( diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java index 455919b5445..151e1a43b93 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java @@ -29,7 +29,7 @@ import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; @@ -79,7 +79,7 @@ public class IndexGeneratorCombinerTest new HyperUniquesAggregatorFactory("unique_hosts", "host") }, new UniformGranularitySpec( - Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(Interval.parse("2010/2011")) + Granularity.DAY, QueryGranularities.NONE, ImmutableList.of(Interval.parse("2010/2011")) ), HadoopDruidIndexerConfig.JSON_MAPPER ), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java index f7e9872cf75..b0785a51088 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java @@ -33,7 +33,7 @@ import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -478,7 +478,7 @@ public class IndexGeneratorJobTest ), aggs, new UniformGranularitySpec( - Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(this.interval) + Granularity.DAY, QueryGranularities.NONE, ImmutableList.of(this.interval) ), mapper ), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java index 64755bf7a13..99580e97c8b 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java @@ -26,7 +26,7 @@ import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.indexing.DataSchema; @@ -81,7 +81,7 @@ public class JobHelperTest ), new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, new UniformGranularitySpec( - Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(this.interval) + Granularity.DAY, QueryGranularities.NONE, ImmutableList.of(this.interval) ), HadoopDruidIndexerConfig.JSON_MAPPER ), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java index 502adef9668..a2b85834dc8 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java @@ -22,7 +22,7 @@ package io.druid.indexer.hadoop; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.TestHelper; import io.druid.timeline.DataSegment; @@ -49,7 +49,7 @@ public class DatasourceIngestionSpecTest null, null, new SelectorDimFilter("dim", "value", null), - QueryGranularity.DAY, + QueryGranularities.DAY, Lists.newArrayList("d1", "d2"), Lists.newArrayList("m1", "m2", "m3"), false @@ -133,7 +133,7 @@ public class DatasourceIngestionSpecTest ) ), new SelectorDimFilter("dim", "value", null), - QueryGranularity.DAY, + QueryGranularities.DAY, Lists.newArrayList("d1", "d2"), Lists.newArrayList("m1", "m2", "m3"), true diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java index b74096a572f..89296080273 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java @@ -33,7 +33,7 @@ import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.guice.GuiceInjectors; import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Self; @@ -276,7 +276,7 @@ public class DatasourcePathSpecTest new LongSumAggregatorFactory("visited_sum", "visited") }, new UniformGranularitySpec( - Granularity.DAY, QueryGranularity.NONE, ImmutableList.of(Interval.parse("2000/3000")) + Granularity.DAY, QueryGranularities.NONE, ImmutableList.of(Interval.parse("2000/3000")) ), HadoopDruidIndexerConfig.JSON_MAPPER ), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java index 99b4d6aed91..b2dd9b75c64 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java @@ -24,7 +24,7 @@ import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.metamx.common.Granularity; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.HadoopIOConfig; import io.druid.indexer.HadoopIngestionSpec; @@ -114,7 +114,7 @@ public class GranularityPathSpecTest new AggregatorFactory[0], new UniformGranularitySpec( Granularity.DAY, - QueryGranularity.MINUTE, + QueryGranularities.MINUTE, ImmutableList.of(new Interval("2015-11-06T00:00Z/2015-11-07T00:00Z")) ), jsonMapper diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java index 09bdda47d93..4de688fbd0e 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java @@ -34,7 +34,7 @@ import io.druid.data.input.impl.DelimitedParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.indexer.HadoopDruidDetermineConfigurationJob; import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.HadoopDruidIndexerJob; @@ -174,7 +174,7 @@ public class HadoopConverterJobTest }, new UniformGranularitySpec( Granularity.MONTH, - QueryGranularity.DAY, + QueryGranularities.DAY, ImmutableList.of(interval) ), HadoopDruidIndexerConfig.JSON_MAPPER diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index f1773d4c3a3..9160138e40e 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -35,7 +35,7 @@ import com.metamx.emitter.EmittingLogger; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.impl.InputRowParser; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.actions.SegmentListUsedAction; @@ -274,7 +274,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactoryof( new LongSumAggregatorFactory(metric, metric) ) - ).granularity(QueryGranularity.ALL) + ).granularity(QueryGranularities.ALL) .intervals("2000/3000") .build(); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index abc6fc8b4d9..8a8123e3170 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -27,7 +27,7 @@ import com.metamx.common.Granularity; import io.druid.client.indexing.ClientAppendQuery; import io.druid.client.indexing.ClientKillQuery; import io.druid.client.indexing.ClientMergeQuery; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.guice.FirehoseModule; import io.druid.indexer.HadoopIOConfig; import io.druid.indexer.HadoopIngestionSpec; @@ -306,7 +306,7 @@ public class TaskSerdeTest "foo", null, new AggregatorFactory[0], - new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null), + new UniformGranularitySpec(Granularity.HOUR, QueryGranularities.NONE, null), jsonMapper ), new RealtimeIOConfig( diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index b131781dc3b..e363bd3fc05 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -44,7 +44,7 @@ import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.MapInputRowParser; import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.guice.GuiceAnnotationIntrospector; import io.druid.guice.GuiceInjectableValues; import io.druid.guice.GuiceInjectors; @@ -132,7 +132,7 @@ public class IngestSegmentFirehoseFactoryTest } ); final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() - .withQueryGranularity(QueryGranularity.NONE) + .withQueryGranularity(QueryGranularities.NONE) .withMinTimestamp(JodaUtils.MIN_INSTANT) .withDimensionsSpec(ROW_PARSER) .withMetrics( diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index daad984c2f0..64fe50f9c19 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -40,7 +40,7 @@ import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.MapInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskToolboxFactory; import io.druid.indexing.common.TestUtils; @@ -209,7 +209,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest { final File persistDir = new File(tmpDir, UUID.randomUUID().toString()); final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() - .withQueryGranularity(QueryGranularity.NONE) + .withQueryGranularity(QueryGranularities.NONE) .withMinTimestamp(JodaUtils.MIN_INSTANT) .withDimensionsSpec(ROW_PARSER) .withMetrics( diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index abc32ebec62..1e7bd3e85c9 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -48,7 +48,7 @@ import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.InputRowParser; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; @@ -1151,7 +1151,7 @@ public class TaskLifecycleTest "test_ds", null, new AggregatorFactory[]{new LongSumAggregatorFactory("count", "rows")}, - new UniformGranularitySpec(Granularity.DAY, QueryGranularity.NONE, null), + new UniformGranularitySpec(Granularity.DAY, QueryGranularities.NONE, null), mapper ); RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig( diff --git a/processing/src/main/java/io/druid/granularity/QueryGranularities.java b/processing/src/main/java/io/druid/granularity/QueryGranularities.java new file mode 100644 index 00000000000..98e8680a7f8 --- /dev/null +++ b/processing/src/main/java/io/druid/granularity/QueryGranularities.java @@ -0,0 +1,45 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.granularity; + +import com.google.common.collect.ImmutableMap; +import org.joda.time.Period; + +import java.util.Map; + +public class QueryGranularities +{ + static final Map CALENDRIC_GRANULARITIES = ImmutableMap.of( + "YEAR", new PeriodGranularity(new Period("P1Y"), null, null), + "MONTH", new PeriodGranularity(new Period("P1M"), null, null), + "QUARTER", new PeriodGranularity(new Period("P3M"), null, null), + "WEEK", new PeriodGranularity(new Period("P1W"), null, null) + ); + public static final QueryGranularity NONE = new NoneGranularity(); + public static final QueryGranularity ALL = new AllGranularity(); + public static final QueryGranularity MINUTE = QueryGranularity.fromString("MINUTE"); + public static final QueryGranularity HOUR = QueryGranularity.fromString("HOUR"); + public static final QueryGranularity DAY = QueryGranularity.fromString("DAY"); + public static final QueryGranularity SECOND = QueryGranularity.fromString("SECOND"); + public static final QueryGranularity WEEK = QueryGranularity.fromString("WEEK"); + public static final QueryGranularity MONTH = QueryGranularity.fromString("MONTH"); + public static final QueryGranularity QUARTER = QueryGranularity.fromString("QUARTER"); + public static final QueryGranularity YEAR = QueryGranularity.fromString("YEAR"); +} diff --git a/processing/src/main/java/io/druid/granularity/QueryGranularity.java b/processing/src/main/java/io/druid/granularity/QueryGranularity.java index 43664eb3681..766d132c148 100644 --- a/processing/src/main/java/io/druid/granularity/QueryGranularity.java +++ b/processing/src/main/java/io/druid/granularity/QueryGranularity.java @@ -20,14 +20,11 @@ package io.druid.granularity; import com.fasterxml.jackson.annotation.JsonCreator; -import com.google.common.collect.ImmutableMap; import com.metamx.common.IAE; import org.joda.time.DateTime; -import org.joda.time.Period; import org.joda.time.ReadableDuration; import java.util.List; -import java.util.Map; import java.util.Objects; public abstract class QueryGranularity @@ -42,71 +39,42 @@ public abstract class QueryGranularity public abstract Iterable iterable(final long start, final long end); - public static final QueryGranularity ALL = new AllGranularity(); - public static final QueryGranularity NONE = new NoneGranularity(); - - private static final Map CALENDRIC_GRANULARITIES = ImmutableMap.of( - "YEAR", new PeriodGranularity(new Period("P1Y"), null, null), - "MONTH", new PeriodGranularity(new Period("P1M"), null, null), - "QUARTER", new PeriodGranularity(new Period("P3M"), null, null), - "WEEK", new PeriodGranularity(new Period("P1W"), null, null) - ); - - public static final QueryGranularity MINUTE = fromString("MINUTE"); - public static final QueryGranularity HOUR = fromString("HOUR"); - public static final QueryGranularity DAY = fromString("DAY"); - public static final QueryGranularity SECOND = fromString("SECOND"); - - public static final QueryGranularity WEEK = fromString("WEEK"); - public static final QueryGranularity MONTH = fromString("MONTH"); - public static final QueryGranularity QUARTER = fromString("QUARTER"); - public static final QueryGranularity YEAR = fromString("YEAR"); - @JsonCreator public static QueryGranularity fromString(String str) { String name = str.toUpperCase(); - if(name.equals("ALL")) - { - return QueryGranularity.ALL; - } - else if(name.equals("NONE")) - { - return QueryGranularity.NONE; - } - else if(CALENDRIC_GRANULARITIES.containsKey(name)) - { - return CALENDRIC_GRANULARITIES.get(name); + if (name.equals("ALL")) { + return QueryGranularities.ALL; + } else if (name.equals("NONE")) { + return QueryGranularities.NONE; + } else if (QueryGranularities.CALENDRIC_GRANULARITIES.containsKey(name)) { + return QueryGranularities.CALENDRIC_GRANULARITIES.get(name); } return new DurationGranularity(convertValue(str), 0); } private static enum MillisIn { - SECOND ( 1000), - MINUTE ( 60 * 1000), - FIFTEEN_MINUTE (15 * 60 * 1000), - THIRTY_MINUTE (30 * 60 * 1000), - HOUR ( 3600 * 1000), - DAY (24 * 3600 * 1000); + SECOND(1000), + MINUTE(60 * 1000), + FIFTEEN_MINUTE(15 * 60 * 1000), + THIRTY_MINUTE(30 * 60 * 1000), + HOUR(3600 * 1000), + DAY(24 * 3600 * 1000); private final long millis; + MillisIn(final long millis) { this.millis = millis; } } private static long convertValue(Object o) { - if(o instanceof String) - { + if (o instanceof String) { return MillisIn.valueOf(((String) o).toUpperCase()).millis; - } - else if(o instanceof ReadableDuration) - { - return ((ReadableDuration)o).getMillis(); - } - else if(o instanceof Number) - { - return ((Number)o).longValue(); + } else if (o instanceof ReadableDuration) { + return ((ReadableDuration) o).getMillis(); + } else if (o instanceof Number) { + return ((Number) o).longValue(); } throw new IAE("Cannot convert [%s] to QueryGranularity", o.getClass()); } diff --git a/processing/src/main/java/io/druid/query/Druids.java b/processing/src/main/java/io/druid/query/Druids.java index 8427a7a2cd8..1963592e80e 100644 --- a/processing/src/main/java/io/druid/query/Druids.java +++ b/processing/src/main/java/io/druid/query/Druids.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.PostAggregator; import io.druid.query.datasourcemetadata.DataSourceMetadataQuery; @@ -341,7 +342,7 @@ public class Druids dataSource = null; querySegmentSpec = null; dimFilter = null; - granularity = QueryGranularity.ALL; + granularity = QueryGranularities.ALL; aggregatorSpecs = Lists.newArrayList(); postAggregatorSpecs = Lists.newArrayList(); context = null; @@ -552,7 +553,7 @@ public class Druids { dataSource = null; dimFilter = null; - granularity = QueryGranularity.ALL; + granularity = QueryGranularities.ALL; limit = 0; querySegmentSpec = null; dimensions = null; @@ -1088,7 +1089,7 @@ public class Druids querySegmentSpec = null; context = null; dimFilter = null; - granularity = QueryGranularity.ALL; + granularity = QueryGranularities.ALL; dimensions = Lists.newArrayList(); metrics = Lists.newArrayList(); pagingSpec = null; diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java b/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java index a81aa80dd0f..40674063637 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentAnalyzer.java @@ -30,7 +30,7 @@ import com.metamx.common.guava.Accumulator; import com.metamx.common.guava.Sequence; import com.metamx.common.logger.Logger; import io.druid.common.utils.StringUtils; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.metadata.metadata.ColumnAnalysis; import io.druid.query.metadata.metadata.SegmentMetadataQuery; @@ -248,7 +248,7 @@ public class SegmentAnalyzer final long end = storageAdapter.getMaxTime().getMillis(); final Sequence cursors = - storageAdapter.makeCursors(null, new Interval(start, end), QueryGranularity.ALL, false); + storageAdapter.makeCursors(null, new Interval(start, end), QueryGranularities.ALL, false); size = cursors.accumulate( 0L, diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java b/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java index e6fb921ff02..552cc05e9f4 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryRunner.java @@ -33,7 +33,7 @@ import com.metamx.common.guava.FunctionalIterable; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.emitter.EmittingLogger; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -165,7 +165,7 @@ public class SearchQueryRunner implements QueryRunner> dimsToSearch = dimensions; } - final Sequence cursors = adapter.makeCursors(filter, segment.getDataInterval(), QueryGranularity.ALL, descending); + final Sequence cursors = adapter.makeCursors(filter, segment.getDataInterval(), QueryGranularities.ALL, descending); final TreeMap retVal = cursors.accumulate( Maps.newTreeMap(query.getSort().getComparator()), diff --git a/processing/src/main/java/io/druid/query/search/search/SearchQuery.java b/processing/src/main/java/io/druid/query/search/search/SearchQuery.java index b724538ac4b..1d67cdb9a58 100644 --- a/processing/src/main/java/io/druid/query/search/search/SearchQuery.java +++ b/processing/src/main/java/io/druid/query/search/search/SearchQuery.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.BaseQuery; import io.druid.query.DataSource; import io.druid.query.Query; @@ -62,7 +63,7 @@ public class SearchQuery extends BaseQuery> super(dataSource, querySegmentSpec, false, context); this.dimFilter = dimFilter; this.sortSpec = sortSpec == null ? new LexicographicSearchSortSpec() : sortSpec; - this.granularity = granularity == null ? QueryGranularity.ALL : granularity; + this.granularity = granularity == null ? QueryGranularities.ALL : granularity; this.limit = (limit == 0) ? 1000 : limit; this.dimensions = dimensions; this.querySpec = querySpec; diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java index 5631a29730e..79095ae6e98 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryBuilder.java @@ -21,6 +21,7 @@ package io.druid.query.topn; import com.google.common.collect.Lists; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.DataSource; import io.druid.query.TableDataSource; import io.druid.query.aggregation.AggregatorFactory; @@ -79,7 +80,7 @@ public class TopNQueryBuilder threshold = 0; querySegmentSpec = null; dimFilter = null; - granularity = QueryGranularity.ALL; + granularity = QueryGranularities.ALL; aggregatorSpecs = Lists.newArrayList(); postAggregatorSpecs = Lists.newArrayList(); context = null; diff --git a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java index 4bf273e17d4..1e2e38140a7 100644 --- a/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java +++ b/processing/src/main/java/io/druid/segment/incremental/IncrementalIndexSchema.java @@ -22,6 +22,7 @@ package io.druid.segment.incremental; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.InputRowParser; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; /** @@ -76,7 +77,7 @@ public class IncrementalIndexSchema public Builder() { this.minTimestamp = 0L; - this.gran = QueryGranularity.NONE; + this.gran = QueryGranularities.NONE; this.dimensionsSpec = new DimensionsSpec(null, null, null); this.metrics = new AggregatorFactory[]{}; } diff --git a/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java b/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java index 1c6d9e1f98b..3caeab6c181 100644 --- a/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java +++ b/processing/src/test/java/io/druid/granularity/QueryGranularityTest.java @@ -36,6 +36,8 @@ import org.joda.time.Years; import org.junit.Assert; import org.junit.Test; +import java.net.URL; +import java.net.URLClassLoader; import java.util.Iterator; import java.util.List; @@ -46,7 +48,7 @@ public class QueryGranularityTest @Test public void testIterableNone() throws Exception { - List millis = Lists.newArrayList(QueryGranularity.NONE.iterable(0, 1000)); + List millis = Lists.newArrayList(QueryGranularities.NONE.iterable(0, 1000)); int count = 0; Assert.assertEquals(1000, millis.size()); for (Long milli : millis) { @@ -66,7 +68,7 @@ public class QueryGranularityTest new DateTime("2011-01-01T09:39:00.000Z"), new DateTime("2011-01-01T09:40:00.000Z") ), - QueryGranularity.MINUTE.iterable(baseTime.getMillis(), baseTime.plus(Minutes.THREE).getMillis()) + QueryGranularities.MINUTE.iterable(baseTime.getMillis(), baseTime.plus(Minutes.THREE).getMillis()) ); } @@ -82,7 +84,7 @@ public class QueryGranularityTest new DateTime("2011-01-01T09:40:00.000Z"), new DateTime("2011-01-01T09:41:00.000Z") ), - QueryGranularity.MINUTE.iterable(baseTime.getMillis(), baseTime.plus(Minutes.THREE).getMillis()) + QueryGranularities.MINUTE.iterable(baseTime.getMillis(), baseTime.plus(Minutes.THREE).getMillis()) ); } @@ -131,7 +133,7 @@ public class QueryGranularityTest new DateTime("2011-01-01T10:00:00.000Z"), new DateTime("2011-01-01T11:00:00.000Z") ), - QueryGranularity.HOUR.iterable(baseTime.getMillis(), baseTime.plus(Hours.hours(3)).getMillis()) + QueryGranularities.HOUR.iterable(baseTime.getMillis(), baseTime.plus(Hours.hours(3)).getMillis()) ); } @@ -147,7 +149,7 @@ public class QueryGranularityTest new DateTime("2011-01-01T11:00:00.000Z"), new DateTime("2011-01-01T12:00:00.000Z") ), - QueryGranularity.HOUR.iterable(baseTime.getMillis(), baseTime.plus(Hours.hours(3)).getMillis()) + QueryGranularities.HOUR.iterable(baseTime.getMillis(), baseTime.plus(Hours.hours(3)).getMillis()) ); } @@ -162,7 +164,7 @@ public class QueryGranularityTest new DateTime("2011-01-02T00:00:00.000Z"), new DateTime("2011-01-03T00:00:00.000Z") ), - QueryGranularity.DAY.iterable(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()) + QueryGranularities.DAY.iterable(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()) ); } @@ -178,7 +180,7 @@ public class QueryGranularityTest new DateTime("2011-01-03T00:00:00.000Z"), new DateTime("2011-01-04T00:00:00.000Z") ), - QueryGranularity.DAY.iterable(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()) + QueryGranularities.DAY.iterable(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()) ); } @@ -193,7 +195,7 @@ public class QueryGranularityTest new DateTime("2011-01-10T00:00:00.000Z"), new DateTime("2011-01-17T00:00:00.000Z") ), - QueryGranularity.WEEK.iterable(baseTime.getMillis(), baseTime.plus(Weeks.THREE).getMillis()) + QueryGranularities.WEEK.iterable(baseTime.getMillis(), baseTime.plus(Weeks.THREE).getMillis()) ); } @@ -209,7 +211,7 @@ public class QueryGranularityTest new DateTime("2011-01-10T00:00:00.000Z"), new DateTime("2011-01-17T00:00:00.000Z") ), - QueryGranularity.WEEK.iterable(baseTime.getMillis(), baseTime.plus(Weeks.THREE).getMillis()) + QueryGranularities.WEEK.iterable(baseTime.getMillis(), baseTime.plus(Weeks.THREE).getMillis()) ); } @@ -224,7 +226,7 @@ public class QueryGranularityTest new DateTime("2011-02-01T00:00:00.000Z"), new DateTime("2011-03-01T00:00:00.000Z") ), - QueryGranularity.MONTH.iterable(baseTime.getMillis(), baseTime.plus(Months.THREE).getMillis()) + QueryGranularities.MONTH.iterable(baseTime.getMillis(), baseTime.plus(Months.THREE).getMillis()) ); } @@ -240,7 +242,7 @@ public class QueryGranularityTest new DateTime("2011-03-01T00:00:00.000Z"), new DateTime("2011-04-01T00:00:00.000Z") ), - QueryGranularity.MONTH.iterable(baseTime.getMillis(), baseTime.plus(Months.THREE).getMillis()) + QueryGranularities.MONTH.iterable(baseTime.getMillis(), baseTime.plus(Months.THREE).getMillis()) ); } @@ -255,7 +257,7 @@ public class QueryGranularityTest new DateTime("2011-04-01T00:00:00.000Z"), new DateTime("2011-07-01T00:00:00.000Z") ), - QueryGranularity.QUARTER.iterable(baseTime.getMillis(), baseTime.plus(Months.NINE).getMillis()) + QueryGranularities.QUARTER.iterable(baseTime.getMillis(), baseTime.plus(Months.NINE).getMillis()) ); } @@ -271,7 +273,7 @@ public class QueryGranularityTest new DateTime("2011-07-01T00:00:00.000Z"), new DateTime("2011-10-01T00:00:00.000Z") ), - QueryGranularity.QUARTER.iterable(baseTime.getMillis(), baseTime.plus(Months.NINE).getMillis()) + QueryGranularities.QUARTER.iterable(baseTime.getMillis(), baseTime.plus(Months.NINE).getMillis()) ); } @@ -286,7 +288,7 @@ public class QueryGranularityTest new DateTime("2012-01-01T00:00:00.000Z"), new DateTime("2013-01-01T00:00:00.000Z") ), - QueryGranularity.YEAR.iterable(baseTime.getMillis(), baseTime.plus(Years.THREE).getMillis()) + QueryGranularities.YEAR.iterable(baseTime.getMillis(), baseTime.plus(Years.THREE).getMillis()) ); } @@ -302,7 +304,7 @@ public class QueryGranularityTest new DateTime("2013-01-01T00:00:00.000Z"), new DateTime("2014-01-01T00:00:00.000Z") ), - QueryGranularity.YEAR.iterable(baseTime.getMillis(), baseTime.plus(Years.THREE).getMillis()) + QueryGranularities.YEAR.iterable(baseTime.getMillis(), baseTime.plus(Years.THREE).getMillis()) ); } @@ -602,7 +604,7 @@ public class QueryGranularityTest assertSame( Lists.newArrayList(baseTime), - QueryGranularity.ALL.iterable(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()) + QueryGranularities.ALL.iterable(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()) ); } @@ -613,7 +615,7 @@ public class QueryGranularityTest assertSame( Lists.newArrayList(baseTime), - QueryGranularity.ALL.iterable(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()) + QueryGranularities.ALL.iterable(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()) ); } @@ -665,17 +667,17 @@ public class QueryGranularityTest ObjectMapper mapper = new DefaultObjectMapper(); Assert.assertEquals( - QueryGranularity.ALL, - mapper.readValue( - mapper.writeValueAsString(QueryGranularity.ALL), + QueryGranularities.ALL, + mapper.readValue( + mapper.writeValueAsString(QueryGranularities.ALL), QueryGranularity.class ) ); Assert.assertEquals( - QueryGranularity.NONE, - mapper.readValue( - mapper.writeValueAsString(QueryGranularity.NONE), + QueryGranularities.NONE, + mapper.readValue( + mapper.writeValueAsString(QueryGranularities.NONE), QueryGranularity.class ) ); @@ -686,19 +688,19 @@ public class QueryGranularityTest { ObjectMapper mapper = new DefaultObjectMapper(); - Assert.assertEquals(QueryGranularity.ALL, mapper.readValue("\"all\"", QueryGranularity.class)); - Assert.assertEquals(QueryGranularity.ALL, mapper.readValue("\"ALL\"", QueryGranularity.class)); - Assert.assertEquals(QueryGranularity.NONE, mapper.readValue("\"none\"", QueryGranularity.class)); - Assert.assertEquals(QueryGranularity.NONE, mapper.readValue("\"NONE\"", QueryGranularity.class)); + Assert.assertEquals(QueryGranularities.ALL, mapper.readValue("\"all\"", QueryGranularity.class)); + Assert.assertEquals(QueryGranularities.ALL, mapper.readValue("\"ALL\"", QueryGranularity.class)); + Assert.assertEquals(QueryGranularities.NONE, mapper.readValue("\"none\"", QueryGranularity.class)); + Assert.assertEquals(QueryGranularities.NONE, mapper.readValue("\"NONE\"", QueryGranularity.class)); - Assert.assertEquals(QueryGranularity.DAY, mapper.readValue("\"day\"", QueryGranularity.class)); - Assert.assertEquals(QueryGranularity.HOUR, mapper.readValue("\"hour\"", QueryGranularity.class)); - Assert.assertEquals(QueryGranularity.MINUTE, mapper.readValue("\"minute\"", QueryGranularity.class)); + Assert.assertEquals(QueryGranularities.DAY, mapper.readValue("\"day\"", QueryGranularity.class)); + Assert.assertEquals(QueryGranularities.HOUR, mapper.readValue("\"hour\"", QueryGranularity.class)); + Assert.assertEquals(QueryGranularities.MINUTE, mapper.readValue("\"minute\"", QueryGranularity.class)); - Assert.assertEquals(QueryGranularity.WEEK, mapper.readValue("\"week\"", QueryGranularity.class)); - Assert.assertEquals(QueryGranularity.QUARTER, mapper.readValue("\"quarter\"", QueryGranularity.class)); - Assert.assertEquals(QueryGranularity.MONTH, mapper.readValue("\"month\"", QueryGranularity.class)); - Assert.assertEquals(QueryGranularity.YEAR, mapper.readValue("\"year\"", QueryGranularity.class)); + Assert.assertEquals(QueryGranularities.WEEK, mapper.readValue("\"week\"", QueryGranularity.class)); + Assert.assertEquals(QueryGranularities.QUARTER, mapper.readValue("\"quarter\"", QueryGranularity.class)); + Assert.assertEquals(QueryGranularities.MONTH, mapper.readValue("\"month\"", QueryGranularity.class)); + Assert.assertEquals(QueryGranularities.YEAR, mapper.readValue("\"year\"", QueryGranularity.class)); QueryGranularity gran = mapper.readValue("\"thirty_minute\"", QueryGranularity.class); Assert.assertEquals(new DurationGranularity(30 * 60 * 1000, null), gran); @@ -712,24 +714,24 @@ public class QueryGranularityTest { Assert.assertNull(QueryGranularity.mergeQueryGranularities(null)); Assert.assertNull(QueryGranularity.mergeQueryGranularities(ImmutableList.of())); - Assert.assertNull(QueryGranularity.mergeQueryGranularities(Lists.newArrayList(null, QueryGranularity.DAY))); - Assert.assertNull(QueryGranularity.mergeQueryGranularities(Lists.newArrayList(QueryGranularity.DAY, null))); + Assert.assertNull(QueryGranularity.mergeQueryGranularities(Lists.newArrayList(null, QueryGranularities.DAY))); + Assert.assertNull(QueryGranularity.mergeQueryGranularities(Lists.newArrayList(QueryGranularities.DAY, null))); Assert.assertNull( QueryGranularity.mergeQueryGranularities( Lists.newArrayList( - QueryGranularity.DAY, + QueryGranularities.DAY, null, - QueryGranularity.DAY + QueryGranularities.DAY ) ) ); Assert.assertNull( - QueryGranularity.mergeQueryGranularities(ImmutableList.of(QueryGranularity.ALL, QueryGranularity.DAY)) + QueryGranularity.mergeQueryGranularities(ImmutableList.of(QueryGranularities.ALL, QueryGranularities.DAY)) ); Assert.assertEquals( - QueryGranularity.ALL, - QueryGranularity.mergeQueryGranularities(ImmutableList.of(QueryGranularity.ALL, QueryGranularity.ALL)) + QueryGranularities.ALL, + QueryGranularity.mergeQueryGranularities(ImmutableList.of(QueryGranularities.ALL, QueryGranularities.ALL)) ); } @@ -746,4 +748,15 @@ public class QueryGranularityTest Assert.assertFalse("actualIter not exhausted!?", actualIter.hasNext()); Assert.assertFalse("expectedIter not exhausted!?", expectedIter.hasNext()); } + + @Test(timeout = 10_000L) + public void testDeadLock() throws Exception + { + final URL[] urls = ((URLClassLoader)QueryGranularity.class.getClassLoader()).getURLs(); + final String className = QueryGranularity.class.getCanonicalName(); + for(int i = 0; i < 1000; ++i) { + final ClassLoader loader = new URLClassLoader(urls, null); + Assert.assertNotNull(Class.forName(className, true, loader)); + } + } } diff --git a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java index 7f7a337f7f7..37457271d9b 100644 --- a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java @@ -33,7 +33,7 @@ import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregationTestHelper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -93,7 +93,7 @@ public class MultiValuedDimensionTest { incrementalIndex = new OnheapIncrementalIndex( 0, - QueryGranularity.NONE, + QueryGranularities.NONE, new AggregatorFactory[]{ new CountAggregatorFactory("count") }, @@ -137,7 +137,7 @@ public class MultiValuedDimensionTest .builder() .setDataSource("xx") .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000")) - .setGranularity(QueryGranularity.ALL) + .setGranularity(QueryGranularities.ALL) .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("tags", "tags"))) .setAggregatorSpecs( Arrays.asList( @@ -185,7 +185,7 @@ public class MultiValuedDimensionTest .builder() .setDataSource("xx") .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000")) - .setGranularity(QueryGranularity.ALL) + .setGranularity(QueryGranularities.ALL) .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("tags", "tags"))) .setAggregatorSpecs( Arrays.asList( @@ -226,7 +226,7 @@ public class MultiValuedDimensionTest .builder() .setDataSource("xx") .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000")) - .setGranularity(QueryGranularity.ALL) + .setGranularity(QueryGranularities.ALL) .setDimensions( Lists.newArrayList( new RegexFilteredDimensionSpec( @@ -268,7 +268,7 @@ public class MultiValuedDimensionTest { TopNQuery query = new TopNQueryBuilder() .dataSource("xx") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .dimension(new ListFilteredDimensionSpec( new DefaultDimensionSpec("tags", "tags"), ImmutableSet.of("t3"), diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index a39473b476b..c3b4b3d02dc 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -29,6 +29,7 @@ import com.metamx.common.guava.MergeSequence; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.js.JavaScriptConfig; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -97,8 +98,8 @@ public class QueryRunnerTestHelper public static final DateTime minTime = new DateTime("2011-01-12T00:00:00.000Z"); - public static final QueryGranularity dayGran = QueryGranularity.DAY; - public static final QueryGranularity allGran = QueryGranularity.ALL; + public static final QueryGranularity dayGran = QueryGranularities.DAY; + public static final QueryGranularity allGran = QueryGranularities.ALL; public static final String marketDimension = "market"; public static final String qualityDimension = "quality"; public static final String placementDimension = "placement"; diff --git a/processing/src/test/java/io/druid/query/ResultGranularTimestampComparatorTest.java b/processing/src/test/java/io/druid/query/ResultGranularTimestampComparatorTest.java index d7e7d8d8aca..e070fe15e29 100644 --- a/processing/src/test/java/io/druid/query/ResultGranularTimestampComparatorTest.java +++ b/processing/src/test/java/io/druid/query/ResultGranularTimestampComparatorTest.java @@ -20,6 +20,7 @@ package io.druid.query; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; @@ -55,7 +56,7 @@ public class ResultGranularTimestampComparatorTest Result r1 = new Result(time, null); Result r2 = new Result(time.plusYears(5), null); - Assert.assertEquals(ResultGranularTimestampComparator.create(QueryGranularity.ALL, descending).compare(r1, r2), 0); + Assert.assertEquals(ResultGranularTimestampComparator.create(QueryGranularities.ALL, descending).compare(r1, r2), 0); } @Test @@ -66,7 +67,7 @@ public class ResultGranularTimestampComparatorTest Result greater = new Result(time.plusHours(25), null); Result less = new Result(time.minusHours(1), null); - QueryGranularity day = QueryGranularity.DAY; + QueryGranularity day = QueryGranularities.DAY; Assert.assertEquals(ResultGranularTimestampComparator.create(day, descending).compare(res, same), 0); Assert.assertEquals(ResultGranularTimestampComparator.create(day, descending).compare(res, greater), descending ? 1 : -1); Assert.assertEquals(ResultGranularTimestampComparator.create(day, descending).compare(res, less), descending ? -1 : 1); @@ -80,7 +81,7 @@ public class ResultGranularTimestampComparatorTest Result greater = new Result(time.plusHours(1), null); Result less = new Result(time.minusHours(1), null); - QueryGranularity hour = QueryGranularity.HOUR; + QueryGranularity hour = QueryGranularities.HOUR; Assert.assertEquals(ResultGranularTimestampComparator.create(hour, descending).compare(res, same), 0); Assert.assertEquals(ResultGranularTimestampComparator.create(hour, descending).compare(res, greater), descending ? 1 : -1); Assert.assertEquals(ResultGranularTimestampComparator.create(hour, descending).compare(res, less), descending ? -1 : 1); diff --git a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java index 3e33dcc047d..75754048e5d 100644 --- a/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/hyperloglog/HyperUniquesAggregationTest.java @@ -23,7 +23,7 @@ import com.google.common.collect.Lists; import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.data.input.MapBasedRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.AggregatorsModule; import io.druid.query.aggregation.AggregationTestHelper; import org.junit.Assert; @@ -88,7 +88,7 @@ public class HyperUniquesAggregationTest parseSpec, metricSpec, 0, - QueryGranularity.NONE, + QueryGranularities.NONE, 50000, query ); diff --git a/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java b/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java index bb012e6afb1..e925e2ffdbe 100644 --- a/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java @@ -27,7 +27,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.MapMaker; import com.metamx.common.guava.Sequences; import io.druid.data.input.MapBasedInputRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Druids; import io.druid.query.Query; @@ -112,7 +112,7 @@ public class DataSourceMetadataQueryTest public void testMaxIngestedEventTime() throws Exception { final IncrementalIndex rtIndex = new OnheapIncrementalIndex( - 0L, QueryGranularity.NONE, new AggregatorFactory[]{new CountAggregatorFactory("count")}, 1000 + 0L, QueryGranularities.NONE, new AggregatorFactory[]{new CountAggregatorFactory("count")}, 1000 ); ; final QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner( diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java index bc1b26f8b14..eb76d935b1e 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerFactoryTest.java @@ -37,7 +37,7 @@ import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Query; import io.druid.query.QueryRunner; @@ -87,7 +87,7 @@ public class GroupByQueryRunnerFactoryTest .builder() .setDataSource("xx") .setQuerySegmentSpec(new LegacySegmentSpec("1970/3000")) - .setGranularity(QueryGranularity.ALL) + .setGranularity(QueryGranularities.ALL) .setDimensions(Lists.newArrayList(new DefaultDimensionSpec("tags", "tags"))) .setAggregatorSpecs( Arrays.asList( @@ -113,7 +113,7 @@ public class GroupByQueryRunnerFactoryTest { IncrementalIndex incrementalIndex = new OnheapIncrementalIndex( 0, - QueryGranularity.NONE, + QueryGranularities.NONE, new AggregatorFactory[]{ new CountAggregatorFactory("count") }, diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java index e1779cd5058..31e01689707 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByQueryRunnerTest.java @@ -36,7 +36,7 @@ import com.metamx.common.parsers.ParseException; import io.druid.collections.StupidPool; import io.druid.data.input.Row; import io.druid.granularity.PeriodGranularity; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.js.JavaScriptConfig; import io.druid.query.BySegmentResultValue; @@ -1250,7 +1250,7 @@ public class GroupByQueryRunnerTest .setGranularity(new PeriodGranularity(new Period("P1M"), null, null)); final GroupByQuery fullQuery = builder.build(); - final GroupByQuery allGranQuery = builder.copy().setGranularity(QueryGranularity.ALL).build(); + final GroupByQuery allGranQuery = builder.copy().setGranularity(QueryGranularities.ALL).build(); QueryRunner mergedRunner = factory.getToolchest().mergeResults( new QueryRunner() @@ -1365,7 +1365,7 @@ public class GroupByQueryRunnerTest new LongSumAggregatorFactory("idx", "index") ) ) - .setGranularity(QueryGranularity.DAY) + .setGranularity(QueryGranularities.DAY) .setLimit(limit) .addOrderByColumn("idx", OrderByColumnSpec.Direction.DESCENDING); @@ -2305,7 +2305,7 @@ public class GroupByQueryRunnerTest new DoubleSumAggregatorFactory("index", "index") ) ) - .setGranularity(QueryGranularity.ALL) + .setGranularity(QueryGranularities.ALL) .setHavingSpec(new GreaterThanHavingSpec("index", 310L)) .setLimitSpec( new DefaultLimitSpec( diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java index 815ad3fce2e..e4d8ec95520 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryTest.java @@ -28,7 +28,7 @@ import com.google.common.collect.Maps; import com.google.common.util.concurrent.MoreExecutors; import com.metamx.common.guava.Sequences; import io.druid.common.utils.JodaUtils; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.BySegmentResultValue; import io.druid.query.BySegmentResultValueClass; @@ -622,7 +622,7 @@ public class SegmentMetadataQueryTest 0, expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(), null, - QueryGranularity.NONE + QueryGranularities.NONE ); QueryToolChest toolChest = FACTORY.getToolchest(); diff --git a/processing/src/test/java/io/druid/query/search/SearchBinaryFnTest.java b/processing/src/test/java/io/druid/query/search/SearchBinaryFnTest.java index 605bf763a00..320221430f5 100644 --- a/processing/src/test/java/io/druid/query/search/SearchBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchBinaryFnTest.java @@ -20,7 +20,7 @@ package io.druid.query.search; import com.google.common.collect.ImmutableList; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Result; import io.druid.query.search.search.LexicographicSearchSortSpec; import io.druid.query.search.search.SearchHit; @@ -94,7 +94,7 @@ public class SearchBinaryFnTest ) ); - Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularity.ALL, Integer.MAX_VALUE).apply(r1, r2); + Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2); Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); assertSearchMergeResult(expected.getValue(), actual.getValue()); } @@ -127,7 +127,7 @@ public class SearchBinaryFnTest ); Result expected = new Result( - new DateTime(QueryGranularity.DAY.truncate(currTime.getMillis())), + new DateTime(QueryGranularities.DAY.truncate(currTime.getMillis())), new SearchResultValue( ImmutableList.of( new SearchHit( @@ -142,7 +142,7 @@ public class SearchBinaryFnTest ) ); - Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularity.DAY, Integer.MAX_VALUE).apply(r1, r2); + Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.DAY, Integer.MAX_VALUE).apply(r1, r2); Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); assertSearchMergeResult(expected.getValue(), actual.getValue()); } @@ -166,7 +166,7 @@ public class SearchBinaryFnTest Result expected = r1; - Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularity.ALL, Integer.MAX_VALUE).apply(r1, r2); + Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2); Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); assertSearchMergeResult(expected.getValue(), actual.getValue()); } @@ -214,7 +214,7 @@ public class SearchBinaryFnTest ) ); - Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularity.ALL, Integer.MAX_VALUE).apply(r1, r2); + Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2); Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); assertSearchMergeResult(expected.getValue(), actual.getValue()); } @@ -240,7 +240,7 @@ public class SearchBinaryFnTest new SearchResultValue(toHits(c, "blah:short", "blah:thisislong")) ); - Result actual = new SearchBinaryFn(searchSortSpec, QueryGranularity.ALL, Integer.MAX_VALUE).apply(r1, r2); + Result actual = new SearchBinaryFn(searchSortSpec, QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2); Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); assertSearchMergeResult(expected.getValue(), actual.getValue()); } @@ -266,7 +266,7 @@ public class SearchBinaryFnTest new SearchResultValue(toHits(c, "blah:short", "blah:thisislong", "blah2:thisislong")) ); - Result actual = new SearchBinaryFn(searchSortSpec, QueryGranularity.ALL, Integer.MAX_VALUE).apply(r1, r2); + Result actual = new SearchBinaryFn(searchSortSpec, QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2); Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); assertSearchMergeResult(expected.getValue(), actual.getValue()); } @@ -301,7 +301,7 @@ public class SearchBinaryFnTest Result expected = r1; - Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularity.ALL, Integer.MAX_VALUE).apply(r1, r2); + Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.ALL, Integer.MAX_VALUE).apply(r1, r2); Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); assertSearchMergeResult(expected.getValue(), actual.getValue()); } @@ -332,7 +332,7 @@ public class SearchBinaryFnTest ) ); Result expected = r1; - Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularity.ALL, 1).apply(r1, r2); + Result actual = new SearchBinaryFn(new LexicographicSearchSortSpec(), QueryGranularities.ALL, 1).apply(r1, r2); Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp()); assertSearchMergeResult(expected.getValue(), actual.getValue()); } diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java index 3245b8ee5d7..a3e9b61b941 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java @@ -21,7 +21,7 @@ package io.druid.query.search; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.CacheStrategy; import io.druid.query.Druids; @@ -47,7 +47,7 @@ public class SearchQueryQueryToolChestTest new SearchQuery( new TableDataSource("dummy"), null, - QueryGranularity.ALL, + QueryGranularities.ALL, 1, new MultipleIntervalSegmentSpec( ImmutableList.of( diff --git a/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java b/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java index 8b0aaae6ced..1408afec6e7 100644 --- a/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java +++ b/processing/src/test/java/io/druid/query/select/MultiSegmentSelectQueryTest.java @@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.io.CharSource; import com.metamx.common.guava.Sequences; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Druids; import io.druid.query.QueryRunner; @@ -176,7 +176,7 @@ public class MultiSegmentSelectQueryTest { final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() .withMinTimestamp(new DateTime(minTimeStamp).getMillis()) - .withQueryGranularity(QueryGranularity.HOUR) + .withQueryGranularity(QueryGranularities.HOUR) .withMetrics(TestIndex.METRIC_AGGS) .build(); return new OnheapIncrementalIndex(schema, true, maxRowCount); diff --git a/processing/src/test/java/io/druid/query/select/SelectBinaryFnTest.java b/processing/src/test/java/io/druid/query/select/SelectBinaryFnTest.java index 0cff751c857..78fed650123 100644 --- a/processing/src/test/java/io/druid/query/select/SelectBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/select/SelectBinaryFnTest.java @@ -22,7 +22,7 @@ package io.druid.query.select; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.metamx.common.ISE; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Result; import org.joda.time.DateTime; import org.junit.Assert; @@ -44,7 +44,7 @@ public class SelectBinaryFnTest @Test public void testApply() throws Exception { - SelectBinaryFn binaryFn = new SelectBinaryFn(QueryGranularity.ALL, new PagingSpec(null, 5), false); + SelectBinaryFn binaryFn = new SelectBinaryFn(QueryGranularities.ALL, new PagingSpec(null, 5), false); Result res1 = new Result<>( new DateTime("2013-01-01"), diff --git a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java index 255b43d6f35..ebce3ecefed 100644 --- a/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/spec/SpecificSegmentQueryRunnerTest.java @@ -29,7 +29,7 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import com.metamx.common.guava.Yielder; import com.metamx.common.guava.YieldingAccumulator; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.Druids; import io.druid.query.Query; @@ -97,7 +97,7 @@ public class SpecificSegmentQueryRunnerTest final Map responseContext = Maps.newHashMap(); TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("foo") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(ImmutableList.of(new Interval("2012-01-01T00:00:00Z/P1D"))) .aggregators( ImmutableList.of( @@ -172,7 +172,7 @@ public class SpecificSegmentQueryRunnerTest final Map responseContext = Maps.newHashMap(); TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("foo") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(ImmutableList.of(new Interval("2012-01-01T00:00:00Z/P1D"))) .aggregators( ImmutableList.of( diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java index ae4dbe7c581..3c72e986357 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesBinaryFnTest.java @@ -20,7 +20,7 @@ package io.druid.query.timeseries; import com.google.common.collect.ImmutableMap; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -77,7 +77,7 @@ public class TimeseriesBinaryFnTest ); Result actual = new TimeseriesBinaryFn( - QueryGranularity.ALL, + QueryGranularities.ALL, aggregatorFactories ).apply( result1, @@ -109,7 +109,7 @@ public class TimeseriesBinaryFnTest ); Result expected = new Result( - new DateTime(QueryGranularity.DAY.truncate(currTime.getMillis())), + new DateTime(QueryGranularities.DAY.truncate(currTime.getMillis())), new TimeseriesResultValue( ImmutableMap.of( "rows", 3L, @@ -119,7 +119,7 @@ public class TimeseriesBinaryFnTest ); Result actual = new TimeseriesBinaryFn( - QueryGranularity.DAY, + QueryGranularities.DAY, aggregatorFactories ).apply( result1, @@ -145,7 +145,7 @@ public class TimeseriesBinaryFnTest Result expected = result1; Result actual = new TimeseriesBinaryFn( - QueryGranularity.ALL, + QueryGranularities.ALL, aggregatorFactories ).apply( result1, @@ -187,7 +187,7 @@ public class TimeseriesBinaryFnTest ); Result actual = new TimeseriesBinaryFn( - QueryGranularity.ALL, + QueryGranularities.ALL, aggregatorFactories ).apply( result1, diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java index 5abb0ebdc83..c005abdf118 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java @@ -22,7 +22,7 @@ package io.druid.query.timeseries; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.CacheStrategy; import io.druid.query.QueryRunnerTestHelper; @@ -74,7 +74,7 @@ public class TimeseriesQueryQueryToolChestTest ), descending, null, - QueryGranularity.ALL, + QueryGranularities.ALL, ImmutableList.of(new CountAggregatorFactory("metric1")), null, null diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java index bb6cfe7b329..39b6b37263e 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import io.druid.data.input.MapBasedInputRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -70,7 +70,7 @@ public class TimeseriesQueryRunnerBonusTest public void testOneRowAtATime() throws Exception { final IncrementalIndex oneRowIndex = new OnheapIncrementalIndex( - new DateTime("2012-01-01T00:00:00Z").getMillis(), QueryGranularity.NONE, new AggregatorFactory[]{}, 1000 + new DateTime("2012-01-01T00:00:00Z").getMillis(), QueryGranularities.NONE, new AggregatorFactory[]{}, 1000 ); List> results; @@ -122,7 +122,7 @@ public class TimeseriesQueryRunnerBonusTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("xxx") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(ImmutableList.of(new Interval("2012-01-01T00:00:00Z/P1D"))) .aggregators( ImmutableList.of( diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java index e70f17bf67d..00741cc60be 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -26,6 +26,7 @@ import com.google.common.collect.Lists; import com.metamx.common.guava.Sequences; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.QueryRunner; import io.druid.query.QueryRunnerTestHelper; @@ -112,7 +113,7 @@ public class TimeseriesQueryRunnerTest @Test public void testFullOnTimeseries() { - QueryGranularity gran = QueryGranularity.DAY; + QueryGranularity gran = QueryGranularities.DAY; TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(gran) @@ -187,7 +188,7 @@ public class TimeseriesQueryRunnerTest { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(QueryRunnerTestHelper.fullOnInterval) .aggregators( Arrays.asList( @@ -509,7 +510,7 @@ public class TimeseriesQueryRunnerTest TimeseriesQuery query1 = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .filters(QueryRunnerTestHelper.marketDimension, "spot", "upfront", "total_market") - .granularity(QueryGranularity.HOUR) + .granularity(QueryGranularities.HOUR) .intervals( Arrays.asList( new Interval( @@ -530,7 +531,7 @@ public class TimeseriesQueryRunnerTest .build(); List> lotsOfZeroes = Lists.newArrayList(); - for (final Long millis : QueryGranularity.HOUR.iterable( + for (final Long millis : QueryGranularities.HOUR.iterable( new DateTime("2011-04-14T01").getMillis(), new DateTime("2011-04-15").getMillis() )) { diff --git a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java index fcb1ff484ad..b1dc77c6860 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java +++ b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnBenchmark.java @@ -23,7 +23,7 @@ import com.google.caliper.Param; import com.google.caliper.Runner; import com.google.caliper.SimpleBenchmark; import com.google.common.collect.Lists; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -116,7 +116,7 @@ public class TopNBinaryFnBenchmark extends SimpleBenchmark ); fn = new TopNBinaryFn( TopNResultMerger.identity, - QueryGranularity.ALL, + QueryGranularities.ALL, new DefaultDimensionSpec("testdim", null), new NumericTopNMetricSpec("index"), 100, diff --git a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java index cb5c0f2903b..fa662460bd3 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNBinaryFnTest.java @@ -22,7 +22,7 @@ package io.druid.query.topn; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -144,7 +144,7 @@ public class TopNBinaryFnTest Result actual = new TopNBinaryFn( TopNResultMerger.identity, - QueryGranularity.ALL, + QueryGranularities.ALL, new DefaultDimensionSpec("testdim", null), new NumericTopNMetricSpec("index"), 2, @@ -207,7 +207,7 @@ public class TopNBinaryFnTest ); Result expected = new Result( - new DateTime(QueryGranularity.DAY.truncate(currTime.getMillis())), + new DateTime(QueryGranularities.DAY.truncate(currTime.getMillis())), new TopNResultValue( ImmutableList.>of( ImmutableMap.of( @@ -226,7 +226,7 @@ public class TopNBinaryFnTest Result actual = new TopNBinaryFn( TopNResultMerger.identity, - QueryGranularity.DAY, + QueryGranularities.DAY, new DefaultDimensionSpec("testdim", null), new NumericTopNMetricSpec("index"), 2, @@ -271,7 +271,7 @@ public class TopNBinaryFnTest Result actual = new TopNBinaryFn( TopNResultMerger.identity, - QueryGranularity.ALL, + QueryGranularities.ALL, new DefaultDimensionSpec("testdim", null), new NumericTopNMetricSpec("index"), 2, @@ -367,7 +367,7 @@ public class TopNBinaryFnTest Result actual = new TopNBinaryFn( TopNResultMerger.identity, - QueryGranularity.ALL, + QueryGranularities.ALL, new DefaultDimensionSpec("testdim", null), new NumericTopNMetricSpec("addrowsindexconstant"), 3, @@ -449,7 +449,7 @@ public class TopNBinaryFnTest Result actual = new TopNBinaryFn( TopNResultMerger.identity, - QueryGranularity.ALL, + QueryGranularities.ALL, new DefaultDimensionSpec("testdim", null), new NumericTopNMetricSpec("index"), 2, @@ -507,7 +507,7 @@ public class TopNBinaryFnTest Result actual = new TopNBinaryFn( TopNResultMerger.identity, - QueryGranularity.ALL, + QueryGranularities.ALL, new DefaultDimensionSpec("INVALID_DIM_NAME", null), new LexicographicTopNMetricSpec(null), 2, diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java index 965a0c6560e..5cf2071b8ff 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java @@ -24,7 +24,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.metamx.common.guava.Sequence; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.CacheStrategy; import io.druid.query.Query; @@ -71,7 +71,7 @@ public class TopNQueryQueryToolChestTest ) ), null, - QueryGranularity.ALL, + QueryGranularities.ALL, ImmutableList.of(new CountAggregatorFactory("metric1")), null, null diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java index 04fb629a8bf..8a2ac0d03d1 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryRunnerTest.java @@ -33,6 +33,7 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.collections.StupidPool; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.js.JavaScriptConfig; import io.druid.query.BySegmentResultValue; import io.druid.query.BySegmentResultValueClass; @@ -1520,7 +1521,7 @@ public class TopNQueryRunnerTest .postAggregators(Arrays.asList(QueryRunnerTestHelper.addRowsIndexConstant)) .build(); - QueryGranularity gran = QueryGranularity.DAY; + QueryGranularity gran = QueryGranularities.DAY; TimeseriesQuery tsQuery = Druids.newTimeseriesQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) .granularity(gran) @@ -3088,7 +3089,7 @@ public class TopNQueryRunnerTest { TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .dimension("partial_null_column") .metric(QueryRunnerTestHelper.uniqueMetric) .threshold(1000) @@ -3125,7 +3126,7 @@ public class TopNQueryRunnerTest { TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .dimension("partial_null_column") .metric(QueryRunnerTestHelper.uniqueMetric) .filters(new SelectorDimFilter("partial_null_column", null, null)) @@ -3157,7 +3158,7 @@ public class TopNQueryRunnerTest { TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .dimension("partial_null_column") .metric(QueryRunnerTestHelper.uniqueMetric) .filters(new SelectorDimFilter("partial_null_column", "value", null)) @@ -3189,7 +3190,7 @@ public class TopNQueryRunnerTest { TopNQuery query = new TopNQueryBuilder() .dataSource(QueryRunnerTestHelper.dataSource) - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .dimension(QueryRunnerTestHelper.marketDimension) .metric(new AlphaNumericTopNMetricSpec(null)) .threshold(2) diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java index e3cfcee0b7f..b46a9d97dd8 100644 --- a/processing/src/test/java/io/druid/segment/AppendTest.java +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.common.Pair; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.QueryRunner; import io.druid.query.Result; @@ -78,7 +79,7 @@ public class AppendTest }; final String dataSource = "testing"; - final QueryGranularity allGran = QueryGranularity.ALL; + final QueryGranularity allGran = QueryGranularities.ALL; final String dimensionValue = "dimension"; final String valueValue = "value"; final String marketDimension = "market"; diff --git a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java index cf14cec1aac..faaa5bf48bf 100644 --- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java @@ -22,7 +22,7 @@ package io.druid.segment; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.collections.bitmap.ConciseBitmapFactory; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.column.Column; import io.druid.segment.incremental.IncrementalIndex; @@ -50,7 +50,7 @@ public class EmptyIndexTest IncrementalIndex emptyIndex = new OnheapIncrementalIndex( 0, - QueryGranularity.NONE, + QueryGranularities.NONE, new AggregatorFactory[0], 1000 ); diff --git a/processing/src/test/java/io/druid/segment/IndexIOTest.java b/processing/src/test/java/io/druid/segment/IndexIOTest.java index ad598fa69db..85d61cc9c91 100644 --- a/processing/src/test/java/io/druid/segment/IndexIOTest.java +++ b/processing/src/test/java/io/druid/segment/IndexIOTest.java @@ -30,7 +30,7 @@ import com.google.common.collect.Maps; import com.metamx.common.UOE; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.DimensionsSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -263,7 +263,7 @@ public class IndexIOTest final IncrementalIndex incrementalIndex1 = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DEFAULT_INTERVAL.getStart().getMillis()) - .withQueryGranularity(QueryGranularity.NONE) + .withQueryGranularity(QueryGranularities.NONE) .withMetrics( new AggregatorFactory[]{ new CountAggregatorFactory( @@ -285,7 +285,7 @@ public class IndexIOTest final IncrementalIndex incrementalIndex2 = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DEFAULT_INTERVAL.getStart().getMillis()) - .withQueryGranularity(QueryGranularity.NONE) + .withQueryGranularity(QueryGranularities.NONE) .withMetrics( new AggregatorFactory[]{ new CountAggregatorFactory( diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTest.java b/processing/src/test/java/io/druid/segment/IndexMergerTest.java index fb4e20fa79a..3ebbb30e00f 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTest.java @@ -32,7 +32,7 @@ import com.metamx.common.IAE; import com.metamx.common.ISE; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.DimensionsSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -184,7 +184,7 @@ public class IndexMergerTest ); Assert.assertEquals( - QueryGranularity.NONE, + QueryGranularities.NONE, index.getMetadata().getQueryGranularity() ); } @@ -272,7 +272,7 @@ public class IndexMergerTest .setAggregators( IncrementalIndexTest.getDefaultCombiningAggregatorFactories() ) - .setQueryGranularity(QueryGranularity.NONE) + .setQueryGranularity(QueryGranularities.NONE) .putAll(metadataElems), index.getMetadata() ); @@ -287,7 +287,7 @@ public class IndexMergerTest IncrementalIndex toPersist2 = new OnheapIncrementalIndex( 0L, - QueryGranularity.NONE, + QueryGranularities.NONE, new AggregatorFactory[]{new CountAggregatorFactory("count")}, 1000 ); @@ -372,13 +372,13 @@ public class IndexMergerTest { final IncrementalIndex toPersist1 = new OnheapIncrementalIndex( 0L, - QueryGranularity.NONE, + QueryGranularities.NONE, new AggregatorFactory[]{}, 10 ); final IncrementalIndex toPersist2 = new OnheapIncrementalIndex( 0L, - QueryGranularity.NONE, + QueryGranularities.NONE, new AggregatorFactory[]{}, 10 ); @@ -887,7 +887,7 @@ public class IndexMergerTest null )) .withMinTimestamp(0L) - .withQueryGranularity(QueryGranularity.NONE) + .withQueryGranularity(QueryGranularities.NONE) .withMetrics(new AggregatorFactory[]{new CountAggregatorFactory("count")}) .build(); @@ -1103,7 +1103,7 @@ public class IndexMergerTest IncrementalIndex toPersistA = new OnheapIncrementalIndex( 0L, - QueryGranularity.NONE, + QueryGranularities.NONE, new AggregatorFactory[]{new CountAggregatorFactory("count")}, 1000 ); @@ -1128,7 +1128,7 @@ public class IndexMergerTest IncrementalIndex toPersistB = new OnheapIncrementalIndex( 0L, - QueryGranularity.NONE, + QueryGranularities.NONE, new AggregatorFactory[]{new CountAggregatorFactory("count")}, 1000 ); @@ -1248,7 +1248,7 @@ public class IndexMergerTest IncrementalIndex toPersistBA2 = new OnheapIncrementalIndex( 0L, - QueryGranularity.NONE, + QueryGranularities.NONE, new AggregatorFactory[]{new CountAggregatorFactory("count")}, 1000 ); @@ -1705,7 +1705,7 @@ public class IndexMergerTest { IncrementalIndex toPersist1 = new OnheapIncrementalIndex( 0L, - QueryGranularity.NONE, + QueryGranularities.NONE, new AggregatorFactory[]{new CountAggregatorFactory("count")}, 1000 ); @@ -1741,7 +1741,7 @@ public class IndexMergerTest { IncrementalIndex toPersist1 = new OnheapIncrementalIndex( 0L, - QueryGranularity.NONE, + QueryGranularities.NONE, new AggregatorFactory[]{new CountAggregatorFactory("count")}, 1000 ); @@ -1767,7 +1767,7 @@ public class IndexMergerTest { IncrementalIndexSchema schema = new IncrementalIndexSchema( 0L, - QueryGranularity.NONE, + QueryGranularities.NONE, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(dims), null, null), new AggregatorFactory[]{new CountAggregatorFactory("count")} ); diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java index 2f2a249dffd..ea988ac54a5 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java @@ -28,7 +28,7 @@ import com.google.common.io.Files; import io.druid.common.utils.JodaUtils; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.data.CompressedObjectStrategy; @@ -166,7 +166,7 @@ public class IndexMergerV9CompatibilityTest { toPersist = new OnheapIncrementalIndex( JodaUtils.MIN_INSTANT, - QueryGranularity.NONE, + QueryGranularities.NONE, DEFAULT_AGG_FACTORIES, 1000000 ); diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java index 7070fbc5628..9a96f669d31 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java @@ -28,7 +28,7 @@ import com.metamx.collections.spatial.search.RectangularBound; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.SpatialDimensionSchema; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryRunner; @@ -103,7 +103,7 @@ public class IndexMergerV9WithSpatialIndexTest { IncrementalIndex theIndex = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -267,7 +267,7 @@ public class IndexMergerV9WithSpatialIndexTest try { IncrementalIndex first = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -291,7 +291,7 @@ public class IndexMergerV9WithSpatialIndexTest ); IncrementalIndex second = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -315,7 +315,7 @@ public class IndexMergerV9WithSpatialIndexTest ); IncrementalIndex third = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -514,7 +514,7 @@ public class IndexMergerV9WithSpatialIndexTest { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("test") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) .filters( new SpatialDimFilter( @@ -568,7 +568,7 @@ public class IndexMergerV9WithSpatialIndexTest { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("test") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) .filters( new SpatialDimFilter( @@ -621,7 +621,7 @@ public class IndexMergerV9WithSpatialIndexTest { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("test") - .granularity(QueryGranularity.DAY) + .granularity(QueryGranularities.DAY) .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) .filters( new SpatialDimFilter( diff --git a/processing/src/test/java/io/druid/segment/MetadataTest.java b/processing/src/test/java/io/druid/segment/MetadataTest.java index eee3e6cb63d..9bace7ead28 100644 --- a/processing/src/test/java/io/druid/segment/MetadataTest.java +++ b/processing/src/test/java/io/druid/segment/MetadataTest.java @@ -21,7 +21,7 @@ package io.druid.segment; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleMaxAggregatorFactory; @@ -50,7 +50,7 @@ public class MetadataTest new LongSumAggregatorFactory("out", "in") }; metadata.setAggregators(aggregators); - metadata.setQueryGranularity(QueryGranularity.ALL); + metadata.setQueryGranularity(QueryGranularities.ALL); Metadata other = jsonMapper.readValue( jsonMapper.writeValueAsString(metadata), @@ -78,12 +78,12 @@ public class MetadataTest Metadata m1 = new Metadata(); m1.put("k", "v"); m1.setAggregators(aggs); - m1.setQueryGranularity(QueryGranularity.ALL); + m1.setQueryGranularity(QueryGranularities.ALL); Metadata m2 = new Metadata(); m2.put("k", "v"); m2.setAggregators(aggs); - m2.setQueryGranularity(QueryGranularity.ALL); + m2.setQueryGranularity(QueryGranularities.ALL); Metadata merged = new Metadata(); merged.put("k", "v"); @@ -92,7 +92,7 @@ public class MetadataTest new LongMaxAggregatorFactory("n", "n") } ); - merged.setQueryGranularity(QueryGranularity.ALL); + merged.setQueryGranularity(QueryGranularities.ALL); Assert.assertEquals(merged, Metadata.merge(ImmutableList.of(m1, m2), null)); //merge check with one metadata being null @@ -116,7 +116,7 @@ public class MetadataTest Metadata.merge(metadataToBeMerged, explicitAggs) ); - merged.setQueryGranularity(QueryGranularity.ALL); + merged.setQueryGranularity(QueryGranularities.ALL); Assert.assertEquals( merged, Metadata.merge(ImmutableList.of(m1, m2), explicitAggs) diff --git a/processing/src/test/java/io/druid/segment/SchemalessIndex.java b/processing/src/test/java/io/druid/segment/SchemalessIndex.java index ab8e2019ab3..649384ff5c0 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessIndex.java +++ b/processing/src/test/java/io/druid/segment/SchemalessIndex.java @@ -31,7 +31,7 @@ import com.google.common.hash.Hashing; import com.metamx.common.Pair; import com.metamx.common.logger.Logger; import io.druid.data.input.MapBasedInputRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -141,7 +141,7 @@ public class SchemalessIndex final long timestamp = new DateTime(event.get(TIMESTAMP)).getMillis(); if (theIndex == null) { - theIndex = new OnheapIncrementalIndex(timestamp, QueryGranularity.MINUTE, METRIC_AGGS, 1000); + theIndex = new OnheapIncrementalIndex(timestamp, QueryGranularities.MINUTE, METRIC_AGGS, 1000); } final List dims = Lists.newArrayList(); @@ -349,7 +349,7 @@ public class SchemalessIndex } final IncrementalIndex rowIndex = new OnheapIncrementalIndex( - timestamp, QueryGranularity.MINUTE, METRIC_AGGS, 1000 + timestamp, QueryGranularities.MINUTE, METRIC_AGGS, 1000 ); rowIndex.add( @@ -379,7 +379,7 @@ public class SchemalessIndex log.info("Realtime loading index file[%s]", filename); final IncrementalIndex retVal = new OnheapIncrementalIndex( - new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularity.MINUTE, aggs, 1000 + new DateTime("2011-01-12T00:00:00.000Z").getMillis(), QueryGranularities.MINUTE, aggs, 1000 ); try { diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java index 9f5ff2dfd33..3032b311cc8 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFull.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFull.java @@ -25,6 +25,7 @@ import com.google.common.collect.Lists; import com.metamx.common.Pair; import com.metamx.common.guava.Sequences; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.QueryRunner; import io.druid.query.Result; @@ -69,7 +70,7 @@ public class SchemalessTestFull final double UNIQUES_1 = 1.0002442201269182d; final String dataSource = "testing"; - final QueryGranularity allGran = QueryGranularity.ALL; + final QueryGranularity allGran = QueryGranularities.ALL; final String dimensionValue = "dimension"; final String valueValue = "value"; final String marketDimension = "market"; diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java index b28881866e5..426e8c285e5 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimple.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.QueryRunner; import io.druid.query.Result; @@ -95,7 +96,7 @@ public class SchemalessTestSimple } final String dataSource = "testing"; - final QueryGranularity allGran = QueryGranularity.ALL; + final QueryGranularity allGran = QueryGranularities.ALL; final String dimensionValue = "dimension"; final String valueValue = "value"; final String marketDimension = "market"; diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 14eaf8c2d7b..c9687520f24 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -30,7 +30,7 @@ import io.druid.data.input.impl.DelimitedParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; @@ -178,7 +178,7 @@ public class TestIndex { final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() .withMinTimestamp(new DateTime("2011-01-12T00:00:00.000Z").getMillis()) - .withQueryGranularity(QueryGranularity.NONE) + .withQueryGranularity(QueryGranularities.NONE) .withMetrics(METRIC_AGGS) .build(); final IncrementalIndex retVal = new OnheapIncrementalIndex(schema, true, 10000); diff --git a/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java index 841e682d0ab..f0a5fd2c722 100644 --- a/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/data/IncrementalIndexTest.java @@ -37,7 +37,7 @@ import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.Row; import io.druid.data.input.impl.DimensionsSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryRunner; @@ -130,7 +130,7 @@ public class IncrementalIndexTest public IncrementalIndex createIndex(AggregatorFactory[] factories) { return new OffheapIncrementalIndex( - 0L, QueryGranularity.NONE, factories, 1000000, + 0L, QueryGranularities.NONE, factories, 1000000, new StupidPool( new Supplier() { @@ -167,7 +167,7 @@ public class IncrementalIndexTest } return new OnheapIncrementalIndex( - 0L, QueryGranularity.NONE, aggregatorFactories, 1000000 + 0L, QueryGranularities.NONE, aggregatorFactories, 1000000 ); } @@ -308,7 +308,7 @@ public class IncrementalIndexTest TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("xxx") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(ImmutableList.of(new Interval("2000/2030"))) .aggregators(queryAggregatorFactories) .build(); @@ -458,7 +458,7 @@ public class IncrementalIndexTest final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("xxx") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(ImmutableList.of(queryInterval)) .aggregators(queryAggregatorFactories) .build(); @@ -536,7 +536,7 @@ public class IncrementalIndexTest ); TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("xxx") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(ImmutableList.of(queryInterval)) .aggregators(queryAggregatorFactories) .build(); @@ -611,7 +611,7 @@ public class IncrementalIndexTest public void testgetDimensions() { final IncrementalIndex incrementalIndex = new OnheapIncrementalIndex( - new IncrementalIndexSchema.Builder().withQueryGranularity(QueryGranularity.NONE) + new IncrementalIndexSchema.Builder().withQueryGranularity(QueryGranularities.NONE) .withMetrics( new AggregatorFactory[]{ new CountAggregatorFactory( diff --git a/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java index a6307342ae1..76b7b08a619 100644 --- a/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java @@ -30,7 +30,7 @@ import com.metamx.common.guava.Sequence; import com.metamx.common.guava.Sequences; import io.druid.common.utils.JodaUtils; import io.druid.data.input.InputRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.FilteredAggregatorFactory; @@ -263,7 +263,7 @@ public abstract class BaseFilterTest final Sequence cursors = adapter.makeCursors( filter, new Interval(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT), - QueryGranularity.ALL, + QueryGranularities.ALL, false ); return Iterables.getOnlyElement(Sequences.toList(cursors, Lists.newArrayList())); diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java index b855bc6dcf9..b7738bab3fa 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -28,7 +28,7 @@ import com.metamx.collections.spatial.search.RectangularBound; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.SpatialDimensionSchema; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryRunner; @@ -118,7 +118,7 @@ public class SpatialFilterBonusTest { IncrementalIndex theIndex = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -257,7 +257,7 @@ public class SpatialFilterBonusTest try { IncrementalIndex first = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -277,7 +277,7 @@ public class SpatialFilterBonusTest ); IncrementalIndex second = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -296,7 +296,7 @@ public class SpatialFilterBonusTest ); IncrementalIndex third = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -457,7 +457,7 @@ public class SpatialFilterBonusTest { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("test") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) .filters( new SpatialDimFilter( @@ -509,7 +509,7 @@ public class SpatialFilterBonusTest { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("test") - .granularity(QueryGranularity.DAY) + .granularity(QueryGranularities.DAY) .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) .filters( new SpatialDimFilter( @@ -597,7 +597,7 @@ public class SpatialFilterBonusTest { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("test") - .granularity(QueryGranularity.DAY) + .granularity(QueryGranularities.DAY) .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) .aggregators( Arrays.asList( diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java index 2f20f61caa7..7d5a702abd5 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -28,7 +28,7 @@ import com.metamx.collections.spatial.search.RectangularBound; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.SpatialDimensionSchema; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryRunner; @@ -111,7 +111,7 @@ public class SpatialFilterTest { IncrementalIndex theIndex = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -275,7 +275,7 @@ public class SpatialFilterTest try { IncrementalIndex first = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -299,7 +299,7 @@ public class SpatialFilterTest ); IncrementalIndex second = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -323,7 +323,7 @@ public class SpatialFilterTest ); IncrementalIndex third = new OnheapIncrementalIndex( new IncrementalIndexSchema.Builder().withMinTimestamp(DATA_INTERVAL.getStartMillis()) - .withQueryGranularity(QueryGranularity.DAY) + .withQueryGranularity(QueryGranularities.DAY) .withMetrics(METRIC_AGGS) .withDimensionsSpec( new DimensionsSpec( @@ -518,7 +518,7 @@ public class SpatialFilterTest { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("test") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) .filters( new SpatialDimFilter( @@ -571,7 +571,7 @@ public class SpatialFilterTest { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("test") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) .filters( new SpatialDimFilter( @@ -623,7 +623,7 @@ public class SpatialFilterTest { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("test") - .granularity(QueryGranularity.DAY) + .granularity(QueryGranularities.DAY) .intervals(Arrays.asList(new Interval("2013-01-01/2013-01-07"))) .filters( new SpatialDimFilter( diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index caf353d3c0f..624d4df8f8f 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -31,7 +31,7 @@ import io.druid.collections.StupidPool; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.js.JavaScriptConfig; import io.druid.query.Result; import io.druid.query.aggregation.AggregatorFactory; @@ -96,7 +96,7 @@ public class IncrementalIndexStorageAdapterTest public IncrementalIndex createIndex() { return new OnheapIncrementalIndex( - 0, QueryGranularity.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 + 0, QueryGranularities.MINUTE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 ); } } @@ -129,7 +129,7 @@ public class IncrementalIndexStorageAdapterTest final Sequence rows = engine.process( GroupByQuery.builder() .setDataSource("test") - .setGranularity(QueryGranularity.ALL) + .setGranularity(QueryGranularities.ALL) .setInterval(new Interval(0, new DateTime().getMillis())) .addDimension("billy") .addDimension("sally") @@ -176,7 +176,7 @@ public class IncrementalIndexStorageAdapterTest final Sequence rows = engine.process( GroupByQuery.builder() .setDataSource("test") - .setGranularity(QueryGranularity.ALL) + .setGranularity(QueryGranularities.ALL) .setInterval(new Interval(0, new DateTime().getMillis())) .addDimension("billy") .addDimension("sally") @@ -263,7 +263,7 @@ public class IncrementalIndexStorageAdapterTest Sequence cursorSequence = adapter.makeCursors( new SelectorFilter("sally", "bo"), interval, - QueryGranularity.NONE, + QueryGranularities.NONE, descending ); @@ -318,7 +318,7 @@ public class IncrementalIndexStorageAdapterTest final Iterable> results = Sequences.toList( engine.query( new TopNQueryBuilder().dataSource("test") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(Lists.newArrayList(new Interval(0, new DateTime().getMillis()))) .dimension("sally") .metric("cnt") @@ -365,7 +365,7 @@ public class IncrementalIndexStorageAdapterTest final Sequence rows = engine.process( GroupByQuery.builder() .setDataSource("test") - .setGranularity(QueryGranularity.ALL) + .setGranularity(QueryGranularities.ALL) .setInterval(new Interval(0, new DateTime().getMillis())) .addDimension("billy") .addDimension("sally") @@ -402,7 +402,7 @@ public class IncrementalIndexStorageAdapterTest final StorageAdapter sa = new IncrementalIndexStorageAdapter(index); Sequence cursors = sa.makeCursors( - null, new Interval(timestamp - 60_000, timestamp + 60_000), QueryGranularity.ALL, false + null, new Interval(timestamp - 60_000, timestamp + 60_000), QueryGranularities.ALL, false ); Sequences.toList( diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java index 9d37eef772a..9b3f1b6552f 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexTest.java @@ -31,7 +31,7 @@ import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.FloatDimensionSchema; import io.druid.data.input.impl.LongDimensionSchema; import io.druid.data.input.impl.StringDimensionSchema; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.FilteredAggregatorFactory; @@ -89,7 +89,7 @@ public class IncrementalIndexTest }; final IncrementalIndexSchema schema = new IncrementalIndexSchema( 0, - QueryGranularity.MINUTE, + QueryGranularities.MINUTE, dimensions, metrics ); diff --git a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index 89a1195e3a8..471a43c68be 100644 --- a/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/io/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -36,6 +36,7 @@ import com.metamx.common.parsers.ParseException; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryRunner; @@ -252,7 +253,7 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark QueryGranularity.class, AggregatorFactory[].class, Integer.TYPE - ).newInstance(0, QueryGranularity.NONE, factories, elementsPerThread * taskCount); + ).newInstance(0, QueryGranularities.NONE, factories, elementsPerThread * taskCount); final ArrayList queryAggregatorFactories = new ArrayList<>(dimensionCount + 1); queryAggregatorFactories.add(new CountAggregatorFactory("rows")); for (int i = 0; i < dimensionCount; ++i) { @@ -339,7 +340,7 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark ); TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("xxx") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(ImmutableList.of(queryInterval)) .aggregators(queryAggregatorFactories) .build(); @@ -376,7 +377,7 @@ public class OnheapIncrementalIndexBenchmark extends AbstractBenchmark ); TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("xxx") - .granularity(QueryGranularity.ALL) + .granularity(QueryGranularities.ALL) .intervals(ImmutableList.of(queryInterval)) .aggregators(queryAggregatorFactories) .build(); diff --git a/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsCompTest.java b/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsCompTest.java index 60da4f5d4c4..6b11057ae7c 100644 --- a/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsCompTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/TimeAndDimsCompTest.java @@ -22,7 +22,7 @@ package io.druid.segment.incremental; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import io.druid.data.input.MapBasedInputRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import org.junit.Assert; @@ -42,7 +42,7 @@ public class TimeAndDimsCompTest public void testBasic() throws IndexSizeExceededException { IncrementalIndex index = new OnheapIncrementalIndex( - 0, QueryGranularity.NONE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 + 0, QueryGranularities.NONE, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000 ); long time = System.currentTimeMillis(); diff --git a/server/src/main/java/io/druid/segment/indexing/granularity/UniformGranularitySpec.java b/server/src/main/java/io/druid/segment/indexing/granularity/UniformGranularitySpec.java index 88b062b6fce..9af0b2f7568 100644 --- a/server/src/main/java/io/druid/segment/indexing/granularity/UniformGranularitySpec.java +++ b/server/src/main/java/io/druid/segment/indexing/granularity/UniformGranularitySpec.java @@ -27,6 +27,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.metamx.common.Granularity; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -36,7 +37,7 @@ import java.util.SortedSet; public class UniformGranularitySpec implements GranularitySpec { private static final Granularity DEFAULT_SEGMENT_GRANULARITY = Granularity.DAY; - private static final QueryGranularity DEFAULT_QUERY_GRANULARITY = QueryGranularity.NONE; + private static final QueryGranularity DEFAULT_QUERY_GRANULARITY = QueryGranularities.NONE; private final Granularity segmentGranularity; private final QueryGranularity queryGranularity; diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 4f09b05cfcf..277669ca06e 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -61,6 +61,7 @@ import io.druid.data.input.MapBasedRow; import io.druid.data.input.Row; import io.druid.granularity.PeriodGranularity; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.BySegmentResultValueClass; import io.druid.query.DataSource; @@ -206,7 +207,7 @@ public class CachingClusteredClientTest ); private static final DimFilter DIM_FILTER = null; private static final List RENAMED_POST_AGGS = ImmutableList.of(); - private static final QueryGranularity GRANULARITY = QueryGranularity.DAY; + private static final QueryGranularity GRANULARITY = QueryGranularities.DAY; private static final DateTimeZone TIMEZONE = DateTimeZone.forID("America/Los_Angeles"); private static final QueryGranularity PT1H_TZ_GRANULARITY = new PeriodGranularity(new Period("PT1H"), null, TIMEZONE); private static final String TOP_DIM = "a_dim"; diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index 16375b1669c..15a6c0b6861 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -34,7 +34,7 @@ import com.metamx.common.guava.YieldingAccumulator; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.MapCache; -import io.druid.granularity.AllGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.CacheStrategy; import io.druid.query.Druids; @@ -98,7 +98,7 @@ public class CachingQueryRunnerTest .threshold(3) .intervals("2011-01-05/2011-01-10") .aggregators(AGGS) - .granularity(AllGranularity.ALL); + .granularity(QueryGranularities.ALL); QueryToolChest toolchest = new TopNQueryQueryToolChest( new TopNQueryConfig(), diff --git a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java index 7bb5f6534c3..3c1e42a7031 100644 --- a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java @@ -29,7 +29,7 @@ import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -70,7 +70,7 @@ public class DataSchemaTest new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric2", "col2"), }, - new ArbitraryGranularitySpec(QueryGranularity.DAY, ImmutableList.of(Interval.parse("2014/2015"))), + new ArbitraryGranularitySpec(QueryGranularities.DAY, ImmutableList.of(Interval.parse("2014/2015"))), jsonMapper ); @@ -99,7 +99,7 @@ public class DataSchemaTest new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric2", "col2"), }, - new ArbitraryGranularitySpec(QueryGranularity.DAY, ImmutableList.of(Interval.parse("2014/2015"))), + new ArbitraryGranularitySpec(QueryGranularities.DAY, ImmutableList.of(Interval.parse("2014/2015"))), jsonMapper ); @@ -128,7 +128,7 @@ public class DataSchemaTest new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric2", "col2"), }, - new ArbitraryGranularitySpec(QueryGranularity.DAY, ImmutableList.of(Interval.parse("2014/2015"))), + new ArbitraryGranularitySpec(QueryGranularities.DAY, ImmutableList.of(Interval.parse("2014/2015"))), jsonMapper ); schema.getParser(); @@ -208,7 +208,7 @@ public class DataSchemaTest ); Assert.assertEquals( actual.getGranularitySpec(), - new ArbitraryGranularitySpec(QueryGranularity.DAY, ImmutableList.of(Interval.parse("2014/2015"))) + new ArbitraryGranularitySpec(QueryGranularities.DAY, ImmutableList.of(Interval.parse("2014/2015"))) ); } } diff --git a/server/src/test/java/io/druid/segment/indexing/granularity/ArbitraryGranularityTest.java b/server/src/test/java/io/druid/segment/indexing/granularity/ArbitraryGranularityTest.java index ba39339855e..8b6ca8ba8ba 100644 --- a/server/src/test/java/io/druid/segment/indexing/granularity/ArbitraryGranularityTest.java +++ b/server/src/test/java/io/druid/segment/indexing/granularity/ArbitraryGranularityTest.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Lists; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -40,7 +40,7 @@ public class ArbitraryGranularityTest public void testSimple() { final GranularitySpec spec = new ArbitraryGranularitySpec( - QueryGranularity.NONE, + QueryGranularities.NONE, Lists.newArrayList( new Interval("2012-01-08T00Z/2012-01-11T00Z"), new Interval("2012-02-01T00Z/2012-03-01T00Z"), @@ -114,7 +114,7 @@ public class ArbitraryGranularityTest boolean thrown = false; try { - final GranularitySpec spec = new ArbitraryGranularitySpec(QueryGranularity.NONE, intervals); + final GranularitySpec spec = new ArbitraryGranularitySpec(QueryGranularities.NONE, intervals); } catch(IllegalArgumentException e) { thrown = true; } @@ -132,7 +132,7 @@ public class ArbitraryGranularityTest boolean thrown = false; try { - final GranularitySpec spec = new ArbitraryGranularitySpec(QueryGranularity.NONE, intervals); + final GranularitySpec spec = new ArbitraryGranularitySpec(QueryGranularities.NONE, intervals); } catch(IllegalArgumentException e) { thrown = true; } @@ -143,7 +143,7 @@ public class ArbitraryGranularityTest @Test public void testJson() { - final GranularitySpec spec = new ArbitraryGranularitySpec(QueryGranularity.NONE, Lists.newArrayList( + final GranularitySpec spec = new ArbitraryGranularitySpec(QueryGranularities.NONE, Lists.newArrayList( new Interval("2012-01-08T00Z/2012-01-11T00Z"), new Interval("2012-02-01T00Z/2012-03-01T00Z"), new Interval("2012-01-07T00Z/2012-01-08T00Z"), diff --git a/server/src/test/java/io/druid/segment/indexing/granularity/UniformGranularityTest.java b/server/src/test/java/io/druid/segment/indexing/granularity/UniformGranularityTest.java index 021e11ac9a7..4d7dd46a35d 100644 --- a/server/src/test/java/io/druid/segment/indexing/granularity/UniformGranularityTest.java +++ b/server/src/test/java/io/druid/segment/indexing/granularity/UniformGranularityTest.java @@ -24,7 +24,7 @@ import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Lists; import com.metamx.common.Granularity; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -200,7 +200,7 @@ public class UniformGranularityTest notEqualsCheck( spec, new UniformGranularitySpec( Granularity.DAY, - QueryGranularity.ALL, + QueryGranularities.ALL, Lists.newArrayList( new Interval("2012-01-08T00Z/2012-01-11T00Z"), new Interval("2012-01-07T00Z/2012-01-08T00Z"), diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index dd290182954..0a8f70c00c2 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -28,7 +28,7 @@ import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -94,7 +94,7 @@ public class FireDepartmentTest new AggregatorFactory[]{ new CountAggregatorFactory("count") }, - new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.MINUTE, null), + new UniformGranularitySpec(Granularity.HOUR, QueryGranularities.MINUTE, null), jsonMapper ), new RealtimeIOConfig( diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index b8edd87bc3d..5e21b355cd3 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -41,7 +41,7 @@ import io.druid.data.input.FirehoseV2; import io.druid.data.input.InputRow; import io.druid.data.input.Row; import io.druid.data.input.impl.InputRowParser; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.BaseQuery; import io.druid.query.Query; @@ -135,14 +135,14 @@ public class RealtimeManagerTest "test", null, new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null), + new UniformGranularitySpec(Granularity.HOUR, QueryGranularities.NONE, null), jsonMapper ); schema2 = new DataSchema( "testV2", null, new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null), + new UniformGranularitySpec(Granularity.HOUR, QueryGranularities.NONE, null), jsonMapper ); RealtimeIOConfig ioConfig = new RealtimeIOConfig( @@ -280,7 +280,7 @@ public class RealtimeManagerTest "testing", null, new AggregatorFactory[]{new CountAggregatorFactory("ignore")}, - new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null), + new UniformGranularitySpec(Granularity.HOUR, QueryGranularities.NONE, null), jsonMapper ); diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java index 3b910bd8cbd..288e522bc5b 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTest.java @@ -29,7 +29,7 @@ import com.metamx.common.guava.Sequences; import io.druid.data.input.Committer; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.Druids; import io.druid.query.Result; import io.druid.query.SegmentDescriptor; @@ -214,7 +214,7 @@ public class AppenderatorTest new LongSumAggregatorFactory("met", "met") ) ) - .granularity(QueryGranularity.DAY) + .granularity(QueryGranularities.DAY) .build(); final List> results1 = Lists.newArrayList(); @@ -240,7 +240,7 @@ public class AppenderatorTest new LongSumAggregatorFactory("met", "met") ) ) - .granularity(QueryGranularity.DAY) + .granularity(QueryGranularities.DAY) .build(); final List> results2 = Lists.newArrayList(); @@ -270,7 +270,7 @@ public class AppenderatorTest new LongSumAggregatorFactory("met", "met") ) ) - .granularity(QueryGranularity.DAY) + .granularity(QueryGranularities.DAY) .build(); final List> results3 = Lists.newArrayList(); @@ -313,7 +313,7 @@ public class AppenderatorTest new LongSumAggregatorFactory("met", "met") ) ) - .granularity(QueryGranularity.DAY) + .granularity(QueryGranularities.DAY) .intervals( new MultipleSpecificSegmentSpec( ImmutableList.of( @@ -349,7 +349,7 @@ public class AppenderatorTest new LongSumAggregatorFactory("met", "met") ) ) - .granularity(QueryGranularity.DAY) + .granularity(QueryGranularities.DAY) .intervals( new MultipleSpecificSegmentSpec( ImmutableList.of( diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index 88dc4b0eb7d..1b73c6f0faf 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -33,7 +33,7 @@ import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.MapInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.IntervalChunkingQueryRunnerDecorator; @@ -115,7 +115,7 @@ public class AppenderatorTester implements AutoCloseable new CountAggregatorFactory("count"), new LongSumAggregatorFactory("met", "met") }, - new UniformGranularitySpec(Granularity.MINUTE, QueryGranularity.NONE, null), + new UniformGranularitySpec(Granularity.MINUTE, QueryGranularities.NONE, null), objectMapper ); diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java index 48fc264afe5..eeb55feb450 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java @@ -26,7 +26,7 @@ import io.druid.data.input.impl.CSVParseSpec; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.segment.IndexIO; @@ -73,7 +73,7 @@ public class IngestSegmentFirehoseTest ImmutableList.of("host"), ImmutableList.of("visited_sum", "unique_hosts"), null, - QueryGranularity.NONE + QueryGranularities.NONE ); int count = 0; @@ -120,7 +120,7 @@ public class IngestSegmentFirehoseTest IncrementalIndex index = null; try { - index = new OnheapIncrementalIndex(0, QueryGranularity.NONE, aggregators, true, true, true, 5000); + index = new OnheapIncrementalIndex(0, QueryGranularities.NONE, aggregators, true, true, true, 5000); for (String line : rows) { index.add(parser.parse(line)); } diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 55eb86dbfc2..348fa9d6b6e 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -36,7 +36,7 @@ import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Query; @@ -141,7 +141,7 @@ public class RealtimePlumberSchoolTest Map.class ), new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null), + new UniformGranularitySpec(Granularity.HOUR, QueryGranularities.NONE, null), jsonMapper ); @@ -157,7 +157,7 @@ public class RealtimePlumberSchoolTest Map.class ), new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularity.YEAR, QueryGranularity.NONE, null), + new UniformGranularitySpec(Granularity.YEAR, QueryGranularities.NONE, null), jsonMapper ); diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index 612853e5594..a086c4ed849 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -24,7 +24,7 @@ import com.google.common.collect.Lists; import com.metamx.common.Granularity; import io.druid.data.input.InputRow; import io.druid.data.input.Row; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -51,7 +51,7 @@ public class SinkTest "test", null, new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.MINUTE, null), + new UniformGranularitySpec(Granularity.HOUR, QueryGranularities.MINUTE, null), new DefaultObjectMapper() ); diff --git a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java index 29ce37c14be..a7c49295d28 100644 --- a/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java +++ b/server/src/test/java/io/druid/server/coordination/ServerManagerTest.java @@ -40,6 +40,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.LocalCacheProvider; import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.jackson.DefaultObjectMapper; import io.druid.query.ConcatQueryRunner; import io.druid.query.Druids; @@ -167,7 +168,7 @@ public class ServerManagerTest public void testSimpleGet() { Future future = assertQueryable( - QueryGranularity.DAY, + QueryGranularities.DAY, "test", new Interval("P1d/2011-04-01"), ImmutableList.>of( @@ -178,7 +179,7 @@ public class ServerManagerTest future = assertQueryable( - QueryGranularity.DAY, + QueryGranularities.DAY, "test", new Interval("P2d/2011-04-02"), ImmutableList.>of( new Pair("1", new Interval("P1d/2011-04-01")), @@ -195,7 +196,7 @@ public class ServerManagerTest final Interval interval = new Interval("2011-04-01/2011-04-02"); Future future = assertQueryable( - QueryGranularity.DAY, + QueryGranularities.DAY, dataSouce, interval, ImmutableList.>of( new Pair("2", interval) @@ -205,7 +206,7 @@ public class ServerManagerTest dropQueryable(dataSouce, "2", interval); future = assertQueryable( - QueryGranularity.DAY, + QueryGranularities.DAY, dataSouce, interval, ImmutableList.>of( new Pair("1", interval) @@ -220,7 +221,7 @@ public class ServerManagerTest loadQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); Future future = assertQueryable( - QueryGranularity.DAY, + QueryGranularities.DAY, "test", new Interval("2011-04-04/2011-04-06"), ImmutableList.>of( new Pair("3", new Interval("2011-04-04/2011-04-05")) @@ -232,7 +233,7 @@ public class ServerManagerTest dropQueryable("test", "1", new Interval("2011-04-04/2011-04-05")); future = assertQueryable( - QueryGranularity.HOUR, + QueryGranularities.HOUR, "test", new Interval("2011-04-04/2011-04-04T06"), ImmutableList.>of( new Pair("2", new Interval("2011-04-04T00/2011-04-04T01")), @@ -245,7 +246,7 @@ public class ServerManagerTest waitForTestVerificationAndCleanup(future); future = assertQueryable( - QueryGranularity.HOUR, + QueryGranularities.HOUR, "test", new Interval("2011-04-04/2011-04-04T03"), ImmutableList.>of( new Pair("2", new Interval("2011-04-04T00/2011-04-04T01")), @@ -256,7 +257,7 @@ public class ServerManagerTest waitForTestVerificationAndCleanup(future); future = assertQueryable( - QueryGranularity.HOUR, + QueryGranularities.HOUR, "test", new Interval("2011-04-04T04/2011-04-04T06"), ImmutableList.>of( new Pair("2", new Interval("2011-04-04T04/2011-04-04T05")), @@ -272,7 +273,7 @@ public class ServerManagerTest loadQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); Future future = assertQueryable( - QueryGranularity.DAY, + QueryGranularities.DAY, "test", new Interval("2011-04-04/2011-04-06"), ImmutableList.>of( new Pair("3", new Interval("2011-04-04/2011-04-05")) @@ -311,7 +312,7 @@ public class ServerManagerTest loadQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); Future future = assertQueryable( - QueryGranularity.DAY, + QueryGranularities.DAY, "test", new Interval("2011-04-04/2011-04-06"), ImmutableList.>of( new Pair("3", new Interval("2011-04-04/2011-04-05")) @@ -354,7 +355,7 @@ public class ServerManagerTest loadQueryable("test", "3", new Interval("2011-04-04/2011-04-05")); Future future = assertQueryable( - QueryGranularity.DAY, + QueryGranularities.DAY, "test", new Interval("2011-04-04/2011-04-06"), ImmutableList.>of( new Pair("3", new Interval("2011-04-04/2011-04-05")) diff --git a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java index bd8e0a4afd5..fcc12774d22 100644 --- a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java @@ -24,7 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Injector; import com.metamx.common.Granularity; import io.airlift.airline.Cli; -import io.druid.granularity.QueryGranularity; +import io.druid.granularity.QueryGranularities; import io.druid.guice.FirehoseModule; import io.druid.guice.GuiceInjectors; import io.druid.indexing.common.task.RealtimeIndexTask; @@ -150,7 +150,7 @@ public class DruidJsonValidatorTest "foo", null, new AggregatorFactory[0], - new UniformGranularitySpec(Granularity.HOUR, QueryGranularity.NONE, null), + new UniformGranularitySpec(Granularity.HOUR, QueryGranularities.NONE, null), jsonMapper ), new RealtimeIOConfig(