mirror of https://github.com/apache/druid.git
Fix testDeadlock timeout delay (#3979)
* No more singleton. Reduce iterations * Granularities * Fix the delay in the test * Add license header * Remove unused imports * Lot more unused imports from all the rearranging * CR feedback * Move javadoc to constructor
This commit is contained in:
parent
ef6a19c81b
commit
5ccfdcc48b
|
@ -31,10 +31,9 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.Interner;
|
||||
import com.google.common.collect.Interners;
|
||||
import com.google.common.collect.Iterables;
|
||||
|
||||
import io.druid.jackson.CommaListJoinDeserializer;
|
||||
import io.druid.jackson.CommaListJoinSerializer;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.SegmentDescriptor;
|
||||
import io.druid.timeline.partition.NoneShardSpec;
|
||||
import io.druid.timeline.partition.ShardSpec;
|
||||
|
@ -288,8 +287,8 @@ public class DataSegment implements Comparable<DataSegment>
|
|||
{
|
||||
int retVal;
|
||||
|
||||
DateTime lhsMonth = Granularity.MONTH.bucketStart(lhs.getInterval().getStart());
|
||||
DateTime rhsMonth = Granularity.MONTH.bucketStart(rhs.getInterval().getStart());
|
||||
DateTime lhsMonth = Granularities.MONTH.bucketStart(lhs.getInterval().getStart());
|
||||
DateTime rhsMonth = Granularities.MONTH.bucketStart(rhs.getInterval().getStart());
|
||||
|
||||
retVal = lhsMonth.compareTo(rhsMonth);
|
||||
|
||||
|
|
|
@ -19,11 +19,25 @@
|
|||
|
||||
package io.druid.benchmark;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.FluentIterable;
|
||||
import io.druid.collections.bitmap.BitmapFactory;
|
||||
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.collections.bitmap.MutableBitmap;
|
||||
import io.druid.collections.bitmap.RoaringBitmapFactory;
|
||||
import io.druid.collections.spatial.ImmutableRTree;
|
||||
import io.druid.extendedset.intset.ConciseSetUtils;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.query.filter.BoundDimFilter;
|
||||
import io.druid.query.ordering.StringComparators;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.data.BitmapSerdeFactory;
|
||||
import io.druid.segment.data.GenericIndexed;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import io.druid.segment.filter.BoundFilter;
|
||||
import io.druid.segment.serde.BitmapIndexColumnPartSupplier;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
|
@ -36,26 +50,10 @@ import org.openjdk.jmh.annotations.Setup;
|
|||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.annotations.Warmup;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.FluentIterable;
|
||||
|
||||
import io.druid.collections.bitmap.BitmapFactory;
|
||||
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.collections.bitmap.MutableBitmap;
|
||||
import io.druid.collections.bitmap.RoaringBitmapFactory;
|
||||
import io.druid.collections.spatial.ImmutableRTree;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.query.filter.BoundDimFilter;
|
||||
import io.druid.query.ordering.StringComparators;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.data.BitmapSerdeFactory;
|
||||
import io.druid.segment.data.GenericIndexed;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import io.druid.segment.filter.BoundFilter;
|
||||
import io.druid.segment.serde.BitmapIndexColumnPartSupplier;
|
||||
import io.druid.extendedset.intset.ConciseSetUtils;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@State(Scope.Benchmark)
|
||||
@Fork(value = 1)
|
||||
|
|
|
@ -20,8 +20,7 @@
|
|||
package io.druid.benchmark;
|
||||
|
||||
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import io.druid.extendedset.intset.ImmutableConciseSet;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Mode;
|
||||
|
@ -31,7 +30,7 @@ import org.openjdk.jmh.annotations.Scope;
|
|||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.infra.Blackhole;
|
||||
|
||||
import io.druid.extendedset.intset.ImmutableConciseSet;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@State(Scope.Benchmark)
|
||||
public class ConciseComplementBenchmark
|
||||
|
|
|
@ -32,7 +32,7 @@ import io.druid.data.input.InputRow;
|
|||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -230,7 +230,7 @@ public class FilterPartitionBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.build(),
|
||||
|
@ -508,7 +508,7 @@ public class FilterPartitionBenchmark
|
|||
|
||||
private Sequence<Cursor> makeCursors(StorageAdapter sa, Filter filter)
|
||||
{
|
||||
return sa.makeCursors(filter, schemaInfo.getDataInterval(), VirtualColumns.EMPTY, Granularity.ALL, false);
|
||||
return sa.makeCursors(filter, schemaInfo.getDataInterval(), VirtualColumns.EMPTY, Granularities.ALL, false);
|
||||
}
|
||||
|
||||
private Sequence<List<String>> readCursors(Sequence<Cursor> cursors, final Blackhole blackhole)
|
||||
|
|
|
@ -31,7 +31,7 @@ import io.druid.data.input.InputRow;
|
|||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -214,7 +214,7 @@ public class FilteredAggregatorBenchmark
|
|||
|
||||
query = Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(intervalSpec)
|
||||
.aggregators(queryAggs)
|
||||
.descending(false)
|
||||
|
@ -231,7 +231,7 @@ public class FilteredAggregatorBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(metrics)
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.build(),
|
||||
|
|
|
@ -39,6 +39,7 @@ import io.druid.data.input.Row;
|
|||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
|
@ -257,7 +258,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
.setAggregatorSpecs(
|
||||
queryAggs
|
||||
)
|
||||
.setGranularity(Granularity.DAY)
|
||||
.setGranularity(Granularities.DAY)
|
||||
.build();
|
||||
|
||||
GroupByQuery queryA = GroupByQuery
|
||||
|
@ -270,7 +271,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
.setAggregatorSpecs(
|
||||
queryAggs
|
||||
)
|
||||
.setGranularity(Granularity.WEEK)
|
||||
.setGranularity(Granularities.WEEK)
|
||||
.build();
|
||||
|
||||
basicQueries.put("nested", queryA);
|
||||
|
@ -434,7 +435,7 @@ public class GroupByTypeInterfaceBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.build(),
|
||||
|
|
|
@ -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.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
|
@ -122,7 +122,7 @@ public class IncrementalIndexRowTypeBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
aggs,
|
||||
false,
|
||||
false,
|
||||
|
|
|
@ -33,7 +33,7 @@ import io.druid.data.input.InputRow;
|
|||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -171,7 +171,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
// Use an IdentityExtractionFn to force usage of DimExtractionTopNAlgorithm
|
||||
TopNQueryBuilder queryBuilderString = new TopNQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.dimension(new ExtractionDimensionSpec("dimSequential", "dimSequential", IdentityExtractionFn.getInstance()))
|
||||
.metric("sumFloatNormal")
|
||||
.intervals(intervalSpec)
|
||||
|
@ -180,7 +180,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
// DimExtractionTopNAlgorithm is always used for numeric columns
|
||||
TopNQueryBuilder queryBuilderLong = new TopNQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.dimension("metLongUniform")
|
||||
.metric("sumFloatNormal")
|
||||
.intervals(intervalSpec)
|
||||
|
@ -188,7 +188,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
|
||||
TopNQueryBuilder queryBuilderFloat = new TopNQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.dimension("metFloatNormal")
|
||||
.metric("sumFloatNormal")
|
||||
.intervals(intervalSpec)
|
||||
|
@ -206,7 +206,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
|
||||
TopNQueryBuilder queryBuilderA = new TopNQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.dimension("dimUniform")
|
||||
.metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC))
|
||||
.intervals(intervalSpec)
|
||||
|
@ -222,7 +222,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
|
||||
TopNQueryBuilder queryBuilderA = new TopNQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.dimension("dimUniform")
|
||||
.metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC))
|
||||
.intervals(intervalSpec)
|
||||
|
@ -316,7 +316,7 @@ public class TopNTypeInterfaceBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.build(),
|
||||
|
|
|
@ -26,7 +26,7 @@ import io.druid.benchmark.datagen.BenchmarkSchemas;
|
|||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -125,7 +125,7 @@ public class IncrementalIndexReadBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.withRollup(rollup)
|
||||
|
@ -203,7 +203,7 @@ public class IncrementalIndexReadBenchmark
|
|||
filter.toFilter(),
|
||||
schemaInfo.getDataInterval(),
|
||||
VirtualColumns.EMPTY,
|
||||
Granularity.ALL,
|
||||
Granularities.ALL,
|
||||
false
|
||||
);
|
||||
}
|
||||
|
|
|
@ -24,8 +24,8 @@ import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
|
|||
import io.druid.benchmark.datagen.BenchmarkSchemas;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
|
@ -106,7 +106,7 @@ public class IndexIngestionBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.withRollup(rollup)
|
||||
|
|
|
@ -28,7 +28,7 @@ import io.druid.data.input.InputRow;
|
|||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||
import io.druid.segment.IndexIO;
|
||||
|
@ -163,7 +163,7 @@ public class IndexMergeBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.withRollup(rollup)
|
||||
|
|
|
@ -28,7 +28,7 @@ import io.druid.data.input.InputRow;
|
|||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||
import io.druid.segment.IndexIO;
|
||||
|
@ -157,7 +157,7 @@ public class IndexPersistBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.withRollup(rollup)
|
||||
|
|
|
@ -39,6 +39,7 @@ import io.druid.data.input.Row;
|
|||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
|
@ -219,7 +220,7 @@ public class GroupByBenchmark
|
|||
.setAggregatorSpecs(
|
||||
queryAggs
|
||||
)
|
||||
.setGranularity(Granularity.DAY)
|
||||
.setGranularity(Granularities.DAY)
|
||||
.build();
|
||||
|
||||
GroupByQuery queryA = GroupByQuery
|
||||
|
@ -232,7 +233,7 @@ public class GroupByBenchmark
|
|||
.setAggregatorSpecs(
|
||||
queryAggs
|
||||
)
|
||||
.setGranularity(Granularity.WEEK)
|
||||
.setGranularity(Granularities.WEEK)
|
||||
.build();
|
||||
|
||||
basicQueries.put("nested", queryA);
|
||||
|
@ -477,7 +478,7 @@ public class GroupByBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.withRollup(withRollup)
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package io.druid.benchmark.query;
|
||||
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.query.BySegmentQueryRunner;
|
||||
import io.druid.query.FinalizeResultsQueryRunner;
|
||||
|
|
|
@ -33,9 +33,9 @@ import io.druid.concurrent.Execs;
|
|||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -192,7 +192,7 @@ public class SearchBenchmark
|
|||
|
||||
return Druids.newSearchQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(intervalSpec)
|
||||
.query("123");
|
||||
}
|
||||
|
@ -221,7 +221,7 @@ public class SearchBenchmark
|
|||
|
||||
return Druids.newSearchQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(intervalSpec)
|
||||
.query("")
|
||||
.dimensions(Lists.newArrayList("dimUniform", "dimHyperUnique"))
|
||||
|
@ -275,7 +275,7 @@ public class SearchBenchmark
|
|||
|
||||
return Druids.newSearchQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(intervalSpec)
|
||||
.query("")
|
||||
.dimensions(Lists.newArrayList("dimUniform"))
|
||||
|
@ -304,7 +304,7 @@ public class SearchBenchmark
|
|||
|
||||
return Druids.newSearchQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(intervalSpec)
|
||||
.query("")
|
||||
.dimensions(Lists.newArrayList("dimUniform"))
|
||||
|
@ -390,7 +390,7 @@ public class SearchBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.build(),
|
||||
|
|
|
@ -30,9 +30,9 @@ import io.druid.concurrent.Execs;
|
|||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -161,7 +161,7 @@ public class SelectBenchmark
|
|||
.dimensionSpecs(DefaultDimensionSpec.toSpec(Arrays.<String>asList()))
|
||||
.metrics(Arrays.<String>asList())
|
||||
.intervals(intervalSpec)
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.descending(false);
|
||||
|
||||
basicQueries.put("A", queryBuilderA);
|
||||
|
@ -247,7 +247,7 @@ public class SelectBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.build(),
|
||||
|
@ -271,7 +271,7 @@ public class SelectBenchmark
|
|||
return Sequences.toList(queryResult, Lists.<T>newArrayList());
|
||||
}
|
||||
|
||||
// don't run this benchmark with a query that doesn't use QueryGranularity.ALL,
|
||||
// don't run this benchmark with a query that doesn't use QueryGranularities.ALL,
|
||||
// this pagination function probably doesn't work correctly in that case.
|
||||
private SelectQuery incrementQueryPagination(SelectQuery query, SelectResultValue prevResult)
|
||||
{
|
||||
|
|
|
@ -29,8 +29,8 @@ import io.druid.benchmark.datagen.BenchmarkSchemas;
|
|||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -190,7 +190,7 @@ public class SqlBenchmark
|
|||
)
|
||||
)
|
||||
.setAggregatorSpecs(Arrays.<AggregatorFactory>asList(new CountAggregatorFactory("c")))
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.build();
|
||||
|
||||
sqlQuery = "SELECT\n"
|
||||
|
|
|
@ -29,9 +29,9 @@ import io.druid.benchmark.datagen.BenchmarkSchemas;
|
|||
import io.druid.concurrent.Execs;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -167,7 +167,7 @@ public class TimeseriesBenchmark
|
|||
TimeseriesQuery queryA =
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(intervalSpec)
|
||||
.aggregators(queryAggs)
|
||||
.descending(false)
|
||||
|
@ -187,7 +187,7 @@ public class TimeseriesBenchmark
|
|||
TimeseriesQuery timeFilterQuery =
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(intervalSpec)
|
||||
.aggregators(queryAggs)
|
||||
.descending(false)
|
||||
|
@ -207,7 +207,7 @@ public class TimeseriesBenchmark
|
|||
TimeseriesQuery timeFilterQuery =
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(intervalSpec)
|
||||
.aggregators(queryAggs)
|
||||
.descending(false)
|
||||
|
@ -224,7 +224,7 @@ public class TimeseriesBenchmark
|
|||
TimeseriesQuery timeFilterQuery =
|
||||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(intervalSpec)
|
||||
.aggregators(queryAggs)
|
||||
.descending(false)
|
||||
|
@ -314,7 +314,7 @@ public class TimeseriesBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.build(),
|
||||
|
|
|
@ -30,9 +30,9 @@ import io.druid.collections.StupidPool;
|
|||
import io.druid.concurrent.Execs;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.hll.HyperLogLogHash;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -167,7 +167,7 @@ public class TopNBenchmark
|
|||
|
||||
TopNQueryBuilder queryBuilderA = new TopNQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.dimension("dimSequential")
|
||||
.metric("sumFloatNormal")
|
||||
.intervals(intervalSpec)
|
||||
|
@ -183,7 +183,7 @@ public class TopNBenchmark
|
|||
|
||||
TopNQueryBuilder queryBuilderA = new TopNQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.dimension("dimUniform")
|
||||
.metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC))
|
||||
.intervals(intervalSpec)
|
||||
|
@ -199,7 +199,7 @@ public class TopNBenchmark
|
|||
|
||||
TopNQueryBuilder queryBuilderA = new TopNQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.dimension("dimUniform")
|
||||
.metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC))
|
||||
.intervals(intervalSpec)
|
||||
|
@ -294,7 +294,7 @@ public class TopNBenchmark
|
|||
{
|
||||
return new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMetrics(schemaInfo.getAggsArray())
|
||||
.withDimensionsSpec(new DimensionsSpec(null, null, null))
|
||||
.build(),
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.Lists;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.QueryRunnerTestHelper;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
|
@ -57,7 +57,7 @@ public class DistinctCountGroupByQueryTest
|
|||
final GroupByQueryRunnerFactory factory = GroupByQueryRunnerTest.makeQueryRunnerFactory(config);
|
||||
|
||||
IncrementalIndex index = new OnheapIncrementalIndex(
|
||||
0, Granularity.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000
|
||||
0, Granularities.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000
|
||||
);
|
||||
String visitor_id = "visitor_id";
|
||||
String client_type = "client_type";
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.Lists;
|
||||
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.QueryRunnerTestHelper;
|
||||
|
@ -52,7 +52,7 @@ public class DistinctCountTimeseriesQueryTest
|
|||
TimeseriesQueryEngine engine = new TimeseriesQueryEngine();
|
||||
|
||||
IncrementalIndex index = new OnheapIncrementalIndex(
|
||||
0, Granularity.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000
|
||||
0, Granularities.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000
|
||||
);
|
||||
String visitor_id = "visitor_id";
|
||||
String client_type = "client_type";
|
||||
|
|
|
@ -25,7 +25,7 @@ import com.google.common.collect.Lists;
|
|||
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.QueryRunnerTestHelper;
|
||||
import io.druid.query.Result;
|
||||
|
@ -68,7 +68,7 @@ public class DistinctCountTopNQueryTest
|
|||
);
|
||||
|
||||
IncrementalIndex index = new OnheapIncrementalIndex(
|
||||
0, Granularity.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000
|
||||
0, Granularities.SECOND, new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, 1000
|
||||
);
|
||||
String visitor_id = "visitor_id";
|
||||
String client_type = "client_type";
|
||||
|
|
|
@ -37,7 +37,7 @@ import io.druid.indexer.HadoopyShardSpec;
|
|||
import io.druid.indexer.IndexGeneratorJob;
|
||||
import io.druid.indexer.JobHelper;
|
||||
import io.druid.indexer.Jobby;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
|
@ -189,7 +189,7 @@ public class OrcIndexGeneratorJobTest
|
|||
),
|
||||
aggs,
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY, Granularity.NONE, ImmutableList.of(this.interval)
|
||||
Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)
|
||||
),
|
||||
mapper
|
||||
),
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.BaseSequence;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
|
@ -113,7 +113,7 @@ public class ScanQueryEngine
|
|||
filter,
|
||||
intervals.get(0),
|
||||
VirtualColumns.EMPTY,
|
||||
Granularity.ALL,
|
||||
Granularities.ALL,
|
||||
query.isDescending()
|
||||
),
|
||||
new Function<Cursor, Sequence<ScanResultValue>>()
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.io.CharSource;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.MergeSequence;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
|
@ -144,7 +144,7 @@ public class MultiSegmentScanQueryTest
|
|||
{
|
||||
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
|
||||
.withMinTimestamp(new DateTime(minTimeStamp).getMillis())
|
||||
.withQueryGranularity(Granularity.HOUR)
|
||||
.withQueryGranularity(Granularities.HOUR)
|
||||
.withMetrics(TestIndex.METRIC_AGGS)
|
||||
.build();
|
||||
return new OnheapIncrementalIndex(schema, true, maxRowCount);
|
||||
|
|
|
@ -25,7 +25,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.io.Resources;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.Result;
|
||||
|
@ -149,7 +149,7 @@ public class TimestampAggregationSelectTest
|
|||
recordParser,
|
||||
aggregator,
|
||||
0,
|
||||
Granularity.MONTH,
|
||||
Granularities.MONTH,
|
||||
100,
|
||||
Resources.toString(Resources.getResource("select.json"), Charsets.UTF_8)
|
||||
);
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.Lists;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.groupby.GroupByQueryConfig;
|
||||
|
@ -175,7 +175,7 @@ public class TimestampGroupByAggregationTest
|
|||
recordParser,
|
||||
aggregator,
|
||||
0,
|
||||
Granularity.MONTH,
|
||||
Granularities.MONTH,
|
||||
100,
|
||||
groupBy
|
||||
);
|
||||
|
|
|
@ -28,7 +28,7 @@ import io.druid.data.input.impl.DimensionsSpec;
|
|||
import io.druid.data.input.impl.StringInputRowParser;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.QueryRunner;
|
||||
|
@ -80,7 +80,7 @@ public class MapVirtualColumnTest
|
|||
|
||||
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
|
||||
.withMinTimestamp(new DateTime("2011-01-12T00:00:00.000Z").getMillis())
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.build();
|
||||
final IncrementalIndex index = new OnheapIncrementalIndex(schema, true, 10000);
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ import com.yahoo.sketches.theta.Sketches;
|
|||
import com.yahoo.sketches.theta.Union;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.aggregation.AggregationTestHelper;
|
||||
|
@ -94,7 +94,7 @@ public class SketchAggregationTest
|
|||
readFileFromClasspathAsString("sketch_test_data_record_parser.json"),
|
||||
readFileFromClasspathAsString("sketch_test_data_aggregators.json"),
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
5,
|
||||
readFileFromClasspathAsString("sketch_test_data_group_by_query.json")
|
||||
);
|
||||
|
@ -135,7 +135,7 @@ public class SketchAggregationTest
|
|||
+ " }"
|
||||
+ "]",
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
5,
|
||||
readFileFromClasspathAsString("simple_test_data_group_by_query.json")
|
||||
);
|
||||
|
@ -333,7 +333,7 @@ public class SketchAggregationTest
|
|||
readFileFromClasspathAsString("simple_test_data_record_parser.json"),
|
||||
readFileFromClasspathAsString("simple_test_data_aggregators.json"),
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
5,
|
||||
readFileFromClasspathAsString("retention_test_data_group_by_query.json")
|
||||
);
|
||||
|
|
|
@ -26,7 +26,7 @@ import com.google.common.collect.Lists;
|
|||
import com.google.common.io.Files;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.Result;
|
||||
|
@ -99,7 +99,7 @@ public class SketchAggregationTestWithSimpleData
|
|||
readFileFromClasspathAsString("simple_test_data_aggregators.json"),
|
||||
s1,
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
5000
|
||||
);
|
||||
|
||||
|
@ -110,7 +110,7 @@ public class SketchAggregationTestWithSimpleData
|
|||
readFileFromClasspathAsString("simple_test_data_aggregators.json"),
|
||||
s2,
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
5000
|
||||
);
|
||||
}
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.io.Files;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.aggregation.AggregationTestHelper;
|
||||
|
@ -86,7 +86,7 @@ public class OldApiSketchAggregationTest
|
|||
readFileFromClasspathAsString("simple_test_data_record_parser.json"),
|
||||
readFileFromClasspathAsString("oldapi/old_simple_test_data_aggregators.json"),
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
5,
|
||||
readFileFromClasspathAsString("oldapi/old_simple_test_data_group_by_query.json")
|
||||
);
|
||||
|
@ -118,7 +118,7 @@ public class OldApiSketchAggregationTest
|
|||
readFileFromClasspathAsString("sketch_test_data_record_parser.json"),
|
||||
readFileFromClasspathAsString("oldapi/old_sketch_test_data_aggregators.json"),
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
5,
|
||||
readFileFromClasspathAsString("oldapi/old_sketch_test_data_group_by_query.json")
|
||||
);
|
||||
|
|
|
@ -21,7 +21,7 @@ package io.druid.query.aggregation.histogram;
|
|||
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.aggregation.AggregationTestHelper;
|
||||
|
@ -135,7 +135,7 @@ public class ApproximateHistogramAggregationTest
|
|||
parseSpec,
|
||||
metricSpec,
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
50000,
|
||||
query
|
||||
);
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
|
@ -176,7 +176,7 @@ public class QuantileSqlAggregatorTest
|
|||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(ImmutableList.of(
|
||||
new ApproximateHistogramAggregatorFactory("a0:agg", "m1", null, null, null, null),
|
||||
new ApproximateHistogramAggregatorFactory("a2:agg", "m1", 200, null, null, null),
|
||||
|
@ -238,7 +238,7 @@ public class QuantileSqlAggregatorTest
|
|||
Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(CalciteTests.DATASOURCE1)
|
||||
.intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity())))
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.aggregators(ImmutableList.of(
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null),
|
||||
new ApproximateHistogramFoldingAggregatorFactory("a2:agg", "hist_m1", 200, null, null, null),
|
||||
|
|
|
@ -36,7 +36,6 @@ import io.druid.indexing.common.TaskLocation;
|
|||
import io.druid.indexing.common.TaskStatus;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.IAE;
|
||||
|
||||
import org.easymock.Capture;
|
||||
import org.easymock.CaptureType;
|
||||
import org.easymock.EasyMockSupport;
|
||||
|
|
|
@ -74,7 +74,7 @@ import io.druid.indexing.test.TestDataSegmentKiller;
|
|||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.CompressionUtils;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.metadata.EntryExistsException;
|
||||
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
|
||||
|
@ -199,7 +199,7 @@ public class KafkaIndexTaskTest
|
|||
Map.class
|
||||
),
|
||||
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
|
||||
new UniformGranularitySpec(Granularity.DAY, Granularity.NONE, null),
|
||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
||||
objectMapper
|
||||
);
|
||||
}
|
||||
|
@ -1607,7 +1607,7 @@ public class KafkaIndexTaskTest
|
|||
ImmutableList.<AggregatorFactory>of(
|
||||
new LongSumAggregatorFactory("rows", "rows")
|
||||
)
|
||||
).granularity(Granularity.ALL)
|
||||
).granularity(Granularities.ALL)
|
||||
.intervals("0000/3000")
|
||||
.build();
|
||||
|
||||
|
|
|
@ -58,7 +58,7 @@ import io.druid.indexing.overlord.TaskStorage;
|
|||
import io.druid.indexing.overlord.supervisor.SupervisorReport;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
|
@ -1749,8 +1749,8 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
),
|
||||
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.HOUR,
|
||||
Granularity.NONE,
|
||||
Granularities.HOUR,
|
||||
Granularities.NONE,
|
||||
ImmutableList.<Interval>of()
|
||||
),
|
||||
objectMapper
|
||||
|
|
|
@ -27,7 +27,6 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.io.ByteStreams;
|
||||
import com.google.common.io.Files;
|
||||
import com.google.common.io.OutputSupplier;
|
||||
|
||||
import io.druid.indexer.updater.HadoopDruidConverterConfig;
|
||||
import io.druid.java.util.common.FileUtils;
|
||||
import io.druid.java.util.common.IAE;
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.timeline.DataSegment;
|
||||
|
@ -76,7 +77,7 @@ public class DatasourceIngestionSpec
|
|||
this.segments = segments;
|
||||
|
||||
this.filter = filter;
|
||||
this.granularity = granularity == null ? Granularity.NONE : granularity;
|
||||
this.granularity = granularity == null ? Granularities.NONE : granularity;
|
||||
|
||||
this.dimensions = dimensions;
|
||||
this.metrics = metrics;
|
||||
|
|
|
@ -35,7 +35,7 @@ import io.druid.data.input.impl.StringInputRowParser;
|
|||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.indexer.hadoop.WindowedDataSegment;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
|
@ -323,7 +323,7 @@ public class BatchDeltaIngestionTest
|
|||
ImmutableList.of("host"),
|
||||
ImmutableList.of("visited_sum", "unique_hosts"),
|
||||
null,
|
||||
Granularity.NONE
|
||||
Granularities.NONE
|
||||
);
|
||||
|
||||
List<InputRow> rows = Lists.newArrayList();
|
||||
|
@ -358,7 +358,7 @@ public class BatchDeltaIngestionTest
|
|||
new HyperUniquesAggregatorFactory("unique_hosts", "host2")
|
||||
},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY, Granularity.NONE, ImmutableList.of(INTERVAL_FULL)
|
||||
Granularities.DAY, Granularities.NONE, ImmutableList.of(INTERVAL_FULL)
|
||||
),
|
||||
MAPPER
|
||||
),
|
||||
|
|
|
@ -27,7 +27,7 @@ import io.druid.data.input.impl.DimensionsSpec;
|
|||
import io.druid.data.input.impl.StringInputRowParser;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.indexer.partitions.HashedPartitionsSpec;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
|
@ -137,8 +137,8 @@ public class DetermineHashedPartitionsJobTest
|
|||
),
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("index", "index")},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY,
|
||||
Granularity.NONE,
|
||||
Granularities.DAY,
|
||||
Granularities.NONE,
|
||||
ImmutableList.of(new Interval(interval))
|
||||
),
|
||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
||||
|
|
|
@ -27,7 +27,7 @@ import io.druid.data.input.impl.DimensionsSpec;
|
|||
import io.druid.data.input.impl.StringInputRowParser;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
|
@ -235,7 +235,7 @@ public class DeterminePartitionsJobTest
|
|||
),
|
||||
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY, Granularity.NONE, ImmutableList.of(new Interval(interval))
|
||||
Granularities.DAY, Granularities.NONE, ImmutableList.of(new Interval(interval))
|
||||
),
|
||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
||||
),
|
||||
|
|
|
@ -27,7 +27,7 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.Lists;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
|
@ -202,8 +202,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
null,
|
||||
new AggregatorFactory[0],
|
||||
new UniformGranularitySpec(
|
||||
Granularity.MINUTE,
|
||||
Granularity.MINUTE,
|
||||
Granularities.MINUTE,
|
||||
Granularities.MINUTE,
|
||||
ImmutableList.of(new Interval("2010-01-01/P1D"))
|
||||
),
|
||||
jsonMapper
|
||||
|
@ -244,7 +244,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 = Granularity.MINUTE.bucketEnd(new DateTime(timestamp)).getMillis();
|
||||
final long nextBucketTimestamp = Granularities.MINUTE.bucketEnd(new DateTime(timestamp)).getMillis();
|
||||
// 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(
|
||||
|
@ -264,8 +264,8 @@ public class HadoopDruidIndexerConfigTest
|
|||
null,
|
||||
new AggregatorFactory[0],
|
||||
new UniformGranularitySpec(
|
||||
Granularity.MINUTE,
|
||||
Granularity.MINUTE,
|
||||
Granularities.MINUTE,
|
||||
Granularities.MINUTE,
|
||||
ImmutableList.of(new Interval("2010-01-01/P1D"))
|
||||
),
|
||||
jsonMapper
|
||||
|
|
|
@ -30,7 +30,7 @@ import io.druid.indexer.partitions.PartitionsSpec;
|
|||
import io.druid.indexer.partitions.SingleDimensionPartitionsSpec;
|
||||
import io.druid.indexer.updater.MetadataStorageUpdaterJobSpec;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.PeriodGranularity;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
|
@ -84,7 +84,7 @@ public class HadoopIngestionSpecTest
|
|||
|
||||
Assert.assertEquals(
|
||||
"getSegmentGranularity",
|
||||
Granularity.HOUR,
|
||||
Granularities.HOUR,
|
||||
granularitySpec.getSegmentGranularity()
|
||||
);
|
||||
}
|
||||
|
|
|
@ -32,7 +32,7 @@ import io.druid.indexer.path.PathSpec;
|
|||
import io.druid.indexer.path.StaticPathSpec;
|
||||
import io.druid.indexer.path.UsedSegmentLister;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
|
||||
|
@ -235,7 +235,7 @@ public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest
|
|||
null,
|
||||
new AggregatorFactory[0],
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
null,
|
||||
ImmutableList.of(
|
||||
new Interval("2010-01-01/P1D")
|
||||
|
|
|
@ -28,7 +28,7 @@ import io.druid.data.input.impl.DimensionsSpec;
|
|||
import io.druid.data.input.impl.StringInputRowParser;
|
||||
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
|
@ -78,7 +78,7 @@ public class IndexGeneratorCombinerTest
|
|||
new HyperUniquesAggregatorFactory("unique_hosts", "host")
|
||||
},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY, Granularity.NONE, ImmutableList.of(Interval.parse("2010/2011"))
|
||||
Granularities.DAY, Granularities.NONE, ImmutableList.of(Interval.parse("2010/2011"))
|
||||
),
|
||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
||||
),
|
||||
|
|
|
@ -32,7 +32,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.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
|
@ -490,7 +490,7 @@ public class IndexGeneratorJobTest
|
|||
),
|
||||
aggs,
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY, Granularity.NONE, ImmutableList.of(this.interval)
|
||||
Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)
|
||||
),
|
||||
mapper
|
||||
),
|
||||
|
|
|
@ -25,7 +25,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.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
|
@ -85,7 +85,7 @@ public class JobHelperTest
|
|||
),
|
||||
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY, Granularity.NONE, ImmutableList.of(this.interval)
|
||||
Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)
|
||||
),
|
||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
||||
),
|
||||
|
|
|
@ -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.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
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),
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
Lists.newArrayList("d1", "d2"),
|
||||
Lists.newArrayList("m1", "m2", "m3"),
|
||||
false
|
||||
|
@ -133,7 +133,7 @@ public class DatasourceIngestionSpecTest
|
|||
)
|
||||
),
|
||||
new SelectorDimFilter("dim", "value", null),
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
Lists.newArrayList("d1", "d2"),
|
||||
Lists.newArrayList("m1", "m2", "m3"),
|
||||
true
|
||||
|
|
|
@ -44,7 +44,7 @@ import io.druid.indexer.hadoop.WindowedDataSegment;
|
|||
import io.druid.initialization.Initialization;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
|
@ -276,7 +276,7 @@ public class DatasourcePathSpecTest
|
|||
new LongSumAggregatorFactory("visited_sum", "visited")
|
||||
},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY, Granularity.NONE, ImmutableList.of(Interval.parse("2000/3000"))
|
||||
Granularities.DAY, Granularities.NONE, ImmutableList.of(Interval.parse("2000/3000"))
|
||||
),
|
||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
||||
),
|
||||
|
|
|
@ -29,6 +29,7 @@ import io.druid.indexer.HadoopIOConfig;
|
|||
import io.druid.indexer.HadoopIngestionSpec;
|
||||
import io.druid.indexer.HadoopTuningConfig;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.PeriodGranularity;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
|
@ -119,7 +120,7 @@ public class GranularityPathSpecTest
|
|||
@Test
|
||||
public void testSetDataGranularity()
|
||||
{
|
||||
Granularity granularity = Granularity.DAY;
|
||||
Granularity granularity = Granularities.DAY;
|
||||
granularityPathSpec.setDataGranularity(granularity);
|
||||
Assert.assertEquals(granularity, granularityPathSpec.getDataGranularity());
|
||||
}
|
||||
|
@ -127,13 +128,13 @@ public class GranularityPathSpecTest
|
|||
@Test
|
||||
public void testSerdeCustomInputFormat() throws Exception
|
||||
{
|
||||
testSerde("/test/path", "*.test", "pat_pat", Granularity.SECOND, TextInputFormat.class);
|
||||
testSerde("/test/path", "*.test", "pat_pat", Granularities.SECOND, TextInputFormat.class);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSerdeNoInputFormat() throws Exception
|
||||
{
|
||||
testSerde("/test/path", "*.test", "pat_pat", Granularity.SECOND, null);
|
||||
testSerde("/test/path", "*.test", "pat_pat", Granularities.SECOND, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -146,8 +147,8 @@ public class GranularityPathSpecTest
|
|||
null,
|
||||
new AggregatorFactory[0],
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY,
|
||||
Granularity.MINUTE,
|
||||
Granularities.DAY,
|
||||
Granularities.MINUTE,
|
||||
ImmutableList.of(new Interval("2015-11-06T00:00Z/2015-11-07T00:00Z"))
|
||||
),
|
||||
jsonMapper
|
||||
|
@ -156,7 +157,7 @@ public class GranularityPathSpecTest
|
|||
DEFAULT_TUNING_CONFIG
|
||||
);
|
||||
|
||||
granularityPathSpec.setDataGranularity(Granularity.HOUR);
|
||||
granularityPathSpec.setDataGranularity(Granularities.HOUR);
|
||||
granularityPathSpec.setFilePattern(".*");
|
||||
granularityPathSpec.setInputFormat(TextInputFormat.class);
|
||||
|
||||
|
@ -197,8 +198,8 @@ public class GranularityPathSpecTest
|
|||
null,
|
||||
new AggregatorFactory[0],
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY,
|
||||
Granularity.ALL,
|
||||
Granularities.DAY,
|
||||
Granularities.ALL,
|
||||
ImmutableList.of(new Interval("2015-01-01T11Z/2015-01-02T05Z"))
|
||||
),
|
||||
jsonMapper
|
||||
|
@ -207,7 +208,7 @@ public class GranularityPathSpecTest
|
|||
DEFAULT_TUNING_CONFIG
|
||||
);
|
||||
|
||||
granularityPathSpec.setDataGranularity(Granularity.HOUR);
|
||||
granularityPathSpec.setDataGranularity(Granularities.HOUR);
|
||||
granularityPathSpec.setPathFormat("yyyy/MM/dd/HH");
|
||||
granularityPathSpec.setFilePattern(".*");
|
||||
granularityPathSpec.setInputFormat(TextInputFormat.class);
|
||||
|
@ -243,7 +244,7 @@ public class GranularityPathSpecTest
|
|||
{
|
||||
final PeriodGranularity pt2S = new PeriodGranularity(new Period("PT2S"), null, DateTimeZone.UTC);
|
||||
Assert.assertNotEquals("\"SECOND\"", jsonMapper.writeValueAsString(pt2S));
|
||||
final Granularity pt1S = Granularity.SECOND;
|
||||
final Granularity pt1S = Granularities.SECOND;
|
||||
Assert.assertEquals("\"SECOND\"", jsonMapper.writeValueAsString(pt1S));
|
||||
}
|
||||
|
||||
|
|
|
@ -42,7 +42,7 @@ import io.druid.indexer.JobHelper;
|
|||
import io.druid.indexer.Jobby;
|
||||
import io.druid.indexer.SQLMetadataStorageUpdaterJobHandler;
|
||||
import io.druid.java.util.common.FileUtils;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.metadata.MetadataSegmentManagerConfig;
|
||||
import io.druid.metadata.MetadataStorageConnectorConfig;
|
||||
import io.druid.metadata.MetadataStorageTablesConfig;
|
||||
|
@ -175,8 +175,8 @@ public class HadoopConverterJobTest
|
|||
new HyperUniquesAggregatorFactory("quality_uniques", "quality")
|
||||
},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.MONTH,
|
||||
Granularity.DAY,
|
||||
Granularities.MONTH,
|
||||
Granularities.DAY,
|
||||
ImmutableList.<Interval>of(interval)
|
||||
),
|
||||
HadoopDruidIndexerConfig.JSON_MAPPER
|
||||
|
|
|
@ -34,11 +34,11 @@ 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.java.util.common.granularity.Granularity;
|
||||
import io.druid.indexing.common.TaskToolbox;
|
||||
import io.druid.indexing.common.TaskToolboxFactory;
|
||||
import io.druid.indexing.common.actions.SegmentListUsedAction;
|
||||
import io.druid.indexing.common.task.NoopTask;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.parsers.ParseException;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.segment.IndexIO;
|
||||
|
@ -274,7 +274,7 @@ public class IngestSegmentFirehoseFactory implements FirehoseFactory<InputRowPar
|
|||
)
|
||||
);
|
||||
|
||||
return new IngestSegmentFirehose(adapters, dims, metricsList, dimFilter, Granularity.NONE);
|
||||
return new IngestSegmentFirehose(adapters, dims, metricsList, dimFilter, Granularities.NONE);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
|
|
|
@ -25,13 +25,13 @@ import com.google.common.collect.FluentIterable;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.indexing.common.TaskLock;
|
||||
import io.druid.indexing.common.task.NoopTask;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.segment.realtime.appenderator.SegmentIdentifier;
|
||||
import io.druid.timeline.DataSegment;
|
||||
import io.druid.timeline.partition.LinearShardSpec;
|
||||
|
@ -60,17 +60,17 @@ public class SegmentAllocateActionTest
|
|||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
Granularity.DAY,
|
||||
Granularity.SIX_HOUR,
|
||||
Granularity.HOUR,
|
||||
Granularity.THIRTY_MINUTE,
|
||||
Granularity.FIFTEEN_MINUTE,
|
||||
Granularity.TEN_MINUTE,
|
||||
Granularity.FIVE_MINUTE,
|
||||
Granularity.MINUTE,
|
||||
Granularity.SECOND
|
||||
Granularities.DAY,
|
||||
Granularities.SIX_HOUR,
|
||||
Granularities.HOUR,
|
||||
Granularities.THIRTY_MINUTE,
|
||||
Granularities.FIFTEEN_MINUTE,
|
||||
Granularities.TEN_MINUTE,
|
||||
Granularities.FIVE_MINUTE,
|
||||
Granularities.MINUTE,
|
||||
Granularities.SECOND
|
||||
),
|
||||
Granularity.granularitiesFinerThan(Granularity.DAY)
|
||||
Granularity.granularitiesFinerThan(Granularities.DAY)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -79,15 +79,15 @@ public class SegmentAllocateActionTest
|
|||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(
|
||||
Granularity.HOUR,
|
||||
Granularity.THIRTY_MINUTE,
|
||||
Granularity.FIFTEEN_MINUTE,
|
||||
Granularity.TEN_MINUTE,
|
||||
Granularity.FIVE_MINUTE,
|
||||
Granularity.MINUTE,
|
||||
Granularity.SECOND
|
||||
Granularities.HOUR,
|
||||
Granularities.THIRTY_MINUTE,
|
||||
Granularities.FIFTEEN_MINUTE,
|
||||
Granularities.TEN_MINUTE,
|
||||
Granularities.FIVE_MINUTE,
|
||||
Granularities.MINUTE,
|
||||
Granularities.SECOND
|
||||
),
|
||||
Granularity.granularitiesFinerThan(Granularity.HOUR)
|
||||
Granularity.granularitiesFinerThan(Granularities.HOUR)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -101,24 +101,24 @@ public class SegmentAllocateActionTest
|
|||
final SegmentIdentifier id1 = allocate(
|
||||
task,
|
||||
PARTY_TIME,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
null
|
||||
);
|
||||
final SegmentIdentifier id2 = allocate(
|
||||
task,
|
||||
PARTY_TIME,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
id1.getIdentifierAsString()
|
||||
);
|
||||
final SegmentIdentifier id3 = allocate(
|
||||
task,
|
||||
PARTY_TIME,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
id2.getIdentifierAsString()
|
||||
);
|
||||
|
@ -141,7 +141,7 @@ public class SegmentAllocateActionTest
|
|||
id1,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
partyLock.getVersion(),
|
||||
new NumberedShardSpec(0, 0)
|
||||
)
|
||||
|
@ -150,7 +150,7 @@ public class SegmentAllocateActionTest
|
|||
id2,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
partyLock.getVersion(),
|
||||
new NumberedShardSpec(1, 0)
|
||||
)
|
||||
|
@ -159,7 +159,7 @@ public class SegmentAllocateActionTest
|
|||
id3,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
partyLock.getVersion(),
|
||||
new NumberedShardSpec(2, 0)
|
||||
)
|
||||
|
@ -176,56 +176,56 @@ public class SegmentAllocateActionTest
|
|||
final SegmentIdentifier id1 = allocate(
|
||||
task,
|
||||
PARTY_TIME,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
null
|
||||
);
|
||||
final SegmentIdentifier id2 = allocate(
|
||||
task,
|
||||
THE_DISTANT_FUTURE,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
id1.getIdentifierAsString()
|
||||
);
|
||||
final SegmentIdentifier id3 = allocate(
|
||||
task,
|
||||
PARTY_TIME,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
id2.getIdentifierAsString()
|
||||
);
|
||||
final SegmentIdentifier id4 = allocate(
|
||||
task,
|
||||
PARTY_TIME,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
id1.getIdentifierAsString()
|
||||
);
|
||||
final SegmentIdentifier id5 = allocate(
|
||||
task,
|
||||
THE_DISTANT_FUTURE,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
id1.getIdentifierAsString()
|
||||
);
|
||||
final SegmentIdentifier id6 = allocate(
|
||||
task,
|
||||
THE_DISTANT_FUTURE,
|
||||
Granularity.NONE,
|
||||
Granularity.MINUTE,
|
||||
Granularities.NONE,
|
||||
Granularities.MINUTE,
|
||||
"s1",
|
||||
id1.getIdentifierAsString()
|
||||
);
|
||||
final SegmentIdentifier id7 = allocate(
|
||||
task,
|
||||
THE_DISTANT_FUTURE,
|
||||
Granularity.NONE,
|
||||
Granularity.DAY,
|
||||
Granularities.NONE,
|
||||
Granularities.DAY,
|
||||
"s1",
|
||||
id1.getIdentifierAsString()
|
||||
);
|
||||
|
@ -261,7 +261,7 @@ public class SegmentAllocateActionTest
|
|||
id1,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
partyLock.getVersion(),
|
||||
new NumberedShardSpec(0, 0)
|
||||
)
|
||||
|
@ -270,7 +270,7 @@ public class SegmentAllocateActionTest
|
|||
id2,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(THE_DISTANT_FUTURE),
|
||||
Granularities.HOUR.bucket(THE_DISTANT_FUTURE),
|
||||
futureLock.getVersion(),
|
||||
new NumberedShardSpec(0, 0)
|
||||
)
|
||||
|
@ -279,7 +279,7 @@ public class SegmentAllocateActionTest
|
|||
id3,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
partyLock.getVersion(),
|
||||
new NumberedShardSpec(1, 0)
|
||||
)
|
||||
|
@ -297,33 +297,33 @@ public class SegmentAllocateActionTest
|
|||
|
||||
taskActionTestKit.getTaskLockbox().add(task);
|
||||
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularity.NONE, Granularity.HOUR, "s1", null);
|
||||
final SegmentIdentifier id2 = allocate(task, PARTY_TIME, Granularity.NONE, Granularity.HOUR, "s2", null);
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
|
||||
final SegmentIdentifier id2 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s2", null);
|
||||
final SegmentIdentifier id3 = allocate(
|
||||
task,
|
||||
PARTY_TIME,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
id1.getIdentifierAsString()
|
||||
);
|
||||
final SegmentIdentifier id4 = allocate(
|
||||
task,
|
||||
THE_DISTANT_FUTURE,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
id3.getIdentifierAsString()
|
||||
);
|
||||
final SegmentIdentifier id5 = allocate(
|
||||
task,
|
||||
THE_DISTANT_FUTURE,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s2",
|
||||
id2.getIdentifierAsString()
|
||||
);
|
||||
final SegmentIdentifier id6 = allocate(task, PARTY_TIME, Granularity.NONE, Granularity.HOUR, "s1", null);
|
||||
final SegmentIdentifier id6 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
|
||||
|
||||
final TaskLock partyLock = Iterables.getOnlyElement(
|
||||
FluentIterable.from(taskActionTestKit.getTaskLockbox().findLocksForTask(task))
|
||||
|
@ -356,7 +356,7 @@ public class SegmentAllocateActionTest
|
|||
id1,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
partyLock.getVersion(),
|
||||
new NumberedShardSpec(0, 0)
|
||||
)
|
||||
|
@ -365,7 +365,7 @@ public class SegmentAllocateActionTest
|
|||
id2,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
partyLock.getVersion(),
|
||||
new NumberedShardSpec(1, 0)
|
||||
)
|
||||
|
@ -374,7 +374,7 @@ public class SegmentAllocateActionTest
|
|||
id3,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
partyLock.getVersion(),
|
||||
new NumberedShardSpec(2, 0)
|
||||
)
|
||||
|
@ -383,7 +383,7 @@ public class SegmentAllocateActionTest
|
|||
id4,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(THE_DISTANT_FUTURE),
|
||||
Granularities.HOUR.bucket(THE_DISTANT_FUTURE),
|
||||
futureLock.getVersion(),
|
||||
new NumberedShardSpec(0, 0)
|
||||
)
|
||||
|
@ -392,7 +392,7 @@ public class SegmentAllocateActionTest
|
|||
id5,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(THE_DISTANT_FUTURE),
|
||||
Granularities.HOUR.bucket(THE_DISTANT_FUTURE),
|
||||
futureLock.getVersion(),
|
||||
new NumberedShardSpec(1, 0)
|
||||
)
|
||||
|
@ -412,13 +412,13 @@ public class SegmentAllocateActionTest
|
|||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new LinearShardSpec(0))
|
||||
.build(),
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new LinearShardSpec(1))
|
||||
.build()
|
||||
|
@ -430,16 +430,16 @@ public class SegmentAllocateActionTest
|
|||
final SegmentIdentifier id1 = allocate(
|
||||
task,
|
||||
PARTY_TIME,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
null
|
||||
);
|
||||
final SegmentIdentifier id2 = allocate(
|
||||
task,
|
||||
PARTY_TIME,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
id1.getIdentifierAsString()
|
||||
);
|
||||
|
@ -448,7 +448,7 @@ public class SegmentAllocateActionTest
|
|||
id1,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
PARTY_TIME.toString(),
|
||||
new LinearShardSpec(2)
|
||||
)
|
||||
|
@ -457,7 +457,7 @@ public class SegmentAllocateActionTest
|
|||
id2,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
PARTY_TIME.toString(),
|
||||
new LinearShardSpec(3)
|
||||
)
|
||||
|
@ -473,13 +473,13 @@ public class SegmentAllocateActionTest
|
|||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new NumberedShardSpec(0, 2))
|
||||
.build(),
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new NumberedShardSpec(1, 2))
|
||||
.build()
|
||||
|
@ -491,16 +491,16 @@ public class SegmentAllocateActionTest
|
|||
final SegmentIdentifier id1 = allocate(
|
||||
task,
|
||||
PARTY_TIME,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
null
|
||||
);
|
||||
final SegmentIdentifier id2 = allocate(
|
||||
task,
|
||||
PARTY_TIME,
|
||||
Granularity.NONE,
|
||||
Granularity.HOUR,
|
||||
Granularities.NONE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
id1.getIdentifierAsString()
|
||||
);
|
||||
|
@ -509,7 +509,7 @@ public class SegmentAllocateActionTest
|
|||
id1,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
PARTY_TIME.toString(),
|
||||
new NumberedShardSpec(2, 2)
|
||||
)
|
||||
|
@ -518,7 +518,7 @@ public class SegmentAllocateActionTest
|
|||
id2,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
PARTY_TIME.toString(),
|
||||
new NumberedShardSpec(3, 2)
|
||||
)
|
||||
|
@ -534,13 +534,13 @@ public class SegmentAllocateActionTest
|
|||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new NumberedShardSpec(0, 2))
|
||||
.build(),
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new NumberedShardSpec(1, 2))
|
||||
.build()
|
||||
|
@ -549,13 +549,13 @@ public class SegmentAllocateActionTest
|
|||
|
||||
taskActionTestKit.getTaskLockbox().add(task);
|
||||
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularity.NONE, Granularity.DAY, "s1", null);
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.DAY, "s1", null);
|
||||
|
||||
assertSameIdentifier(
|
||||
id1,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
PARTY_TIME.toString(),
|
||||
new NumberedShardSpec(2, 2)
|
||||
)
|
||||
|
@ -571,13 +571,13 @@ public class SegmentAllocateActionTest
|
|||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new NumberedShardSpec(0, 2))
|
||||
.build(),
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new NumberedShardSpec(1, 2))
|
||||
.build()
|
||||
|
@ -586,13 +586,13 @@ public class SegmentAllocateActionTest
|
|||
|
||||
taskActionTestKit.getTaskLockbox().add(task);
|
||||
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularity.NONE, Granularity.MINUTE, "s1", null);
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.MINUTE, "s1", null);
|
||||
|
||||
assertSameIdentifier(
|
||||
id1,
|
||||
new SegmentIdentifier(
|
||||
DATA_SOURCE,
|
||||
Granularity.HOUR.bucket(PARTY_TIME),
|
||||
Granularities.HOUR.bucket(PARTY_TIME),
|
||||
PARTY_TIME.toString(),
|
||||
new NumberedShardSpec(2, 2)
|
||||
)
|
||||
|
@ -608,13 +608,13 @@ public class SegmentAllocateActionTest
|
|||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new NumberedShardSpec(0, 2))
|
||||
.build(),
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new NumberedShardSpec(1, 2))
|
||||
.build()
|
||||
|
@ -623,7 +623,7 @@ public class SegmentAllocateActionTest
|
|||
|
||||
taskActionTestKit.getTaskLockbox().add(task);
|
||||
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularity.DAY, Granularity.DAY, "s1", null);
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.DAY, Granularities.DAY, "s1", null);
|
||||
|
||||
Assert.assertNull(id1);
|
||||
}
|
||||
|
@ -634,7 +634,7 @@ public class SegmentAllocateActionTest
|
|||
final Task task = new NoopTask(null, 0, 0, null, null, null);
|
||||
taskActionTestKit.getTaskLockbox().add(task);
|
||||
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularity.DAY, Granularity.HOUR, "s1", null);
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.DAY, Granularities.HOUR, "s1", null);
|
||||
|
||||
Assert.assertNull(id1);
|
||||
}
|
||||
|
@ -648,13 +648,13 @@ public class SegmentAllocateActionTest
|
|||
ImmutableSet.of(
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new SingleDimensionShardSpec("foo", null, "bar", 0))
|
||||
.build(),
|
||||
DataSegment.builder()
|
||||
.dataSource(DATA_SOURCE)
|
||||
.interval(Granularity.HOUR.bucket(PARTY_TIME))
|
||||
.interval(Granularities.HOUR.bucket(PARTY_TIME))
|
||||
.version(PARTY_TIME.toString())
|
||||
.shardSpec(new SingleDimensionShardSpec("foo", "bar", null, 1))
|
||||
.build()
|
||||
|
@ -663,7 +663,7 @@ public class SegmentAllocateActionTest
|
|||
|
||||
taskActionTestKit.getTaskLockbox().add(task);
|
||||
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularity.NONE, Granularity.HOUR, "s1", null);
|
||||
final SegmentIdentifier id1 = allocate(task, PARTY_TIME, Granularities.NONE, Granularities.HOUR, "s1", null);
|
||||
|
||||
Assert.assertNull(id1);
|
||||
}
|
||||
|
@ -674,8 +674,8 @@ public class SegmentAllocateActionTest
|
|||
final SegmentAllocateAction action = new SegmentAllocateAction(
|
||||
DATA_SOURCE,
|
||||
PARTY_TIME,
|
||||
Granularity.MINUTE,
|
||||
Granularity.HOUR,
|
||||
Granularities.MINUTE,
|
||||
Granularities.HOUR,
|
||||
"s1",
|
||||
"prev"
|
||||
);
|
||||
|
@ -688,8 +688,8 @@ public class SegmentAllocateActionTest
|
|||
|
||||
Assert.assertEquals(DATA_SOURCE, action2.getDataSource());
|
||||
Assert.assertEquals(PARTY_TIME, action2.getTimestamp());
|
||||
Assert.assertEquals(Granularity.MINUTE, action2.getQueryGranularity());
|
||||
Assert.assertEquals(Granularity.HOUR, action2.getPreferredSegmentGranularity());
|
||||
Assert.assertEquals(Granularities.MINUTE, action2.getQueryGranularity());
|
||||
Assert.assertEquals(Granularities.HOUR, action2.getPreferredSegmentGranularity());
|
||||
Assert.assertEquals("s1", action2.getSequenceName());
|
||||
Assert.assertEquals("prev", action2.getPreviousSegmentId());
|
||||
}
|
||||
|
|
|
@ -36,7 +36,7 @@ import io.druid.indexing.common.actions.SegmentTransactionalInsertAction;
|
|||
import io.druid.indexing.common.actions.TaskAction;
|
||||
import io.druid.indexing.common.actions.TaskActionClient;
|
||||
import io.druid.indexing.overlord.SegmentPublishResult;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.segment.IndexIO;
|
||||
|
@ -186,7 +186,7 @@ public class IndexTaskTest
|
|||
createIngestionSpec(
|
||||
tmpDir,
|
||||
new ArbitraryGranularitySpec(
|
||||
Granularity.MINUTE,
|
||||
Granularities.MINUTE,
|
||||
Arrays.asList(new Interval("2014/2015"))
|
||||
),
|
||||
10,
|
||||
|
@ -221,8 +221,8 @@ public class IndexTaskTest
|
|||
createIngestionSpec(
|
||||
tmpDir,
|
||||
new UniformGranularitySpec(
|
||||
Granularity.HOUR,
|
||||
Granularity.HOUR,
|
||||
Granularities.HOUR,
|
||||
Granularities.HOUR,
|
||||
Arrays.asList(new Interval("2015-03-01T08:00:00Z/2015-03-01T09:00:00Z"))
|
||||
),
|
||||
50,
|
||||
|
@ -324,8 +324,8 @@ public class IndexTaskTest
|
|||
createIngestionSpec(
|
||||
tmpDir,
|
||||
new UniformGranularitySpec(
|
||||
Granularity.HOUR,
|
||||
Granularity.MINUTE,
|
||||
Granularities.HOUR,
|
||||
Granularities.MINUTE,
|
||||
null
|
||||
),
|
||||
2,
|
||||
|
@ -468,8 +468,8 @@ public class IndexTaskTest
|
|||
new LongSumAggregatorFactory("val", "val")
|
||||
},
|
||||
granularitySpec != null ? granularitySpec : new UniformGranularitySpec(
|
||||
Granularity.DAY,
|
||||
Granularity.MINUTE,
|
||||
Granularities.DAY,
|
||||
Granularities.MINUTE,
|
||||
Arrays.asList(new Interval("2014/2015"))
|
||||
),
|
||||
jsonMapper
|
||||
|
|
|
@ -67,7 +67,7 @@ import io.druid.indexing.test.TestIndexerMetadataStorageCoordinator;
|
|||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.parsers.ParseException;
|
||||
import io.druid.metadata.EntryExistsException;
|
||||
|
@ -805,8 +805,8 @@ public class RealtimeIndexTaskTest
|
|||
directory,
|
||||
task1.getId(),
|
||||
task1.getDataSource(),
|
||||
Granularity.DAY.bucketStart(now),
|
||||
Granularity.DAY.bucketEnd(now)
|
||||
Granularities.DAY.bucketStart(now),
|
||||
Granularities.DAY.bucketEnd(now)
|
||||
)
|
||||
);
|
||||
|
||||
|
@ -888,7 +888,7 @@ public class RealtimeIndexTaskTest
|
|||
"test_ds",
|
||||
null,
|
||||
new AggregatorFactory[]{new CountAggregatorFactory("rows"), new LongSumAggregatorFactory("met1", "met1")},
|
||||
new UniformGranularitySpec(Granularity.DAY, Granularity.NONE, null),
|
||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
||||
objectMapper
|
||||
);
|
||||
RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig(
|
||||
|
@ -1078,7 +1078,7 @@ public class RealtimeIndexTaskTest
|
|||
ImmutableList.<AggregatorFactory>of(
|
||||
new LongSumAggregatorFactory(metric, metric)
|
||||
)
|
||||
).granularity(Granularity.ALL)
|
||||
).granularity(Granularities.ALL)
|
||||
.intervals("2000/3000")
|
||||
.build();
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ import io.druid.guice.FirehoseModule;
|
|||
import io.druid.indexer.HadoopIOConfig;
|
||||
import io.druid.indexer.HadoopIngestionSpec;
|
||||
import io.druid.indexing.common.TestUtils;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
|
@ -180,7 +180,7 @@ public class TaskSerdeTest
|
|||
null,
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
null,
|
||||
ImmutableList.of(new Interval("2010-01-01/P2D"))
|
||||
),
|
||||
|
@ -245,7 +245,7 @@ public class TaskSerdeTest
|
|||
null,
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
null,
|
||||
ImmutableList.of(new Interval("2010-01-01/P2D"))
|
||||
),
|
||||
|
@ -436,7 +436,7 @@ public class TaskSerdeTest
|
|||
"foo",
|
||||
null,
|
||||
new AggregatorFactory[0],
|
||||
new UniformGranularitySpec(Granularity.HOUR, Granularity.NONE, null),
|
||||
new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null),
|
||||
jsonMapper
|
||||
),
|
||||
new RealtimeIOConfig(
|
||||
|
@ -487,7 +487,7 @@ public class TaskSerdeTest
|
|||
.getTuningConfig().getWindowPeriod()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
Granularity.HOUR,
|
||||
Granularities.HOUR,
|
||||
task.getRealtimeIngestionSchema().getDataSchema().getGranularitySpec().getSegmentGranularity()
|
||||
);
|
||||
Assert.assertTrue(task.getRealtimeIngestionSchema().getTuningConfig().isReportParseExceptions());
|
||||
|
@ -723,7 +723,7 @@ public class TaskSerdeTest
|
|||
new HadoopIngestionSpec(
|
||||
new DataSchema(
|
||||
"foo", null, new AggregatorFactory[0], new UniformGranularitySpec(
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
null,
|
||||
ImmutableList.of(new Interval("2010-01-01/P1D"))
|
||||
),
|
||||
|
|
|
@ -41,7 +41,6 @@ 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.java.util.common.granularity.Granularity;
|
||||
import io.druid.guice.GuiceAnnotationIntrospector;
|
||||
import io.druid.guice.GuiceInjectableValues;
|
||||
import io.druid.guice.GuiceInjectors;
|
||||
|
@ -55,6 +54,7 @@ import io.druid.indexing.common.config.TaskStorageConfig;
|
|||
import io.druid.indexing.overlord.HeapMemoryTaskStorage;
|
||||
import io.druid.indexing.overlord.TaskLockbox;
|
||||
import io.druid.indexing.overlord.supervisor.SupervisorManager;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
import io.druid.metadata.IndexerSQLMetadataStorageCoordinator;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
|
@ -132,7 +132,7 @@ public class IngestSegmentFirehoseFactoryTest
|
|||
}
|
||||
);
|
||||
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMinTimestamp(JodaUtils.MIN_INSTANT)
|
||||
.withDimensionsSpec(ROW_PARSER)
|
||||
.withMetrics(
|
||||
|
|
|
@ -40,7 +40,6 @@ 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.java.util.common.granularity.Granularity;
|
||||
import io.druid.indexing.common.SegmentLoaderFactory;
|
||||
import io.druid.indexing.common.TaskToolboxFactory;
|
||||
import io.druid.indexing.common.TestUtils;
|
||||
|
@ -50,6 +49,7 @@ import io.druid.indexing.common.actions.TaskActionClient;
|
|||
import io.druid.indexing.common.actions.TaskActionClientFactory;
|
||||
import io.druid.indexing.common.config.TaskConfig;
|
||||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.filter.NoopDimFilter;
|
||||
|
@ -211,7 +211,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
|
|||
{
|
||||
final File persistDir = new File(tmpDir, UUID.randomUUID().toString());
|
||||
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMinTimestamp(JodaUtils.MIN_INSTANT)
|
||||
.withDimensionsSpec(ROW_PARSER)
|
||||
.withMetrics(
|
||||
|
|
|
@ -70,7 +70,7 @@ import io.druid.indexing.test.TestIndexerMetadataStorageCoordinator;
|
|||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Comparators;
|
||||
import io.druid.metadata.SQLMetadataStorageActionHandlerFactory;
|
||||
import io.druid.metadata.TestDerbyConnector;
|
||||
|
@ -646,7 +646,7 @@ public class TaskLifecycleTest
|
|||
null,
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
null,
|
||||
ImmutableList.of(new Interval("2010-01-01/P2D"))
|
||||
),
|
||||
|
@ -704,7 +704,7 @@ public class TaskLifecycleTest
|
|||
null,
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
null,
|
||||
ImmutableList.of(new Interval("2010-01-01/P1D"))
|
||||
),
|
||||
|
@ -1063,7 +1063,7 @@ public class TaskLifecycleTest
|
|||
null,
|
||||
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
|
||||
new UniformGranularitySpec(
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
null,
|
||||
ImmutableList.of(new Interval("2010-01-01/P2D"))
|
||||
),
|
||||
|
@ -1168,7 +1168,7 @@ public class TaskLifecycleTest
|
|||
"test_ds",
|
||||
null,
|
||||
new AggregatorFactory[]{new LongSumAggregatorFactory("count", "rows")},
|
||||
new UniformGranularitySpec(Granularity.DAY, Granularity.NONE, null),
|
||||
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
|
||||
mapper
|
||||
);
|
||||
RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig(
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.joda.time.format.DateTimeFormatter;
|
|||
/**
|
||||
* AllGranularty buckets everything into a single bucket
|
||||
*/
|
||||
public final class AllGranularity extends Granularity
|
||||
public class AllGranularity extends Granularity
|
||||
{
|
||||
// These constants are from JodaUtils in druid-common.
|
||||
// Creates circular dependency.
|
||||
|
@ -38,14 +38,11 @@ public final class AllGranularity extends Granularity
|
|||
private final DateTime maxDateTime = new DateTime(MAX_INSTANT);
|
||||
private final DateTime minDateTime = new DateTime(MIN_INSTANT);
|
||||
|
||||
private static final AllGranularity INSTANCE = new AllGranularity();
|
||||
|
||||
private AllGranularity() {}
|
||||
|
||||
public static AllGranularity getInstance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
/**
|
||||
* This constructor is public b/c it is serialized and deserialized
|
||||
* based on type in GranularityModule
|
||||
*/
|
||||
public AllGranularity() {}
|
||||
|
||||
@Override
|
||||
public DateTimeFormatter getFormatter(Formatter type)
|
||||
|
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* 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.java.util.common.granularity;
|
||||
|
||||
/**
|
||||
* This class was created b/c sometimes static initializers of a class that use a subclass can deadlock.
|
||||
* See: #2979, #3979
|
||||
*/
|
||||
public class Granularities
|
||||
{
|
||||
public static final Granularity SECOND = GranularityType.SECOND.getDefaultGranularity();
|
||||
public static final Granularity MINUTE = GranularityType.MINUTE.getDefaultGranularity();
|
||||
public static final Granularity FIVE_MINUTE = GranularityType.FIVE_MINUTE.getDefaultGranularity();
|
||||
public static final Granularity TEN_MINUTE = GranularityType.TEN_MINUTE.getDefaultGranularity();
|
||||
public static final Granularity FIFTEEN_MINUTE = GranularityType.FIFTEEN_MINUTE.getDefaultGranularity();
|
||||
public static final Granularity THIRTY_MINUTE = GranularityType.THIRTY_MINUTE.getDefaultGranularity();
|
||||
public static final Granularity HOUR = GranularityType.HOUR.getDefaultGranularity();
|
||||
public static final Granularity SIX_HOUR = GranularityType.SIX_HOUR.getDefaultGranularity();
|
||||
public static final Granularity DAY = GranularityType.DAY.getDefaultGranularity();
|
||||
public static final Granularity WEEK = GranularityType.WEEK.getDefaultGranularity();
|
||||
public static final Granularity MONTH = GranularityType.MONTH.getDefaultGranularity();
|
||||
public static final Granularity QUARTER = GranularityType.QUARTER.getDefaultGranularity();
|
||||
public static final Granularity YEAR = GranularityType.YEAR.getDefaultGranularity();
|
||||
public static final Granularity ALL = GranularityType.ALL.getDefaultGranularity();
|
||||
public static final Granularity NONE = GranularityType.NONE.getDefaultGranularity();
|
||||
|
||||
}
|
|
@ -20,7 +20,6 @@
|
|||
package io.druid.java.util.common.granularity;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.primitives.Longs;
|
||||
import io.druid.java.util.common.Cacheable;
|
||||
|
@ -28,7 +27,6 @@ import io.druid.java.util.common.IAE;
|
|||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.Interval;
|
||||
import org.joda.time.Period;
|
||||
import org.joda.time.format.DateTimeFormatter;
|
||||
|
||||
import java.util.Collections;
|
||||
|
@ -42,32 +40,6 @@ import java.util.regex.Pattern;
|
|||
|
||||
public abstract class Granularity implements Cacheable
|
||||
{
|
||||
|
||||
public static final Granularity SECOND = GranularityType.SECOND.defaultGranularity;
|
||||
public static final Granularity MINUTE = GranularityType.MINUTE.defaultGranularity;
|
||||
public static final Granularity FIVE_MINUTE = GranularityType.FIVE_MINUTE.defaultGranularity;
|
||||
public static final Granularity TEN_MINUTE = GranularityType.TEN_MINUTE.defaultGranularity;
|
||||
public static final Granularity FIFTEEN_MINUTE = GranularityType.FIFTEEN_MINUTE.defaultGranularity;
|
||||
public static final Granularity THIRTY_MINUTE = GranularityType.THIRTY_MINUTE.defaultGranularity;
|
||||
public static final Granularity HOUR = GranularityType.HOUR.defaultGranularity;
|
||||
public static final Granularity SIX_HOUR = GranularityType.SIX_HOUR.defaultGranularity;
|
||||
public static final Granularity DAY = GranularityType.DAY.defaultGranularity;
|
||||
public static final Granularity WEEK = GranularityType.WEEK.defaultGranularity;
|
||||
public static final Granularity MONTH = GranularityType.MONTH.defaultGranularity;
|
||||
public static final Granularity QUARTER = GranularityType.QUARTER.defaultGranularity;
|
||||
public static final Granularity YEAR = GranularityType.YEAR.defaultGranularity;
|
||||
public static final Granularity ALL = GranularityType.ALL.defaultGranularity;
|
||||
public static final Granularity NONE = GranularityType.NONE.defaultGranularity;
|
||||
|
||||
/**
|
||||
* For a select subset of granularites, users can specify them directly as string.
|
||||
* These are "predefined granularities".
|
||||
* For all others, the users will have to use "Duration" or "Period" type granularities
|
||||
*/
|
||||
static final List<Granularity> PREDEFINED_GRANULARITIES = ImmutableList.of(
|
||||
SECOND, MINUTE, FIVE_MINUTE, TEN_MINUTE, FIFTEEN_MINUTE, THIRTY_MINUTE,
|
||||
HOUR, SIX_HOUR, DAY, WEEK, MONTH, QUARTER, YEAR);
|
||||
|
||||
/**
|
||||
* Default patterns for parsing paths.
|
||||
*/
|
||||
|
@ -81,7 +53,7 @@ public abstract class Granularity implements Cacheable
|
|||
@JsonCreator
|
||||
public static Granularity fromString(String str)
|
||||
{
|
||||
return GranularityType.valueOf(str.toUpperCase()).defaultGranularity;
|
||||
return GranularityType.valueOf(str.toUpperCase()).getDefaultGranularity();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -219,222 +191,6 @@ public abstract class Granularity implements Cacheable
|
|||
LOWER_DEFAULT
|
||||
}
|
||||
|
||||
/**
|
||||
* Only to create a mapping of the granularity and all the supported file patterns
|
||||
* namely: default, lowerDefault and hive.
|
||||
*/
|
||||
public enum GranularityType
|
||||
{
|
||||
SECOND(
|
||||
"'dt'=yyyy-MM-dd-HH-mm-ss",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'h'=HH/'m'=mm/'s'=ss",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'H'=HH/'M'=mm/'S'=ss",
|
||||
6,
|
||||
"PT1S"
|
||||
),
|
||||
MINUTE(
|
||||
"'dt'=yyyy-MM-dd-HH-mm",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'h'=HH/'m'=mm",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'H'=HH/'M'=mm",
|
||||
5,
|
||||
"PT1M"
|
||||
),
|
||||
FIVE_MINUTE(MINUTE, "PT5M"),
|
||||
TEN_MINUTE(MINUTE, "PT10M"),
|
||||
FIFTEEN_MINUTE(MINUTE, "PT15M"),
|
||||
THIRTY_MINUTE(MINUTE, "PT30M"),
|
||||
HOUR(
|
||||
"'dt'=yyyy-MM-dd-HH",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'h'=HH",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'H'=HH",
|
||||
4,
|
||||
"PT1H"
|
||||
),
|
||||
SIX_HOUR(HOUR, "PT6H"),
|
||||
DAY(
|
||||
"'dt'=yyyy-MM-dd",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd",
|
||||
3,
|
||||
"P1D"
|
||||
),
|
||||
WEEK(DAY, "P1W"),
|
||||
MONTH(
|
||||
"'dt'=yyyy-MM",
|
||||
"'y'=yyyy/'m'=MM",
|
||||
"'y'=yyyy/'m'=MM",
|
||||
2,
|
||||
"P1M"
|
||||
),
|
||||
QUARTER(MONTH, "P3M"),
|
||||
YEAR(
|
||||
"'dt'=yyyy",
|
||||
"'y'=yyyy",
|
||||
"'y'=yyyy",
|
||||
1,
|
||||
"P1Y"
|
||||
),
|
||||
ALL(AllGranularity.getInstance()),
|
||||
NONE(NoneGranularity.getInstance());
|
||||
|
||||
private final String hiveFormat;
|
||||
private final String lowerDefaultFormat;
|
||||
private final String defaultFormat;
|
||||
private final int dateValuePositions;
|
||||
private final Period period;
|
||||
private final Granularity defaultGranularity;
|
||||
|
||||
GranularityType(Granularity specialGranularity)
|
||||
{
|
||||
this.hiveFormat = null;
|
||||
this.lowerDefaultFormat = null;
|
||||
this.defaultFormat = null;
|
||||
this.dateValuePositions = 0;
|
||||
this.period = null;
|
||||
this.defaultGranularity = specialGranularity;
|
||||
}
|
||||
|
||||
GranularityType(
|
||||
final String hiveFormat,
|
||||
final String lowerDefaultFormat,
|
||||
final String defaultFormat,
|
||||
final int dateValuePositions,
|
||||
final String period
|
||||
)
|
||||
{
|
||||
this.hiveFormat = hiveFormat;
|
||||
this.lowerDefaultFormat = lowerDefaultFormat;
|
||||
this.defaultFormat = defaultFormat;
|
||||
this.dateValuePositions = dateValuePositions;
|
||||
this.period = new Period(period);
|
||||
this.defaultGranularity = new PeriodGranularity(this.period, null, null);
|
||||
}
|
||||
|
||||
GranularityType(GranularityType granularityType, String period)
|
||||
{
|
||||
this(
|
||||
granularityType.getHiveFormat(),
|
||||
granularityType.getLowerDefaultFormat(),
|
||||
granularityType.getDefaultFormat(),
|
||||
granularityType.dateValuePositions,
|
||||
period
|
||||
);
|
||||
}
|
||||
|
||||
Granularity create(DateTime origin, DateTimeZone tz)
|
||||
{
|
||||
if (period != null && (origin != null || tz != null)) {
|
||||
return new PeriodGranularity(period, origin, tz);
|
||||
} else {
|
||||
// If All or None granularity, or if origin and tz are both null, return the cached granularity
|
||||
return defaultGranularity;
|
||||
}
|
||||
}
|
||||
|
||||
public static DateTime getDateTime(GranularityType granularityType, Integer[] vals)
|
||||
{
|
||||
if (granularityType.dateValuePositions == 0) {
|
||||
// All or None granularity
|
||||
return null;
|
||||
}
|
||||
for (int i = 1; i <= granularityType.dateValuePositions; i++) {
|
||||
if (vals[i] == null) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
return new DateTime(
|
||||
vals[1],
|
||||
granularityType.dateValuePositions >= 2 ? vals[2] : 1,
|
||||
granularityType.dateValuePositions >= 3 ? vals[3] : 1,
|
||||
granularityType.dateValuePositions >= 4 ? vals[4] : 0,
|
||||
granularityType.dateValuePositions >= 5 ? vals[5] : 0,
|
||||
granularityType.dateValuePositions >= 6 ? vals[6] : 0,
|
||||
0
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* Note: This is only an estimate based on the values in period.
|
||||
* This will not work for complicated periods that represent say 1 year 1 day
|
||||
*/
|
||||
public static GranularityType fromPeriod(Period period)
|
||||
{
|
||||
int[] vals = period.getValues();
|
||||
int index = -1;
|
||||
for (int i = 0; i < vals.length; i++) {
|
||||
if (vals[i] != 0) {
|
||||
if (index < 0) {
|
||||
index = i;
|
||||
} else {
|
||||
throw new IAE("Granularity is not supported. [%s]", period);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
switch (index) {
|
||||
case 0:
|
||||
return GranularityType.YEAR;
|
||||
case 1:
|
||||
if (vals[index] == 4) {
|
||||
return GranularityType.QUARTER;
|
||||
}
|
||||
else if (vals[index] == 1) {
|
||||
return GranularityType.MONTH;
|
||||
}
|
||||
break;
|
||||
case 2:
|
||||
return GranularityType.WEEK;
|
||||
case 3:
|
||||
return GranularityType.DAY;
|
||||
case 4:
|
||||
if (vals[index] == 6) {
|
||||
return GranularityType.SIX_HOUR;
|
||||
}
|
||||
else if (vals[index] == 1) {
|
||||
return GranularityType.HOUR;
|
||||
}
|
||||
break;
|
||||
case 5:
|
||||
if (vals[index] == 30) {
|
||||
return GranularityType.THIRTY_MINUTE;
|
||||
}
|
||||
else if (vals[index] == 15) {
|
||||
return GranularityType.FIFTEEN_MINUTE;
|
||||
}
|
||||
else if (vals[index] == 10) {
|
||||
return GranularityType.TEN_MINUTE;
|
||||
}
|
||||
else if (vals[index] == 5) {
|
||||
return GranularityType.FIVE_MINUTE;
|
||||
}
|
||||
else if (vals[index] == 1) {
|
||||
return GranularityType.MINUTE;
|
||||
}
|
||||
break;
|
||||
case 6:
|
||||
return GranularityType.SECOND;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
throw new IAE("Granularity is not supported. [%s]", period);
|
||||
}
|
||||
|
||||
public String getHiveFormat()
|
||||
{
|
||||
return hiveFormat;
|
||||
}
|
||||
|
||||
public String getLowerDefaultFormat()
|
||||
{
|
||||
return lowerDefaultFormat;
|
||||
}
|
||||
|
||||
public String getDefaultFormat()
|
||||
{
|
||||
return defaultFormat;
|
||||
}
|
||||
}
|
||||
|
||||
private class IntervalIterable implements Iterable<Interval>
|
||||
{
|
||||
private final Interval inputInterval;
|
||||
|
|
|
@ -0,0 +1,262 @@
|
|||
/*
|
||||
* 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.java.util.common.granularity;
|
||||
|
||||
import io.druid.java.util.common.IAE;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.Period;
|
||||
|
||||
/**
|
||||
* Only to create a mapping of the granularity and all the supported file patterns
|
||||
* namely: default, lowerDefault and hive.
|
||||
*/
|
||||
public enum GranularityType
|
||||
{
|
||||
SECOND(
|
||||
"'dt'=yyyy-MM-dd-HH-mm-ss",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'h'=HH/'m'=mm/'s'=ss",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'H'=HH/'M'=mm/'S'=ss",
|
||||
6,
|
||||
"PT1S"
|
||||
),
|
||||
MINUTE(
|
||||
"'dt'=yyyy-MM-dd-HH-mm",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'h'=HH/'m'=mm",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'H'=HH/'M'=mm",
|
||||
5,
|
||||
"PT1M"
|
||||
),
|
||||
FIVE_MINUTE(MINUTE, "PT5M"),
|
||||
TEN_MINUTE(MINUTE, "PT10M"),
|
||||
FIFTEEN_MINUTE(MINUTE, "PT15M"),
|
||||
THIRTY_MINUTE(MINUTE, "PT30M"),
|
||||
HOUR(
|
||||
"'dt'=yyyy-MM-dd-HH",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'h'=HH",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd/'H'=HH",
|
||||
4,
|
||||
"PT1H"
|
||||
),
|
||||
SIX_HOUR(HOUR, "PT6H"),
|
||||
DAY(
|
||||
"'dt'=yyyy-MM-dd",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd",
|
||||
"'y'=yyyy/'m'=MM/'d'=dd",
|
||||
3,
|
||||
"P1D"
|
||||
),
|
||||
WEEK(DAY, "P1W"),
|
||||
MONTH(
|
||||
"'dt'=yyyy-MM",
|
||||
"'y'=yyyy/'m'=MM",
|
||||
"'y'=yyyy/'m'=MM",
|
||||
2,
|
||||
"P1M"
|
||||
),
|
||||
QUARTER(MONTH, "P3M"),
|
||||
YEAR(
|
||||
"'dt'=yyyy",
|
||||
"'y'=yyyy",
|
||||
"'y'=yyyy",
|
||||
1,
|
||||
"P1Y"
|
||||
),
|
||||
ALL(new AllGranularity()),
|
||||
NONE(new NoneGranularity());
|
||||
|
||||
private final String hiveFormat;
|
||||
private final String lowerDefaultFormat;
|
||||
private final String defaultFormat;
|
||||
private final int dateValuePositions;
|
||||
private final Period period;
|
||||
private final Granularity defaultGranularity;
|
||||
|
||||
GranularityType(Granularity specialGranularity)
|
||||
{
|
||||
this.hiveFormat = null;
|
||||
this.lowerDefaultFormat = null;
|
||||
this.defaultFormat = null;
|
||||
this.dateValuePositions = 0;
|
||||
this.period = null;
|
||||
this.defaultGranularity = specialGranularity;
|
||||
}
|
||||
|
||||
GranularityType(
|
||||
final String hiveFormat,
|
||||
final String lowerDefaultFormat,
|
||||
final String defaultFormat,
|
||||
final int dateValuePositions,
|
||||
final String period
|
||||
)
|
||||
{
|
||||
this.hiveFormat = hiveFormat;
|
||||
this.lowerDefaultFormat = lowerDefaultFormat;
|
||||
this.defaultFormat = defaultFormat;
|
||||
this.dateValuePositions = dateValuePositions;
|
||||
this.period = new Period(period);
|
||||
this.defaultGranularity = new PeriodGranularity(this.period, null, null);
|
||||
}
|
||||
|
||||
GranularityType(GranularityType granularityType, String period)
|
||||
{
|
||||
this(
|
||||
granularityType.getHiveFormat(),
|
||||
granularityType.getLowerDefaultFormat(),
|
||||
granularityType.getDefaultFormat(),
|
||||
granularityType.dateValuePositions,
|
||||
period
|
||||
);
|
||||
}
|
||||
|
||||
Granularity create(DateTime origin, DateTimeZone tz)
|
||||
{
|
||||
if (period != null && (origin != null || tz != null)) {
|
||||
return new PeriodGranularity(period, origin, tz);
|
||||
} else {
|
||||
// If All or None granularity, or if origin and tz are both null, return the cached granularity
|
||||
return defaultGranularity;
|
||||
}
|
||||
}
|
||||
|
||||
public Granularity getDefaultGranularity()
|
||||
{
|
||||
return defaultGranularity;
|
||||
}
|
||||
|
||||
public DateTime getDateTime(Integer[] vals)
|
||||
{
|
||||
if (dateValuePositions == 0) {
|
||||
// All or None granularity
|
||||
return null;
|
||||
}
|
||||
for (int i = 1; i <= dateValuePositions; i++) {
|
||||
if (vals[i] == null) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
return new DateTime(
|
||||
vals[1],
|
||||
dateValuePositions >= 2 ? vals[2] : 1,
|
||||
dateValuePositions >= 3 ? vals[3] : 1,
|
||||
dateValuePositions >= 4 ? vals[4] : 0,
|
||||
dateValuePositions >= 5 ? vals[5] : 0,
|
||||
dateValuePositions >= 6 ? vals[6] : 0,
|
||||
0
|
||||
);
|
||||
}
|
||||
|
||||
/**
|
||||
* For a select subset of granularites, users can specify them directly as string.
|
||||
* These are "predefined granularities" or "standard" granularities.
|
||||
* For all others, the users will have to use "Duration" or "Period" type granularities
|
||||
*/
|
||||
public static boolean isStandard(Granularity granularity)
|
||||
{
|
||||
final GranularityType[] values = GranularityType.values();
|
||||
for (GranularityType value : values) {
|
||||
if (value.getDefaultGranularity().equals(granularity)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Note: This is only an estimate based on the values in period.
|
||||
* This will not work for complicated periods that represent say 1 year 1 day
|
||||
*/
|
||||
public static GranularityType fromPeriod(Period period)
|
||||
{
|
||||
int[] vals = period.getValues();
|
||||
int index = -1;
|
||||
for (int i = 0; i < vals.length; i++) {
|
||||
if (vals[i] != 0) {
|
||||
if (index < 0) {
|
||||
index = i;
|
||||
} else {
|
||||
throw new IAE("Granularity is not supported. [%s]", period);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
switch (index) {
|
||||
case 0:
|
||||
return GranularityType.YEAR;
|
||||
case 1:
|
||||
if (vals[index] == 4) {
|
||||
return GranularityType.QUARTER;
|
||||
}
|
||||
else if (vals[index] == 1) {
|
||||
return GranularityType.MONTH;
|
||||
}
|
||||
break;
|
||||
case 2:
|
||||
return GranularityType.WEEK;
|
||||
case 3:
|
||||
return GranularityType.DAY;
|
||||
case 4:
|
||||
if (vals[index] == 6) {
|
||||
return GranularityType.SIX_HOUR;
|
||||
}
|
||||
else if (vals[index] == 1) {
|
||||
return GranularityType.HOUR;
|
||||
}
|
||||
break;
|
||||
case 5:
|
||||
if (vals[index] == 30) {
|
||||
return GranularityType.THIRTY_MINUTE;
|
||||
}
|
||||
else if (vals[index] == 15) {
|
||||
return GranularityType.FIFTEEN_MINUTE;
|
||||
}
|
||||
else if (vals[index] == 10) {
|
||||
return GranularityType.TEN_MINUTE;
|
||||
}
|
||||
else if (vals[index] == 5) {
|
||||
return GranularityType.FIVE_MINUTE;
|
||||
}
|
||||
else if (vals[index] == 1) {
|
||||
return GranularityType.MINUTE;
|
||||
}
|
||||
break;
|
||||
case 6:
|
||||
return GranularityType.SECOND;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
throw new IAE("Granularity is not supported. [%s]", period);
|
||||
}
|
||||
|
||||
public String getHiveFormat()
|
||||
{
|
||||
return hiveFormat;
|
||||
}
|
||||
|
||||
public String getLowerDefaultFormat()
|
||||
{
|
||||
return lowerDefaultFormat;
|
||||
}
|
||||
|
||||
public String getDefaultFormat()
|
||||
{
|
||||
return defaultFormat;
|
||||
}
|
||||
}
|
|
@ -25,16 +25,13 @@ import org.joda.time.format.DateTimeFormatter;
|
|||
/**
|
||||
* NoneGranularity does not bucket data
|
||||
*/
|
||||
public final class NoneGranularity extends Granularity
|
||||
public class NoneGranularity extends Granularity
|
||||
{
|
||||
private static final NoneGranularity INSTANCE = new NoneGranularity();
|
||||
|
||||
private NoneGranularity() {}
|
||||
|
||||
public static NoneGranularity getInstance()
|
||||
{
|
||||
return INSTANCE;
|
||||
}
|
||||
/**
|
||||
* This constructor is public b/c it is serialized and deserialized
|
||||
* based on type in GranularityModule
|
||||
*/
|
||||
public NoneGranularity() {}
|
||||
|
||||
@Override
|
||||
public DateTimeFormatter getFormatter(Formatter type)
|
||||
|
|
|
@ -131,7 +131,7 @@ public class PeriodGranularity extends Granularity implements JsonSerializable
|
|||
Integer[] vals = getDateValues(filePath, formatter);
|
||||
GranularityType granularityType = GranularityType.fromPeriod(period);
|
||||
|
||||
DateTime date = GranularityType.getDateTime(granularityType, vals);
|
||||
DateTime date = granularityType.getDateTime(vals);
|
||||
|
||||
if (date != null) {
|
||||
return bucketStart(date);
|
||||
|
@ -437,9 +437,9 @@ public class PeriodGranularity extends Granularity implements JsonSerializable
|
|||
JsonGenerator jsonGenerator, SerializerProvider serializerProvider
|
||||
) throws IOException, JsonProcessingException
|
||||
{
|
||||
// Retain the same behavior as pre-refactor granularity code.
|
||||
// Retain the same behavior as before #3850.
|
||||
// i.e. when Granularity class was an enum.
|
||||
if (PREDEFINED_GRANULARITIES.contains(this)) {
|
||||
if (GranularityType.isStandard(this)) {
|
||||
jsonGenerator.writeString(GranularityType.fromPeriod(getPeriod()).toString());
|
||||
} else {
|
||||
jsonGenerator.writeStartObject();
|
||||
|
|
|
@ -19,7 +19,9 @@
|
|||
|
||||
package io.druid.java.util.common;
|
||||
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.GranularityType;
|
||||
import io.druid.java.util.common.granularity.PeriodGranularity;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
@ -34,15 +36,15 @@ import java.util.NoSuchElementException;
|
|||
|
||||
public class GranularityTest {
|
||||
|
||||
final Granularity SECOND = Granularity.SECOND;
|
||||
final Granularity MINUTE = Granularity.MINUTE;
|
||||
final Granularity HOUR = Granularity.HOUR;
|
||||
final Granularity SIX_HOUR = Granularity.SIX_HOUR;
|
||||
final Granularity FIFTEEN_MINUTE = Granularity.FIFTEEN_MINUTE;
|
||||
final Granularity DAY = Granularity.DAY;
|
||||
final Granularity WEEK = Granularity.WEEK;
|
||||
final Granularity MONTH = Granularity.MONTH;
|
||||
final Granularity YEAR = Granularity.YEAR;
|
||||
final Granularity SECOND = Granularities.SECOND;
|
||||
final Granularity MINUTE = Granularities.MINUTE;
|
||||
final Granularity HOUR = Granularities.HOUR;
|
||||
final Granularity SIX_HOUR = Granularities.SIX_HOUR;
|
||||
final Granularity FIFTEEN_MINUTE = Granularities.FIFTEEN_MINUTE;
|
||||
final Granularity DAY = Granularities.DAY;
|
||||
final Granularity WEEK = Granularities.WEEK;
|
||||
final Granularity MONTH = Granularities.MONTH;
|
||||
final Granularity YEAR = Granularities.YEAR;
|
||||
|
||||
@Test
|
||||
public void testHiveFormat() {
|
||||
|
@ -412,7 +414,7 @@ public class GranularityTest {
|
|||
{
|
||||
try {
|
||||
Period p = Period.years(6).withMonths(3).withSeconds(23);
|
||||
Granularity.GranularityType.fromPeriod(p);
|
||||
GranularityType.fromPeriod(p);
|
||||
Assert.fail("Complicated period creation should fail b/c of unsupported granularity type.");
|
||||
}
|
||||
catch (IAE e) {
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.PostAggregator;
|
||||
|
@ -345,7 +346,7 @@ public class Druids
|
|||
descending = false;
|
||||
virtualColumns = null;
|
||||
dimFilter = null;
|
||||
granularity = Granularity.ALL;
|
||||
granularity = Granularities.ALL;
|
||||
aggregatorSpecs = Lists.newArrayList();
|
||||
postAggregatorSpecs = Lists.newArrayList();
|
||||
context = null;
|
||||
|
@ -573,7 +574,7 @@ public class Druids
|
|||
{
|
||||
dataSource = null;
|
||||
dimFilter = null;
|
||||
granularity = Granularity.ALL;
|
||||
granularity = Granularities.ALL;
|
||||
limit = 0;
|
||||
querySegmentSpec = null;
|
||||
dimensions = null;
|
||||
|
@ -1132,7 +1133,7 @@ public class Druids
|
|||
querySegmentSpec = null;
|
||||
context = null;
|
||||
dimFilter = null;
|
||||
granularity = Granularity.ALL;
|
||||
granularity = Granularities.ALL;
|
||||
dimensions = Lists.newArrayList();
|
||||
metrics = Lists.newArrayList();
|
||||
pagingSpec = null;
|
||||
|
|
|
@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import com.google.common.base.Preconditions;
|
||||
import io.druid.common.guava.GuavaUtils;
|
||||
import io.druid.java.util.common.StringUtils;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
@ -53,7 +54,7 @@ public class TimeFormatExtractionFn implements ExtractionFn
|
|||
this.format = format;
|
||||
this.tz = tz;
|
||||
this.locale = localeString == null ? null : Locale.forLanguageTag(localeString);
|
||||
this.granularity = granularity == null ? Granularity.NONE : granularity;
|
||||
this.granularity = granularity == null ? Granularities.NONE : granularity;
|
||||
|
||||
if (asMillis && format == null) {
|
||||
Preconditions.checkArgument(tz == null, "timeZone requires a format");
|
||||
|
|
|
@ -34,6 +34,7 @@ import com.google.common.primitives.Longs;
|
|||
import io.druid.data.input.Row;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
|
@ -354,7 +355,7 @@ public class GroupByQuery extends BaseQuery<Row>
|
|||
|
||||
private Comparator<Row> getTimeComparator(boolean granular)
|
||||
{
|
||||
if (Granularity.ALL.equals(granularity)) {
|
||||
if (Granularities.ALL.equals(granularity)) {
|
||||
return null;
|
||||
} else if (granular) {
|
||||
return new Comparator<Row>()
|
||||
|
|
|
@ -32,6 +32,7 @@ import io.druid.data.input.impl.DimensionsSpec;
|
|||
import io.druid.data.input.impl.StringDimensionSchema;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.guava.Accumulator;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
|
@ -69,7 +70,7 @@ public class GroupByQueryHelper
|
|||
final long timeStart = query.getIntervals().get(0).getStartMillis();
|
||||
|
||||
long granTimeStart = timeStart;
|
||||
if (!(Granularity.ALL.equals(gran))) {
|
||||
if (!(Granularities.ALL.equals(gran))) {
|
||||
granTimeStart = gran.bucketStart(new DateTime(timeStart)).getMillis();
|
||||
}
|
||||
|
||||
|
|
|
@ -37,6 +37,7 @@ import io.druid.data.input.Row;
|
|||
import io.druid.guice.annotations.Global;
|
||||
import io.druid.guice.annotations.Merging;
|
||||
import io.druid.guice.annotations.Smile;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
|
@ -116,7 +117,7 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
|||
|
||||
if (!timestampStringFromContext.isEmpty()) {
|
||||
return new DateTime(Long.parseLong(timestampStringFromContext));
|
||||
} else if (Granularity.ALL.equals(gran)) {
|
||||
} else if (Granularities.ALL.equals(gran)) {
|
||||
final long timeStart = query.getIntervals().get(0).getStartMillis();
|
||||
return gran.getIterable(new Interval(timeStart, timeStart + 1)).iterator().next().getStart();
|
||||
} else {
|
||||
|
|
|
@ -27,7 +27,7 @@ import com.google.common.collect.Maps;
|
|||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Longs;
|
||||
import io.druid.common.utils.StringUtils;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Accumulator;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -253,7 +253,7 @@ public class SegmentAnalyzer
|
|||
null,
|
||||
new Interval(start, end),
|
||||
VirtualColumns.EMPTY,
|
||||
Granularity.ALL,
|
||||
Granularities.ALL,
|
||||
false
|
||||
);
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.query.search.search;
|
|||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.query.BaseQuery;
|
||||
import io.druid.query.DataSource;
|
||||
|
@ -67,7 +68,7 @@ public class SearchQuery extends BaseQuery<Result<SearchResultValue>>
|
|||
|
||||
this.dimFilter = dimFilter;
|
||||
this.sortSpec = sortSpec == null ? DEFAULT_SORT_SPEC : sortSpec;
|
||||
this.granularity = granularity == null ? Granularity.ALL : granularity;
|
||||
this.granularity = granularity == null ? Granularities.ALL : granularity;
|
||||
this.limit = (limit == 0) ? 1000 : limit;
|
||||
this.dimensions = dimensions;
|
||||
this.querySpec = querySpec == null ? new AllSearchQuerySpec() : querySpec;
|
||||
|
|
|
@ -22,8 +22,8 @@ package io.druid.query.timeboundary;
|
|||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.inject.Inject;
|
||||
import io.druid.java.util.common.granularity.AllGranularity;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.BaseSequence;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
|
@ -114,7 +114,7 @@ public class TimeBoundaryQueryRunnerFactory
|
|||
legacyQuery.getQuerySegmentSpec().getIntervals(),
|
||||
Filters.toFilter(legacyQuery.getDimensionsFilter()), VirtualColumns.EMPTY,
|
||||
descending,
|
||||
AllGranularity.getInstance(),
|
||||
Granularities.ALL,
|
||||
this.skipToFirstMatching
|
||||
);
|
||||
final List<Result<DateTime>> resultList = Sequences.toList(
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package io.druid.query.topn;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.query.DataSource;
|
||||
import io.druid.query.TableDataSource;
|
||||
|
@ -84,7 +85,7 @@ public class TopNQueryBuilder
|
|||
threshold = 0;
|
||||
querySegmentSpec = null;
|
||||
dimFilter = null;
|
||||
granularity = Granularity.ALL;
|
||||
granularity = Granularities.ALL;
|
||||
aggregatorSpecs = Lists.newArrayList();
|
||||
postAggregatorSpecs = Lists.newArrayList();
|
||||
context = null;
|
||||
|
|
|
@ -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.data.input.impl.TimestampSpec;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.segment.VirtualColumns;
|
||||
|
@ -106,7 +107,7 @@ public class IncrementalIndexSchema
|
|||
public Builder()
|
||||
{
|
||||
this.minTimestamp = 0L;
|
||||
this.gran = Granularity.NONE;
|
||||
this.gran = Granularities.NONE;
|
||||
this.virtualColumns = VirtualColumns.EMPTY;
|
||||
this.dimensionsSpec = new DimensionsSpec(null, null, null);
|
||||
this.metrics = new AggregatorFactory[]{};
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.collect.Iterables;
|
|||
import com.google.common.collect.Lists;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.DurationGranularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.PeriodGranularity;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -53,7 +54,7 @@ public class QueryGranularityTest
|
|||
@Test
|
||||
public void testIterableNone() throws Exception
|
||||
{
|
||||
final Iterator<Interval> iterator = Granularity.NONE.getIterable(new Interval(0, 1000)).iterator();
|
||||
final Iterator<Interval> iterator = Granularities.NONE.getIterable(new Interval(0, 1000)).iterator();
|
||||
int count = 0;
|
||||
while (iterator.hasNext()) {
|
||||
Assert.assertEquals(count, iterator.next().getStartMillis());
|
||||
|
@ -72,7 +73,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-01-01T09:39:00.000Z"),
|
||||
new DateTime("2011-01-01T09:40:00.000Z")
|
||||
),
|
||||
Granularity.MINUTE.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Minutes.THREE).getMillis()))
|
||||
Granularities.MINUTE.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Minutes.THREE).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -88,7 +89,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-01-01T09:40:00.000Z"),
|
||||
new DateTime("2011-01-01T09:41:00.000Z")
|
||||
),
|
||||
Granularity.MINUTE.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Minutes.THREE).getMillis()))
|
||||
Granularities.MINUTE.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Minutes.THREE).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -103,7 +104,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-01-01T09:45:00.000Z"),
|
||||
new DateTime("2011-01-01T10:00:00.000Z")
|
||||
),
|
||||
Granularity.FIFTEEN_MINUTE.getIterable(
|
||||
Granularities.FIFTEEN_MINUTE.getIterable(
|
||||
new Interval(
|
||||
baseTime.getMillis(), baseTime.plus(Minutes.minutes(45)).getMillis()
|
||||
))
|
||||
|
@ -122,7 +123,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-01-01T10:00:00.000Z"),
|
||||
new DateTime("2011-01-01T10:15:00.000Z")
|
||||
),
|
||||
Granularity.FIFTEEN_MINUTE.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Minutes.minutes(45)).getMillis()))
|
||||
Granularities.FIFTEEN_MINUTE.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Minutes.minutes(45)).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -136,7 +137,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-01-01T09:00:00.000Z"),
|
||||
new DateTime("2011-01-01T10:00:00.000Z"),
|
||||
new DateTime("2011-01-01T11:00:00.000Z")
|
||||
), Granularity.HOUR.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Hours.hours(3)).getMillis()))
|
||||
), Granularities.HOUR.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Hours.hours(3)).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -151,7 +152,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-01-01T10:00:00.000Z"),
|
||||
new DateTime("2011-01-01T11:00:00.000Z"),
|
||||
new DateTime("2011-01-01T12:00:00.000Z")
|
||||
), Granularity.HOUR.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Hours.hours(3)).getMillis()))
|
||||
), Granularities.HOUR.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Hours.hours(3)).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -166,7 +167,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-01-02T00:00:00.000Z"),
|
||||
new DateTime("2011-01-03T00:00:00.000Z")
|
||||
),
|
||||
Granularity.DAY.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()))
|
||||
Granularities.DAY.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -182,7 +183,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-01-03T00:00:00.000Z"),
|
||||
new DateTime("2011-01-04T00:00:00.000Z")
|
||||
),
|
||||
Granularity.DAY.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()))
|
||||
Granularities.DAY.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -197,7 +198,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-01-10T00:00:00.000Z"),
|
||||
new DateTime("2011-01-17T00:00:00.000Z")
|
||||
),
|
||||
Granularity.WEEK.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Weeks.THREE).getMillis()))
|
||||
Granularities.WEEK.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Weeks.THREE).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -213,7 +214,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-01-10T00:00:00.000Z"),
|
||||
new DateTime("2011-01-17T00:00:00.000Z")
|
||||
),
|
||||
Granularity.WEEK.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Weeks.THREE).getMillis()))
|
||||
Granularities.WEEK.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Weeks.THREE).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -228,7 +229,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-02-01T00:00:00.000Z"),
|
||||
new DateTime("2011-03-01T00:00:00.000Z")
|
||||
),
|
||||
Granularity.MONTH.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Months.THREE).getMillis()))
|
||||
Granularities.MONTH.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Months.THREE).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -244,7 +245,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-03-01T00:00:00.000Z"),
|
||||
new DateTime("2011-04-01T00:00:00.000Z")
|
||||
),
|
||||
Granularity.MONTH.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Months.THREE).getMillis()))
|
||||
Granularities.MONTH.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Months.THREE).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -259,7 +260,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-04-01T00:00:00.000Z"),
|
||||
new DateTime("2011-07-01T00:00:00.000Z")
|
||||
),
|
||||
Granularity.QUARTER.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Months.NINE).getMillis()))
|
||||
Granularities.QUARTER.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Months.NINE).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -275,7 +276,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2011-07-01T00:00:00.000Z"),
|
||||
new DateTime("2011-10-01T00:00:00.000Z")
|
||||
),
|
||||
Granularity.QUARTER.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Months.NINE).getMillis()))
|
||||
Granularities.QUARTER.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Months.NINE).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -290,7 +291,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2012-01-01T00:00:00.000Z"),
|
||||
new DateTime("2013-01-01T00:00:00.000Z")
|
||||
),
|
||||
Granularity.YEAR.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Years.THREE).getMillis()))
|
||||
Granularities.YEAR.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Years.THREE).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -306,7 +307,7 @@ public class QueryGranularityTest
|
|||
new DateTime("2013-01-01T00:00:00.000Z"),
|
||||
new DateTime("2014-01-01T00:00:00.000Z")
|
||||
),
|
||||
Granularity.YEAR.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Years.THREE).getMillis()))
|
||||
Granularities.YEAR.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Years.THREE).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -606,7 +607,7 @@ public class QueryGranularityTest
|
|||
|
||||
assertSameInterval(
|
||||
Lists.newArrayList(baseTime),
|
||||
Granularity.ALL.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()))
|
||||
Granularities.ALL.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -617,7 +618,7 @@ public class QueryGranularityTest
|
|||
|
||||
assertSameInterval(
|
||||
Lists.newArrayList(baseTime),
|
||||
Granularity.ALL.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()))
|
||||
Granularities.ALL.getIterable(new Interval(baseTime.getMillis(), baseTime.plus(Days.days(3)).getMillis()))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -684,17 +685,17 @@ public class QueryGranularityTest
|
|||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
|
||||
Assert.assertEquals(
|
||||
Granularity.ALL,
|
||||
Granularities.ALL,
|
||||
mapper.readValue(
|
||||
mapper.writeValueAsString(Granularity.ALL),
|
||||
mapper.writeValueAsString(Granularities.ALL),
|
||||
Granularity.class
|
||||
)
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
mapper.readValue(
|
||||
mapper.writeValueAsString(Granularity.NONE),
|
||||
mapper.writeValueAsString(Granularities.NONE),
|
||||
Granularity.class
|
||||
)
|
||||
);
|
||||
|
@ -705,20 +706,20 @@ public class QueryGranularityTest
|
|||
{
|
||||
ObjectMapper mapper = new DefaultObjectMapper();
|
||||
|
||||
Assert.assertEquals(Granularity.ALL, mapper.readValue("\"all\"", Granularity.class));
|
||||
Assert.assertEquals(Granularity.ALL, mapper.readValue("\"ALL\"", Granularity.class));
|
||||
Assert.assertEquals(Granularity.NONE, mapper.readValue("\"none\"", Granularity.class));
|
||||
Assert.assertEquals(Granularity.NONE, mapper.readValue("\"NONE\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.ALL, mapper.readValue("\"all\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.ALL, mapper.readValue("\"ALL\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.NONE, mapper.readValue("\"none\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.NONE, mapper.readValue("\"NONE\"", Granularity.class));
|
||||
|
||||
Assert.assertEquals(Granularity.DAY, mapper.readValue("\"day\"", Granularity.class));
|
||||
Assert.assertEquals(Granularity.HOUR, mapper.readValue("\"hour\"", Granularity.class));
|
||||
Assert.assertEquals(Granularity.MINUTE, mapper.readValue("\"minute\"", Granularity.class));
|
||||
Assert.assertEquals(Granularity.FIFTEEN_MINUTE, mapper.readValue("\"fifteen_minute\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.DAY, mapper.readValue("\"day\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.HOUR, mapper.readValue("\"hour\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.MINUTE, mapper.readValue("\"minute\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.FIFTEEN_MINUTE, mapper.readValue("\"fifteen_minute\"", Granularity.class));
|
||||
|
||||
Assert.assertEquals(Granularity.WEEK, mapper.readValue("\"week\"", Granularity.class));
|
||||
Assert.assertEquals(Granularity.QUARTER, mapper.readValue("\"quarter\"", Granularity.class));
|
||||
Assert.assertEquals(Granularity.MONTH, mapper.readValue("\"month\"", Granularity.class));
|
||||
Assert.assertEquals(Granularity.YEAR, mapper.readValue("\"year\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.WEEK, mapper.readValue("\"week\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.QUARTER, mapper.readValue("\"quarter\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.MONTH, mapper.readValue("\"month\"", Granularity.class));
|
||||
Assert.assertEquals(Granularities.YEAR, mapper.readValue("\"year\"", Granularity.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -726,24 +727,24 @@ public class QueryGranularityTest
|
|||
{
|
||||
Assert.assertNull(Granularity.mergeGranularities(null));
|
||||
Assert.assertNull(Granularity.mergeGranularities(ImmutableList.<Granularity>of()));
|
||||
Assert.assertNull(Granularity.mergeGranularities(Lists.newArrayList(null, Granularity.DAY)));
|
||||
Assert.assertNull(Granularity.mergeGranularities(Lists.newArrayList(Granularity.DAY, null)));
|
||||
Assert.assertNull(Granularity.mergeGranularities(Lists.newArrayList(null, Granularities.DAY)));
|
||||
Assert.assertNull(Granularity.mergeGranularities(Lists.newArrayList(Granularities.DAY, null)));
|
||||
Assert.assertNull(
|
||||
Granularity.mergeGranularities(
|
||||
Lists.newArrayList(
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
null,
|
||||
Granularity.DAY
|
||||
Granularities.DAY
|
||||
)
|
||||
)
|
||||
);
|
||||
Assert.assertNull(
|
||||
Granularity.mergeGranularities(ImmutableList.of(Granularity.ALL, Granularity.DAY))
|
||||
Granularity.mergeGranularities(ImmutableList.of(Granularities.ALL, Granularities.DAY))
|
||||
);
|
||||
|
||||
Assert.assertEquals(
|
||||
Granularity.ALL,
|
||||
Granularity.mergeGranularities(ImmutableList.of(Granularity.ALL, Granularity.ALL))
|
||||
Granularities.ALL,
|
||||
Granularity.mergeGranularities(ImmutableList.of(Granularities.ALL, Granularities.ALL))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -773,14 +774,14 @@ public class QueryGranularityTest
|
|||
Assert.assertFalse("expectedIter not exhausted!?", expectedIter.hasNext());
|
||||
}
|
||||
|
||||
@Test(timeout = 60_000L)
|
||||
@Test(timeout = 10_000L)
|
||||
public void testDeadLock() throws Exception
|
||||
{
|
||||
final URL[] urls = ((URLClassLoader)Granularity.class.getClassLoader()).getURLs();
|
||||
final String className = Granularity.class.getCanonicalName();
|
||||
for(int i = 0; i < 1000; ++i) {
|
||||
final ClassLoader loader = new URLClassLoader(urls, null);
|
||||
Assert.assertNotNull(Class.forName(className, true, loader));
|
||||
Assert.assertNotNull(String.valueOf(i), Class.forName(className, true, loader));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,7 +31,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.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.aggregation.AggregationTestHelper;
|
||||
|
@ -115,7 +115,7 @@ public class MultiValuedDimensionTest
|
|||
{
|
||||
incrementalIndex = new OnheapIncrementalIndex(
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
new AggregatorFactory[]{
|
||||
new CountAggregatorFactory("count")
|
||||
},
|
||||
|
@ -160,7 +160,7 @@ public class MultiValuedDimensionTest
|
|||
.builder()
|
||||
.setDataSource("xx")
|
||||
.setQuerySegmentSpec(new LegacySegmentSpec("1970/3000"))
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("tags", "tags")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.asList(
|
||||
|
@ -201,7 +201,7 @@ public class MultiValuedDimensionTest
|
|||
.builder()
|
||||
.setDataSource("xx")
|
||||
.setQuerySegmentSpec(new LegacySegmentSpec("1970/3000"))
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("tags", "tags")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.asList(
|
||||
|
@ -242,7 +242,7 @@ public class MultiValuedDimensionTest
|
|||
.builder()
|
||||
.setDataSource("xx")
|
||||
.setQuerySegmentSpec(new LegacySegmentSpec("1970/3000"))
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(
|
||||
Lists.<DimensionSpec>newArrayList(
|
||||
new RegexFilteredDimensionSpec(
|
||||
|
@ -284,7 +284,7 @@ public class MultiValuedDimensionTest
|
|||
{
|
||||
TopNQuery query = new TopNQueryBuilder()
|
||||
.dataSource("xx")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.dimension(new ListFilteredDimensionSpec(
|
||||
new DefaultDimensionSpec("tags", "tags"),
|
||||
ImmutableSet.of("t3"),
|
||||
|
|
|
@ -26,6 +26,7 @@ import com.google.common.collect.Iterables;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import io.druid.java.util.common.UOE;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.guava.MergeSequence;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
|
@ -104,9 +105,9 @@ public class QueryRunnerTestHelper
|
|||
|
||||
public static final DateTime minTime = new DateTime("2011-01-12T00:00:00.000Z");
|
||||
|
||||
public static final Granularity dayGran = Granularity.DAY;
|
||||
public static final Granularity allGran = Granularity.ALL;
|
||||
public static final Granularity monthGran = Granularity.MONTH;
|
||||
public static final Granularity dayGran = Granularities.DAY;
|
||||
public static final Granularity allGran = Granularities.ALL;
|
||||
public static final Granularity monthGran = Granularities.MONTH;
|
||||
public static final String timeDimension = "__time";
|
||||
public static final String marketDimension = "market";
|
||||
public static final String qualityDimension = "quality";
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package io.druid.query;
|
||||
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.Assert;
|
||||
|
@ -55,7 +56,7 @@ public class ResultGranularTimestampComparatorTest
|
|||
Result<Object> r1 = new Result<Object>(time, null);
|
||||
Result<Object> r2 = new Result<Object>(time.plusYears(5), null);
|
||||
|
||||
Assert.assertEquals(ResultGranularTimestampComparator.create(Granularity.ALL, descending).compare(r1, r2), 0);
|
||||
Assert.assertEquals(ResultGranularTimestampComparator.create(Granularities.ALL, descending).compare(r1, r2), 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -66,7 +67,7 @@ public class ResultGranularTimestampComparatorTest
|
|||
Result<Object> greater = new Result<Object>(time.plusHours(25), null);
|
||||
Result<Object> less = new Result<Object>(time.minusHours(1), null);
|
||||
|
||||
Granularity day = Granularity.DAY;
|
||||
Granularity day = Granularities.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<Object> greater = new Result<Object>(time.plusHours(1), null);
|
||||
Result<Object> less = new Result<Object>(time.minusHours(1), null);
|
||||
|
||||
Granularity hour = Granularity.HOUR;
|
||||
Granularity hour = Granularities.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);
|
||||
|
|
|
@ -21,8 +21,8 @@ package io.druid.query.aggregation.hyperloglog;
|
|||
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.jackson.AggregatorsModule;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.aggregation.AggregationTestHelper;
|
||||
|
@ -114,7 +114,7 @@ public class HyperUniquesAggregationTest
|
|||
parseSpec,
|
||||
metricSpec,
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
50000,
|
||||
query
|
||||
);
|
||||
|
@ -176,7 +176,7 @@ public class HyperUniquesAggregationTest
|
|||
parseSpec,
|
||||
metricSpec,
|
||||
0,
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
50000,
|
||||
query
|
||||
);
|
||||
|
|
|
@ -25,7 +25,7 @@ import com.google.common.collect.Maps;
|
|||
import com.google.common.collect.Ordering;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.jackson.AggregatorsModule;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.aggregation.AggregationTestHelper;
|
||||
|
@ -229,7 +229,7 @@ public class FinalizingFieldAccessPostAggregatorTest
|
|||
parseSpec,
|
||||
metricSpec,
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
50000,
|
||||
query
|
||||
);
|
||||
|
|
|
@ -26,8 +26,8 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
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, Granularity.NONE, new AggregatorFactory[]{new CountAggregatorFactory("count")}, 1000
|
||||
0L, Granularities.NONE, new AggregatorFactory[]{new CountAggregatorFactory("count")}, 1000
|
||||
);
|
||||
;
|
||||
final QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner(
|
||||
|
|
|
@ -21,6 +21,7 @@ package io.druid.query.extraction;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
@ -52,7 +53,7 @@ public class TimeFormatExtractionFnTest
|
|||
Assert.assertEquals("Saturday", fn.apply(timestamps[4]));
|
||||
Assert.assertEquals("Monday", fn.apply(timestamps[5]));
|
||||
|
||||
testSerde(fn, "EEEE", null, null, Granularity.NONE);
|
||||
testSerde(fn, "EEEE", null, null, Granularities.NONE);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -66,13 +67,13 @@ public class TimeFormatExtractionFnTest
|
|||
Assert.assertEquals("laugardagur", fn.apply(timestamps[4]));
|
||||
Assert.assertEquals("mánudagur", fn.apply(timestamps[5]));
|
||||
|
||||
testSerde(fn, "EEEE", null, "is", Granularity.NONE);
|
||||
testSerde(fn, "EEEE", null, "is", Granularities.NONE);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGranularExtractionWithNullPattern() throws Exception
|
||||
{
|
||||
TimeFormatExtractionFn fn = new TimeFormatExtractionFn(null, null, null, Granularity.DAY, false);
|
||||
TimeFormatExtractionFn fn = new TimeFormatExtractionFn(null, null, null, Granularities.DAY, false);
|
||||
Assert.assertEquals("2015-01-01T00:00:00.000Z", fn.apply(timestamps[0]));
|
||||
Assert.assertEquals("2015-01-02T00:00:00.000Z", fn.apply(timestamps[1]));
|
||||
Assert.assertEquals("2015-03-03T00:00:00.000Z", fn.apply(timestamps[2]));
|
||||
|
@ -80,7 +81,7 @@ public class TimeFormatExtractionFnTest
|
|||
Assert.assertEquals("2015-05-02T00:00:00.000Z", fn.apply(timestamps[4]));
|
||||
Assert.assertEquals("2015-12-21T00:00:00.000Z", fn.apply(timestamps[5]));
|
||||
|
||||
testSerde(fn, null, null, null, Granularity.DAY);
|
||||
testSerde(fn, null, null, null, Granularities.DAY);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -100,7 +101,7 @@ public class TimeFormatExtractionFnTest
|
|||
Assert.assertEquals("In Berlin ist es schon Sonntag", fn.apply(timestamps[4]));
|
||||
Assert.assertEquals("In Berlin ist es schon Dienstag", fn.apply(timestamps[5]));
|
||||
|
||||
testSerde(fn, "'In Berlin ist es schon 'EEEE", DateTimeZone.forID("Europe/Berlin"), "de", Granularity.NONE);
|
||||
testSerde(fn, "'In Berlin ist es schon 'EEEE", DateTimeZone.forID("Europe/Berlin"), "de", Granularities.NONE);
|
||||
}
|
||||
|
||||
public void testSerde(
|
||||
|
|
|
@ -21,8 +21,8 @@ package io.druid.query.groupby;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -66,7 +66,7 @@ public class GroupByQueryConfigTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource("test")
|
||||
.setInterval(new Interval("2000/P1D"))
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.build()
|
||||
);
|
||||
|
||||
|
@ -88,7 +88,7 @@ public class GroupByQueryConfigTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource("test")
|
||||
.setInterval(new Interval("2000/P1D"))
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setContext(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"groupByStrategy", "v1",
|
||||
|
|
|
@ -30,7 +30,7 @@ import io.druid.collections.BlockingPool;
|
|||
import io.druid.collections.ReferenceCountingResourceHolder;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.DruidProcessingConfig;
|
||||
import io.druid.query.QueryContextKeys;
|
||||
import io.druid.query.QueryDataSource;
|
||||
|
@ -227,7 +227,7 @@ public class GroupByQueryMergeBufferTest
|
|||
final GroupByQuery query = GroupByQuery
|
||||
.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(new LongSumAggregatorFactory("rows", "rows")))
|
||||
.setContext(ImmutableMap.<String, Object>of(QueryContextKeys.TIMEOUT, Integers.valueOf(500)))
|
||||
|
@ -249,13 +249,13 @@ public class GroupByQueryMergeBufferTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(new LongSumAggregatorFactory("rows", "rows")))
|
||||
.setContext(ImmutableMap.<String, Object>of(QueryContextKeys.TIMEOUT, Integers.valueOf(500)))
|
||||
|
@ -279,7 +279,7 @@ public class GroupByQueryMergeBufferTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(
|
||||
new DefaultDimensionSpec("quality", "alias"),
|
||||
new DefaultDimensionSpec("market", null)
|
||||
|
@ -288,13 +288,13 @@ public class GroupByQueryMergeBufferTest
|
|||
.build()
|
||||
)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(new LongSumAggregatorFactory("rows", "rows")))
|
||||
.setContext(ImmutableMap.<String, Object>of(QueryContextKeys.TIMEOUT, Integers.valueOf(500)))
|
||||
|
@ -321,7 +321,7 @@ public class GroupByQueryMergeBufferTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(
|
||||
new DefaultDimensionSpec("quality", "alias"),
|
||||
new DefaultDimensionSpec("market", null),
|
||||
|
@ -331,7 +331,7 @@ public class GroupByQueryMergeBufferTest
|
|||
.build()
|
||||
)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(
|
||||
new DefaultDimensionSpec("quality", "alias"),
|
||||
new DefaultDimensionSpec("market", null)
|
||||
|
@ -340,13 +340,13 @@ public class GroupByQueryMergeBufferTest
|
|||
.build()
|
||||
)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(new LongSumAggregatorFactory("rows", "rows")))
|
||||
.setContext(ImmutableMap.<String, Object>of(QueryContextKeys.TIMEOUT, Integers.valueOf(500)))
|
||||
|
|
|
@ -28,7 +28,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.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.MergeSequence;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
|
@ -68,7 +68,7 @@ public class GroupByQueryRunnerFactoryTest
|
|||
.builder()
|
||||
.setDataSource("xx")
|
||||
.setQuerySegmentSpec(new LegacySegmentSpec("1970/3000"))
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("tags", "tags")))
|
||||
.setAggregatorSpecs(
|
||||
Arrays.asList(
|
||||
|
@ -129,7 +129,7 @@ public class GroupByQueryRunnerFactoryTest
|
|||
{
|
||||
IncrementalIndex incrementalIndex = new OnheapIncrementalIndex(
|
||||
0,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
new AggregatorFactory[]{
|
||||
new CountAggregatorFactory("count")
|
||||
},
|
||||
|
|
|
@ -30,7 +30,7 @@ import io.druid.collections.BlockingPool;
|
|||
import io.druid.collections.ReferenceCountingResourceHolder;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.DruidProcessingConfig;
|
||||
import io.druid.query.InsufficientResourcesException;
|
||||
import io.druid.query.QueryContextKeys;
|
||||
|
@ -192,13 +192,13 @@ public class GroupByQueryRunnerFailureTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(new LongSumAggregatorFactory("rows", "rows")))
|
||||
.setContext(ImmutableMap.<String, Object>of(QueryContextKeys.TIMEOUT, Integers.valueOf(500)))
|
||||
|
@ -221,7 +221,7 @@ public class GroupByQueryRunnerFailureTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(
|
||||
new DefaultDimensionSpec("quality", "alias"),
|
||||
new DefaultDimensionSpec("market", null)
|
||||
|
@ -230,13 +230,13 @@ public class GroupByQueryRunnerFailureTest
|
|||
.build()
|
||||
)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(new LongSumAggregatorFactory("rows", "rows")))
|
||||
.setContext(ImmutableMap.<String, Object>of(QueryContextKeys.TIMEOUT, Integers.valueOf(500)))
|
||||
|
@ -256,13 +256,13 @@ public class GroupByQueryRunnerFailureTest
|
|||
GroupByQuery.builder()
|
||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setDimensions(Lists.<DimensionSpec>newArrayList(new DefaultDimensionSpec("quality", "alias")))
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(QueryRunnerTestHelper.rowsCount))
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(QueryRunnerTestHelper.firstToThird)
|
||||
.setAggregatorSpecs(Lists.<AggregatorFactory>newArrayList(new LongSumAggregatorFactory("rows", "rows")))
|
||||
.setContext(ImmutableMap.<String, Object>of(QueryContextKeys.TIMEOUT, Integers.valueOf(500)))
|
||||
|
|
|
@ -35,10 +35,10 @@ import com.google.common.util.concurrent.MoreExecutors;
|
|||
import io.druid.collections.BlockingPool;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.granularity.PeriodGranularity;
|
||||
import io.druid.java.util.common.guava.MergeSequence;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
|
@ -2150,7 +2150,7 @@ public class GroupByQueryRunnerTest
|
|||
.setGranularity(new PeriodGranularity(new Period("P1M"), null, null));
|
||||
|
||||
final GroupByQuery fullQuery = builder.build();
|
||||
final GroupByQuery allGranQuery = builder.copy().setGranularity(Granularity.ALL).build();
|
||||
final GroupByQuery allGranQuery = builder.copy().setGranularity(Granularities.ALL).build();
|
||||
|
||||
QueryRunner mergedRunner = factory.getToolchest().mergeResults(
|
||||
new QueryRunner<Row>()
|
||||
|
@ -2268,7 +2268,7 @@ public class GroupByQueryRunnerTest
|
|||
new LongSumAggregatorFactory("idx", "index")
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularity.DAY)
|
||||
.setGranularity(Granularities.DAY)
|
||||
.setLimit(limit)
|
||||
.addOrderByColumn("idx", OrderByColumnSpec.Direction.DESCENDING);
|
||||
|
||||
|
@ -2318,7 +2318,7 @@ public class GroupByQueryRunnerTest
|
|||
new LongSumAggregatorFactory("idx", "expr")
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularity.DAY)
|
||||
.setGranularity(Granularities.DAY)
|
||||
.setLimit(limit)
|
||||
.addOrderByColumn("idx", OrderByColumnSpec.Direction.DESCENDING);
|
||||
|
||||
|
@ -3279,7 +3279,7 @@ public class GroupByQueryRunnerTest
|
|||
new DoubleSumAggregatorFactory("index", "index")
|
||||
)
|
||||
)
|
||||
.setGranularity(Granularity.ALL)
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setHavingSpec(new GreaterThanHavingSpec("index", 310L))
|
||||
.setLimitSpec(
|
||||
new DefaultLimitSpec(
|
||||
|
|
|
@ -25,7 +25,7 @@ import com.google.common.collect.Lists;
|
|||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import io.druid.data.input.MapBasedRow;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.Druids;
|
||||
|
@ -140,7 +140,7 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest
|
|||
{
|
||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||
.aggregators(
|
||||
Arrays.asList(
|
||||
|
|
|
@ -28,8 +28,8 @@ import com.google.common.collect.Maps;
|
|||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.BySegmentResultValue;
|
||||
import io.druid.query.BySegmentResultValueClass;
|
||||
|
@ -785,7 +785,7 @@ public class SegmentMetadataQueryTest
|
|||
expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(),
|
||||
null,
|
||||
null,
|
||||
Granularity.NONE,
|
||||
Granularities.NONE,
|
||||
null
|
||||
);
|
||||
|
||||
|
|
|
@ -20,7 +20,7 @@
|
|||
package io.druid.query.search;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.ordering.StringComparators;
|
||||
import io.druid.query.search.search.SearchHit;
|
||||
|
@ -97,7 +97,7 @@ public class SearchBinaryFnTest
|
|||
)
|
||||
);
|
||||
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularity.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
|
||||
assertSearchMergeResult(expected.getValue(), actual.getValue());
|
||||
}
|
||||
|
@ -130,7 +130,7 @@ public class SearchBinaryFnTest
|
|||
);
|
||||
|
||||
Result<SearchResultValue> expected = new Result<SearchResultValue>(
|
||||
Granularity.DAY.bucketStart(currTime),
|
||||
Granularities.DAY.bucketStart(currTime),
|
||||
new SearchResultValue(
|
||||
ImmutableList.<SearchHit>of(
|
||||
new SearchHit(
|
||||
|
@ -145,7 +145,7 @@ public class SearchBinaryFnTest
|
|||
)
|
||||
);
|
||||
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularity.DAY, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularities.DAY, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
|
||||
assertSearchMergeResult(expected.getValue(), actual.getValue());
|
||||
}
|
||||
|
@ -169,7 +169,7 @@ public class SearchBinaryFnTest
|
|||
|
||||
Result<SearchResultValue> expected = r1;
|
||||
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularity.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
|
||||
assertSearchMergeResult(expected.getValue(), actual.getValue());
|
||||
}
|
||||
|
@ -217,7 +217,7 @@ public class SearchBinaryFnTest
|
|||
)
|
||||
);
|
||||
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularity.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
|
||||
assertSearchMergeResult(expected.getValue(), actual.getValue());
|
||||
}
|
||||
|
@ -243,7 +243,7 @@ public class SearchBinaryFnTest
|
|||
new SearchResultValue(toHits(c, "blah:short", "blah:thisislong"))
|
||||
);
|
||||
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(searchSortSpec, Granularity.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(searchSortSpec, Granularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
|
||||
assertSearchMergeResult(expected.getValue(), actual.getValue());
|
||||
}
|
||||
|
@ -269,7 +269,7 @@ public class SearchBinaryFnTest
|
|||
new SearchResultValue(toHits(c, "blah:short", "blah:thisislong", "blah2:thisislong"))
|
||||
);
|
||||
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(searchSortSpec, Granularity.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(searchSortSpec, Granularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
|
||||
assertSearchMergeResult(expected.getValue(), actual.getValue());
|
||||
}
|
||||
|
@ -296,7 +296,7 @@ public class SearchBinaryFnTest
|
|||
);
|
||||
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(
|
||||
searchSortSpec, Granularity.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
searchSortSpec, Granularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
|
||||
assertSearchMergeResult(expected.getValue(), actual.getValue());
|
||||
}
|
||||
|
@ -331,7 +331,7 @@ public class SearchBinaryFnTest
|
|||
|
||||
Result<SearchResultValue> expected = r1;
|
||||
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularity.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
|
||||
assertSearchMergeResult(expected.getValue(), actual.getValue());
|
||||
}
|
||||
|
@ -362,7 +362,7 @@ public class SearchBinaryFnTest
|
|||
)
|
||||
);
|
||||
Result<SearchResultValue> expected = r1;
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularity.ALL, 1).apply(r1, r2);
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularities.ALL, 1).apply(r1, r2);
|
||||
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
|
||||
assertSearchMergeResult(expected.getValue(), actual.getValue());
|
||||
}
|
||||
|
@ -396,7 +396,7 @@ public class SearchBinaryFnTest
|
|||
|
||||
Result<SearchResultValue> expected = r1;
|
||||
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularity.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Result<SearchResultValue> actual = new SearchBinaryFn(new SearchSortSpec(StringComparators.LEXICOGRAPHIC), Granularities.ALL, Integer.MAX_VALUE).apply(r1, r2);
|
||||
Assert.assertEquals(expected.getTimestamp(), actual.getTimestamp());
|
||||
assertSearchMergeResult(expected.getValue(), actual.getValue());
|
||||
}
|
||||
|
|
|
@ -21,8 +21,8 @@ package io.druid.query.search;
|
|||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.Result;
|
||||
|
@ -47,7 +47,7 @@ public class SearchQueryQueryToolChestTest
|
|||
new SearchQuery(
|
||||
new TableDataSource("dummy"),
|
||||
null,
|
||||
Granularity.ALL,
|
||||
Granularities.ALL,
|
||||
1,
|
||||
new MultipleIntervalSegmentSpec(
|
||||
ImmutableList.of(
|
||||
|
|
|
@ -23,7 +23,7 @@ import com.google.common.base.Suppliers;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.data.input.MapBasedInputRow;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.java.util.common.logger.Logger;
|
||||
|
@ -746,7 +746,7 @@ public class SearchQueryRunnerTest
|
|||
{
|
||||
IncrementalIndex<Aggregator> index = new OnheapIncrementalIndex(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withQueryGranularity(Granularity.NONE)
|
||||
.withQueryGranularity(Granularities.NONE)
|
||||
.withMinTimestamp(new DateTime("2011-01-12T00:00:00.000Z").getMillis()).build(),
|
||||
true,
|
||||
10
|
||||
|
|
|
@ -22,8 +22,8 @@ package io.druid.query.select;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.io.CharSource;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.QueryRunner;
|
||||
|
@ -177,7 +177,7 @@ public class MultiSegmentSelectQueryTest
|
|||
{
|
||||
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
|
||||
.withMinTimestamp(new DateTime(minTimeStamp).getMillis())
|
||||
.withQueryGranularity(Granularity.HOUR)
|
||||
.withQueryGranularity(Granularities.HOUR)
|
||||
.withMetrics(TestIndex.METRIC_AGGS)
|
||||
.build();
|
||||
return new OnheapIncrementalIndex(schema, true, maxRowCount);
|
||||
|
|
|
@ -23,8 +23,8 @@ import com.google.common.collect.ImmutableMap;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.Result;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.Assert;
|
||||
|
@ -47,7 +47,7 @@ public class SelectBinaryFnTest
|
|||
@Test
|
||||
public void testApply() throws Exception
|
||||
{
|
||||
SelectBinaryFn binaryFn = new SelectBinaryFn(Granularity.ALL, new PagingSpec(null, 5), false);
|
||||
SelectBinaryFn binaryFn = new SelectBinaryFn(Granularities.ALL, new PagingSpec(null, 5), false);
|
||||
|
||||
Result<SelectResultValue> res1 = new Result<>(
|
||||
new DateTime("2013-01-01"),
|
||||
|
@ -213,7 +213,7 @@ public class SelectBinaryFnTest
|
|||
@Test
|
||||
public void testColumnMerge() throws Exception
|
||||
{
|
||||
SelectBinaryFn binaryFn = new SelectBinaryFn(Granularity.ALL, new PagingSpec(null, 5), false);
|
||||
SelectBinaryFn binaryFn = new SelectBinaryFn(Granularities.ALL, new PagingSpec(null, 5), false);
|
||||
|
||||
Result<SelectResultValue> res1 = new Result<>(
|
||||
new DateTime("2013-01-01"),
|
||||
|
|
|
@ -24,8 +24,8 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Accumulator;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
|
@ -98,7 +98,7 @@ public class SpecificSegmentQueryRunnerTest
|
|||
Map<String, Object> responseContext = Maps.newHashMap();
|
||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource("foo")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(ImmutableList.of(new Interval("2012-01-01T00:00:00Z/P1D")))
|
||||
.aggregators(
|
||||
ImmutableList.<AggregatorFactory>of(
|
||||
|
@ -175,7 +175,7 @@ public class SpecificSegmentQueryRunnerTest
|
|||
final Map<String, Object> responseContext = Maps.newHashMap();
|
||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource("foo")
|
||||
.granularity(Granularity.ALL)
|
||||
.granularity(Granularities.ALL)
|
||||
.intervals(ImmutableList.of(new Interval("2012-01-01T00:00:00Z/P1D")))
|
||||
.aggregators(
|
||||
ImmutableList.<AggregatorFactory>of(
|
||||
|
|
|
@ -24,7 +24,7 @@ import com.google.common.collect.Iterables;
|
|||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.MapMaker;
|
||||
import com.google.common.io.CharSource;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.QueryRunner;
|
||||
|
@ -116,7 +116,7 @@ public class TimeBoundaryQueryRunnerTest
|
|||
{
|
||||
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
|
||||
.withMinTimestamp(new DateTime(minTimeStamp).getMillis())
|
||||
.withQueryGranularity(Granularity.HOUR)
|
||||
.withQueryGranularity(Granularities.HOUR)
|
||||
.withMetrics(TestIndex.METRIC_AGGS)
|
||||
.build();
|
||||
return new OnheapIncrementalIndex(schema, true, maxRowCount);
|
||||
|
|
|
@ -20,7 +20,7 @@
|
|||
package io.druid.query.timeseries;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.druid.java.util.common.granularity.Granularity;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
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<TimeseriesResultValue> actual = new TimeseriesBinaryFn(
|
||||
Granularity.ALL,
|
||||
Granularities.ALL,
|
||||
aggregatorFactories
|
||||
).apply(
|
||||
result1,
|
||||
|
@ -109,7 +109,7 @@ public class TimeseriesBinaryFnTest
|
|||
);
|
||||
|
||||
Result<TimeseriesResultValue> expected = new Result<TimeseriesResultValue>(
|
||||
Granularity.DAY.bucketStart(currTime),
|
||||
Granularities.DAY.bucketStart(currTime),
|
||||
new TimeseriesResultValue(
|
||||
ImmutableMap.<String, Object>of(
|
||||
"rows", 3L,
|
||||
|
@ -119,7 +119,7 @@ public class TimeseriesBinaryFnTest
|
|||
);
|
||||
|
||||
Result<TimeseriesResultValue> actual = new TimeseriesBinaryFn(
|
||||
Granularity.DAY,
|
||||
Granularities.DAY,
|
||||
aggregatorFactories
|
||||
).apply(
|
||||
result1,
|
||||
|
@ -145,7 +145,7 @@ public class TimeseriesBinaryFnTest
|
|||
Result<TimeseriesResultValue> expected = result1;
|
||||
|
||||
Result<TimeseriesResultValue> actual = new TimeseriesBinaryFn(
|
||||
Granularity.ALL,
|
||||
Granularities.ALL,
|
||||
aggregatorFactories
|
||||
).apply(
|
||||
result1,
|
||||
|
@ -187,7 +187,7 @@ public class TimeseriesBinaryFnTest
|
|||
);
|
||||
|
||||
Result<TimeseriesResultValue> actual = new TimeseriesBinaryFn(
|
||||
Granularity.ALL,
|
||||
Granularities.ALL,
|
||||
aggregatorFactories
|
||||
).apply(
|
||||
result1,
|
||||
|
|
|
@ -22,8 +22,8 @@ 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.java.util.common.granularity.Granularity;
|
||||
import io.druid.jackson.DefaultObjectMapper;
|
||||
import io.druid.java.util.common.granularity.Granularities;
|
||||
import io.druid.query.CacheStrategy;
|
||||
import io.druid.query.QueryRunnerTestHelper;
|
||||
import io.druid.query.Result;
|
||||
|
@ -76,7 +76,7 @@ public class TimeseriesQueryQueryToolChestTest
|
|||
descending,
|
||||
VirtualColumns.EMPTY,
|
||||
null,
|
||||
Granularity.ALL,
|
||||
Granularities.ALL,
|
||||
ImmutableList.<AggregatorFactory>of(new CountAggregatorFactory("metric1")),
|
||||
null,
|
||||
null
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue