Add long/float ColumnSelectorStrategy implementations (#3838)

* Add long/float ColumnSelectorStrategy implementations

* Address PR comments

* Add String strategy with internal dictionary to V2 groupby, remove dict from numeric wrapping selectors, more tests

* PR comments

* Use BaseSingleValueDimensionSelector for long/float wrapping

* remove unused import

* Address PR comments

* PR comments

* PR comments

* More PR comments

* Fix failing calcite histogram subquery tests

* ScanQuery test and comment about isInputRaw

* Add outputType to extractionDimensionSpec, tweak SQL tests

* Fix limit spec optimization for numerics

* Add cardinality sanity checks to TopN

* Fix import from merge

* Add tests for filtered dimension spec outputType

* Address PR comments

* Allow filtered dimspecs on numerics

* More comments
This commit is contained in:
Jonathan Wei 2017-02-08 20:39:29 -08:00 committed by Gian Merlino
parent e08cd0066b
commit ca2b04f0fd
113 changed files with 10169 additions and 868 deletions

View File

@ -29,6 +29,7 @@ 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.DruidFloatPredicate;
import io.druid.query.filter.DruidLongPredicate;
import io.druid.query.filter.DruidPredicateFactory;
import io.druid.segment.column.BitmapIndex;
@ -86,14 +87,13 @@ public class DimensionPredicateFilterBenchmark
@Override
public DruidLongPredicate makeLongPredicate()
{
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return false;
}
};
return DruidLongPredicate.ALWAYS_FALSE;
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
return DruidFloatPredicate.ALWAYS_FALSE;
}
},
null

View File

@ -46,6 +46,7 @@ import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.DruidFloatPredicate;
import io.druid.query.filter.DruidLongPredicate;
import io.druid.query.filter.DruidPredicateFactory;
import io.druid.query.filter.Filter;
@ -626,14 +627,13 @@ public class FilterPartitionBenchmark
@Override
public DruidLongPredicate makeLongPredicate()
{
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return false;
}
};
return DruidLongPredicate.ALWAYS_FALSE;
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
return DruidFloatPredicate.ALWAYS_FALSE;
}
};

View File

@ -0,0 +1,860 @@
/*
* 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.benchmark;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.dataformat.smile.SmileFactory;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.hash.Hashing;
import com.google.common.io.Files;
import io.druid.benchmark.datagen.BenchmarkDataGenerator;
import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
import io.druid.benchmark.datagen.BenchmarkSchemas;
import io.druid.benchmark.query.QueryBenchmarkUtil;
import io.druid.collections.BlockingPool;
import io.druid.collections.StupidPool;
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.granularity.QueryGranularities;
import io.druid.granularity.QueryGranularity;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.offheap.OffheapBufferGenerator;
import io.druid.query.DruidProcessingConfig;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.groupby.GroupByQuery;
import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryEngine;
import io.druid.query.groupby.GroupByQueryQueryToolChest;
import io.druid.query.groupby.GroupByQueryRunnerFactory;
import io.druid.query.groupby.strategy.GroupByStrategySelector;
import io.druid.query.groupby.strategy.GroupByStrategyV1;
import io.druid.query.groupby.strategy.GroupByStrategyV2;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.query.spec.QuerySegmentSpec;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMergerV9;
import io.druid.segment.IndexSpec;
import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexSegment;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema;
import io.druid.segment.incremental.OnheapIncrementalIndex;
import io.druid.segment.serde.ComplexMetrics;
import org.apache.commons.io.FileUtils;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Level;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
// Benchmark for determining the interface overhead of GroupBy with multiple type implementations
@State(Scope.Benchmark)
@Fork(jvmArgsPrepend = "-server", value = 1)
@Warmup(iterations = 15)
@Measurement(iterations = 30)
public class GroupByTypeInterfaceBenchmark
{
@Param({"4"})
private int numSegments;
@Param({"4"})
private int numProcessingThreads;
@Param({"-1"})
private int initialBuckets;
@Param({"100000"})
private int rowsPerSegment;
@Param({"v2"})
private String defaultStrategy;
@Param({"all"})
private String queryGranularity;
private static final Logger log = new Logger(GroupByTypeInterfaceBenchmark.class);
private static final int RNG_SEED = 9999;
private static final IndexMergerV9 INDEX_MERGER_V9;
private static final IndexIO INDEX_IO;
public static final ObjectMapper JSON_MAPPER;
private File tmpDir;
private IncrementalIndex anIncrementalIndex;
private List<QueryableIndex> queryableIndexes;
private QueryRunnerFactory<Row, GroupByQuery> factory;
private BenchmarkSchemaInfo schemaInfo;
private GroupByQuery stringQuery;
private GroupByQuery longFloatQuery;
private GroupByQuery floatQuery;
private GroupByQuery longQuery;
private ExecutorService executorService;
static {
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
new ColumnConfig()
{
@Override
public int columnCacheSizeBytes()
{
return 0;
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
}
private static final Map<String, Map<String, GroupByQuery>> SCHEMA_QUERY_MAP = new LinkedHashMap<>();
private void setupQueries()
{
// queries for the basic schema
Map<String, GroupByQuery> basicQueries = new LinkedHashMap<>();
BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic");
{ // basic.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory(
"sumLongSequential",
"sumLongSequential"
));
GroupByQuery queryString = GroupByQuery
.builder()
.setDataSource("blah")
.setQuerySegmentSpec(intervalSpec)
.setDimensions(Lists.<DimensionSpec>newArrayList(
new DefaultDimensionSpec("dimSequential", null)
))
.setAggregatorSpecs(
queryAggs
)
.setGranularity(QueryGranularity.fromString(queryGranularity))
.build();
GroupByQuery queryLongFloat = GroupByQuery
.builder()
.setDataSource("blah")
.setQuerySegmentSpec(intervalSpec)
.setDimensions(Lists.<DimensionSpec>newArrayList(
new DefaultDimensionSpec("metLongUniform", null),
new DefaultDimensionSpec("metFloatNormal", null)
))
.setAggregatorSpecs(
queryAggs
)
.setGranularity(QueryGranularity.fromString(queryGranularity))
.build();
GroupByQuery queryLong = GroupByQuery
.builder()
.setDataSource("blah")
.setQuerySegmentSpec(intervalSpec)
.setDimensions(Lists.<DimensionSpec>newArrayList(
new DefaultDimensionSpec("metLongUniform", null)
))
.setAggregatorSpecs(
queryAggs
)
.setGranularity(QueryGranularity.fromString(queryGranularity))
.build();
GroupByQuery queryFloat = GroupByQuery
.builder()
.setDataSource("blah")
.setQuerySegmentSpec(intervalSpec)
.setDimensions(Lists.<DimensionSpec>newArrayList(
new DefaultDimensionSpec("metFloatNormal", null)
))
.setAggregatorSpecs(
queryAggs
)
.setGranularity(QueryGranularity.fromString(queryGranularity))
.build();
basicQueries.put("string", queryString);
basicQueries.put("longFloat", queryLongFloat);
basicQueries.put("long", queryLong);
basicQueries.put("float", queryFloat);
}
{ // basic.nested
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory(
"sumLongSequential",
"sumLongSequential"
));
GroupByQuery subqueryA = GroupByQuery
.builder()
.setDataSource("blah")
.setQuerySegmentSpec(intervalSpec)
.setDimensions(Lists.<DimensionSpec>newArrayList(
new DefaultDimensionSpec("dimSequential", null),
new DefaultDimensionSpec("dimZipf", null)
))
.setAggregatorSpecs(
queryAggs
)
.setGranularity(QueryGranularities.DAY)
.build();
GroupByQuery queryA = GroupByQuery
.builder()
.setDataSource(subqueryA)
.setQuerySegmentSpec(intervalSpec)
.setDimensions(Lists.<DimensionSpec>newArrayList(
new DefaultDimensionSpec("dimSequential", null)
))
.setAggregatorSpecs(
queryAggs
)
.setGranularity(QueryGranularities.WEEK)
.build();
basicQueries.put("nested", queryA);
}
SCHEMA_QUERY_MAP.put("basic", basicQueries);
}
@Setup(Level.Trial)
public void setup() throws IOException
{
log.info("SETUP CALLED AT %d", System.currentTimeMillis());
if (ComplexMetrics.getSerdeForType("hyperUnique") == null) {
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128()));
}
executorService = Execs.multiThreaded(numProcessingThreads, "GroupByThreadPool[%d]");
setupQueries();
String schemaName = "basic";
schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName);
stringQuery = SCHEMA_QUERY_MAP.get(schemaName).get("string");
longFloatQuery = SCHEMA_QUERY_MAP.get(schemaName).get("longFloat");
longQuery = SCHEMA_QUERY_MAP.get(schemaName).get("long");
floatQuery = SCHEMA_QUERY_MAP.get(schemaName).get("float");
final BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(
schemaInfo.getColumnSchemas(),
RNG_SEED + 1,
schemaInfo.getDataInterval(),
rowsPerSegment
);
tmpDir = Files.createTempDir();
log.info("Using temp dir: %s", tmpDir.getAbsolutePath());
// queryableIndexes -> numSegments worth of on-disk segments
// anIncrementalIndex -> the last incremental index
anIncrementalIndex = null;
queryableIndexes = new ArrayList<>(numSegments);
for (int i = 0; i < numSegments; i++) {
log.info("Generating rows for segment %d/%d", i + 1, numSegments);
final IncrementalIndex index = makeIncIndex();
for (int j = 0; j < rowsPerSegment; j++) {
final InputRow row = dataGenerator.nextRow();
if (j % 20000 == 0) {
log.info("%,d/%,d rows generated.", i * rowsPerSegment + j, rowsPerSegment * numSegments);
}
index.add(row);
}
log.info(
"%,d/%,d rows generated, persisting segment %d/%d.",
(i + 1) * rowsPerSegment,
rowsPerSegment * numSegments,
i + 1,
numSegments
);
final File file = INDEX_MERGER_V9.persist(
index,
new File(tmpDir, String.valueOf(i)),
new IndexSpec()
);
queryableIndexes.add(INDEX_IO.loadIndex(file));
if (i == numSegments - 1) {
anIncrementalIndex = index;
} else {
index.close();
}
}
StupidPool<ByteBuffer> bufferPool = new StupidPool<>(
"GroupByBenchmark-computeBufferPool",
new OffheapBufferGenerator("compute", 250_000_000),
0,
Integer.MAX_VALUE
);
// limit of 2 is required since we simulate both historical merge and broker merge in the same process
BlockingPool<ByteBuffer> mergePool = new BlockingPool<>(
new OffheapBufferGenerator("merge", 250_000_000),
2
);
final GroupByQueryConfig config = new GroupByQueryConfig()
{
@Override
public String getDefaultStrategy()
{
return defaultStrategy;
}
@Override
public int getBufferGrouperInitialBuckets()
{
return initialBuckets;
}
@Override
public long getMaxOnDiskStorage()
{
return 1_000_000_000L;
}
};
config.setSingleThreaded(false);
config.setMaxIntermediateRows(Integer.MAX_VALUE);
config.setMaxResults(Integer.MAX_VALUE);
DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig()
{
@Override
public int getNumThreads()
{
// Used by "v2" strategy for concurrencyHint
return numProcessingThreads;
}
@Override
public String getFormatString()
{
return null;
}
};
final Supplier<GroupByQueryConfig> configSupplier = Suppliers.ofInstance(config);
final GroupByStrategySelector strategySelector = new GroupByStrategySelector(
configSupplier,
new GroupByStrategyV1(
configSupplier,
new GroupByQueryEngine(configSupplier, bufferPool),
QueryBenchmarkUtil.NOOP_QUERYWATCHER,
bufferPool
),
new GroupByStrategyV2(
druidProcessingConfig,
configSupplier,
bufferPool,
mergePool,
new ObjectMapper(new SmileFactory()),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
)
);
factory = new GroupByQueryRunnerFactory(
strategySelector,
new GroupByQueryQueryToolChest(
configSupplier,
strategySelector,
bufferPool,
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
)
);
}
private IncrementalIndex makeIncIndex()
{
return new OnheapIncrementalIndex(
new IncrementalIndexSchema.Builder()
.withQueryGranularity(QueryGranularities.NONE)
.withMetrics(schemaInfo.getAggsArray())
.withDimensionsSpec(new DimensionsSpec(null, null, null))
.build(),
true,
false,
true,
rowsPerSegment
);
}
@TearDown(Level.Trial)
public void tearDown()
{
try {
if (anIncrementalIndex != null) {
anIncrementalIndex.close();
}
if (queryableIndexes != null) {
for (QueryableIndex index : queryableIndexes) {
index.close();
}
}
if (tmpDir != null) {
FileUtils.deleteDirectory(tmpDir);
}
}
catch (IOException e) {
log.warn(e, "Failed to tear down, temp dir was: %s", tmpDir);
throw Throwables.propagate(e);
}
}
private static <T> List<T> runQuery(QueryRunnerFactory factory, QueryRunner runner, Query<T> query)
{
QueryToolChest toolChest = factory.getToolchest();
QueryRunner<T> theRunner = new FinalizeResultsQueryRunner<>(
toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)),
toolChest
);
Sequence<T> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
return Sequences.toList(queryResult, Lists.<T>newArrayList());
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexStringOnly(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexLongOnly(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexFloatOnly(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexNumericOnly(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexNumericThenString(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
for (Row result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexLongThenString(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Row result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexLongThenFloat(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Row result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexStringThenNumeric(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Row result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longFloatQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexStringThenLong(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Row result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexStringTwice(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Row result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexLongTwice(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Row result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexFloatTwice(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Row result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexFloatThenLong(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Row result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexFloatThenString(Blackhole blackhole) throws Exception
{
QueryRunner<Row> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
List<Row> results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Row result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", queryableIndexes.get(0))
);
results = GroupByTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Row result : results) {
blackhole.consume(result);
}
}
}

View File

@ -0,0 +1,645 @@
/*
* 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.benchmark;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.hash.Hashing;
import com.google.common.io.Files;
import io.druid.benchmark.datagen.BenchmarkDataGenerator;
import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
import io.druid.benchmark.datagen.BenchmarkSchemas;
import io.druid.benchmark.query.QueryBenchmarkUtil;
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.granularity.QueryGranularities;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.offheap.OffheapBufferGenerator;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerFactory;
import io.druid.query.QueryToolChest;
import io.druid.query.Result;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleMinAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.aggregation.LongMaxAggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
import io.druid.query.dimension.ExtractionDimensionSpec;
import io.druid.query.extraction.IdentityExtractionFn;
import io.druid.query.ordering.StringComparators;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.query.spec.QuerySegmentSpec;
import io.druid.query.topn.DimensionTopNMetricSpec;
import io.druid.query.topn.TopNQuery;
import io.druid.query.topn.TopNQueryBuilder;
import io.druid.query.topn.TopNQueryConfig;
import io.druid.query.topn.TopNQueryQueryToolChest;
import io.druid.query.topn.TopNQueryRunnerFactory;
import io.druid.query.topn.TopNResultValue;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMergerV9;
import io.druid.segment.IndexSpec;
import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexSegment;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema;
import io.druid.segment.incremental.OnheapIncrementalIndex;
import io.druid.segment.serde.ComplexMetrics;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
import org.openjdk.jmh.annotations.Measurement;
import org.openjdk.jmh.annotations.Mode;
import org.openjdk.jmh.annotations.OutputTimeUnit;
import org.openjdk.jmh.annotations.Param;
import org.openjdk.jmh.annotations.Scope;
import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
// Benchmark for determining the interface overhead of TopN with multiple type implementations
@State(Scope.Benchmark)
@Fork(jvmArgsPrepend = "-server", value = 1)
@Warmup(iterations = 10)
@Measurement(iterations = 25)
public class TopNTypeInterfaceBenchmark
{
@Param({"1"})
private int numSegments;
@Param({"750000"})
private int rowsPerSegment;
@Param({"basic.A"})
private String schemaAndQuery;
@Param({"10"})
private int threshold;
private static final Logger log = new Logger(TopNTypeInterfaceBenchmark.class);
private static final int RNG_SEED = 9999;
private static final IndexMergerV9 INDEX_MERGER_V9;
private static final IndexIO INDEX_IO;
public static final ObjectMapper JSON_MAPPER;
private List<IncrementalIndex> incIndexes;
private List<QueryableIndex> qIndexes;
private QueryRunnerFactory factory;
private BenchmarkSchemaInfo schemaInfo;
private TopNQueryBuilder queryBuilder;
private TopNQuery stringQuery;
private TopNQuery longQuery;
private TopNQuery floatQuery;
private ExecutorService executorService;
static {
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
new ColumnConfig()
{
@Override
public int columnCacheSizeBytes()
{
return 0;
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
}
private static final Map<String, Map<String, TopNQueryBuilder>> SCHEMA_QUERY_MAP = new LinkedHashMap<>();
private void setupQueries()
{
// queries for the basic schema
Map<String, TopNQueryBuilder> basicQueries = new LinkedHashMap<>();
BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic");
{ // basic.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
queryAggs.add(new LongMaxAggregatorFactory("maxLongUniform", "maxLongUniform"));
queryAggs.add(new DoubleSumAggregatorFactory("sumFloatNormal", "sumFloatNormal"));
queryAggs.add(new DoubleMinAggregatorFactory("minFloatZipf", "minFloatZipf"));
queryAggs.add(new HyperUniquesAggregatorFactory("hyperUniquesMet", "hyper"));
// Use an IdentityExtractionFn to force usage of DimExtractionTopNAlgorithm
TopNQueryBuilder queryBuilderString = new TopNQueryBuilder()
.dataSource("blah")
.granularity(QueryGranularities.ALL)
.dimension(new ExtractionDimensionSpec("dimSequential", "dimSequential", IdentityExtractionFn.getInstance()))
.metric("sumFloatNormal")
.intervals(intervalSpec)
.aggregators(queryAggs);
// DimExtractionTopNAlgorithm is always used for numeric columns
TopNQueryBuilder queryBuilderLong = new TopNQueryBuilder()
.dataSource("blah")
.granularity(QueryGranularities.ALL)
.dimension("metLongUniform")
.metric("sumFloatNormal")
.intervals(intervalSpec)
.aggregators(queryAggs);
TopNQueryBuilder queryBuilderFloat = new TopNQueryBuilder()
.dataSource("blah")
.granularity(QueryGranularities.ALL)
.dimension("metFloatNormal")
.metric("sumFloatNormal")
.intervals(intervalSpec)
.aggregators(queryAggs);
basicQueries.put("string", queryBuilderString);
basicQueries.put("long", queryBuilderLong);
basicQueries.put("float", queryBuilderFloat);
}
{ // basic.numericSort
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
TopNQueryBuilder queryBuilderA = new TopNQueryBuilder()
.dataSource("blah")
.granularity(QueryGranularities.ALL)
.dimension("dimUniform")
.metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC))
.intervals(intervalSpec)
.aggregators(queryAggs);
basicQueries.put("numericSort", queryBuilderA);
}
{ // basic.alphanumericSort
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
TopNQueryBuilder queryBuilderA = new TopNQueryBuilder()
.dataSource("blah")
.granularity(QueryGranularities.ALL)
.dimension("dimUniform")
.metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC))
.intervals(intervalSpec)
.aggregators(queryAggs);
basicQueries.put("alphanumericSort", queryBuilderA);
}
SCHEMA_QUERY_MAP.put("basic", basicQueries);
}
@Setup
public void setup() throws IOException
{
log.info("SETUP CALLED AT " + System.currentTimeMillis());
if (ComplexMetrics.getSerdeForType("hyperUnique") == null) {
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128()));
}
executorService = Execs.multiThreaded(numSegments, "TopNThreadPool");
setupQueries();
schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get("basic");
queryBuilder = SCHEMA_QUERY_MAP.get("basic").get("string");
queryBuilder.threshold(threshold);
stringQuery = queryBuilder.build();
TopNQueryBuilder longBuilder = SCHEMA_QUERY_MAP.get("basic").get("long");
longBuilder.threshold(threshold);
longQuery = longBuilder.build();
TopNQueryBuilder floatBuilder = SCHEMA_QUERY_MAP.get("basic").get("float");
floatBuilder.threshold(threshold);
floatQuery = floatBuilder.build();
incIndexes = new ArrayList<>();
for (int i = 0; i < numSegments; i++) {
log.info("Generating rows for segment " + i);
BenchmarkDataGenerator gen = new BenchmarkDataGenerator(
schemaInfo.getColumnSchemas(),
RNG_SEED + i,
schemaInfo.getDataInterval(),
rowsPerSegment
);
IncrementalIndex incIndex = makeIncIndex();
for (int j = 0; j < rowsPerSegment; j++) {
InputRow row = gen.nextRow();
if (j % 10000 == 0) {
log.info(j + " rows generated.");
}
incIndex.add(row);
}
incIndexes.add(incIndex);
}
File tmpFile = Files.createTempDir();
log.info("Using temp dir: " + tmpFile.getAbsolutePath());
tmpFile.deleteOnExit();
qIndexes = new ArrayList<>();
for (int i = 0; i < numSegments; i++) {
File indexFile = INDEX_MERGER_V9.persist(
incIndexes.get(i),
tmpFile,
new IndexSpec()
);
QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile);
qIndexes.add(qIndex);
}
factory = new TopNQueryRunnerFactory(
new StupidPool<>(
"TopNBenchmark-compute-bufferPool",
new OffheapBufferGenerator("compute", 250000000),
0,
Integer.MAX_VALUE
),
new TopNQueryQueryToolChest(new TopNQueryConfig(), QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()),
QueryBenchmarkUtil.NOOP_QUERYWATCHER
);
}
private IncrementalIndex makeIncIndex()
{
return new OnheapIncrementalIndex(
new IncrementalIndexSchema.Builder()
.withQueryGranularity(QueryGranularities.NONE)
.withMetrics(schemaInfo.getAggsArray())
.withDimensionsSpec(new DimensionsSpec(null, null, null))
.build(),
true,
false,
true,
rowsPerSegment
);
}
private static <T> List<T> runQuery(QueryRunnerFactory factory, QueryRunner runner, Query<T> query)
{
QueryToolChest toolChest = factory.getToolchest();
QueryRunner<T> theRunner = new FinalizeResultsQueryRunner<>(
toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)),
toolChest
);
Sequence<T> queryResult = theRunner.run(query, Maps.<String, Object>newHashMap());
return Sequences.toList(queryResult, Lists.<T>newArrayList());
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexStringOnly(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexStringTwice(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexStringThenLong(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexStringThenFloat(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexLongOnly(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexLongTwice(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexLongThenString(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexLongThenFloat(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexFloatOnly(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexFloatTwice(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexFloatThenString(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, stringQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void querySingleQueryableIndexFloatThenLong(Blackhole blackhole) throws Exception
{
QueryRunner<Result<TopNResultValue>> runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
List<Result<TopNResultValue>> results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, floatQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
runner = QueryBenchmarkUtil.makeQueryRunner(
factory,
"qIndex",
new QueryableIndexSegment("qIndex", qIndexes.get(0))
);
results = TopNTypeInterfaceBenchmark.runQuery(factory, runner, longQuery);
for (Result<TopNResultValue> result : results) {
blackhole.consume(result);
}
}
}

View File

@ -29,16 +29,19 @@ public class BenchmarkSchemaInfo
private List<BenchmarkColumnSchema> columnSchemas;
private List<AggregatorFactory> aggs;
private Interval dataInterval;
private boolean withRollup;
public BenchmarkSchemaInfo (
List<BenchmarkColumnSchema> columnSchemas,
List<AggregatorFactory> aggs,
Interval dataInterval
Interval dataInterval,
boolean withRollup
)
{
this.columnSchemas = columnSchemas;
this.aggs = aggs;
this.dataInterval = dataInterval;
this.withRollup = withRollup;
}
public List<BenchmarkColumnSchema> getColumnSchemas()
@ -61,4 +64,8 @@ public class BenchmarkSchemaInfo
return dataInterval;
}
public boolean isWithRollup()
{
return withRollup;
}
}

View File

@ -89,8 +89,71 @@ public class BenchmarkSchemas
BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo(
basicSchemaColumns,
basicSchemaIngestAggs,
basicSchemaDataInterval
basicSchemaDataInterval,
true
);
SCHEMA_MAP.put("basic", basicSchema);
}
static { // simple single string column and count agg schema, no rollup
List<BenchmarkColumnSchema> basicSchemaColumns = ImmutableList.of(
// dims
BenchmarkColumnSchema.makeSequential("dimSequential", ValueType.STRING, false, 1, null, 0, 1000000)
);
List<AggregatorFactory> basicSchemaIngestAggs = new ArrayList<>();
basicSchemaIngestAggs.add(new CountAggregatorFactory("rows"));
Interval basicSchemaDataInterval = new Interval(0, 1000000);
BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo(
basicSchemaColumns,
basicSchemaIngestAggs,
basicSchemaDataInterval,
false
);
SCHEMA_MAP.put("simple", basicSchema);
}
static { // simple single long column and count agg schema, no rollup
List<BenchmarkColumnSchema> basicSchemaColumns = ImmutableList.of(
// dims, ingest as a metric for now with rollup off, until numeric dims at ingestion are supported
BenchmarkColumnSchema.makeSequential("dimSequential", ValueType.LONG, true, 1, null, 0, 1000000)
);
List<AggregatorFactory> basicSchemaIngestAggs = new ArrayList<>();
basicSchemaIngestAggs.add(new LongSumAggregatorFactory("dimSequential", "dimSequential"));
basicSchemaIngestAggs.add(new CountAggregatorFactory("rows"));
Interval basicSchemaDataInterval = new Interval(0, 1000000);
BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo(
basicSchemaColumns,
basicSchemaIngestAggs,
basicSchemaDataInterval,
false
);
SCHEMA_MAP.put("simpleLong", basicSchema);
}
static { // simple single float column and count agg schema, no rollup
List<BenchmarkColumnSchema> basicSchemaColumns = ImmutableList.of(
// dims, ingest as a metric for now with rollup off, until numeric dims at ingestion are supported
BenchmarkColumnSchema.makeSequential("dimSequential", ValueType.FLOAT, true, 1, null, 0, 1000000)
);
List<AggregatorFactory> basicSchemaIngestAggs = new ArrayList<>();
basicSchemaIngestAggs.add(new DoubleSumAggregatorFactory("dimSequential", "dimSequential"));
basicSchemaIngestAggs.add(new CountAggregatorFactory("rows"));
Interval basicSchemaDataInterval = new Interval(0, 1000000);
BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo(
basicSchemaColumns,
basicSchemaIngestAggs,
basicSchemaDataInterval,
false
);
SCHEMA_MAP.put("simpleFloat", basicSchema);
}
}

View File

@ -73,6 +73,7 @@ import io.druid.segment.IndexSpec;
import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexSegment;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.column.ValueType;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema;
import io.druid.segment.incremental.OnheapIncrementalIndex;
@ -237,8 +238,90 @@ public class GroupByBenchmark
basicQueries.put("nested", queryA);
}
SCHEMA_QUERY_MAP.put("basic", basicQueries);
// simple one column schema, for testing performance difference between querying on numeric values as Strings and
// directly as longs
Map<String, GroupByQuery> simpleQueries = new LinkedHashMap<>();
BenchmarkSchemaInfo simpleSchema = BenchmarkSchemas.SCHEMA_MAP.get("simple");
{ // simple.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory(
"rows",
"rows"
));
GroupByQuery queryA = GroupByQuery
.builder()
.setDataSource("blah")
.setQuerySegmentSpec(intervalSpec)
.setDimensions(Lists.<DimensionSpec>newArrayList(
new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.STRING)
))
.setAggregatorSpecs(
queryAggs
)
.setGranularity(QueryGranularity.fromString(queryGranularity))
.build();
simpleQueries.put("A", queryA);
}
SCHEMA_QUERY_MAP.put("simple", simpleQueries);
Map<String, GroupByQuery> simpleLongQueries = new LinkedHashMap<>();
BenchmarkSchemaInfo simpleLongSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleLong");
{ // simpleLong.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleLongSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory(
"rows",
"rows"
));
GroupByQuery queryA = GroupByQuery
.builder()
.setDataSource("blah")
.setQuerySegmentSpec(intervalSpec)
.setDimensions(Lists.<DimensionSpec>newArrayList(
new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.LONG)
))
.setAggregatorSpecs(
queryAggs
)
.setGranularity(QueryGranularity.fromString(queryGranularity))
.build();
simpleLongQueries.put("A", queryA);
}
SCHEMA_QUERY_MAP.put("simpleLong", simpleLongQueries);
Map<String, GroupByQuery> simpleFloatQueries = new LinkedHashMap<>();
BenchmarkSchemaInfo simpleFloatSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleFloat");
{ // simpleFloat.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(simpleFloatSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory(
"rows",
"rows"
));
GroupByQuery queryA = GroupByQuery
.builder()
.setDataSource("blah")
.setQuerySegmentSpec(intervalSpec)
.setDimensions(Lists.<DimensionSpec>newArrayList(
new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.FLOAT)
))
.setAggregatorSpecs(
queryAggs
)
.setGranularity(QueryGranularity.fromString(queryGranularity))
.build();
simpleFloatQueries.put("A", queryA);
}
SCHEMA_QUERY_MAP.put("simpleFloat", simpleFloatQueries);
}
@Setup(Level.Trial)
@ -278,7 +361,7 @@ public class GroupByBenchmark
for (int i = 0; i < numSegments; i++) {
log.info("Generating rows for segment %d/%d", i + 1, numSegments);
final IncrementalIndex index = makeIncIndex();
final IncrementalIndex index = makeIncIndex(schemaInfo.isWithRollup());
for (int j = 0; j < rowsPerSegment; j++) {
final InputRow row = dataGenerator.nextRow();
@ -393,13 +476,14 @@ public class GroupByBenchmark
);
}
private IncrementalIndex makeIncIndex()
private IncrementalIndex makeIncIndex(boolean withRollup)
{
return new OnheapIncrementalIndex(
new IncrementalIndexSchema.Builder()
.withQueryGranularity(QueryGranularities.NONE)
.withMetrics(schemaInfo.getAggsArray())
.withDimensionsSpec(new DimensionsSpec(null, null, null))
.withRollup(withRollup)
.build(),
true,
false,

View File

@ -15,9 +15,18 @@ The following JSON fields can be used in a query to operate on dimension values.
Returns dimension values as is and optionally renames the dimension.
```json
{ "type" : "default", "dimension" : <dimension>, "outputName": <output_name> }
{
"type" : "default",
"dimension" : <dimension>,
"outputName": <output_name>,
"outputType": <"STRING"|"LONG"|"FLOAT">
}
```
When specifying a DimensionSpec on a numeric column, the user should include the type of the column in the `outputType` field. If left unspecified, the `outputType` defaults to STRING.
Please refer to the [Output Types](#output-types) section for more details.
### Extraction DimensionSpec
Returns dimension values transformed using the given [extraction function](#extraction-functions).
@ -27,10 +36,15 @@ Returns dimension values transformed using the given [extraction function](#extr
"type" : "extraction",
"dimension" : <dimension>,
"outputName" : <output_name>,
"outputType": <"STRING"|"LONG"|"FLOAT">,
"extractionFn" : <extraction_function>
}
```
`outputType` may also be specified in an ExtractionDimensionSpec to apply type conversion to results before merging. If left unspecified, the `outputType` defaults to STRING.
Please refer to the [Output Types](#output-types) section for more details.
### Filtered DimensionSpecs
These are only useful for multi-value dimensions. If you have a row in druid that has a multi-value dimension with values ["v1", "v2", "v3"] and you send a groupBy/topN query grouping by that dimension with [query filter](filters.html) for value "v1". In the response you will get 3 rows containing "v1", "v2" and "v3". This behavior might be unintuitive for some use cases.
@ -96,6 +110,21 @@ The second kind where it is not possible to pass at query time due to their size
}
```
## Output Types
The dimension specs provide an option to specify the output type of a column's values. This is necessary as it is possible for a column with given name to have different value types in different segments; results will be converted to the type specified by `outputType` before merging.
Note that not all use cases for DimensionSpec currently support `outputType`, the table below shows which use cases support this option:
|Query Type|Supported?|
|--------|---------|
|GroupBy (v1)|no|
|GroupBy (v2)|yes|
|TopN|yes|
|Search|no|
|Select|no|
|Cardinality Aggregator|no|
## Extraction Functions
Extraction functions define the transformation applied to each dimension value.

View File

@ -21,4 +21,3 @@ The query context is used for various query configuration parameters.
|`maxResults`|500000|Maximum number of results groupBy query can process. Default value used can be changed by `druid.query.groupBy.maxResults` in druid configuration at broker and historical nodes. At query time you can only lower the value.|
|`maxIntermediateRows`|50000|Maximum number of intermediate rows while processing single segment for groupBy query. Default value used can be changed by `druid.query.groupBy.maxIntermediateRows` in druid configuration at broker and historical nodes. At query time you can only lower the value.|
|`groupByIsSingleThreaded`|false|Whether to run single threaded group By queries. Default value used can be changed by `druid.query.groupBy.singleThreaded` in druid configuration at historical nodes.|

View File

@ -160,8 +160,6 @@ Druid does not support all SQL features. Most of these are due to missing featur
language. Some unsupported SQL features include:
- Grouping on functions of multiple columns, like concatenation: `SELECT COUNT(*) FROM data_source GROUP BY dim1 || ' ' || dim2`
- Grouping on long and float columns.
- Filtering on float columns.
- Filtering on non-boolean interactions between columns, like two columns equaling each other: `SELECT COUNT(*) FROM data_source WHERE dim1 = dim2`.
- A number of miscellaneous functions, like `TRIM`.
- Joins, other than semi-joins as described above.

View File

@ -70,37 +70,37 @@ public class MultiSegmentScanQueryTest
new ScanQueryEngine()
);
// time modified version of druid.sample.tsv
// time modified version of druid.sample.numeric.tsv
public static final String[] V_0112 = {
"2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000",
"2011-01-12T01:00:00.000Z spot business preferred bpreferred 100.000000",
"2011-01-12T02:00:00.000Z spot entertainment preferred epreferred 100.000000",
"2011-01-12T03:00:00.000Z spot health preferred hpreferred 100.000000",
"2011-01-12T04:00:00.000Z spot mezzanine preferred mpreferred 100.000000",
"2011-01-12T05:00:00.000Z spot news preferred npreferred 100.000000",
"2011-01-12T06:00:00.000Z spot premium preferred ppreferred 100.000000",
"2011-01-12T07:00:00.000Z spot technology preferred tpreferred 100.000000",
"2011-01-12T08:00:00.000Z spot travel preferred tpreferred 100.000000",
"2011-01-12T09:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000",
"2011-01-12T10:00:00.000Z total_market premium preferred ppreferred 1000.000000",
"2011-01-12T11:00:00.000Z upfront mezzanine preferred mpreferred 800.000000 value",
"2011-01-12T12:00:00.000Z upfront premium preferred ppreferred 800.000000 value",
"2011-01-12T13:00:00.000Z upfront premium preferred ppreferred2 800.000000 value"
"2011-01-12T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 100.000000",
"2011-01-12T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000",
"2011-01-12T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 100.000000",
"2011-01-12T03:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 100.000000",
"2011-01-12T04:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 100.000000",
"2011-01-12T05:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 100.000000",
"2011-01-12T06:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 100.000000",
"2011-01-12T07:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 100.000000",
"2011-01-12T08:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 100.000000",
"2011-01-12T09:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1000.000000",
"2011-01-12T10:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1000.000000",
"2011-01-12T11:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 800.000000 value",
"2011-01-12T12:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 800.000000 value"
};
public static final String[] V_0113 = {
"2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713",
"2011-01-13T01:00:00.000Z spot business preferred bpreferred 103.629399",
"2011-01-13T02:00:00.000Z spot entertainment preferred epreferred 110.087299",
"2011-01-13T03:00:00.000Z spot health preferred hpreferred 114.947403",
"2011-01-13T04:00:00.000Z spot mezzanine preferred mpreferred 104.465767",
"2011-01-13T05:00:00.000Z spot news preferred npreferred 102.851683",
"2011-01-13T06:00:00.000Z spot premium preferred ppreferred 108.863011",
"2011-01-13T07:00:00.000Z spot technology preferred tpreferred 111.356672",
"2011-01-13T08:00:00.000Z spot travel preferred tpreferred 106.236928",
"2011-01-13T09:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505",
"2011-01-13T10:00:00.000Z total_market premium preferred ppreferred 1689.012875",
"2011-01-13T11:00:00.000Z upfront mezzanine preferred mpreferred 826.060182 value",
"2011-01-13T12:00:00.000Z upfront premium preferred ppreferred 1564.617729 value"
"2011-01-13T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713",
"2011-01-13T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399",
"2011-01-13T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299",
"2011-01-13T03:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 114.947403",
"2011-01-13T04:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 104.465767",
"2011-01-13T05:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 102.851683",
"2011-01-13T06:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 108.863011",
"2011-01-13T07:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 111.356672",
"2011-01-13T08:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 106.236928",
"2011-01-13T09:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1040.945505",
"2011-01-13T10:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1689.012875",
"2011-01-13T11:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 826.060182 value",
"2011-01-13T12:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 1564.617729 value"
};
private static Segment segment0;

View File

@ -58,40 +58,40 @@ import java.util.Set;
@RunWith(Parameterized.class)
public class ScanQueryRunnerTest
{
// copied from druid.sample.tsv
// copied from druid.sample.numeric.tsv
public static final String[] V_0112 = {
"2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000",
"2011-01-12T00:00:00.000Z spot business preferred bpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot entertainment preferred epreferred 100.000000",
"2011-01-12T00:00:00.000Z spot health preferred hpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot mezzanine preferred mpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot news preferred npreferred 100.000000",
"2011-01-12T00:00:00.000Z spot premium preferred ppreferred 100.000000",
"2011-01-12T00:00:00.000Z spot technology preferred tpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot travel preferred tpreferred 100.000000",
"2011-01-12T00:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000",
"2011-01-12T00:00:00.000Z total_market premium preferred ppreferred 1000.000000",
"2011-01-12T00:00:00.000Z upfront mezzanine preferred mpreferred 800.000000 value",
"2011-01-12T00:00:00.000Z upfront premium preferred ppreferred 800.000000 value"
"2011-01-12T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 100.000000",
"2011-01-12T00:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 100.000000",
"2011-01-12T00:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 100.000000",
"2011-01-12T00:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 100.000000",
"2011-01-12T00:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 100.000000",
"2011-01-12T00:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1000.000000",
"2011-01-12T00:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1000.000000",
"2011-01-12T00:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 800.000000 value",
"2011-01-12T00:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 800.000000 value"
};
public static final String[] V_0113 = {
"2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713",
"2011-01-13T00:00:00.000Z spot business preferred bpreferred 103.629399",
"2011-01-13T00:00:00.000Z spot entertainment preferred epreferred 110.087299",
"2011-01-13T00:00:00.000Z spot health preferred hpreferred 114.947403",
"2011-01-13T00:00:00.000Z spot mezzanine preferred mpreferred 104.465767",
"2011-01-13T00:00:00.000Z spot news preferred npreferred 102.851683",
"2011-01-13T00:00:00.000Z spot premium preferred ppreferred 108.863011",
"2011-01-13T00:00:00.000Z spot technology preferred tpreferred 111.356672",
"2011-01-13T00:00:00.000Z spot travel preferred tpreferred 106.236928",
"2011-01-13T00:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505",
"2011-01-13T00:00:00.000Z total_market premium preferred ppreferred 1689.012875",
"2011-01-13T00:00:00.000Z upfront mezzanine preferred mpreferred 826.060182 value",
"2011-01-13T00:00:00.000Z upfront premium preferred ppreferred 1564.617729 value"
"2011-01-13T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713",
"2011-01-13T00:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399",
"2011-01-13T00:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299",
"2011-01-13T00:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 114.947403",
"2011-01-13T00:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 104.465767",
"2011-01-13T00:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 102.851683",
"2011-01-13T00:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 108.863011",
"2011-01-13T00:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 111.356672",
"2011-01-13T00:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 106.236928",
"2011-01-13T00:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1040.945505",
"2011-01-13T00:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1689.012875",
"2011-01-13T00:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 826.060182 value",
"2011-01-13T00:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 1564.617729 value"
};
public static final QuerySegmentSpec I_0112_0114 = new LegacySegmentSpec(
new Interval("2011-01-12/2011-01-14")
new Interval("2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z")
);
public static final String[] V_0112_0114 = ObjectArrays.concat(V_0112, V_0113, String.class);
@ -133,6 +133,7 @@ public class ScanQueryRunnerTest
ScanResultValue.timestampKey,
"market",
"quality",
"qualityNumericString",
"placement",
"placementish",
"partial_null_column",
@ -140,7 +141,9 @@ public class ScanQueryRunnerTest
"index",
"indexMin",
"indexMaxPlusTen",
"quality_uniques"
"quality_uniques",
"qualityLong",
"qualityFloat"
);
ScanQuery query = newTestQuery()
.intervals(I_0112_0114)
@ -168,6 +171,9 @@ public class ScanQueryRunnerTest
ScanResultValue.timestampKey,
"market",
"quality",
"qualityLong",
"qualityFloat",
"qualityNumericString",
"placement",
"placementish",
"partial_null_column",
@ -219,6 +225,9 @@ public class ScanQueryRunnerTest
null,
null,
null,
null,
null,
null,
QueryRunnerTestHelper.indexMetric + ":FLOAT"
},
V_0112_0114
@ -253,6 +262,9 @@ public class ScanQueryRunnerTest
null,
null,
null,
null,
null,
null,
QueryRunnerTestHelper.indexMetric + ":FLOAT"
},
V_0112_0114
@ -441,6 +453,9 @@ public class ScanQueryRunnerTest
ScanResultValue.timestampKey + ":TIME",
QueryRunnerTestHelper.marketDimension + ":STRING",
QueryRunnerTestHelper.qualityDimension + ":STRING",
"qualityLong" + ":LONG",
"qualityFloat" + ":FLOAT",
"qualityNumericString" + ":STRING",
QueryRunnerTestHelper.placementDimension + ":STRING",
QueryRunnerTestHelper.placementishDimension + ":STRINGS",
QueryRunnerTestHelper.indexMetric + ":FLOAT",

View File

@ -21,6 +21,7 @@ package io.druid.query.aggregation.cardinality.types;
import io.druid.java.util.common.IAE;
import io.druid.query.dimension.ColumnSelectorStrategyFactory;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ValueType;
@ -29,13 +30,17 @@ public class CardinalityAggregatorColumnSelectorStrategyFactory
{
@Override
public CardinalityAggregatorColumnSelectorStrategy makeColumnSelectorStrategy(
ColumnCapabilities capabilities
ColumnCapabilities capabilities, ColumnValueSelector selector
)
{
ValueType type = capabilities.getType();
switch(type) {
case STRING:
return new StringCardinalityAggregatorColumnSelectorStrategy();
case LONG:
return new LongCardinalityAggregatorColumnSelectorStrategy();
case FLOAT:
return new FloatCardinalityAggregatorColumnSelectorStrategy();
default:
throw new IAE("Cannot create query type helper from invalid type [%s]", type);
}

View File

@ -0,0 +1,41 @@
/*
* 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.query.aggregation.cardinality.types;
import com.google.common.hash.Hasher;
import io.druid.hll.HyperLogLogCollector;
import io.druid.query.aggregation.cardinality.CardinalityAggregator;
import io.druid.segment.FloatColumnSelector;
public class FloatCardinalityAggregatorColumnSelectorStrategy
implements CardinalityAggregatorColumnSelectorStrategy<FloatColumnSelector>
{
@Override
public void hashRow(FloatColumnSelector dimSelector, Hasher hasher)
{
hasher.putFloat(dimSelector.get());
}
@Override
public void hashValues(FloatColumnSelector dimSelector, HyperLogLogCollector collector)
{
collector.add(CardinalityAggregator.hashFn.hashInt(Float.floatToIntBits(dimSelector.get())).asBytes());
}
}

View File

@ -0,0 +1,41 @@
/*
* 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.query.aggregation.cardinality.types;
import com.google.common.hash.Hasher;
import io.druid.hll.HyperLogLogCollector;
import io.druid.query.aggregation.cardinality.CardinalityAggregator;
import io.druid.segment.LongColumnSelector;
public class LongCardinalityAggregatorColumnSelectorStrategy
implements CardinalityAggregatorColumnSelectorStrategy<LongColumnSelector>
{
@Override
public void hashRow(LongColumnSelector dimSelector, Hasher hasher)
{
hasher.putLong(dimSelector.get());
}
@Override
public void hashValues(LongColumnSelector dimSelector, HyperLogLogCollector collector)
{
collector.add(CardinalityAggregator.hashFn.hashLong(dimSelector.get()).asBytes());
}
}

View File

@ -22,6 +22,7 @@ package io.druid.query.dimension;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import io.druid.query.extraction.ExtractionFn;
import io.druid.segment.column.ValueType;
/**
*/
@ -54,12 +55,24 @@ public abstract class BaseFilteredDimensionSpec implements DimensionSpec
return delegate.getOutputName();
}
@Override
public ValueType getOutputType()
{
return delegate.getOutputType();
}
@Override
public ExtractionFn getExtractionFn()
{
return delegate.getExtractionFn();
}
@Override
public boolean mustDecorate()
{
return true;
}
@Override
public boolean preservesOrdering()
{

View File

@ -19,9 +19,10 @@
package io.druid.query.dimension;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.column.ColumnCapabilities;
public interface ColumnSelectorStrategyFactory<ColumnSelectorStrategyClass extends ColumnSelectorStrategy>
{
ColumnSelectorStrategyClass makeColumnSelectorStrategy(ColumnCapabilities capabilities);
ColumnSelectorStrategyClass makeColumnSelectorStrategy(ColumnCapabilities capabilities, ColumnValueSelector selector);
}

View File

@ -27,6 +27,7 @@ import com.google.common.collect.Lists;
import io.druid.java.util.common.StringUtils;
import io.druid.query.extraction.ExtractionFn;
import io.druid.segment.DimensionSelector;
import io.druid.segment.column.ValueType;
import java.nio.ByteBuffer;
import java.util.Arrays;
@ -65,19 +66,30 @@ public class DefaultDimensionSpec implements DimensionSpec
private static final byte CACHE_TYPE_ID = 0x0;
private final String dimension;
private final String outputName;
private final ValueType outputType;
@JsonCreator
public DefaultDimensionSpec(
@JsonProperty("dimension") String dimension,
@JsonProperty("outputName") String outputName
@JsonProperty("outputName") String outputName,
@JsonProperty("outputType") ValueType outputType
)
{
this.dimension = dimension;
this.outputType = outputType == null ? ValueType.STRING : outputType;
// Do null check for legacy backwards compatibility, callers should be setting the value.
this.outputName = outputName == null ? dimension : outputName;
}
public DefaultDimensionSpec(
String dimension,
String outputName
)
{
this(dimension, outputName, ValueType.STRING);
}
@Override
@JsonProperty
public String getDimension()
@ -92,6 +104,13 @@ public class DefaultDimensionSpec implements DimensionSpec
return outputName;
}
@Override
@JsonProperty
public ValueType getOutputType()
{
return outputType;
}
@Override
public ExtractionFn getExtractionFn()
{
@ -104,6 +123,12 @@ public class DefaultDimensionSpec implements DimensionSpec
return selector;
}
@Override
public boolean mustDecorate()
{
return false;
}
@Override
public byte[] getCacheKey()
{
@ -127,6 +152,7 @@ public class DefaultDimensionSpec implements DimensionSpec
return "DefaultDimensionSpec{" +
"dimension='" + dimension + '\'' +
", outputName='" + outputName + '\'' +
", outputType='" + outputType + '\'' +
'}';
}
@ -149,6 +175,9 @@ public class DefaultDimensionSpec implements DimensionSpec
if (outputName != null ? !outputName.equals(that.outputName) : that.outputName != null) {
return false;
}
if (outputType != null ? !outputType.equals(that.outputType) : that.outputType != null) {
return false;
}
return true;
}
@ -158,6 +187,7 @@ public class DefaultDimensionSpec implements DimensionSpec
{
int result = dimension != null ? dimension.hashCode() : 0;
result = 31 * result + (outputName != null ? outputName.hashCode() : 0);
result = 31 * result + (outputType != null ? outputType.hashCode() : 0);
return result;
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import io.druid.query.extraction.ExtractionFn;
import io.druid.segment.DimensionSelector;
import io.druid.segment.column.ValueType;
/**
*/
@ -40,12 +41,19 @@ public interface DimensionSpec
String getOutputName();
ValueType getOutputType();
//ExtractionFn can be implemented with decorate(..) fn
@Deprecated
ExtractionFn getExtractionFn();
DimensionSelector decorate(DimensionSelector selector);
/**
* Does this DimensionSpec require that decorate() be called to produce correct results?
*/
boolean mustDecorate();
byte[] getCacheKey();
boolean preservesOrdering();

View File

@ -25,6 +25,7 @@ import com.google.common.base.Preconditions;
import io.druid.java.util.common.StringUtils;
import io.druid.query.extraction.ExtractionFn;
import io.druid.segment.DimensionSelector;
import io.druid.segment.column.ValueType;
import java.nio.ByteBuffer;
@ -37,11 +38,13 @@ public class ExtractionDimensionSpec implements DimensionSpec
private final String dimension;
private final ExtractionFn extractionFn;
private final String outputName;
private final ValueType outputType;
@JsonCreator
public ExtractionDimensionSpec(
@JsonProperty("dimension") String dimension,
@JsonProperty("outputName") String outputName,
@JsonProperty("outputType") ValueType outputType,
@JsonProperty("extractionFn") ExtractionFn extractionFn,
// for backwards compatibility
@Deprecated @JsonProperty("dimExtractionFn") ExtractionFn dimExtractionFn
@ -52,6 +55,7 @@ public class ExtractionDimensionSpec implements DimensionSpec
this.dimension = dimension;
this.extractionFn = extractionFn != null ? extractionFn : dimExtractionFn;
this.outputType = outputType == null ? ValueType.STRING : outputType;
// Do null check for backwards compatibility
this.outputName = outputName == null ? dimension : outputName;
@ -59,7 +63,12 @@ public class ExtractionDimensionSpec implements DimensionSpec
public ExtractionDimensionSpec(String dimension, String outputName, ExtractionFn extractionFn)
{
this(dimension, outputName, extractionFn, null);
this(dimension, outputName, null, extractionFn, null);
}
public ExtractionDimensionSpec(String dimension, String outputName, ValueType outputType, ExtractionFn extractionFn)
{
this(dimension, outputName, outputType, extractionFn, null);
}
@Override
@ -76,6 +85,13 @@ public class ExtractionDimensionSpec implements DimensionSpec
return outputName;
}
@Override
@JsonProperty
public ValueType getOutputType()
{
return outputType;
}
@Override
@JsonProperty
public ExtractionFn getExtractionFn()
@ -89,6 +105,12 @@ public class ExtractionDimensionSpec implements DimensionSpec
return selector;
}
@Override
public boolean mustDecorate()
{
return false;
}
@Override
public byte[] getCacheKey()
{
@ -115,6 +137,7 @@ public class ExtractionDimensionSpec implements DimensionSpec
"dimension='" + dimension + '\'' +
", extractionFn=" + extractionFn +
", outputName='" + outputName + '\'' +
", outputType='" + outputType + '\'' +
'}';
}
@ -139,6 +162,9 @@ public class ExtractionDimensionSpec implements DimensionSpec
if (outputName != null ? !outputName.equals(that.outputName) : that.outputName != null) {
return false;
}
if (outputType != null ? !outputType.equals(that.outputType) : that.outputType != null) {
return false;
}
return true;
}
@ -149,6 +175,7 @@ public class ExtractionDimensionSpec implements DimensionSpec
int result = dimension != null ? dimension.hashCode() : 0;
result = 31 * result + (extractionFn != null ? extractionFn.hashCode() : 0);
result = 31 * result + (outputName != null ? outputName.hashCode() : 0);
result = 31 * result + (outputType != null ? outputType.hashCode() : 0);
return result;
}
}

View File

@ -31,6 +31,7 @@ import io.druid.query.lookup.LookupExtractionFn;
import io.druid.query.lookup.LookupExtractor;
import io.druid.query.lookup.LookupReferencesManager;
import io.druid.segment.DimensionSelector;
import io.druid.segment.column.ValueType;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
@ -109,6 +110,13 @@ public class LookupDimensionSpec implements DimensionSpec
return outputName;
}
@Override
public ValueType getOutputType()
{
// Extraction functions always output String
return ValueType.STRING;
}
@JsonProperty
@Nullable
public LookupExtractor getLookup()
@ -149,6 +157,12 @@ public class LookupDimensionSpec implements DimensionSpec
return selector;
}
@Override
public boolean mustDecorate()
{
return false;
}
@Override
public byte[] getCacheKey()
{

View File

@ -19,12 +19,14 @@
package io.druid.query.extraction;
import java.util.Objects;
public abstract class DimExtractionFn implements ExtractionFn
{
@Override
public String apply(Object value)
{
return apply(value == null ? null : value.toString());
return apply(Objects.toString(value, null));
}
@Override

View File

@ -27,6 +27,7 @@ import com.google.common.collect.BoundType;
import com.google.common.collect.Range;
import com.google.common.collect.RangeSet;
import com.google.common.collect.TreeRangeSet;
import com.google.common.primitives.Floats;
import io.druid.common.guava.GuavaUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.query.extraction.ExtractionFn;
@ -47,6 +48,7 @@ public class BoundDimFilter implements DimFilter
private final ExtractionFn extractionFn;
private final StringComparator ordering;
private final Supplier<DruidLongPredicate> longPredicateSupplier;
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
@JsonCreator
public BoundDimFilter(
@ -87,6 +89,7 @@ public class BoundDimFilter implements DimFilter
}
this.extractionFn = extractionFn;
this.longPredicateSupplier = makeLongPredicateSupplier();
this.floatPredicateSupplier = makeFloatPredicateSupplier();
}
@JsonProperty
@ -146,6 +149,10 @@ public class BoundDimFilter implements DimFilter
return longPredicateSupplier;
}
public Supplier<DruidFloatPredicate> getFloatPredicateSupplier() {
return floatPredicateSupplier;
}
@Override
public byte[] getCacheKey()
{
@ -260,7 +267,6 @@ public class BoundDimFilter implements DimFilter
return false;
}
return getOrdering().equals(that.getOrdering());
}
@Override
@ -317,89 +323,45 @@ public class BoundDimFilter implements DimFilter
class BoundLongPredicateSupplier implements Supplier<DruidLongPredicate>
{
private final Object initLock = new Object();
// longsInitialized is volatile since it establishes the happens-before relationship on
// writes/reads to the rest of the fields (it's written last and read first).
private volatile boolean longsInitialized = false;
// Other fields are not volatile.
private boolean matchesNothing;
private boolean hasLowerLongBound;
private boolean hasUpperLongBound;
private long lowerLongBound;
private long upperLongBound;
private DruidLongPredicate predicate;
@Override
public DruidLongPredicate get()
{
initLongData();
if (matchesNothing) {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return false;
}
};
}
return new DruidLongPredicate()
{
private final boolean hasLowerLongBound = BoundLongPredicateSupplier.this.hasLowerLongBound;
private final boolean hasUpperLongBound = BoundLongPredicateSupplier.this.hasUpperLongBound;
private final long lowerLongBound = hasLowerLongBound ? BoundLongPredicateSupplier.this.lowerLongBound : 0L;
private final long upperLongBound = hasUpperLongBound ? BoundLongPredicateSupplier.this.upperLongBound : 0L;
@Override
public boolean applyLong(long input)
{
int lowerComparing = 1;
int upperComparing = 1;
if (hasLowerLongBound) {
lowerComparing = Long.compare(input, lowerLongBound);
}
if (hasUpperLongBound) {
upperComparing = Long.compare(upperLongBound, input);
}
if (lowerStrict && upperStrict) {
return ((lowerComparing > 0)) && (upperComparing > 0);
} else if (lowerStrict) {
return (lowerComparing > 0) && (upperComparing >= 0);
} else if (upperStrict) {
return (lowerComparing >= 0) && (upperComparing > 0);
}
return (lowerComparing >= 0) && (upperComparing >= 0);
}
};
initPredicate();
return predicate;
}
private void initLongData()
private void initPredicate()
{
if (longsInitialized) {
if (predicate != null) {
return;
}
synchronized (initLock) {
if (longsInitialized) {
if (predicate != null) {
return;
}
matchesNothing = false;
final boolean hasLowerLongBound;
final boolean hasUpperLongBound;
final long lowerLongBound;
final long upperLongBound;
boolean matchesNothing = false;
if (hasLowerBound()) {
final Long lowerLong = GuavaUtils.tryParseLong(lower);
if (lowerLong == null) {
// Unparseable values fall before all actual numbers, so all numbers will match the lower bound.
hasLowerLongBound = false;
lowerLongBound = 0L;
} else {
hasLowerLongBound = true;
lowerLongBound = lowerLong;
}
} else {
hasLowerLongBound = false;
lowerLongBound = 0L;
}
if (hasUpperBound()) {
@ -407,19 +369,320 @@ public class BoundDimFilter implements DimFilter
if (upperLong == null) {
// Unparseable values fall before all actual numbers, so no numbers can match the upper bound.
matchesNothing = true;
return;
hasUpperLongBound = false;
upperLongBound = 0L;
} else {
hasUpperLongBound = true;
upperLongBound = upperLong;
}
hasUpperLongBound = true;
upperLongBound = upperLong;
} else {
hasUpperLongBound = false;
upperLongBound = 0L;
}
longsInitialized = true;
if (matchesNothing) {
predicate = DruidLongPredicate.ALWAYS_FALSE;
} else {
predicate = makeLongPredicateFromBounds(
hasLowerLongBound,
hasUpperLongBound,
lowerStrict,
upperStrict,
lowerLongBound,
upperLongBound
);
}
}
}
}
return new BoundLongPredicateSupplier();
}
private Supplier<DruidFloatPredicate> makeFloatPredicateSupplier()
{
class BoundFloatPredicateSupplier implements Supplier<DruidFloatPredicate>
{
private final Object initLock = new Object();
private DruidFloatPredicate predicate;
@Override
public DruidFloatPredicate get()
{
initPredicate();
return predicate;
}
private void initPredicate()
{
if (predicate != null) {
return;
}
synchronized (initLock) {
if (predicate != null) {
return;
}
final boolean hasLowerFloatBound;
final boolean hasUpperFloatBound;
final float lowerFloatBound;
final float upperFloatBound;
boolean matchesNothing = false;
if (hasLowerBound()) {
final Float lowerFloat = Floats.tryParse(lower);
if (lowerFloat == null) {
// Unparseable values fall before all actual numbers, so all numbers will match the lower bound.
hasLowerFloatBound = false;
lowerFloatBound = 0L;
} else {
hasLowerFloatBound = true;
lowerFloatBound = lowerFloat;
}
} else {
hasLowerFloatBound = false;
lowerFloatBound = 0L;
}
if (hasUpperBound()) {
Float upperFloat = Floats.tryParse(upper);
if (upperFloat == null) {
// Unparseable values fall before all actual numbers, so no numbers can match the upper bound.
matchesNothing = true;
hasUpperFloatBound = false;
upperFloatBound = 0L;
} else {
hasUpperFloatBound = true;
upperFloatBound = upperFloat;
}
} else {
hasUpperFloatBound = false;
upperFloatBound = 0L;
}
if (matchesNothing) {
predicate = DruidFloatPredicate.ALWAYS_FALSE;
} else {
predicate = makeFloatPredicateFromBounds(
hasLowerFloatBound,
hasUpperFloatBound,
lowerStrict,
upperStrict,
lowerFloatBound,
upperFloatBound
);
}
}
}
}
return new BoundFloatPredicateSupplier();
}
private static DruidLongPredicate makeLongPredicateFromBounds(
final boolean hasLowerLongBound,
final boolean hasUpperLongBound,
final boolean lowerStrict,
final boolean upperStrict,
final long lowerLongBound,
final long upperLongBound
)
{
if (hasLowerLongBound && hasUpperLongBound) {
if (upperStrict && lowerStrict) {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
final int lowerComparing = Long.compare(input, lowerLongBound);
final int upperComparing = Long.compare(upperLongBound, input);
return ((lowerComparing > 0)) && (upperComparing > 0);
}
};
} else if (lowerStrict) {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
final int lowerComparing = Long.compare(input, lowerLongBound);
final int upperComparing = Long.compare(upperLongBound, input);
return (lowerComparing > 0) && (upperComparing >= 0);
}
};
} else if (upperStrict) {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
final int lowerComparing = Long.compare(input, lowerLongBound);
final int upperComparing = Long.compare(upperLongBound, input);
return (lowerComparing >= 0) && (upperComparing > 0);
}
};
} else {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
final int lowerComparing = Long.compare(input, lowerLongBound);
final int upperComparing = Long.compare(upperLongBound, input);
return (lowerComparing >= 0) && (upperComparing >= 0);
}
};
}
} else if (hasUpperLongBound) {
if (upperStrict) {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
final int upperComparing = Long.compare(upperLongBound, input);
return upperComparing > 0;
}
};
} else {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
final int upperComparing = Long.compare(upperLongBound, input);
return upperComparing >= 0;
}
};
}
} else if (hasLowerLongBound) {
if (lowerStrict) {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
final int lowerComparing = Long.compare(input, lowerLongBound);
return lowerComparing > 0;
}
};
} else {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
final int lowerComparing = Long.compare(input, lowerLongBound);
return lowerComparing >= 0;
}
};
}
} else {
return DruidLongPredicate.ALWAYS_TRUE;
}
}
private static DruidFloatPredicate makeFloatPredicateFromBounds(
final boolean hasLowerFloatBound,
final boolean hasUpperFloatBound,
final boolean lowerStrict,
final boolean upperStrict,
final float lowerFloatBound,
final float upperFloatBound
)
{
if (hasLowerFloatBound && hasUpperFloatBound) {
if (upperStrict && lowerStrict) {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
final int lowerComparing = Float.compare(input, lowerFloatBound);
final int upperComparing = Float.compare(upperFloatBound, input);
return ((lowerComparing > 0)) && (upperComparing > 0);
}
};
} else if (lowerStrict) {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
final int lowerComparing = Float.compare(input, lowerFloatBound);
final int upperComparing = Float.compare(upperFloatBound, input);
return (lowerComparing > 0) && (upperComparing >= 0);
}
};
} else if (upperStrict) {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
final int lowerComparing = Float.compare(input, lowerFloatBound);
final int upperComparing = Float.compare(upperFloatBound, input);
return (lowerComparing >= 0) && (upperComparing > 0);
}
};
} else {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
final int lowerComparing = Float.compare(input, lowerFloatBound);
final int upperComparing = Float.compare(upperFloatBound, input);
return (lowerComparing >= 0) && (upperComparing >= 0);
}
};
}
} else if (hasUpperFloatBound) {
if (upperStrict) {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
final int upperComparing = Float.compare(upperFloatBound, input);
return upperComparing > 0;
}
};
} else {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
final int upperComparing = Float.compare(upperFloatBound, input);
return upperComparing >= 0;
}
};
}
} else if (hasLowerFloatBound) {
if (lowerStrict) {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
final int lowerComparing = Float.compare(input, lowerFloatBound);
return lowerComparing > 0;
}
};
} else {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
final int lowerComparing = Float.compare(input, lowerFloatBound);
return lowerComparing >= 0;
}
};
}
} else {
return DruidFloatPredicate.ALWAYS_TRUE;
}
}
}

View File

@ -0,0 +1,47 @@
/*
* 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.query.filter;
/**
* FloatPredicate is only supported in Java 8+, so use this to avoid boxing when a float predicate is needed.
*/
public interface DruidFloatPredicate
{
DruidFloatPredicate ALWAYS_FALSE = new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return false;
}
};
DruidFloatPredicate ALWAYS_TRUE = new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return true;
}
};
boolean applyFloat(float input);
}

View File

@ -24,5 +24,23 @@ package io.druid.query.filter;
*/
public interface DruidLongPredicate
{
DruidLongPredicate ALWAYS_FALSE = new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return false;
}
};
DruidLongPredicate ALWAYS_TRUE = new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return true;
}
};
boolean applyLong(long input);
}

View File

@ -26,4 +26,6 @@ public interface DruidPredicateFactory
public Predicate<String> makeStringPredicate();
public DruidLongPredicate makeLongPredicate();
public DruidFloatPredicate makeFloatPredicate();
}

View File

@ -0,0 +1,62 @@
/*
* 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.query.filter;
import io.druid.segment.DimensionHandlerUtils;
import io.druid.segment.FloatColumnSelector;
import io.druid.segment.filter.BooleanValueMatcher;
public class FloatValueMatcherColumnSelectorStrategy implements ValueMatcherColumnSelectorStrategy<FloatColumnSelector>
{
@Override
public ValueMatcher makeValueMatcher(final FloatColumnSelector selector, final String value)
{
final Float matchVal = DimensionHandlerUtils.convertObjectToFloat(value);
if (matchVal == null) {
return BooleanValueMatcher.of(false);
}
final int matchValIntBits = Float.floatToIntBits(matchVal);
return new ValueMatcher()
{
@Override
public boolean matches()
{
return Float.floatToIntBits(selector.get()) == matchValIntBits;
}
};
}
@Override
public ValueMatcher makeValueMatcher(
final FloatColumnSelector selector, DruidPredicateFactory predicateFactory
)
{
final DruidFloatPredicate predicate = predicateFactory.makeFloatPredicate();
return new ValueMatcher()
{
@Override
public boolean matches()
{
return predicate.applyFloat(selector.get());
}
};
}
}

View File

@ -31,18 +31,22 @@ import com.google.common.collect.Iterables;
import com.google.common.collect.Range;
import com.google.common.collect.RangeSet;
import com.google.common.collect.TreeRangeSet;
import com.google.common.primitives.Floats;
import io.druid.common.guava.GuavaUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.lookup.LookupExtractionFn;
import io.druid.query.lookup.LookupExtractor;
import io.druid.segment.filter.InFilter;
import it.unimi.dsi.fastutil.ints.IntArrayList;
import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
import it.unimi.dsi.fastutil.longs.LongArrayList;
import it.unimi.dsi.fastutil.longs.LongOpenHashSet;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Set;
@ -51,12 +55,13 @@ public class InDimFilter implements DimFilter
{
// determined through benchmark that binary search on long[] is faster than HashSet until ~16 elements
// Hashing threshold is not applied to String for now, String still uses ImmutableSortedSet
public static final int LONG_HASHING_THRESHOLD = 16;
public static final int NUMERIC_HASHING_THRESHOLD = 16;
private final ImmutableSortedSet<String> values;
private final String dimension;
private final ExtractionFn extractionFn;
private final Supplier<DruidLongPredicate> longPredicateSupplier;
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
@JsonCreator
public InDimFilter(
@ -83,6 +88,7 @@ public class InDimFilter implements DimFilter
this.dimension = dimension;
this.extractionFn = extractionFn;
this.longPredicateSupplier = getLongPredicateSupplier();
this.floatPredicateSupplier = getFloatPredicateSupplier();
}
@JsonProperty
@ -182,7 +188,7 @@ public class InDimFilter implements DimFilter
@Override
public Filter toFilter()
{
return new InFilter(dimension, values, longPredicateSupplier, extractionFn);
return new InFilter(dimension, values, longPredicateSupplier, floatPredicateSupplier, extractionFn);
}
@Override
@ -258,23 +264,21 @@ public class InDimFilter implements DimFilter
return new Supplier<DruidLongPredicate>()
{
private final Object initLock = new Object();
private volatile boolean longsInitialized = false;
private volatile boolean useLongHash;
private volatile long[] longArray;
private volatile HashSet<Long> longHashSet;
private DruidLongPredicate predicate;
private void initLongValues()
{
if (longsInitialized) {
if (predicate != null) {
return;
}
synchronized (initLock) {
if (longsInitialized) {
if (predicate != null) {
return;
}
List<Long> longs = new ArrayList<>();
LongArrayList longs = new LongArrayList(values.size());
for (String value : values) {
Long longValue = GuavaUtils.tryParseLong(value);
if (longValue != null) {
@ -282,18 +286,30 @@ public class InDimFilter implements DimFilter
}
}
useLongHash = longs.size() > LONG_HASHING_THRESHOLD;
if (useLongHash) {
longHashSet = new HashSet<Long>(longs);
} else {
longArray = new long[longs.size()];
for (int i = 0; i < longs.size(); i++) {
longArray[i] = longs.get(i).longValue();
}
Arrays.sort(longArray);
}
if (longs.size() > NUMERIC_HASHING_THRESHOLD) {
final LongOpenHashSet longHashSet = new LongOpenHashSet(longs);
longsInitialized = true;
predicate = new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return longHashSet.contains(input);
}
};
} else {
final long[] longArray = longs.toLongArray();
Arrays.sort(longArray);
predicate = new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return Arrays.binarySearch(longArray, input) >= 0;
}
};
}
}
}
@ -301,26 +317,69 @@ public class InDimFilter implements DimFilter
public DruidLongPredicate get()
{
initLongValues();
return predicate;
}
};
}
if (useLongHash) {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return longHashSet.contains(input);
}
};
} else {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return Arrays.binarySearch(longArray, input) >= 0;
}
};
private Supplier<DruidFloatPredicate> getFloatPredicateSupplier()
{
return new Supplier<DruidFloatPredicate>()
{
private final Object initLock = new Object();
private DruidFloatPredicate predicate;
private void initFloatValues()
{
if (predicate != null) {
return;
}
synchronized (initLock) {
if (predicate != null) {
return;
}
IntArrayList floatBits = new IntArrayList(values.size());
for (String value : values) {
Float floatValue = Floats.tryParse(value);
if (floatValue != null) {
floatBits.add(Float.floatToIntBits(floatValue));
}
}
if (floatBits.size() > NUMERIC_HASHING_THRESHOLD) {
final IntOpenHashSet floatBitsHashSet = new IntOpenHashSet(floatBits);
predicate = new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return floatBitsHashSet.contains(Float.floatToIntBits(input));
}
};
} else {
final int[] floatBitsArray = floatBits.toIntArray();
Arrays.sort(floatBitsArray);
predicate = new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return Arrays.binarySearch(floatBitsArray, Float.floatToIntBits(input)) >= 0;
}
};
}
}
}
@Override
public DruidFloatPredicate get()
{
initFloatValues();
return predicate;
}
};
}

View File

@ -217,6 +217,20 @@ public class JavaScriptDimFilter implements DimFilter
};
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
// Can't avoid boxing here because the Mozilla JS Function.call() only accepts Object[]
return applyObject(input);
}
};
}
public boolean applyObject(final Object input)
{
// one and only one context per thread

View File

@ -224,6 +224,30 @@ public class LikeDimFilter implements DimFilter
};
}
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
if (extractionFn != null) {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return matches(extractionFn.apply(input));
}
};
} else {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return matches(String.valueOf(input));
}
};
}
}
};
}

View File

@ -0,0 +1,61 @@
/*
* 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.query.filter;
import io.druid.segment.DimensionHandlerUtils;
import io.druid.segment.LongColumnSelector;
import io.druid.segment.filter.BooleanValueMatcher;
public class LongValueMatcherColumnSelectorStrategy implements ValueMatcherColumnSelectorStrategy<LongColumnSelector>
{
@Override
public ValueMatcher makeValueMatcher(final LongColumnSelector selector, final String value)
{
final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value);
if (matchVal == null) {
return BooleanValueMatcher.of(false);
}
final long matchValLong = matchVal;
return new ValueMatcher()
{
@Override
public boolean matches()
{
return selector.get() == matchValLong;
}
};
}
@Override
public ValueMatcher makeValueMatcher(
final LongColumnSelector selector, DruidPredicateFactory predicateFactory
)
{
final DruidLongPredicate predicate = predicateFactory.makeLongPredicate();
return new ValueMatcher()
{
@Override
public boolean matches()
{
return predicate.applyLong(selector.get());
}
};
}
}

View File

@ -23,11 +23,13 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.base.Predicates;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Range;
import com.google.common.collect.RangeSet;
import com.google.common.collect.TreeRangeSet;
import com.google.common.primitives.Floats;
import io.druid.common.guava.GuavaUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.query.extraction.ExtractionFn;
@ -46,9 +48,9 @@ public class SelectorDimFilter implements DimFilter
private final ExtractionFn extractionFn;
private final Object initLock = new Object();
private volatile boolean longsInitialized = false;
private volatile Long valueAsLong;
private DruidLongPredicate longPredicate;
private DruidFloatPredicate floatPredicate;
@JsonCreator
public SelectorDimFilter(
@ -100,42 +102,21 @@ public class SelectorDimFilter implements DimFilter
@Override
public Predicate<String> makeStringPredicate()
{
return new Predicate<String>()
{
@Override
public boolean apply(String input)
{
return Objects.equals(valueOrNull, input);
}
};
return Predicates.equalTo(valueOrNull);
}
@Override
public DruidLongPredicate makeLongPredicate()
{
initLongValue();
initLongPredicate();
return longPredicate;
}
if (valueAsLong == null) {
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return false;
}
};
} else {
// store the primitive, so we don't unbox for every comparison
final long unboxedLong = valueAsLong.longValue();
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return input == unboxedLong;
}
};
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
initFloatPredicate();
return floatPredicate;
}
};
return new DimensionPredicateFilter(dimension, predicateFactory, extractionFn);
@ -212,17 +193,57 @@ public class SelectorDimFilter implements DimFilter
}
private void initLongValue()
private void initLongPredicate()
{
if (longsInitialized) {
if (longPredicate != null) {
return;
}
synchronized (initLock) {
if (longsInitialized) {
if (longPredicate != null) {
return;
}
valueAsLong = GuavaUtils.tryParseLong(value);
longsInitialized = true;
final Long valueAsLong = GuavaUtils.tryParseLong(value);
if (valueAsLong == null) {
longPredicate = DruidLongPredicate.ALWAYS_FALSE;
} else {
// store the primitive, so we don't unbox for every comparison
final long unboxedLong = valueAsLong.longValue();
longPredicate = new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
return input == unboxedLong;
}
};
}
}
}
private void initFloatPredicate()
{
if (floatPredicate != null) {
return;
}
synchronized (initLock) {
if (floatPredicate != null) {
return;
}
final Float valueAsFloat = Floats.tryParse(value);
if (valueAsFloat == null) {
floatPredicate = DruidFloatPredicate.ALWAYS_FALSE;
} else {
final int floatBits = Float.floatToIntBits(valueAsFloat);
floatPredicate = new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return Float.floatToIntBits(input) == floatBits;
}
};
}
}
}
}

View File

@ -21,6 +21,7 @@ package io.druid.query.filter;
import io.druid.java.util.common.IAE;
import io.druid.query.dimension.ColumnSelectorStrategyFactory;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ValueType;
@ -41,15 +42,19 @@ public class ValueMatcherColumnSelectorStrategyFactory
@Override
public ValueMatcherColumnSelectorStrategy makeColumnSelectorStrategy(
ColumnCapabilities capabilities
ColumnCapabilities capabilities, ColumnValueSelector selector
)
{
ValueType type = capabilities.getType();
switch (type) {
case STRING:
return new StringValueMatcherColumnSelectorStrategy();
case LONG:
return new LongValueMatcherColumnSelectorStrategy();
case FLOAT:
return new FloatValueMatcherColumnSelectorStrategy();
default:
throw new IAE("Cannot create query type helper from invalid type [%s]", type);
throw new IAE("Cannot create column selector strategy from invalid type [%s]", type);
}
}
}

View File

@ -49,6 +49,7 @@ import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector;
import io.druid.segment.StorageAdapter;
import io.druid.segment.VirtualColumns;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.filter.Filters;
import org.joda.time.DateTime;
@ -326,8 +327,18 @@ public class GroupByQueryEngine
for (int i = 0; i < dimensionSpecs.size(); ++i) {
final DimensionSpec dimSpec = dimensionSpecs.get(i);
if (dimSpec.getOutputType() != ValueType.STRING) {
throw new UnsupportedOperationException(
"GroupBy v1 only supports dimensions with an outputType of STRING."
);
}
final DimensionSelector selector = cursor.makeDimensionSelector(dimSpec);
if (selector != null) {
if (selector.getValueCardinality() == DimensionSelector.CARDINALITY_UNKNOWN) {
throw new UnsupportedOperationException(
"GroupBy v1 does not support dimension selectors with unknown cardinality.");
}
dimensions.add(selector);
dimNames.add(dimSpec.getOutputName());
}

View File

@ -237,7 +237,7 @@ public class GroupByQueryHelper
final ImmutableMap.Builder<String, ValueType> types = ImmutableMap.builder();
for (DimensionSpec dimensionSpec : query.getDimensions()) {
types.put(dimensionSpec.getOutputName(), ValueType.STRING);
types.put(dimensionSpec.getOutputName(), dimensionSpec.getOutputType());
}
for (AggregatorFactory aggregatorFactory : query.getAggregatorSpecs()) {

View File

@ -179,6 +179,7 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner
Pair<Grouper<RowBasedKey>, Accumulator<Grouper<RowBasedKey>, Row>> pair = RowBasedGrouperHelper.createGrouperAccumulatorPair(
query,
false,
null,
config,
mergeBufferHolder.get(),
concurrencyHint,

View File

@ -22,7 +22,6 @@ package io.druid.query.groupby.epinephelinae;
import com.google.common.base.Function;
import com.google.common.base.Strings;
import com.google.common.collect.Maps;
import com.google.common.primitives.Ints;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidPool;
import io.druid.data.input.MapBasedRow;
@ -35,10 +34,15 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.query.ColumnSelectorPlus;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.dimension.ColumnSelectorStrategy;
import io.druid.query.dimension.ColumnSelectorStrategyFactory;
import io.druid.query.groupby.GroupByQuery;
import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.epinephelinae.column.DictionaryBuildingStringGroupByColumnSelectorStrategy;
import io.druid.query.groupby.epinephelinae.column.FloatGroupByColumnSelectorStrategy;
import io.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorPlus;
import io.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorStrategy;
import io.druid.query.groupby.epinephelinae.column.LongGroupByColumnSelectorStrategy;
import io.druid.query.groupby.epinephelinae.column.StringGroupByColumnSelectorStrategy;
import io.druid.query.groupby.strategy.GroupByStrategyV2;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.Cursor;
@ -48,7 +52,6 @@ import io.druid.segment.StorageAdapter;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ValueType;
import io.druid.segment.VirtualColumns;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.filter.Filters;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -174,173 +177,28 @@ public class GroupByQueryEngineV2
{
@Override
public GroupByColumnSelectorStrategy makeColumnSelectorStrategy(
ColumnCapabilities capabilities
ColumnCapabilities capabilities, ColumnValueSelector selector
)
{
ValueType type = capabilities.getType();
switch(type) {
case STRING:
return new StringGroupByColumnSelectorStrategy();
DimensionSelector dimSelector = (DimensionSelector) selector;
if (dimSelector.getValueCardinality() >= 0) {
return new StringGroupByColumnSelectorStrategy();
} else {
return new DictionaryBuildingStringGroupByColumnSelectorStrategy();
}
case LONG:
return new LongGroupByColumnSelectorStrategy();
case FLOAT:
return new FloatGroupByColumnSelectorStrategy();
default:
throw new IAE("Cannot create query type helper from invalid type [%s]", type);
}
}
}
/**
* Contains a collection of query processing methods for type-specific operations used exclusively by
* GroupByQueryEngineV2.
*
* Each GroupByColumnSelectorStrategy is associated with a single dimension.
*/
private interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy
{
/**
* Return the size, in bytes, of this dimension's values in the grouping key.
*
* For example, a String implementation would return 4, the size of an int.
*
* @return size, in bytes, of this dimension's values in the grouping key.
*/
int getGroupingKeySize();
/**
* Read a value from a grouping key and add it to the group by query result map, using the output name specified
* in a DimensionSpec.
*
* An implementation may choose to not add anything to the result map
* (e.g., as the String implementation does for empty rows)
*
* selectorPlus provides access to:
* - the keyBufferPosition offset from which to read the value
* - the dimension value selector
* - the DimensionSpec for this dimension from the query
*
* @param selectorPlus dimension info containing the key offset, value selector, and dimension spec
* @param resultMap result map for the group by query being served
* @param key grouping key
*/
void processValueFromGroupingKey(
GroupByColumnSelectorPlus selectorPlus,
ByteBuffer key,
Map<String, Object> resultMap
);
/**
* Retrieve a row object from the ColumnSelectorPlus and put it in valuess at columnIndex.
*
* @param selector Value selector for a column.
* @param columnIndex Index of the column within the row values array
* @param valuess Row values array, one index per column
*/
void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess);
/**
* Read the first value within a row values object (IndexedInts, IndexedLongs, etc.) and write that value
* to the keyBuffer at keyBufferPosition. If rowSize is 0, write GROUP_BY_MISSING_VALUE instead.
*
* If the size of the row is > 0, write 1 to stack[] at columnIndex, otherwise write 0.
*
* @param keyBufferPosition Starting offset for this column's value within the grouping key.
* @param columnIndex Index of the column within the row values array
* @param rowObj Row value object for this column (e.g., IndexedInts)
* @param keyBuffer grouping key
* @param stack array containing the current within-row value index for each column
*/
void initGroupingKeyColumnValue(int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack);
/**
* If rowValIdx is less than the size of rowObj (haven't handled all of the row values):
* First, read the value at rowValIdx from a rowObj and write that value to the keyBuffer at keyBufferPosition.
* Then return true
*
* Otherwise, return false.
*
* @param keyBufferPosition Starting offset for this column's value within the grouping key.
* @param rowObj Row value object for this column (e.g., IndexedInts)
* @param rowValIdx Index of the current value being grouped on within the row
* @param keyBuffer grouping key
* @return true if rowValIdx < size of rowObj, false otherwise
*/
boolean checkRowIndexAndAddValueToGroupingKey(int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer);
}
private static class StringGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy
{
private static final int GROUP_BY_MISSING_VALUE = -1;
@Override
public int getGroupingKeySize()
{
return Ints.BYTES;
}
@Override
public void processValueFromGroupingKey(GroupByColumnSelectorPlus selectorPlus, ByteBuffer key, Map<String, Object> resultMap)
{
final int id = key.getInt(selectorPlus.getKeyBufferPosition());
// GROUP_BY_MISSING_VALUE is used to indicate empty rows, which are omitted from the result map.
if (id != GROUP_BY_MISSING_VALUE) {
resultMap.put(
selectorPlus.getOutputName(),
((DimensionSelector) selectorPlus.getSelector()).lookupName(id)
);
} else {
resultMap.put(selectorPlus.getOutputName(), "");
}
}
@Override
public void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess)
{
DimensionSelector dimSelector = (DimensionSelector) selector;
IndexedInts row = dimSelector.getRow();
valuess[columnIndex] = row;
}
@Override
public void initGroupingKeyColumnValue(int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack)
{
IndexedInts row = (IndexedInts) rowObj;
int rowSize = row.size();
initializeGroupingKeyV2Dimension(row, rowSize, keyBuffer, keyBufferPosition);
stack[columnIndex] = rowSize == 0 ? 0 : 1;
}
@Override
public boolean checkRowIndexAndAddValueToGroupingKey(int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer)
{
IndexedInts row = (IndexedInts) rowObj;
int rowSize = row.size();
if (rowValIdx < rowSize) {
keyBuffer.putInt(
keyBufferPosition,
row.get(rowValIdx)
);
return true;
} else {
return false;
}
}
private void initializeGroupingKeyV2Dimension(
final IndexedInts values,
final int rowSize,
final ByteBuffer keyBuffer,
final int keyBufferPosition
)
{
if (rowSize == 0) {
keyBuffer.putInt(keyBufferPosition, GROUP_BY_MISSING_VALUE);
} else {
keyBuffer.putInt(keyBufferPosition, values.get(0));
}
}
}
private static class GroupByEngineIterator implements Iterator<Row>, Closeable
{
private final GroupByQuery query;
@ -594,28 +452,4 @@ outer:
// No state, nothing to reset
}
}
private static class GroupByColumnSelectorPlus extends ColumnSelectorPlus<GroupByColumnSelectorStrategy>
{
/**
* Indicates the offset of this dimension's value within the grouping key.
*/
private int keyBufferPosition;
public GroupByColumnSelectorPlus(ColumnSelectorPlus<GroupByColumnSelectorStrategy> baseInfo, int keyBufferPosition)
{
super(
baseInfo.getName(),
baseInfo.getOutputName(),
baseInfo.getColumnSelectorStrategy(),
baseInfo.getSelector()
);
this.keyBufferPosition = keyBufferPosition;
}
public int getKeyBufferPosition()
{
return keyBufferPosition;
}
}
}

View File

@ -156,6 +156,7 @@ public class GroupByRowProcessor
Pair<Grouper<RowBasedKey>, Accumulator<Grouper<RowBasedKey>, Row>> pair = RowBasedGrouperHelper.createGrouperAccumulatorPair(
query,
true,
rowSignature,
querySpecificConfig,
mergeBufferHolder.get(),
-1,

View File

@ -31,12 +31,28 @@ public class Groupers
// No instantiation
}
private static final int C1 = 0xcc9e2d51;
private static final int C2 = 0x1b873593;
/*
* This method was rewritten in Java from an intermediate step of the Murmur hash function in
* https://github.com/aappleby/smhasher/blob/master/src/MurmurHash3.cpp, which contained the
* following header:
*
* MurmurHash3 was written by Austin Appleby, and is placed in the public domain. The author
* hereby disclaims copyright to this source code.
*/
static int smear(int hashCode) {
return C2 * Integer.rotateLeft(hashCode * C1, 15);
}
public static int hash(final Object obj)
{
// Mask off the high bit so we can use that to determine if a bucket is used or not.
// Also apply the same XOR transformation that j.u.HashMap applies, to improve distribution.
// Also apply the smear function, to improve distribution.
final int code = obj.hashCode();
return (code ^ (code >>> 16)) & 0x7fffffff;
return smear(code) & 0x7fffffff;
}
public static <KeyType> Iterator<Grouper.Entry<KeyType>> mergeIterators(

View File

@ -25,30 +25,39 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.base.Supplier;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.primitives.Chars;
import com.google.common.primitives.Floats;
import com.google.common.primitives.Ints;
import com.google.common.primitives.Longs;
import io.druid.data.input.MapBasedRow;
import io.druid.data.input.Row;
import io.druid.granularity.AllGranularity;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.Pair;
import io.druid.java.util.common.guava.Accumulator;
import io.druid.query.QueryInterruptedException;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.groupby.GroupByQuery;
import io.druid.query.groupby.GroupByQueryConfig;
import io.druid.query.groupby.GroupByQueryHelper;
import io.druid.query.groupby.RowBasedColumnSelectorFactory;
import io.druid.query.groupby.strategy.GroupByStrategyV2;
import io.druid.segment.ColumnSelectorFactory;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.DimensionHandlerUtils;
import io.druid.segment.DimensionSelector;
import io.druid.segment.FloatColumnSelector;
import io.druid.segment.LongColumnSelector;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.IndexedInts;
import org.joda.time.DateTime;
import java.io.Closeable;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
@ -57,10 +66,15 @@ import java.util.Map;
// this class contains shared code between GroupByMergingQueryRunnerV2 and GroupByRowProcessor
public class RowBasedGrouperHelper
{
/**
* If isInputRaw is true, transformations such as timestamp truncation and extraction functions have not
* been applied to the input rows yet, for example, in a nested query, if an extraction function is being
* applied in the outer query to a field of the inner query. This method must apply those transformations.
*/
public static Pair<Grouper<RowBasedKey>, Accumulator<Grouper<RowBasedKey>, Row>> createGrouperAccumulatorPair(
final GroupByQuery query,
final boolean isInputRaw,
final Map<String, ValueType> rawInputRowSignature,
final GroupByQueryConfig config,
final ByteBuffer buffer,
final int concurrencyHint,
@ -72,18 +86,21 @@ public class RowBasedGrouperHelper
// concurrencyHint >= 1 for concurrent groupers, -1 for single-threaded
Preconditions.checkArgument(concurrencyHint >= 1 || concurrencyHint == -1, "invalid concurrencyHint");
final List<ValueType> valueTypes = DimensionHandlerUtils.getValueTypesFromDimensionSpecs(query.getDimensions());
final GroupByQueryConfig querySpecificConfig = config.withOverrides(query);
final boolean includeTimestamp = GroupByStrategyV2.getUniversalTimestamp(query) == null;
final Grouper.KeySerdeFactory<RowBasedKey> keySerdeFactory = new RowBasedKeySerdeFactory(
includeTimestamp,
query.getContextSortByDimsFirst(),
query.getDimensions().size(),
querySpecificConfig.getMaxMergingDictionarySize() / (concurrencyHint == -1 ? 1 : concurrencyHint)
querySpecificConfig.getMaxMergingDictionarySize() / (concurrencyHint == -1 ? 1 : concurrencyHint),
valueTypes
);
final ThreadLocal<Row> columnSelectorRow = new ThreadLocal<>();
final ColumnSelectorFactory columnSelectorFactory = RowBasedColumnSelectorFactory.create(
columnSelectorRow,
GroupByQueryHelper.rowSignatureFor(query)
rawInputRowSignature
);
final Grouper<RowBasedKey> grouper;
if (concurrencyHint == -1) {
@ -114,14 +131,15 @@ public class RowBasedGrouperHelper
);
}
final DimensionSelector[] dimensionSelectors;
final Supplier[] inputRawSuppliers;
if (isInputRaw) {
dimensionSelectors = new DimensionSelector[query.getDimensions().size()];
for (int i = 0; i < dimensionSelectors.length; i++) {
dimensionSelectors[i] = columnSelectorFactory.makeDimensionSelector(query.getDimensions().get(i));
}
inputRawSuppliers = getValueSuppliersForDimensions(
columnSelectorFactory,
query.getDimensions(),
rawInputRowSignature
);
} else {
dimensionSelectors = null;
inputRawSuppliers = null;
}
final Accumulator<Grouper<RowBasedKey>, Row> accumulator = new Accumulator<Grouper<RowBasedKey>, Row>()
@ -168,14 +186,34 @@ public class RowBasedGrouperHelper
}
for (int i = dimStart; i < key.length; i++) {
final String value;
final ValueType type = valueTypes.get(i - dimStart);
Object valObj;
if (isInputRaw) {
IndexedInts index = dimensionSelectors[i - dimStart].getRow();
value = index.size() == 0 ? "" : dimensionSelectors[i - dimStart].lookupName(index.get(0));
valObj = inputRawSuppliers[i - dimStart].get();
} else {
value = (String) row.getRaw(query.getDimensions().get(i - dimStart).getOutputName());
valObj = row.getRaw(query.getDimensions().get(i - dimStart).getOutputName());
}
key[i] = Strings.nullToEmpty(value);
// convert values to the output type specified by the DimensionSpec, for merging purposes
switch (type) {
case STRING:
valObj = valObj == null ? "" : valObj.toString();
break;
case LONG:
valObj = DimensionHandlerUtils.convertObjectToLong(valObj);
if (valObj == null) {
valObj = 0L;
}
break;
case FLOAT:
valObj = DimensionHandlerUtils.convertObjectToFloat(valObj);
if (valObj == null) {
valObj = 0.0f;
}
break;
default:
throw new IAE("invalid type: [%s]", type);
}
key[i] = (Comparable) valObj;
}
final boolean didAggregate = theGrouper.aggregate(new RowBasedKey(key));
@ -224,9 +262,10 @@ public class RowBasedGrouperHelper
// Add dimensions.
for (int i = dimStart; i < entry.getKey().getKey().length; i++) {
Object dimVal = entry.getKey().getKey()[i];
theMap.put(
query.getDimensions().get(i - dimStart).getOutputName(),
Strings.emptyToNull((String) entry.getKey().getKey()[i])
dimVal instanceof String ? Strings.emptyToNull((String)dimVal) : dimVal
);
}
@ -254,9 +293,14 @@ public class RowBasedGrouperHelper
@JsonCreator
public static RowBasedKey fromJsonArray(final Object[] key)
{
// Type info is lost during serde. We know we don't want ints as timestamps, so adjust.
if (key.length > 0 && key[0] instanceof Integer) {
key[0] = ((Integer) key[0]).longValue();
// Type info is lost during serde:
// Floats may be deserialized as doubles, Longs may be deserialized as integers, convert them back
for (int i = 0; i < key.length; i++) {
if (key[i] instanceof Integer) {
key[i] = ((Integer) key[i]).longValue();
} else if (key[i] instanceof Double) {
key[i] = ((Double) key[i]).floatValue();
}
}
return new RowBasedKey(key);
@ -296,25 +340,94 @@ public class RowBasedGrouperHelper
}
}
private static Supplier[] getValueSuppliersForDimensions(
final ColumnSelectorFactory columnSelectorFactory,
final List<DimensionSpec> dimensions,
final Map<String, ValueType> rawInputRowSignature
)
{
final Supplier[] inputRawSuppliers = new Supplier[dimensions.size()];
for (int i = 0; i < dimensions.size(); i++) {
final ColumnValueSelector selector = DimensionHandlerUtils.getColumnValueSelectorFromDimensionSpec(
dimensions.get(i),
columnSelectorFactory
);
ValueType type = rawInputRowSignature.get(dimensions.get(i).getDimension());
if (type == null) {
// Subquery post-aggs aren't added to the rowSignature (see rowSignatureFor() in GroupByQueryHelper) because
// their types aren't known, so default to String handling.
type = ValueType.STRING;
}
switch (type) {
case STRING:
inputRawSuppliers[i] = new Supplier()
{
@Override
public Object get()
{
final String value;
IndexedInts index = ((DimensionSelector) selector).getRow();
value = index.size() == 0
? ""
: ((DimensionSelector) selector).lookupName(index.get(0));
return Strings.nullToEmpty(value);
}
};
break;
case LONG:
inputRawSuppliers[i] = new Supplier()
{
@Override
public Object get()
{
return ((LongColumnSelector) selector).get();
}
};
break;
case FLOAT:
inputRawSuppliers[i] = new Supplier()
{
@Override
public Object get()
{
return ((FloatColumnSelector) selector).get();
}
};
break;
default:
throw new IAE("invalid type: [%s]", type);
}
}
return inputRawSuppliers;
}
private static class RowBasedKeySerdeFactory implements Grouper.KeySerdeFactory<RowBasedKey>
{
private final boolean includeTimestamp;
private final boolean sortByDimsFirst;
private final int dimCount;
private final long maxDictionarySize;
private final List<ValueType> valueTypes;
RowBasedKeySerdeFactory(boolean includeTimestamp, boolean sortByDimsFirst, int dimCount, long maxDictionarySize)
RowBasedKeySerdeFactory(
boolean includeTimestamp,
boolean sortByDimsFirst,
int dimCount,
long maxDictionarySize,
List<ValueType> valueTypes
)
{
this.includeTimestamp = includeTimestamp;
this.sortByDimsFirst = sortByDimsFirst;
this.dimCount = dimCount;
this.maxDictionarySize = maxDictionarySize;
this.valueTypes = valueTypes;
}
@Override
public Grouper.KeySerde<RowBasedKey> factorize()
{
return new RowBasedKeySerde(includeTimestamp, sortByDimsFirst, dimCount, maxDictionarySize);
return new RowBasedKeySerde(includeTimestamp, sortByDimsFirst, dimCount, maxDictionarySize, valueTypes);
}
@Override
@ -366,7 +479,7 @@ public class RowBasedGrouperHelper
private static int compareDimsInRows(RowBasedKey key1, RowBasedKey key2, int dimStart)
{
for (int i = dimStart; i < key1.getKey().length; i++) {
final int cmp = ((String) key1.getKey()[i]).compareTo((String) key2.getKey()[i]);
final int cmp = ((Comparable) key1.getKey()[i]).compareTo(key2.getKey()[i]);
if (cmp != 0) {
return cmp;
}
@ -388,6 +501,8 @@ public class RowBasedGrouperHelper
private final ByteBuffer keyBuffer;
private final List<String> dictionary = Lists.newArrayList();
private final Map<String, Integer> reverseDictionary = Maps.newHashMap();
private final List<ValueType> valueTypes;
private final List<RowBasedKeySerdeHelper> serdeHelpers;
// Size limiting for the dictionary, in (roughly estimated) bytes.
private final long maxDictionarySize;
@ -400,14 +515,17 @@ public class RowBasedGrouperHelper
final boolean includeTimestamp,
final boolean sortByDimsFirst,
final int dimCount,
final long maxDictionarySize
final long maxDictionarySize,
final List<ValueType> valueTypes
)
{
this.includeTimestamp = includeTimestamp;
this.sortByDimsFirst = sortByDimsFirst;
this.dimCount = dimCount;
this.maxDictionarySize = maxDictionarySize;
this.keySize = (includeTimestamp ? Longs.BYTES : 0) + dimCount * Ints.BYTES;
this.valueTypes = valueTypes;
this.serdeHelpers = makeSerdeHelpers();
this.keySize = (includeTimestamp ? Longs.BYTES : 0) + getTotalKeySize();
this.keyBuffer = ByteBuffer.allocate(keySize);
}
@ -435,13 +553,10 @@ public class RowBasedGrouperHelper
} else {
dimStart = 0;
}
for (int i = dimStart; i < key.getKey().length; i++) {
final int id = addToDictionary((String) key.getKey()[i]);
if (id < 0) {
if (!serdeHelpers.get(i - dimStart).putToKeyBuffer(key, i)) {
return null;
}
keyBuffer.putInt(id);
}
keyBuffer.flip();
@ -467,7 +582,8 @@ public class RowBasedGrouperHelper
}
for (int i = dimStart; i < key.length; i++) {
key[i] = dictionary.get(buffer.getInt(dimsPosition + (Ints.BYTES * (i - dimStart))));
// Writes value from buffer to key[i]
serdeHelpers.get(i - dimStart).getFromByteBuffer(buffer, dimsPosition, i, key);
}
return new RowBasedKey(key);
@ -496,6 +612,7 @@ public class RowBasedGrouperHelper
public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition)
{
final int cmp = compareDimsInBuffersForNullFudgeTimestamp(
serdeHelpers,
sortableIds,
dimCount,
lhsBuffer,
@ -523,6 +640,7 @@ public class RowBasedGrouperHelper
}
return compareDimsInBuffersForNullFudgeTimestamp(
serdeHelpers,
sortableIds,
dimCount,
lhsBuffer,
@ -540,9 +658,11 @@ public class RowBasedGrouperHelper
public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition)
{
for (int i = 0; i < dimCount; i++) {
final int cmp = Ints.compare(
sortableIds[lhsBuffer.getInt(lhsPosition + (Ints.BYTES * i))],
sortableIds[rhsBuffer.getInt(rhsPosition + (Ints.BYTES * i))]
final int cmp = serdeHelpers.get(i).compare(
lhsBuffer,
rhsBuffer,
lhsPosition,
rhsPosition
);
if (cmp != 0) {
@ -557,6 +677,7 @@ public class RowBasedGrouperHelper
}
private static int compareDimsInBuffersForNullFudgeTimestamp(
List<RowBasedKeySerdeHelper> serdeHelpers,
int[] sortableIds,
int dimCount,
ByteBuffer lhsBuffer,
@ -566,11 +687,12 @@ public class RowBasedGrouperHelper
)
{
for (int i = 0; i < dimCount; i++) {
final int cmp = Ints.compare(
sortableIds[lhsBuffer.getInt(lhsPosition + Longs.BYTES + (Ints.BYTES * i))],
sortableIds[rhsBuffer.getInt(rhsPosition + Longs.BYTES + (Ints.BYTES * i))]
final int cmp = serdeHelpers.get(i).compare(
lhsBuffer,
rhsBuffer,
lhsPosition + Longs.BYTES,
rhsPosition + Longs.BYTES
);
if (cmp != 0) {
return cmp;
}
@ -612,5 +734,204 @@ public class RowBasedGrouperHelper
}
return idx;
}
private int getTotalKeySize()
{
int size = 0;
for (RowBasedKeySerdeHelper helper : serdeHelpers) {
size += helper.getKeyBufferValueSize();
}
return size;
}
private List<RowBasedKeySerdeHelper> makeSerdeHelpers()
{
List<RowBasedKeySerdeHelper> helpers = new ArrayList<>();
int keyBufferPosition = 0;
for (ValueType valType : valueTypes) {
RowBasedKeySerdeHelper helper;
switch (valType) {
case STRING:
helper = new StringRowBasedKeySerdeHelper(keyBufferPosition);
break;
case LONG:
helper = new LongRowBasedKeySerdeHelper(keyBufferPosition);
break;
case FLOAT:
helper = new FloatRowBasedKeySerdeHelper(keyBufferPosition);
break;
default:
throw new IAE("invalid type: %s", valType);
}
keyBufferPosition += helper.getKeyBufferValueSize();
helpers.add(helper);
}
return helpers;
}
private interface RowBasedKeySerdeHelper
{
/**
* @return The size in bytes for a value of the column handled by this SerdeHelper.
*/
int getKeyBufferValueSize();
/**
* Read a value from RowBasedKey at `idx` and put the value at the current position of RowBasedKeySerde's keyBuffer.
* advancing the position by the size returned by getKeyBufferValueSize().
*
* If an internal resource limit has been reached and the value could not be added to the keyBuffer,
* (e.g., maximum dictionary size exceeded for Strings), this method returns false.
*
* @param key RowBasedKey containing the grouping key values for a row.
* @param idx Index of the grouping key column within that this SerdeHelper handles
* @return true if the value was added to the key, false otherwise
*/
boolean putToKeyBuffer(RowBasedKey key, int idx);
/**
* Read a value from a ByteBuffer containing a grouping key in the same format as RowBasedKeySerde's keyBuffer and
* put the value in `dimValues` at `dimValIdx`.
*
* The value to be read resides in the buffer at position (`initialOffset` + the SerdeHelper's keyBufferPosition).
*
* @param buffer ByteBuffer containing an array of grouping keys for a row
* @param initialOffset Offset where non-timestamp grouping key columns start, needed because timestamp is not
* always included in the buffer.
* @param dimValIdx Index within dimValues to store the value read from the buffer
* @param dimValues Output array containing grouping key values for a row
*/
void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues);
/**
* Compare the values at lhsBuffer[lhsPosition] and rhsBuffer[rhsPosition] using the natural ordering
* for this SerdeHelper's value type.
*
* @param lhsBuffer ByteBuffer containing an array of grouping keys for a row
* @param rhsBuffer ByteBuffer containing an array of grouping keys for a row
* @param lhsPosition Position of value within lhsBuffer
* @param rhsPosition Position of value within rhsBuffer
* @return Negative number if lhs < rhs, positive if lhs > rhs, 0 if lhs == rhs
*/
int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition);
}
private class StringRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper
{
final int keyBufferPosition;
public StringRowBasedKeySerdeHelper(int keyBufferPosition)
{
this.keyBufferPosition = keyBufferPosition;
}
@Override
public int getKeyBufferValueSize()
{
return Ints.BYTES;
}
@Override
public boolean putToKeyBuffer(RowBasedKey key, int idx)
{
final int id = addToDictionary((String) key.getKey()[idx]);
if (id < 0) {
return false;
}
keyBuffer.putInt(id);
return true;
}
@Override
public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues)
{
dimValues[dimValIdx] = dictionary.get(buffer.getInt(initialOffset + keyBufferPosition));
}
@Override
public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition)
{
return Ints.compare(
sortableIds[lhsBuffer.getInt(lhsPosition + keyBufferPosition)],
sortableIds[rhsBuffer.getInt(rhsPosition + keyBufferPosition)]
);
}
}
private class LongRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper
{
final int keyBufferPosition;
public LongRowBasedKeySerdeHelper(int keyBufferPosition)
{
this.keyBufferPosition = keyBufferPosition;
}
@Override
public int getKeyBufferValueSize()
{
return Longs.BYTES;
}
@Override
public boolean putToKeyBuffer(RowBasedKey key, int idx)
{
keyBuffer.putLong((Long) key.getKey()[idx]);
return true;
}
@Override
public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues)
{
dimValues[dimValIdx] = buffer.getLong(initialOffset + keyBufferPosition);
}
@Override
public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition)
{
return Longs.compare(
lhsBuffer.getLong(lhsPosition + keyBufferPosition),
rhsBuffer.getLong(rhsPosition + keyBufferPosition)
);
}
}
private class FloatRowBasedKeySerdeHelper implements RowBasedKeySerdeHelper
{
final int keyBufferPosition;
public FloatRowBasedKeySerdeHelper(int keyBufferPosition)
{
this.keyBufferPosition = keyBufferPosition;
}
@Override
public int getKeyBufferValueSize()
{
return Floats.BYTES;
}
@Override
public boolean putToKeyBuffer(RowBasedKey key, int idx)
{
keyBuffer.putFloat((Float) key.getKey()[idx]);
return true;
}
@Override
public void getFromByteBuffer(ByteBuffer buffer, int initialOffset, int dimValIdx, Comparable[] dimValues)
{
dimValues[dimValIdx] = buffer.getFloat(initialOffset + keyBufferPosition);
}
@Override
public int compare(ByteBuffer lhsBuffer, ByteBuffer rhsBuffer, int lhsPosition, int rhsPosition)
{
return Float.compare(
lhsBuffer.getFloat(lhsPosition + keyBufferPosition),
rhsBuffer.getFloat(rhsPosition + keyBufferPosition)
);
}
}
}
}

View File

@ -0,0 +1,86 @@
/*
* 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.query.groupby.epinephelinae.column;
import com.google.common.collect.Lists;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.DimensionSelector;
import io.druid.segment.data.ArrayBasedIndexedInts;
import io.druid.segment.data.IndexedInts;
import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
/**
* A String strategy that builds an internal String<->Integer dictionary for
* DimensionSelectors that return false for nameLookupPossibleInAdvance()
*/
public class DictionaryBuildingStringGroupByColumnSelectorStrategy extends StringGroupByColumnSelectorStrategy
{
private static final int GROUP_BY_MISSING_VALUE = -1;
private int nextId = 0;
private final List<String> dictionary = Lists.newArrayList();
private final Object2IntOpenHashMap<String> reverseDictionary = new Object2IntOpenHashMap<>();
{
reverseDictionary.defaultReturnValue(-1);
}
@Override
public void processValueFromGroupingKey(GroupByColumnSelectorPlus selectorPlus, ByteBuffer key, Map<String, Object> resultMap)
{
final int id = key.getInt(selectorPlus.getKeyBufferPosition());
// GROUP_BY_MISSING_VALUE is used to indicate empty rows, which are omitted from the result map.
if (id != GROUP_BY_MISSING_VALUE) {
final String value = dictionary.get(id);
resultMap.put(
selectorPlus.getOutputName(),
value
);
} else {
resultMap.put(selectorPlus.getOutputName(), "");
}
}
@Override
public void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess)
{
final DimensionSelector dimSelector = (DimensionSelector) selector;
final IndexedInts row = dimSelector.getRow();
final int[] newIds = new int[row.size()];
for (int i = 0; i < row.size(); i++) {
final String value = dimSelector.lookupName(row.get(i));
final int dictId = reverseDictionary.getInt(value);
if (dictId < 0) {
dictionary.add(value);
reverseDictionary.put(value, nextId);
newIds[i] = nextId;
nextId++;
} else {
newIds[i] = dictId;
}
}
valuess[columnIndex] = ArrayBasedIndexedInts.of(newIds);
}
}

View File

@ -0,0 +1,71 @@
/*
* 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.query.groupby.epinephelinae.column;
import com.google.common.primitives.Floats;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.FloatColumnSelector;
import java.nio.ByteBuffer;
import java.util.Map;
public class FloatGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy
{
@Override
public int getGroupingKeySize()
{
return Floats.BYTES;
}
@Override
public void processValueFromGroupingKey(
GroupByColumnSelectorPlus selectorPlus, ByteBuffer key, Map<String, Object> resultMap
)
{
final float val = key.getFloat(selectorPlus.getKeyBufferPosition());
resultMap.put(selectorPlus.getOutputName(), val);
}
@Override
public void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess)
{
valuess[columnIndex] = ((FloatColumnSelector) selector).get();
}
@Override
public void initGroupingKeyColumnValue(
int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack
)
{
keyBuffer.putFloat(keyBufferPosition, (Float) rowObj);
stack[columnIndex] = 1;
}
@Override
public boolean checkRowIndexAndAddValueToGroupingKey(
int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer
)
{
// rows from a float column always have a single value, multi-value is not currently supported
// this method handles row values after the first in a multivalued row, so just return false
return false;
}
}

View File

@ -0,0 +1,46 @@
/*
* 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.query.groupby.epinephelinae.column;
import io.druid.query.ColumnSelectorPlus;
public class GroupByColumnSelectorPlus extends ColumnSelectorPlus<GroupByColumnSelectorStrategy>
{
/**
* Indicates the offset of this dimension's value within the grouping key.
*/
private int keyBufferPosition;
public GroupByColumnSelectorPlus(ColumnSelectorPlus<GroupByColumnSelectorStrategy> baseInfo, int keyBufferPosition)
{
super(
baseInfo.getName(),
baseInfo.getOutputName(),
baseInfo.getColumnSelectorStrategy(),
baseInfo.getSelector()
);
this.keyBufferPosition = keyBufferPosition;
}
public int getKeyBufferPosition()
{
return keyBufferPosition;
}
}

View File

@ -0,0 +1,104 @@
/*
* 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.query.groupby.epinephelinae.column;
import io.druid.query.dimension.ColumnSelectorStrategy;
import io.druid.segment.ColumnValueSelector;
import java.nio.ByteBuffer;
import java.util.Map;
/**
* Contains a collection of query processing methods for type-specific operations used exclusively by
* GroupByQueryEngineV2.
*
* Each GroupByColumnSelectorStrategy is associated with a single dimension.
*/
public interface GroupByColumnSelectorStrategy extends ColumnSelectorStrategy
{
/**
* Return the size, in bytes, of this dimension's values in the grouping key.
*
* For example, a String implementation would return 4, the size of an int.
*
* @return size, in bytes, of this dimension's values in the grouping key.
*/
int getGroupingKeySize();
/**
* Read a value from a grouping key and add it to the group by query result map, using the output name specified
* in a DimensionSpec.
*
* An implementation may choose to not add anything to the result map
* (e.g., as the String implementation does for empty rows)
*
* selectorPlus provides access to:
* - the keyBufferPosition offset from which to read the value
* - the dimension value selector
* - the DimensionSpec for this dimension from the query
*
* @param selectorPlus dimension info containing the key offset, value selector, and dimension spec
* @param resultMap result map for the group by query being served
* @param key grouping key
*/
void processValueFromGroupingKey(
GroupByColumnSelectorPlus selectorPlus,
ByteBuffer key,
Map<String, Object> resultMap
);
/**
* Retrieve a row object from the ColumnSelectorPlus and put it in valuess at columnIndex.
*
* @param selector Value selector for a column.
* @param columnIndex Index of the column within the row values array
* @param valuess Row values array, one index per column
*/
void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess);
/**
* Read the first value within a row values object (IndexedInts, IndexedLongs, etc.) and write that value
* to the keyBuffer at keyBufferPosition. If rowSize is 0, write GROUP_BY_MISSING_VALUE instead.
*
* If the size of the row is > 0, write 1 to stack[] at columnIndex, otherwise write 0.
*
* @param keyBufferPosition Starting offset for this column's value within the grouping key.
* @param columnIndex Index of the column within the row values array
* @param rowObj Row value object for this column (e.g., IndexedInts)
* @param keyBuffer grouping key
* @param stack array containing the current within-row value index for each column
*/
void initGroupingKeyColumnValue(int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack);
/**
* If rowValIdx is less than the size of rowObj (haven't handled all of the row values):
* First, read the value at rowValIdx from a rowObj and write that value to the keyBuffer at keyBufferPosition.
* Then return true
*
* Otherwise, return false.
*
* @param keyBufferPosition Starting offset for this column's value within the grouping key.
* @param rowObj Row value object for this column (e.g., IndexedInts)
* @param rowValIdx Index of the current value being grouped on within the row
* @param keyBuffer grouping key
* @return true if rowValIdx < size of rowObj, false otherwise
*/
boolean checkRowIndexAndAddValueToGroupingKey(int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer);
}

View File

@ -0,0 +1,71 @@
/*
* 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.query.groupby.epinephelinae.column;
import com.google.common.primitives.Longs;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.LongColumnSelector;
import java.nio.ByteBuffer;
import java.util.Map;
public class LongGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy
{
@Override
public int getGroupingKeySize()
{
return Longs.BYTES;
}
@Override
public void processValueFromGroupingKey(
GroupByColumnSelectorPlus selectorPlus, ByteBuffer key, Map<String, Object> resultMap
)
{
final long val = key.getLong(selectorPlus.getKeyBufferPosition());
resultMap.put(selectorPlus.getOutputName(), val);
}
@Override
public void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess)
{
valuess[columnIndex] = ((LongColumnSelector) selector).get();
}
@Override
public void initGroupingKeyColumnValue(
int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack
)
{
keyBuffer.putLong(keyBufferPosition, (Long) rowObj);
stack[columnIndex] = 1;
}
@Override
public boolean checkRowIndexAndAddValueToGroupingKey(
int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer
)
{
// rows from a long column always have a single value, multi-value is not currently supported
// this method handles row values after the first in a multivalued row, so just return false
return false;
}
}

View File

@ -0,0 +1,104 @@
/*
* 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.query.groupby.epinephelinae.column;
import com.google.common.primitives.Ints;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.DimensionSelector;
import io.druid.segment.data.IndexedInts;
import java.nio.ByteBuffer;
import java.util.Map;
public class StringGroupByColumnSelectorStrategy implements GroupByColumnSelectorStrategy
{
private static final int GROUP_BY_MISSING_VALUE = -1;
@Override
public int getGroupingKeySize()
{
return Ints.BYTES;
}
@Override
public void processValueFromGroupingKey(GroupByColumnSelectorPlus selectorPlus, ByteBuffer key, Map<String, Object> resultMap)
{
final int id = key.getInt(selectorPlus.getKeyBufferPosition());
// GROUP_BY_MISSING_VALUE is used to indicate empty rows, which are omitted from the result map.
if (id != GROUP_BY_MISSING_VALUE) {
resultMap.put(
selectorPlus.getOutputName(),
((DimensionSelector) selectorPlus.getSelector()).lookupName(id)
);
} else {
resultMap.put(selectorPlus.getOutputName(), "");
}
}
@Override
public void initColumnValues(ColumnValueSelector selector, int columnIndex, Object[] valuess)
{
DimensionSelector dimSelector = (DimensionSelector) selector;
IndexedInts row = dimSelector.getRow();
valuess[columnIndex] = row;
}
@Override
public void initGroupingKeyColumnValue(int keyBufferPosition, int columnIndex, Object rowObj, ByteBuffer keyBuffer, int[] stack)
{
IndexedInts row = (IndexedInts) rowObj;
int rowSize = row.size();
initializeGroupingKeyV2Dimension(row, rowSize, keyBuffer, keyBufferPosition);
stack[columnIndex] = rowSize == 0 ? 0 : 1;
}
@Override
public boolean checkRowIndexAndAddValueToGroupingKey(int keyBufferPosition, Object rowObj, int rowValIdx, ByteBuffer keyBuffer)
{
IndexedInts row = (IndexedInts) rowObj;
int rowSize = row.size();
if (rowValIdx < rowSize) {
keyBuffer.putInt(
keyBufferPosition,
row.get(rowValIdx)
);
return true;
} else {
return false;
}
}
private void initializeGroupingKeyV2Dimension(
final IndexedInts values,
final int rowSize,
final ByteBuffer keyBuffer,
final int keyBufferPosition
)
{
if (rowSize == 0) {
keyBuffer.putInt(keyBufferPosition, GROUP_BY_MISSING_VALUE);
} else {
keyBuffer.putInt(keyBufferPosition, values.get(0));
}
}
}

View File

@ -39,6 +39,7 @@ import io.druid.query.aggregation.PostAggregator;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.ordering.StringComparator;
import io.druid.query.ordering.StringComparators;
import io.druid.segment.column.ValueType;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
@ -107,10 +108,25 @@ public class DefaultLimitSpec implements LimitSpec
for (int i = 0; i < columns.size(); i++) {
final OrderByColumnSpec columnSpec = columns.get(i);
if (aggAndPostAggNames.contains(columnSpec.getDimension())) {
sortingNeeded = true;
break;
}
final ValueType columnType = getOrderByType(columnSpec, dimensions);
final StringComparator naturalComparator;
if (columnType == ValueType.STRING) {
naturalComparator = StringComparators.LEXICOGRAPHIC;
} else if (columnType == ValueType.LONG || columnType == ValueType.FLOAT) {
naturalComparator = StringComparators.NUMERIC;
} else {
sortingNeeded = true;
break;
}
if (columnSpec.getDirection() != OrderByColumnSpec.Direction.ASCENDING
|| !columnSpec.getDimensionComparator().equals(StringComparators.LEXICOGRAPHIC)
|| !columnSpec.getDimension().equals(dimensions.get(i).getOutputName())
|| aggAndPostAggNames.contains(columnSpec.getDimension())) {
|| !columnSpec.getDimensionComparator().equals(naturalComparator)
|| !columnSpec.getDimension().equals(dimensions.get(i).getOutputName())) {
sortingNeeded = true;
break;
}
@ -137,6 +153,17 @@ public class DefaultLimitSpec implements LimitSpec
return this;
}
private ValueType getOrderByType(final OrderByColumnSpec columnSpec, final List<DimensionSpec> dimensions)
{
for (DimensionSpec dimSpec : dimensions) {
if (columnSpec.getDimension().equals(dimSpec.getOutputName())) {
return dimSpec.getOutputType();
}
}
throw new ISE("Unknown column in order clause[%s]", columnSpec);
}
private Ordering<Row> makeComparator(
List<DimensionSpec> dimensions, List<AggregatorFactory> aggs, List<PostAggregator> postAggs
)

View File

@ -39,6 +39,8 @@ import io.druid.query.search.search.SearchQueryExecutor;
import io.druid.query.search.search.SearchQuerySpec;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.DimensionSelector;
import io.druid.segment.FloatColumnSelector;
import io.druid.segment.LongColumnSelector;
import io.druid.segment.Segment;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ValueType;
@ -70,13 +72,17 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
{
@Override
public SearchColumnSelectorStrategy makeColumnSelectorStrategy(
ColumnCapabilities capabilities
ColumnCapabilities capabilities, ColumnValueSelector selector
)
{
ValueType type = capabilities.getType();
switch (type) {
case STRING:
return new StringSearchColumnSelectorStrategy();
case LONG:
return new LongSearchColumnSelectorStrategy();
case FLOAT:
return new FloatSearchColumnSelectorStrategy();
default:
throw new IAE("Cannot create query type helper from invalid type [%s]", type);
}
@ -135,6 +141,47 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
}
}
public static class LongSearchColumnSelectorStrategy implements SearchColumnSelectorStrategy<LongColumnSelector>
{
@Override
public void updateSearchResultSet(
String outputName,
LongColumnSelector selector,
SearchQuerySpec searchQuerySpec,
int limit,
Object2IntRBTreeMap<SearchHit> set
)
{
if (selector != null) {
final String dimVal = String.valueOf(selector.get());
if (searchQuerySpec.accept(dimVal)) {
set.addTo(new SearchHit(outputName, dimVal), 1);
}
}
}
}
public static class FloatSearchColumnSelectorStrategy implements SearchColumnSelectorStrategy<FloatColumnSelector>
{
@Override
public void updateSearchResultSet(
String outputName,
FloatColumnSelector selector,
SearchQuerySpec searchQuerySpec,
int limit,
Object2IntRBTreeMap<SearchHit> set
)
{
if (selector != null) {
final String dimVal = String.valueOf(selector.get());
if (searchQuerySpec.accept(dimVal)) {
set.addTo(new SearchHit(outputName, dimVal), 1);
}
}
}
}
@Override
public Sequence<Result<SearchResultValue>> run(
final Query<Result<SearchResultValue>> input,

View File

@ -39,6 +39,7 @@ import io.druid.segment.ColumnValueSelector;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionHandlerUtils;
import io.druid.segment.DimensionSelector;
import io.druid.segment.FloatColumnSelector;
import io.druid.segment.LongColumnSelector;
import io.druid.segment.ObjectColumnSelector;
import io.druid.segment.Segment;
@ -67,13 +68,17 @@ public class SelectQueryEngine
{
@Override
public SelectColumnSelectorStrategy makeColumnSelectorStrategy(
ColumnCapabilities capabilities
ColumnCapabilities capabilities, ColumnValueSelector selector
)
{
ValueType type = capabilities.getType();
switch(type) {
case STRING:
return new StringSelectColumnSelectorStrategy();
case LONG:
return new LongSelectColumnSelectorStrategy();
case FLOAT:
return new FloatSelectColumnSelectorStrategy();
default:
throw new IAE("Cannot create query type helper from invalid type [%s]", type);
}
@ -122,6 +127,37 @@ public class SelectQueryEngine
}
}
public static class LongSelectColumnSelectorStrategy implements SelectColumnSelectorStrategy<LongColumnSelector>
{
@Override
public void addRowValuesToSelectResult(
String outputName, LongColumnSelector dimSelector, Map<String, Object> resultMap
)
{
if (dimSelector == null) {
resultMap.put(outputName, null);
} else {
resultMap.put(outputName, dimSelector.get());
}
}
}
public static class FloatSelectColumnSelectorStrategy implements SelectColumnSelectorStrategy<FloatColumnSelector>
{
@Override
public void addRowValuesToSelectResult(
String outputName, FloatColumnSelector dimSelector, Map<String, Object> resultMap
)
{
if (dimSelector == null) {
resultMap.put(outputName, null);
} else {
resultMap.put(outputName, dimSelector.get());
}
}
}
public Sequence<Result<SelectResultValue>> process(final SelectQuery query, final Segment segment)
{
final StorageAdapter adapter = segment.asStorageAdapter();

View File

@ -128,6 +128,10 @@ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm<int[], T
private int[] getDimValSelectorForTopNMetric(TopNParams params, TopNResultBuilder resultBuilder)
{
if (params.getCardinality() < 0) {
throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality");
}
int[] dimValSelector = new int[params.getCardinality()];
Arrays.fill(dimValSelector, SKIP_POSITION_VALUE);

View File

@ -24,6 +24,7 @@ import io.druid.java.util.common.Pair;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.BufferAggregator;
import io.druid.query.topn.types.TopNColumnSelectorStrategy;
import io.druid.segment.Capabilities;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector;
@ -73,10 +74,23 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
TopNResultBuilder resultBuilder,
DimValSelector dimValSelector
)
{
if (params.getCardinality() != TopNColumnSelectorStrategy.CARDINALITY_UNKNOWN) {
runWithCardinalityKnown(params, resultBuilder, dimValSelector);
} else {
runWithCardinalityUnknown(params, resultBuilder);
}
}
private void runWithCardinalityKnown(
Parameters params,
TopNResultBuilder resultBuilder,
DimValSelector dimValSelector
)
{
boolean hasDimValSelector = (dimValSelector != null);
final int cardinality = params.getCardinality();
int cardinality = params.getCardinality();
int numProcessed = 0;
while (numProcessed < cardinality) {
final int numToProcess;
@ -105,6 +119,25 @@ public abstract class BaseTopNAlgorithm<DimValSelector, DimValAggregateStore, Pa
}
}
/**
* This function currently handles TopNs on long and float columns, which do not provide cardinality or an ID lookup.
* When cardinality is unknown, process everything in one pass.
* Existing implementations of makeDimValSelector() require cardinality as well, so the DimValSelector is not used.
* @param params TopN parameters from run()
* @param resultBuilder Result builder from run()
*/
private void runWithCardinalityUnknown(
Parameters params,
TopNResultBuilder resultBuilder
)
{
DimValAggregateStore aggregatesStore = makeDimValAggregateStore(params);
scanAndAggregate(params, null, aggregatesStore, 0);
updateResults(params, null, aggregatesStore, resultBuilder);
closeAggregators(aggregatesStore);
params.getCursor().reset();
}
protected abstract DimValSelector makeDimValSelector(Parameters params, int numProcessed, int numToProcess);
/**

View File

@ -19,7 +19,7 @@
package io.druid.query.topn;
import com.google.common.collect.Maps;
import com.google.common.base.Function;
import io.druid.query.ColumnSelectorPlus;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.topn.types.TopNColumnSelectorStrategy;
@ -61,6 +61,9 @@ public class DimExtractionTopNAlgorithm extends BaseTopNAlgorithm<Aggregator[][]
@Override
protected Aggregator[][] makeDimValSelector(TopNParams params, int numProcessed, int numToProcess)
{
if (params.getCardinality() < 0) {
throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality");
}
ColumnSelectorPlus<TopNColumnSelectorStrategy> selectorPlus = params.getSelectorPlus();
return selectorPlus.getColumnSelectorStrategy().getDimExtractionRowSelector(query, params, capabilities);
}
@ -74,7 +77,8 @@ public class DimExtractionTopNAlgorithm extends BaseTopNAlgorithm<Aggregator[][]
@Override
protected Map<Comparable, Aggregator[]> makeDimValAggregateStore(TopNParams params)
{
return Maps.newHashMap();
final ColumnSelectorPlus<TopNColumnSelectorStrategy> selectorPlus = params.getSelectorPlus();
return selectorPlus.getColumnSelectorStrategy().makeDimExtractionAggregateStore();
}
@Override
@ -88,16 +92,13 @@ public class DimExtractionTopNAlgorithm extends BaseTopNAlgorithm<Aggregator[][]
final Cursor cursor = params.getCursor();
final ColumnSelectorPlus<TopNColumnSelectorStrategy> selectorPlus = params.getSelectorPlus();
while (!cursor.isDone()) {
selectorPlus.getColumnSelectorStrategy().dimExtractionScanAndAggregate(
query,
selectorPlus.getSelector(),
cursor,
rowSelector,
aggregatesStore
);
cursor.advance();
}
selectorPlus.getColumnSelectorStrategy().dimExtractionScanAndAggregate(
query,
selectorPlus.getSelector(),
cursor,
rowSelector,
aggregatesStore
);
}
@Override
@ -108,21 +109,17 @@ public class DimExtractionTopNAlgorithm extends BaseTopNAlgorithm<Aggregator[][]
TopNResultBuilder resultBuilder
)
{
for (Map.Entry<Comparable, Aggregator[]> entry : aggregatesStore.entrySet()) {
Aggregator[] aggs = entry.getValue();
if (aggs != null && aggs.length > 0) {
Object[] vals = new Object[aggs.length];
for (int i = 0; i < aggs.length; i++) {
vals[i] = aggs[i].get();
}
final ColumnSelectorPlus<TopNColumnSelectorStrategy> selectorPlus = params.getSelectorPlus();
final boolean needsResultTypeConversion = needsResultTypeConversion(params);
final Function<Object, Object> valueTransformer = TopNMapFn.getValueTransformer(
query.getDimensionSpec().getOutputType()
);
resultBuilder.addEntry(
entry.getKey() == null ? null : entry.getKey().toString(),
entry.getKey(),
vals
);
}
}
selectorPlus.getColumnSelectorStrategy().updateDimExtractionResults(
aggregatesStore,
needsResultTypeConversion ? valueTransformer : null,
resultBuilder
);
}
@Override
@ -139,4 +136,11 @@ public class DimExtractionTopNAlgorithm extends BaseTopNAlgorithm<Aggregator[][]
public void cleanup(TopNParams params)
{
}
private boolean needsResultTypeConversion(TopNParams params)
{
ColumnSelectorPlus<TopNColumnSelectorStrategy> selectorPlus = params.getSelectorPlus();
TopNColumnSelectorStrategy strategy = selectorPlus.getColumnSelectorStrategy();
return query.getDimensionSpec().getOutputType() != strategy.getValueType();
}
}

View File

@ -26,13 +26,13 @@ import java.util.Map;
public class DimValHolder
{
private final Object topNMetricVal;
private final String dimName;
private final Comparable dimName;
private final Object dimValIndex;
private final Map<String, Object> metricValues;
public DimValHolder(
Object topNMetricVal,
String dimName,
Comparable dimName,
Object dimValIndex,
Map<String, Object> metricValues
)
@ -48,7 +48,7 @@ public class DimValHolder
return topNMetricVal;
}
public String getDimName()
public Comparable getDimName()
{
return dimName;
}
@ -66,7 +66,7 @@ public class DimValHolder
public static class Builder
{
private Object topNMetricVal;
private String dimName;
private Comparable dimName;
private Object dimValIndex;
private Map<String, Object> metricValues;
@ -84,7 +84,7 @@ public class DimValHolder
return this;
}
public Builder withDimName(String dimName)
public Builder withDimName(Comparable dimName)
{
this.dimName = dimName;
return this;

View File

@ -38,11 +38,6 @@ public class DimensionAndMetricValueExtractor extends MetricValueExtractor
this.value = value;
}
public String getStringDimensionValue(String dimension)
{
return (String) value.get(dimension);
}
public Object getDimensionValue(String dimension)
{
return value.get(dimension);

View File

@ -19,6 +19,7 @@
package io.druid.query.topn;
import com.google.common.base.Function;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidPool;
import io.druid.java.util.common.Pair;
@ -28,6 +29,7 @@ import io.druid.query.ColumnSelectorPlus;
import io.druid.segment.Capabilities;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.IndexedInts;
import java.nio.ByteBuffer;
@ -190,6 +192,10 @@ public class PooledTopNAlgorithm
final int numProcessed
)
{
if (params.getCardinality() < 0) {
throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality");
}
final ByteBuffer resultsBuf = params.getResultsBuf();
final int numBytesPerRecord = params.getNumBytesPerRecord();
final int[] aggregatorSizes = params.getAggregatorSizes();
@ -457,10 +463,18 @@ public class PooledTopNAlgorithm
TopNResultBuilder resultBuilder
)
{
if (params.getCardinality() < 0) {
throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality");
}
final ByteBuffer resultsBuf = params.getResultsBuf();
final int[] aggregatorSizes = params.getAggregatorSizes();
final DimensionSelector dimSelector = params.getDimSelector();
final ValueType outType = query.getDimensionSpec().getOutputType();
final boolean needsResultConversion = outType != ValueType.STRING;
final Function<Object, Object> valueTransformer = TopNMapFn.getValueTransformer(outType);
for (int i = 0; i < positions.length; i++) {
int position = positions[i];
if (position >= 0) {
@ -470,8 +484,14 @@ public class PooledTopNAlgorithm
position += aggregatorSizes[j];
}
Object retVal = dimSelector.lookupName(i);
if (needsResultConversion) {
retVal = valueTransformer.apply(retVal);
}
resultBuilder.addEntry(
dimSelector.lookupName(i),
(Comparable) retVal,
i,
vals
);

View File

@ -73,6 +73,10 @@ public class TimeExtractionTopNAlgorithm extends BaseTopNAlgorithm<int[], Map<St
TopNParams params, int[] dimValSelector, Map<String, Aggregator[]> aggregatesStore, int numProcessed
)
{
if (params.getCardinality() < 0) {
throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality");
}
final Cursor cursor = params.getCursor();
final DimensionSelector dimSelector = params.getDimSelector();

View File

@ -84,16 +84,16 @@ public class TopNBinaryFn implements BinaryFn<Result<TopNResultValue>, Result<To
return merger.getResult(arg1, comparator);
}
Map<String, DimensionAndMetricValueExtractor> retVals = new LinkedHashMap<>();
Map<Object, DimensionAndMetricValueExtractor> retVals = new LinkedHashMap<>();
TopNResultValue arg1Vals = arg1.getValue();
TopNResultValue arg2Vals = arg2.getValue();
for (DimensionAndMetricValueExtractor arg1Val : arg1Vals) {
retVals.put(arg1Val.getStringDimensionValue(dimension), arg1Val);
retVals.put(arg1Val.getDimensionValue(dimension), arg1Val);
}
for (DimensionAndMetricValueExtractor arg2Val : arg2Vals) {
final String dimensionValue = arg2Val.getStringDimensionValue(dimension);
final Object dimensionValue = arg2Val.getDimensionValue(dimension);
DimensionAndMetricValueExtractor arg1Val = retVals.get(dimensionValue);
if (arg1Val != null) {

View File

@ -32,6 +32,7 @@ import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.PriorityQueue;
/**
@ -81,11 +82,12 @@ public class TopNLexicographicResultBuilder implements TopNResultBuilder
@Override
public TopNResultBuilder addEntry(
String dimName,
Comparable dimNameObj,
Object dimValIndex,
Object[] metricVals
)
{
final String dimName = Objects.toString(dimNameObj, null);
final Map<String, Object> metricValues = Maps.newHashMapWithExpectedSize(metricVals.length + 1);
if (shouldAdd(dimName)) {
@ -131,7 +133,7 @@ public class TopNLexicographicResultBuilder implements TopNResultBuilder
public TopNResultBuilder addEntry(DimensionAndMetricValueExtractor dimensionAndMetricValueExtractor)
{
Object dimensionValueObj = dimensionAndMetricValueExtractor.getDimensionValue(dimSpec.getOutputName());
String dimensionValue = dimensionValueObj == null ? null : dimensionValueObj.toString();
String dimensionValue = Objects.toString(dimensionValueObj, null);
if (shouldAdd(dimensionValue)) {
pQueue.add(

View File

@ -20,15 +20,62 @@
package io.druid.query.topn;
import com.google.common.base.Function;
import io.druid.java.util.common.IAE;
import io.druid.query.ColumnSelectorPlus;
import io.druid.query.Result;
import io.druid.query.topn.types.TopNStrategyFactory;
import io.druid.query.topn.types.TopNColumnSelectorStrategyFactory;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionHandlerUtils;
import io.druid.segment.column.ValueType;
import java.util.Objects;
public class TopNMapFn implements Function<Cursor, Result<TopNResultValue>>
{
private static final TopNStrategyFactory STRATEGY_FACTORY = new TopNStrategyFactory();
public static Function<Object, Object> getValueTransformer(ValueType outputType)
{
switch (outputType) {
case STRING:
return STRING_TRANSFORMER;
case LONG:
return LONG_TRANSFORMER;
case FLOAT:
return FLOAT_TRANSFORMER;
default:
throw new IAE("invalid type: %s", outputType);
}
}
private static Function<Object, Object> STRING_TRANSFORMER = new Function<Object, Object>()
{
@Override
public Object apply(Object input)
{
return Objects.toString(input, null);
}
};
private static Function<Object, Object> LONG_TRANSFORMER = new Function<Object, Object>()
{
@Override
public Object apply(Object input)
{
final Long longVal = DimensionHandlerUtils.convertObjectToLong(input);
return longVal == null ? 0L : longVal;
}
};
private static Function<Object, Object> FLOAT_TRANSFORMER = new Function<Object, Object>()
{
@Override
public Object apply(Object input)
{
final Float floatVal = DimensionHandlerUtils.convertObjectToFloat(input);
return floatVal == null ? 0.0f : floatVal;
}
};
private static final TopNColumnSelectorStrategyFactory STRATEGY_FACTORY = new TopNColumnSelectorStrategyFactory();
private final TopNQuery query;
private final TopNAlgorithm topNAlgorithm;

View File

@ -50,10 +50,10 @@ public class TopNNumericResultBuilder implements TopNResultBuilder
private final PriorityQueue<DimValHolder> pQueue;
private final Comparator<DimValHolder> dimValComparator;
private final String[] aggFactoryNames;
private static final Comparator<String> dimNameComparator = new Comparator<String>()
private static final Comparator<Comparable> dimNameComparator = new Comparator<Comparable>()
{
@Override
public int compare(String o1, String o2)
public int compare(Comparable o1, Comparable o2)
{
int retval;
if (null == o1) {
@ -114,7 +114,7 @@ public class TopNNumericResultBuilder implements TopNResultBuilder
@Override
public TopNNumericResultBuilder addEntry(
String dimName,
Comparable dimName,
Object dimValIndex,
Object[] metricVals
)
@ -196,7 +196,7 @@ public class TopNNumericResultBuilder implements TopNResultBuilder
if (shouldAdd(dimValue)) {
final DimValHolder valHolder = new DimValHolder.Builder()
.withTopNMetricVal(dimValue)
.withDimName(dimensionAndMetricValueExtractor.getStringDimensionValue(dimSpec.getOutputName()))
.withDimName((Comparable) dimensionAndMetricValueExtractor.getDimensionValue(dimSpec.getOutputName()))
.withMetricValues(dimensionAndMetricValueExtractor.getBaseObject())
.build();
pQueue.add(valHolder);

View File

@ -43,10 +43,6 @@ public class TopNParams
this.cursor = cursor;
this.cardinality = selectorPlus.getColumnSelectorStrategy().getCardinality(selectorPlus.getSelector());
this.numValuesPerPass = numValuesPerPass;
if (cardinality < 0) {
throw new UnsupportedOperationException("Cannot operate on a dimension without a dictionary");
}
}
// Only used by TopN algorithms that support String exclusively

View File

@ -37,6 +37,8 @@ import io.druid.segment.SegmentMissingException;
import io.druid.segment.StorageAdapter;
import io.druid.segment.VirtualColumns;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ValueType;
import io.druid.segment.filter.Filters;
import org.joda.time.Interval;
@ -104,6 +106,8 @@ public class TopNQueryEngine
final TopNAlgorithmSelector selector = new TopNAlgorithmSelector(cardinality, numBytesPerRecord);
query.initTopNAlgorithmSelector(selector);
final ColumnCapabilities columnCapabilities = adapter.getColumnCapabilities(dimension);
final TopNAlgorithm topNAlgorithm;
if (
selector.isHasExtractionFn() &&
@ -117,6 +121,9 @@ public class TopNQueryEngine
topNAlgorithm = new TimeExtractionTopNAlgorithm(capabilities, query);
} else if (selector.isHasExtractionFn()) {
topNAlgorithm = new DimExtractionTopNAlgorithm(capabilities, query);
} else if (columnCapabilities != null && columnCapabilities.getType() != ValueType.STRING) {
// force non-Strings to use DimExtraction for now, do a typed PooledTopN later
topNAlgorithm = new DimExtractionTopNAlgorithm(capabilities, query);
} else if (selector.isAggregateAllMetrics()) {
topNAlgorithm = new PooledTopNAlgorithm(capabilities, query, bufferPool);
} else if (selector.isAggregateTopNMetricFirst() || query.getContextBoolean("doAggregateTopNMetricFirst", false)) {

View File

@ -28,7 +28,7 @@ import java.util.Iterator;
public interface TopNResultBuilder
{
public TopNResultBuilder addEntry(
String dimName,
Comparable dimNameObj,
Object dimValIndex,
Object[] metricVals
);

View File

@ -0,0 +1,115 @@
/*
* 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.query.topn.types;
import com.google.common.base.Function;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.topn.BaseTopNAlgorithm;
import io.druid.query.topn.TopNParams;
import io.druid.query.topn.TopNQuery;
import io.druid.query.topn.TopNResultBuilder;
import io.druid.segment.Capabilities;
import io.druid.segment.Cursor;
import io.druid.segment.FloatColumnSelector;
import io.druid.segment.column.ValueType;
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
public class FloatTopNColumnSelectorStrategy
implements TopNColumnSelectorStrategy<FloatColumnSelector, Int2ObjectMap<Aggregator[]>>
{
@Override
public int getCardinality(FloatColumnSelector selector)
{
return TopNColumnSelectorStrategy.CARDINALITY_UNKNOWN;
}
@Override
public ValueType getValueType()
{
return ValueType.FLOAT;
}
@Override
public Aggregator[][] getDimExtractionRowSelector(
TopNQuery query, TopNParams params, Capabilities capabilities
)
{
return null;
}
@Override
public Int2ObjectMap<Aggregator[]> makeDimExtractionAggregateStore()
{
return new Int2ObjectOpenHashMap<>();
}
@Override
public void dimExtractionScanAndAggregate(
TopNQuery query,
FloatColumnSelector selector,
Cursor cursor,
Aggregator[][] rowSelector,
Int2ObjectMap<Aggregator[]> aggregatesStore
)
{
while (!cursor.isDone()) {
int key = Float.floatToIntBits(selector.get());
Aggregator[] theAggregators = aggregatesStore.get(key);
if (theAggregators == null) {
theAggregators = BaseTopNAlgorithm.makeAggregators(cursor, query.getAggregatorSpecs());
aggregatesStore.put(key, theAggregators);
}
for (Aggregator aggregator : theAggregators) {
aggregator.aggregate();
}
cursor.advance();
}
}
@Override
public void updateDimExtractionResults(
final Int2ObjectMap<Aggregator[]> aggregatesStore,
final Function<Object, Object> valueTransformer,
final TopNResultBuilder resultBuilder
)
{
for (Int2ObjectMap.Entry<Aggregator[]> entry : aggregatesStore.int2ObjectEntrySet()) {
Aggregator[] aggs = entry.getValue();
if (aggs != null && aggs.length > 0) {
Object[] vals = new Object[aggs.length];
for (int i = 0; i < aggs.length; i++) {
vals[i] = aggs[i].get();
}
Comparable key = Float.intBitsToFloat(entry.getIntKey());
if (valueTransformer != null) {
key = (Comparable) valueTransformer.apply(key);
}
resultBuilder.addEntry(
key,
key,
vals
);
}
}
}
}

View File

@ -0,0 +1,115 @@
/*
* 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.query.topn.types;
import com.google.common.base.Function;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.topn.BaseTopNAlgorithm;
import io.druid.query.topn.TopNParams;
import io.druid.query.topn.TopNQuery;
import io.druid.query.topn.TopNResultBuilder;
import io.druid.segment.Capabilities;
import io.druid.segment.Cursor;
import io.druid.segment.LongColumnSelector;
import io.druid.segment.column.ValueType;
import it.unimi.dsi.fastutil.longs.Long2ObjectMap;
import it.unimi.dsi.fastutil.longs.Long2ObjectOpenHashMap;
public class LongTopNColumnSelectorStrategy
implements TopNColumnSelectorStrategy<LongColumnSelector, Long2ObjectMap<Aggregator[]>>
{
@Override
public int getCardinality(LongColumnSelector selector)
{
return TopNColumnSelectorStrategy.CARDINALITY_UNKNOWN;
}
@Override
public ValueType getValueType()
{
return ValueType.LONG;
}
@Override
public Aggregator[][] getDimExtractionRowSelector(
TopNQuery query, TopNParams params, Capabilities capabilities
)
{
return null;
}
@Override
public Long2ObjectMap<Aggregator[]> makeDimExtractionAggregateStore()
{
return new Long2ObjectOpenHashMap<>();
}
@Override
public void dimExtractionScanAndAggregate(
TopNQuery query,
LongColumnSelector selector,
Cursor cursor,
Aggregator[][] rowSelector,
Long2ObjectMap<Aggregator[]> aggregatesStore
)
{
while (!cursor.isDone()) {
long key = selector.get();
Aggregator[] theAggregators = aggregatesStore.get(key);
if (theAggregators == null) {
theAggregators = BaseTopNAlgorithm.makeAggregators(cursor, query.getAggregatorSpecs());
aggregatesStore.put(key, theAggregators);
}
for (Aggregator aggregator : theAggregators) {
aggregator.aggregate();
}
cursor.advance();
}
}
@Override
public void updateDimExtractionResults(
final Long2ObjectMap<Aggregator[]> aggregatesStore,
final Function<Object, Object> valueTransformer,
final TopNResultBuilder resultBuilder
)
{
for (Long2ObjectMap.Entry<Aggregator[]> entry : aggregatesStore.long2ObjectEntrySet()) {
Aggregator[] aggs = entry.getValue();
if (aggs != null && aggs.length > 0) {
Object[] vals = new Object[aggs.length];
for (int i = 0; i < aggs.length; i++) {
vals[i] = aggs[i].get();
}
Comparable key = entry.getLongKey();
if (valueTransformer != null) {
key = (Comparable) valueTransformer.apply(key);
}
resultBuilder.addEntry(
key,
key,
vals
);
}
}
}
}

View File

@ -19,18 +19,23 @@
package io.druid.query.topn.types;
import com.google.common.base.Function;
import com.google.common.collect.Maps;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.topn.BaseTopNAlgorithm;
import io.druid.query.topn.TopNParams;
import io.druid.query.topn.TopNQuery;
import io.druid.query.topn.TopNResultBuilder;
import io.druid.segment.Capabilities;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.IndexedInts;
import java.util.Map;
public class StringTopNColumnSelectorStrategy implements TopNColumnSelectorStrategy<DimensionSelector>
public class StringTopNColumnSelectorStrategy
implements TopNColumnSelectorStrategy<DimensionSelector, Map<String, Aggregator[]>>
{
@Override
public int getCardinality(DimensionSelector selector)
@ -38,14 +43,23 @@ public class StringTopNColumnSelectorStrategy implements TopNColumnSelectorStrat
return selector.getValueCardinality();
}
@Override
public ValueType getValueType()
{
return ValueType.STRING;
}
@Override
public Aggregator[][] getDimExtractionRowSelector(TopNQuery query, TopNParams params, Capabilities capabilities)
{
if (params.getCardinality() < 0) {
throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality");
}
// This method is used for the DimExtractionTopNAlgorithm only.
// Unlike regular topN we cannot rely on ordering to optimize.
// Optimization possibly requires a reverse lookup from value to ID, which is
// not possible when applying an extraction function
final BaseTopNAlgorithm.AggregatorArrayProvider provider = new BaseTopNAlgorithm.AggregatorArrayProvider(
(DimensionSelector) params.getSelectorPlus().getSelector(),
query,
@ -56,33 +70,111 @@ public class StringTopNColumnSelectorStrategy implements TopNColumnSelectorStrat
return provider.build();
}
@Override
public Map<String, Aggregator[]> makeDimExtractionAggregateStore()
{
return Maps.newHashMap();
}
@Override
public void dimExtractionScanAndAggregate(
final TopNQuery query,
TopNQuery query,
DimensionSelector selector,
Cursor cursor,
Aggregator[][] rowSelector,
Map<Comparable, Aggregator[]> aggregatesStore
Map<String, Aggregator[]> aggregatesStore
)
{
final IndexedInts dimValues = selector.getRow();
if (selector.getValueCardinality() != DimensionSelector.CARDINALITY_UNKNOWN) {
dimExtractionScanAndAggregateWithCardinalityKnown(query, cursor, selector, rowSelector, aggregatesStore);
} else {
dimExtractionScanAndAggregateWithCardinalityUnknown(query, cursor, selector, aggregatesStore);
}
}
for (int i = 0; i < dimValues.size(); ++i) {
final int dimIndex = dimValues.get(i);
Aggregator[] theAggregators = rowSelector[dimIndex];
if (theAggregators == null) {
@Override
public void updateDimExtractionResults(
final Map<String, Aggregator[]> aggregatesStore,
final Function<Object, Object> valueTransformer,
final TopNResultBuilder resultBuilder
)
{
for (Map.Entry<String, Aggregator[]> entry : aggregatesStore.entrySet()) {
Aggregator[] aggs = entry.getValue();
if (aggs != null && aggs.length > 0) {
Object[] vals = new Object[aggs.length];
for (int i = 0; i < aggs.length; i++) {
vals[i] = aggs[i].get();
}
Comparable key = entry.getKey();
if (valueTransformer != null) {
key = (Comparable) valueTransformer.apply(key);
}
resultBuilder.addEntry(
key,
key,
vals
);
}
}
}
private void dimExtractionScanAndAggregateWithCardinalityKnown(
TopNQuery query,
Cursor cursor,
DimensionSelector selector,
Aggregator[][] rowSelector,
Map<String, Aggregator[]> aggregatesStore
)
{
while (!cursor.isDone()) {
final IndexedInts dimValues = selector.getRow();
for (int i = 0; i < dimValues.size(); ++i) {
final int dimIndex = dimValues.get(i);
Aggregator[] theAggregators = rowSelector[dimIndex];
if (theAggregators == null) {
final String key = selector.lookupName(dimIndex);
theAggregators = aggregatesStore.get(key);
if (theAggregators == null) {
theAggregators = BaseTopNAlgorithm.makeAggregators(cursor, query.getAggregatorSpecs());
aggregatesStore.put(key, theAggregators);
}
rowSelector[dimIndex] = theAggregators;
}
for (Aggregator aggregator : theAggregators) {
aggregator.aggregate();
}
}
cursor.advance();
}
}
private void dimExtractionScanAndAggregateWithCardinalityUnknown(
TopNQuery query,
Cursor cursor,
DimensionSelector selector,
Map<String, Aggregator[]> aggregatesStore
)
{
while (!cursor.isDone()) {
final IndexedInts dimValues = selector.getRow();
for (int i = 0; i < dimValues.size(); ++i) {
final int dimIndex = dimValues.get(i);
final String key = selector.lookupName(dimIndex);
theAggregators = aggregatesStore.get(key);
Aggregator[] theAggregators = aggregatesStore.get(key);
if (theAggregators == null) {
theAggregators = BaseTopNAlgorithm.makeAggregators(cursor, query.getAggregatorSpecs());
aggregatesStore.put(key, theAggregators);
}
rowSelector[dimIndex] = theAggregators;
}
for (Aggregator aggregator : theAggregators) {
aggregator.aggregate();
for (Aggregator aggregator : theAggregators) {
aggregator.aggregate();
}
}
cursor.advance();
}
}
}

View File

@ -19,20 +19,30 @@
package io.druid.query.topn.types;
import com.google.common.base.Function;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.dimension.ColumnSelectorStrategy;
import io.druid.query.topn.TopNParams;
import io.druid.query.topn.TopNQuery;
import io.druid.query.topn.TopNResultBuilder;
import io.druid.segment.Capabilities;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.Cursor;
import io.druid.segment.column.ValueType;
import javax.annotation.Nullable;
import java.util.Map;
public interface TopNColumnSelectorStrategy<ValueSelectorType extends ColumnValueSelector> extends ColumnSelectorStrategy
public interface TopNColumnSelectorStrategy
<ValueSelectorType extends ColumnValueSelector, DimExtractionAggregateStoreType extends Map>
extends ColumnSelectorStrategy
{
int CARDINALITY_UNKNOWN = -1;
int getCardinality(ValueSelectorType selector);
ValueType getValueType();
/**
* Used by DimExtractionTopNAlgorithm.
*
@ -50,11 +60,20 @@ public interface TopNColumnSelectorStrategy<ValueSelectorType extends ColumnValu
*/
Aggregator[][] getDimExtractionRowSelector(TopNQuery query, TopNParams params, Capabilities capabilities);
/**
* Used by DimExtractionTopNAlgorithm.
*
* Creates an aggregate store map suitable for this strategy's type that will be
* passed to dimExtractionScanAndAggregate() and updateDimExtractionResults().
*
* @return Aggregate store map
*/
DimExtractionAggregateStoreType makeDimExtractionAggregateStore();
/**
* Used by DimExtractionTopNAlgorithm.
*
* Read the current row from a dimension value selector, and for each row value:
* Iterate through the cursor, reading the current row from a dimension value selector, and for each row value:
* 1. Retrieve the Aggregator[] for the row value from rowSelector (fast integer lookup) or from
* aggregatesStore (slower map).
*
@ -77,6 +96,22 @@ public interface TopNColumnSelectorStrategy<ValueSelectorType extends ColumnValu
ValueSelectorType selector,
Cursor cursor,
Aggregator[][] rowSelector,
Map<Comparable, Aggregator[]> aggregatesStore
DimExtractionAggregateStoreType aggregatesStore
);
/**
* Used by DimExtractionTopNAlgorithm.
*
* Read entries from the aggregates store, adding the keys and associated values to the resultBuilder, applying the
* valueTransformer to the keys if present
*
* @param aggregatesStore Map created by makeDimExtractionAggregateStore()
* @param valueTransformer Converts keys to different types, if null no conversion is needed
* @param resultBuilder TopN result builder
*/
void updateDimExtractionResults(
DimExtractionAggregateStoreType aggregatesStore,
@Nullable Function<Object, Object> valueTransformer,
TopNResultBuilder resultBuilder
);
}

View File

@ -21,20 +21,25 @@ package io.druid.query.topn.types;
import io.druid.java.util.common.IAE;
import io.druid.query.dimension.ColumnSelectorStrategyFactory;
import io.druid.segment.ColumnValueSelector;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ValueType;
public class TopNStrategyFactory implements ColumnSelectorStrategyFactory<TopNColumnSelectorStrategy>
public class TopNColumnSelectorStrategyFactory implements ColumnSelectorStrategyFactory<TopNColumnSelectorStrategy>
{
@Override
public TopNColumnSelectorStrategy makeColumnSelectorStrategy(
ColumnCapabilities capabilities
ColumnCapabilities capabilities, ColumnValueSelector selector
)
{
ValueType type = capabilities.getType();
switch(type) {
case STRING:
return new StringTopNColumnSelectorStrategy();
case LONG:
return new LongTopNColumnSelectorStrategy();
case FLOAT:
return new FloatTopNColumnSelectorStrategy();
default:
throw new IAE("Cannot create query type helper from invalid type [%s]", type);
}

View File

@ -20,8 +20,11 @@
package io.druid.segment;
import com.google.common.collect.ImmutableList;
import com.google.common.primitives.Floats;
import io.druid.common.guava.GuavaUtils;
import io.druid.data.input.impl.DimensionSchema.MultiValueHandling;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.parsers.ParseException;
import io.druid.query.ColumnSelectorPlus;
import io.druid.query.dimension.ColumnSelectorStrategy;
import io.druid.query.dimension.ColumnSelectorStrategyFactory;
@ -30,6 +33,7 @@ import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ColumnCapabilitiesImpl;
import io.druid.segment.column.ValueType;
import java.util.ArrayList;
import java.util.List;
public final class DimensionHandlerUtils
@ -64,6 +68,15 @@ public final class DimensionHandlerUtils
return new StringDimensionHandler(dimensionName, multiValueHandling);
}
public static List<ValueType> getValueTypesFromDimensionSpecs(List<DimensionSpec> dimSpecs)
{
List<ValueType> types = new ArrayList<>(dimSpecs.size());
for (DimensionSpec dimSpec : dimSpecs) {
types.add(dimSpec.getOutputType());
}
return types;
}
/**
* Convenience function equivalent to calling
* {@link #createColumnSelectorPluses(ColumnSelectorStrategyFactory, List, ColumnSelectorFactory)} with a singleton
@ -113,15 +126,16 @@ public final class DimensionHandlerUtils
for (int i = 0; i < dimCount; i++) {
final DimensionSpec dimSpec = dimensionSpecs.get(i);
final String dimName = dimSpec.getDimension();
ColumnSelectorStrategyClass strategy = makeStrategy(
strategyFactory,
dimName,
cursor.getColumnCapabilities(dimSpec.getDimension())
);
final ColumnValueSelector selector = getColumnValueSelectorFromDimensionSpec(
dimSpec,
cursor
);
ColumnSelectorStrategyClass strategy = makeStrategy(
strategyFactory,
dimSpec,
cursor.getColumnCapabilities(dimSpec.getDimension()),
selector
);
final ColumnSelectorPlus<ColumnSelectorStrategyClass> selectorPlus = new ColumnSelectorPlus<>(
dimName,
dimSpec.getOutputName(),
@ -133,11 +147,31 @@ public final class DimensionHandlerUtils
return dims;
}
public static ColumnValueSelector getColumnValueSelectorFromDimensionSpec(
DimensionSpec dimSpec,
ColumnSelectorFactory columnSelectorFactory
)
{
String dimName = dimSpec.getDimension();
ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(dimName);
capabilities = getEffectiveCapabilities(dimSpec, capabilities);
switch (capabilities.getType()) {
case STRING:
return columnSelectorFactory.makeDimensionSelector(dimSpec);
case LONG:
return columnSelectorFactory.makeLongColumnSelector(dimSpec.getDimension());
case FLOAT:
return columnSelectorFactory.makeFloatColumnSelector(dimSpec.getDimension());
default:
return null;
}
}
// When determining the capabilites of a column during query processing, this function
// adjusts the capabilities for columns that cannot be handled as-is to manageable defaults
// (e.g., treating missing columns as empty String columns)
private static ColumnCapabilities getEffectiveCapabilities(
String dimName,
DimensionSpec dimSpec,
ColumnCapabilities capabilities
)
{
@ -145,37 +179,80 @@ public final class DimensionHandlerUtils
capabilities = DEFAULT_STRING_CAPABILITIES;
}
// non-Strings aren't actually supported yet
if (capabilities.getType() != ValueType.STRING) {
// Complex dimension type is not supported
if (capabilities.getType() == ValueType.COMPLEX) {
capabilities = DEFAULT_STRING_CAPABILITIES;
}
// Currently, all extractionFns output Strings, so the column will return String values via a
// DimensionSelector if an extractionFn is present.
if (dimSpec.getExtractionFn() != null) {
capabilities = DEFAULT_STRING_CAPABILITIES;
}
// DimensionSpec's decorate only operates on DimensionSelectors, so if a spec mustDecorate(),
// we need to wrap selectors on numeric columns with a string casting DimensionSelector.
if (capabilities.getType() == ValueType.LONG || capabilities.getType() == ValueType.FLOAT) {
if (dimSpec.mustDecorate()) {
capabilities = DEFAULT_STRING_CAPABILITIES;
}
}
return capabilities;
}
private static ColumnValueSelector getColumnValueSelectorFromDimensionSpec(
private static <ColumnSelectorStrategyClass extends ColumnSelectorStrategy> ColumnSelectorStrategyClass makeStrategy(
ColumnSelectorStrategyFactory<ColumnSelectorStrategyClass> strategyFactory,
DimensionSpec dimSpec,
ColumnSelectorFactory columnSelectorFactory
ColumnCapabilities capabilities,
ColumnValueSelector selector
)
{
String dimName = dimSpec.getDimension();
ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(dimName);
capabilities = getEffectiveCapabilities(dimName, capabilities);
switch (capabilities.getType()) {
case STRING:
return columnSelectorFactory.makeDimensionSelector(dimSpec);
default:
return null;
capabilities = getEffectiveCapabilities(dimSpec, capabilities);
return strategyFactory.makeColumnSelectorStrategy(capabilities, selector);
}
public static Long convertObjectToLong(Object valObj)
{
if (valObj == null) {
return 0L;
}
if (valObj instanceof Long) {
return (Long) valObj;
} else if (valObj instanceof Number) {
return ((Number) valObj).longValue();
} else if (valObj instanceof String) {
try {
return GuavaUtils.tryParseLong((String) valObj);
}
catch (Exception e) {
throw new ParseException(e, "Unable to parse value[%s] as long", valObj);
}
} else {
throw new ParseException("Unknown type[%s]", valObj.getClass());
}
}
private static <ColumnSelectorStrategyClass extends ColumnSelectorStrategy> ColumnSelectorStrategyClass makeStrategy(
ColumnSelectorStrategyFactory<ColumnSelectorStrategyClass> strategyFactory,
String dimName,
ColumnCapabilities capabilities
)
public static Float convertObjectToFloat(Object valObj)
{
capabilities = getEffectiveCapabilities(dimName, capabilities);
return strategyFactory.makeColumnSelectorStrategy(capabilities);
if (valObj == null) {
return 0.0f;
}
if (valObj instanceof Float) {
return (Float) valObj;
} else if (valObj instanceof Number) {
return ((Number) valObj).floatValue();
} else if (valObj instanceof String) {
try {
return Floats.tryParse((String) valObj);
}
catch (Exception e) {
throw new ParseException(e, "Unable to parse value[%s] as float", valObj);
}
} else {
throw new ParseException("Unknown type[%s]", valObj.getClass());
}
}
}

View File

@ -0,0 +1,45 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.segment;
import io.druid.query.extraction.ExtractionFn;
import io.druid.segment.virtual.BaseSingleValueDimensionSelector;
public class FloatWrappingDimensionSelector extends BaseSingleValueDimensionSelector
{
private final FloatColumnSelector selector;
private final ExtractionFn extractionFn;
public FloatWrappingDimensionSelector(FloatColumnSelector selector, ExtractionFn extractionFn)
{
this.selector = selector;
this.extractionFn = extractionFn;
}
@Override
protected String getValue()
{
if (extractionFn == null) {
return String.valueOf(selector.get());
} else {
return extractionFn.apply(selector.get());
}
}
}

View File

@ -0,0 +1,45 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.segment;
import io.druid.query.extraction.ExtractionFn;
import io.druid.segment.virtual.BaseSingleValueDimensionSelector;
public class LongWrappingDimensionSelector extends BaseSingleValueDimensionSelector
{
private final LongColumnSelector selector;
private final ExtractionFn extractionFn;
public LongWrappingDimensionSelector(LongColumnSelector selector, ExtractionFn extractionFn)
{
this.selector = selector;
this.extractionFn = extractionFn;
}
@Override
protected String getValue()
{
if (extractionFn == null) {
return String.valueOf(selector.get());
} else {
return extractionFn.apply(selector.get());
}
}
}

View File

@ -454,6 +454,14 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
);
}
if (columnDesc.getCapabilities().getType() == ValueType.LONG) {
return new LongWrappingDimensionSelector(makeLongColumnSelector(dimension), extractionFn);
}
if (columnDesc.getCapabilities().getType() == ValueType.FLOAT) {
return new FloatWrappingDimensionSelector(makeFloatColumnSelector(dimension), extractionFn);
}
DictionaryEncodedColumn<String> cachedColumn = dictionaryColumnCache.get(dimension);
if (cachedColumn == null) {
cachedColumn = columnDesc.getDictionaryEncoding();

View File

@ -19,6 +19,8 @@
package io.druid.segment.column;
import com.fasterxml.jackson.annotation.JsonCreator;
/**
*/
public enum ValueType
@ -26,5 +28,14 @@ public enum ValueType
FLOAT,
LONG,
STRING,
COMPLEX
COMPLEX;
@JsonCreator
public static ValueType fromString(String name)
{
if (name == null) {
return null;
}
return valueOf(name.toUpperCase());
}
}

View File

@ -26,6 +26,7 @@ import io.druid.java.util.common.Pair;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DruidFloatPredicate;
import io.druid.query.filter.DruidLongPredicate;
import io.druid.query.filter.DruidPredicateFactory;
import io.druid.query.filter.Filter;
@ -46,6 +47,7 @@ public class BoundFilter implements Filter
private final ExtractionFn extractionFn;
private final Supplier<DruidLongPredicate> longPredicateSupplier;
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
public BoundFilter(final BoundDimFilter boundDimFilter)
{
@ -53,6 +55,7 @@ public class BoundFilter implements Filter
this.comparator = boundDimFilter.getOrdering();
this.extractionFn = boundDimFilter.getExtractionFn();
this.longPredicateSupplier = boundDimFilter.getLongPredicateSupplier();
this.floatPredicateSupplier = boundDimFilter.getFloatPredicateSupplier();
}
@Override
@ -234,6 +237,32 @@ public class BoundFilter implements Filter
};
}
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
if (extractionFn != null) {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return doesMatch(extractionFn.apply(input));
}
};
} else if (boundDimFilter.getOrdering().equals(StringComparators.NUMERIC)) {
return floatPredicateSupplier.get();
} else {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return doesMatch(String.valueOf(input));
}
};
}
}
};
}

View File

@ -24,6 +24,7 @@ import com.google.common.base.Predicate;
import io.druid.collections.bitmap.ImmutableBitmap;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.DruidFloatPredicate;
import io.druid.query.filter.DruidLongPredicate;
import io.druid.query.filter.DruidPredicateFactory;
import io.druid.query.filter.Filter;
@ -83,6 +84,19 @@ public class DimensionPredicateFilter implements Filter
}
};
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return baseStringPredicate.apply(extractionFn.apply(input));
}
};
}
};
}
}

View File

@ -64,7 +64,11 @@ import java.util.NoSuchElementException;
*/
public class Filters
{
public static final List<ValueType> FILTERABLE_TYPES = ImmutableList.of(ValueType.STRING, ValueType.LONG);
public static final List<ValueType> FILTERABLE_TYPES = ImmutableList.of(
ValueType.STRING,
ValueType.LONG,
ValueType.FLOAT
);
private static final String CTX_KEY_USE_FILTER_CNF = "useFilterCNF";
/**
@ -121,16 +125,6 @@ public class Filters
final String value
)
{
final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(columnName);
// This should be folded into the ValueMatcherColumnSelectorStrategy once that can handle LONG typed columns.
if (capabilities != null && capabilities.getType() == ValueType.LONG) {
return getLongValueMatcher(
columnSelectorFactory.makeLongColumnSelector(columnName),
value
);
}
final ColumnSelectorPlus<ValueMatcherColumnSelectorStrategy> selector =
DimensionHandlerUtils.createColumnSelectorPlus(
ValueMatcherColumnSelectorStrategyFactory.instance(),

View File

@ -25,6 +25,7 @@ import com.google.common.base.Supplier;
import io.druid.collections.bitmap.ImmutableBitmap;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.DruidFloatPredicate;
import io.druid.query.filter.DruidLongPredicate;
import io.druid.query.filter.DruidPredicateFactory;
import io.druid.query.filter.Filter;
@ -48,11 +49,13 @@ public class InFilter implements Filter
private final Set<String> values;
private final ExtractionFn extractionFn;
private final Supplier<DruidLongPredicate> longPredicateSupplier;
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
public InFilter(
String dimension,
Set<String> values,
Supplier<DruidLongPredicate> longPredicateSupplier,
Supplier<DruidFloatPredicate> floatPredicateSupplier,
ExtractionFn extractionFn
)
{
@ -60,6 +63,7 @@ public class InFilter implements Filter
this.values = values;
this.extractionFn = extractionFn;
this.longPredicateSupplier = longPredicateSupplier;
this.floatPredicateSupplier = floatPredicateSupplier;
}
@Override
@ -192,6 +196,23 @@ public class InFilter implements Filter
return longPredicateSupplier.get();
}
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
if (extractionFn != null) {
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return values.contains(extractionFn.apply(input));
}
};
} else {
return floatPredicateSupplier.get();
}
}
};
}
}

View File

@ -21,6 +21,7 @@ package io.druid.segment.filter;
import com.google.common.base.Predicate;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.DruidFloatPredicate;
import io.druid.query.filter.DruidLongPredicate;
import io.druid.query.filter.DruidPredicateFactory;
@ -66,6 +67,19 @@ public class RegexFilter extends DimensionPredicateFilter
};
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return pattern.matcher(String.valueOf(input)).find();
}
};
}
@Override
public String toString()
{

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Predicate;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.DruidFloatPredicate;
import io.druid.query.filter.DruidLongPredicate;
import io.druid.query.filter.DruidPredicateFactory;
import io.druid.query.search.search.SearchQuerySpec;
@ -69,6 +70,19 @@ public class SearchQueryFilter extends DimensionPredicateFilter
}
};
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return query.accept(String.valueOf(input));
}
};
}
},
extractionFn
);

View File

@ -23,6 +23,7 @@ import com.google.common.base.Predicate;
import io.druid.collections.bitmap.ImmutableBitmap;
import io.druid.collections.spatial.search.Bound;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.DruidFloatPredicate;
import io.druid.query.filter.DruidLongPredicate;
import io.druid.query.filter.DruidPredicateFactory;
import io.druid.query.filter.Filter;
@ -82,15 +83,15 @@ public class SpatialFilter implements Filter
@Override
public DruidLongPredicate makeLongPredicate()
{
return new DruidLongPredicate()
{
@Override
public boolean applyLong(long input)
{
// SpatialFilter does not currently support longs
return false;
}
};
// SpatialFilter does not currently support longs
return DruidLongPredicate.ALWAYS_FALSE;
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
// SpatialFilter does not currently support floats
return DruidFloatPredicate.ALWAYS_FALSE;
}
}
);

View File

@ -38,7 +38,9 @@ import io.druid.segment.DimensionHandler;
import io.druid.segment.DimensionIndexer;
import io.druid.segment.DimensionSelector;
import io.druid.segment.FloatColumnSelector;
import io.druid.segment.FloatWrappingDimensionSelector;
import io.druid.segment.LongColumnSelector;
import io.druid.segment.LongWrappingDimensionSelector;
import io.druid.segment.Metadata;
import io.druid.segment.NullDimensionSelector;
import io.druid.segment.ObjectColumnSelector;
@ -49,6 +51,7 @@ import io.druid.segment.ZeroFloatColumnSelector;
import io.druid.segment.ZeroLongColumnSelector;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.Indexed;
import io.druid.segment.data.ListIndexed;
import io.druid.segment.filter.BooleanValueMatcher;
@ -341,6 +344,13 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
return virtualColumns.makeDimensionSelector(dimensionSpec, this);
}
return dimensionSpec.decorate(makeDimensionSelectorUndecorated(dimensionSpec));
}
private DimensionSelector makeDimensionSelectorUndecorated(
DimensionSpec dimensionSpec
)
{
final String dimension = dimensionSpec.getDimension();
final ExtractionFn extractionFn = dimensionSpec.getExtractionFn();
@ -350,16 +360,26 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
extractionFn,
descending
);
return dimensionSpec.decorate(selector);
return selector;
}
ColumnCapabilities capabilities = getColumnCapabilities(dimension);
if (capabilities != null) {
if (capabilities.getType() == ValueType.LONG) {
return new LongWrappingDimensionSelector(makeLongColumnSelector(dimension), extractionFn);
}
if (capabilities.getType() == ValueType.FLOAT) {
return new FloatWrappingDimensionSelector(makeFloatColumnSelector(dimension), extractionFn);
}
}
final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimensionSpec.getDimension());
if (dimensionDesc == null) {
return dimensionSpec.decorate(NullDimensionSelector.instance());
return NullDimensionSelector.instance();
}
final DimensionIndexer indexer = dimensionDesc.getIndexer();
return dimensionSpec.decorate((DimensionSelector) indexer.makeColumnValueSelector(dimensionSpec, currEntry, dimensionDesc));
return (DimensionSelector) indexer.makeColumnValueSelector(dimensionSpec, currEntry, dimensionDesc);
}
@Override

View File

@ -349,7 +349,7 @@ public class SchemaEvolutionTest
// Only float(3) -- which we can't filter, but can aggregate
Assert.assertEquals(
timeseriesResult(ImmutableMap.of("a", 0L, "b", 0.0, "c", 0L)),
timeseriesResult(ImmutableMap.of("a", 19L, "b", 19.100000381469727, "c", 2L)),
runQuery(query, factory, ImmutableList.of(index3))
);
@ -362,9 +362,9 @@ public class SchemaEvolutionTest
// string(1) + long(2) + float(3) + nonexistent(4)
Assert.assertEquals(
timeseriesResult(ImmutableMap.of(
"a", 19L,
"b", 19.0,
"c", 4L
"a", 38L,
"b", 38.10000038146973,
"c", 6L
)),
runQuery(query, factory, ImmutableList.of(index1, index2, index3, index4))
);

View File

@ -21,6 +21,7 @@ package io.druid.query.dimension;
import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.segment.column.ValueType;
import org.junit.Assert;
import org.junit.Test;
@ -42,4 +43,16 @@ public class DefaultDimensionSpecTest
Assert.assertEquals(spec, other);
Assert.assertEquals(spec.hashCode(), other.hashCode());
}
@Test
public void testEqualsSerdeWithType() throws IOException
{
final String name = "foo";
final DimensionSpec spec = new DefaultDimensionSpec(name, name, ValueType.FLOAT);
final String json = mapper.writeValueAsString(spec);
final DimensionSpec other = mapper.readValue(json, DimensionSpec.class);
Assert.assertEquals(spec.toString(), other.toString());
Assert.assertEquals(spec, other);
Assert.assertEquals(spec.hashCode(), other.hashCode());
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.extraction.MatchingDimExtractionFn;
import io.druid.query.extraction.RegexDimExtractionFn;
import io.druid.segment.column.ValueType;
import org.junit.Assert;
import org.junit.Test;
@ -48,6 +49,40 @@ public class ExtractionDimensionSpecTest
Assert.assertEquals("first3Letters", extractionDimensionSpec.getOutputName());
Assert.assertEquals("myDim", extractionDimensionSpec.getDimension());
Assert.assertNotNull(extractionDimensionSpec.getExtractionFn());
Assert.assertEquals(ValueType.STRING, extractionDimensionSpec.getOutputType());
Assert.assertTrue(extractionDimensionSpec.getExtractionFn() instanceof RegexDimExtractionFn);
Assert.assertEquals(
extractionDimensionSpec,
objectMapper.readValue(
objectMapper.writeValueAsBytes(extractionDimensionSpec),
DimensionSpec.class
)
);
}
@Test
public void testSerdeWithType() throws Exception
{
final ObjectMapper objectMapper = new DefaultObjectMapper();
final String oldJson = "{\n"
+ " \"type\": \"extraction\",\n"
+ " \"outputName\": \"first3Letters\",\n"
+ " \"outputType\": \"LONG\",\n"
+ " \"dimension\": \"myDim\","
+ " \"extractionFn\": {\n"
+ " \"type\": \"regex\",\n"
+ " \"expr\": \"(...).*\"\n"
+ " }\n"
+ "}";
final ExtractionDimensionSpec extractionDimensionSpec = (ExtractionDimensionSpec) objectMapper.readValue(oldJson, DimensionSpec.class);
Assert.assertEquals("first3Letters", extractionDimensionSpec.getOutputName());
Assert.assertEquals("myDim", extractionDimensionSpec.getDimension());
Assert.assertNotNull(extractionDimensionSpec.getExtractionFn());
Assert.assertEquals(ValueType.LONG, extractionDimensionSpec.getOutputType());
Assert.assertTrue(extractionDimensionSpec.getExtractionFn() instanceof RegexDimExtractionFn);
Assert.assertEquals(

View File

@ -199,7 +199,7 @@ public class SegmentMetadataQueryTest
null,
null
)
), mmap1 ? 93744 : 94517,
), mmap1 ? 123969 : 124664,
1209,
null,
null,
@ -243,7 +243,7 @@ public class SegmentMetadataQueryTest
null
)
// null_column will be included only for incremental index, which makes a little bigger result than expected
), mmap2 ? 93744 : 94517,
), mmap2 ? 123969 : 124664,
1209,
null,
null,

View File

@ -110,7 +110,7 @@ public class SegmentMetadataUnionQueryTest
null
)
),
mmap ? 374976 : 378068,
mmap ? 495876 : 498656,
4836,
null,
null,

View File

@ -25,12 +25,16 @@ import com.google.common.collect.Lists;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.js.JavaScriptConfig;
import io.druid.query.Druids;
import io.druid.query.Query;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
import io.druid.query.Result;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.ExtractionDimensionSpec;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.JavaScriptExtractionFn;
import io.druid.query.extraction.MapLookupExtractor;
import io.druid.query.extraction.TimeFormatExtractionFn;
import io.druid.query.filter.AndDimFilter;
@ -47,6 +51,7 @@ import io.druid.query.search.search.SearchSortSpec;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.segment.TestHelper;
import io.druid.segment.column.Column;
import io.druid.segment.column.ValueType;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
@ -369,8 +374,7 @@ public class SearchQueryRunnerTest
new ExtractionDimensionSpec(
QueryRunnerTestHelper.qualityDimension,
null,
lookupExtractionFn,
null
lookupExtractionFn
)
)
.query("")
@ -636,6 +640,97 @@ public class SearchQueryRunnerTest
checkSearchQuery(searchQuery, expectedHits);
}
@Test
public void testSearchOnLongColumn()
{
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dimensions(
new DefaultDimensionSpec(
Column.TIME_COLUMN_NAME, Column.TIME_COLUMN_NAME,
ValueType.LONG
)
)
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.query("1297123200000")
.build();
List<SearchHit> expectedHits = Lists.newLinkedList();
expectedHits.add(new SearchHit(Column.TIME_COLUMN_NAME, "1297123200000", 13));
checkSearchQuery(searchQuery, expectedHits);
}
@Test
public void testSearchOnLongColumnWithExFn()
{
String jsFn = "function(str) { return 'super-' + str; }";
ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault());
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dimensions(
new ExtractionDimensionSpec(
Column.TIME_COLUMN_NAME, Column.TIME_COLUMN_NAME,
jsExtractionFn
)
)
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.query("1297123200000")
.build();
List<SearchHit> expectedHits = Lists.newLinkedList();
expectedHits.add(new SearchHit(Column.TIME_COLUMN_NAME, "super-1297123200000", 13));
checkSearchQuery(searchQuery, expectedHits);
}
@Test
public void testSearchOnFloatColumn()
{
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dimensions(
new DefaultDimensionSpec(
QueryRunnerTestHelper.indexMetric, QueryRunnerTestHelper.indexMetric,
ValueType.FLOAT
)
)
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.query("100.7")
.build();
List<SearchHit> expectedHits = Lists.newLinkedList();
expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "100.706055", 1));
expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "100.7756", 1));
checkSearchQuery(searchQuery, expectedHits);
}
@Test
public void testSearchOnFloatColumnWithExFn()
{
String jsFn = "function(str) { return 'super-' + str; }";
ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault());
SearchQuery searchQuery = Druids.newSearchQueryBuilder()
.dimensions(
new ExtractionDimensionSpec(
QueryRunnerTestHelper.indexMetric, QueryRunnerTestHelper.indexMetric,
jsExtractionFn
)
)
.dataSource(QueryRunnerTestHelper.dataSource)
.granularity(QueryRunnerTestHelper.allGran)
.intervals(QueryRunnerTestHelper.fullOnInterval)
.query("100.7")
.build();
List<SearchHit> expectedHits = Lists.newLinkedList();
expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "super-100.7060546875", 1));
expectedHits.add(new SearchHit(QueryRunnerTestHelper.indexMetric, "super-100.77559661865234", 1));
checkSearchQuery(searchQuery, expectedHits);
}
private void checkSearchQuery(Query searchQuery, List<SearchHit> expectedResults)
{

View File

@ -81,10 +81,10 @@ public class SearchQueryRunnerWithCaseTest
configs[2].setSearchStrategy(AutoStrategy.NAME);
CharSource input = CharSource.wrap(
"2011-01-12T00:00:00.000Z\tspot\tAutoMotive\tPREFERRED\ta\u0001preferred\t100.000000\n" +
"2011-01-12T00:00:00.000Z\tSPot\tbusiness\tpreferred\tb\u0001Preferred\t100.000000\n" +
"2011-01-12T00:00:00.000Z\tspot\tentertainment\tPREFERRed\te\u0001preferred\t100.000000\n" +
"2011-01-13T00:00:00.000Z\tspot\tautomotive\tpreferred\ta\u0001preferred\t94.874713"
"2011-01-12T00:00:00.000Z\tspot\tAutoMotive\t1000\t10000.0\t100000\tPREFERRED\ta\u0001preferred\t100.000000\n" +
"2011-01-12T00:00:00.000Z\tSPot\tbusiness\t1100\t11000.0\t110000\tpreferred\tb\u0001Preferred\t100.000000\n" +
"2011-01-12T00:00:00.000Z\tspot\tentertainment\t1200\t12000.0\t120000\tPREFERRed\te\u0001preferred\t100.000000\n" +
"2011-01-13T00:00:00.000Z\tspot\tautomotive\t1000\t10000.0\t100000\tpreferred\ta\u0001preferred\t94.874713"
);
IncrementalIndex index1 = TestIndex.makeRealtimeIndex(input);

View File

@ -76,43 +76,44 @@ public class MultiSegmentSelectQueryTest
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
// time modified version of druid.sample.tsv
// time modified version of druid.sample.numeric.tsv
public static final String[] V_0112 = {
"2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000",
"2011-01-12T01:00:00.000Z spot business preferred bpreferred 100.000000",
"2011-01-12T02:00:00.000Z spot entertainment preferred epreferred 100.000000",
"2011-01-12T03:00:00.000Z spot health preferred hpreferred 100.000000",
"2011-01-12T04:00:00.000Z spot mezzanine preferred mpreferred 100.000000",
"2011-01-12T05:00:00.000Z spot news preferred npreferred 100.000000",
"2011-01-12T06:00:00.000Z spot premium preferred ppreferred 100.000000",
"2011-01-12T07:00:00.000Z spot technology preferred tpreferred 100.000000",
"2011-01-12T08:00:00.000Z spot travel preferred tpreferred 100.000000",
"2011-01-12T09:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000",
"2011-01-12T10:00:00.000Z total_market premium preferred ppreferred 1000.000000",
"2011-01-12T11:00:00.000Z upfront mezzanine preferred mpreferred 800.000000 value",
"2011-01-12T12:00:00.000Z upfront premium preferred ppreferred 800.000000 value"
"2011-01-12T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 100.000000",
"2011-01-12T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000",
"2011-01-12T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 100.000000",
"2011-01-12T03:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 100.000000",
"2011-01-12T04:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 100.000000",
"2011-01-12T05:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 100.000000",
"2011-01-12T06:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 100.000000",
"2011-01-12T07:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 100.000000",
"2011-01-12T08:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 100.000000",
"2011-01-12T09:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1000.000000",
"2011-01-12T10:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1000.000000",
"2011-01-12T11:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 800.000000 value",
"2011-01-12T12:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 800.000000 value"
};
public static final String[] V_0113 = {
"2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713",
"2011-01-13T01:00:00.000Z spot business preferred bpreferred 103.629399",
"2011-01-13T02:00:00.000Z spot entertainment preferred epreferred 110.087299",
"2011-01-13T03:00:00.000Z spot health preferred hpreferred 114.947403",
"2011-01-13T04:00:00.000Z spot mezzanine preferred mpreferred 104.465767",
"2011-01-13T05:00:00.000Z spot news preferred npreferred 102.851683",
"2011-01-13T06:00:00.000Z spot premium preferred ppreferred 108.863011",
"2011-01-13T07:00:00.000Z spot technology preferred tpreferred 111.356672",
"2011-01-13T08:00:00.000Z spot travel preferred tpreferred 106.236928",
"2011-01-13T09:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505",
"2011-01-13T10:00:00.000Z total_market premium preferred ppreferred 1689.012875",
"2011-01-13T11:00:00.000Z upfront mezzanine preferred mpreferred 826.060182 value",
"2011-01-13T12:00:00.000Z upfront premium preferred ppreferred 1564.617729 value"
"2011-01-13T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713",
"2011-01-13T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399",
"2011-01-13T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299",
"2011-01-13T03:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 114.947403",
"2011-01-13T04:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 104.465767",
"2011-01-13T05:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 102.851683",
"2011-01-13T06:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 108.863011",
"2011-01-13T07:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 111.356672",
"2011-01-13T08:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 106.236928",
"2011-01-13T09:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1040.945505",
"2011-01-13T10:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1689.012875",
"2011-01-13T11:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 826.060182 value",
"2011-01-13T12:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 1564.617729 value"
};
public static final String[] V_OVERRIDE = {
"2011-01-12T04:00:00.000Z spot automotive preferred apreferred 999.000000",
"2011-01-12T05:00:00.000Z spot business preferred bpreferred 999.000000",
"2011-01-12T06:00:00.000Z spot entertainment preferred epreferred 999.000000",
"2011-01-12T07:00:00.000Z spot health preferred hpreferred 999.000000"
"2011-01-12T04:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 999.000000",
"2011-01-12T05:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 999.000000",
"2011-01-12T06:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 999.000000",
"2011-01-12T07:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 999.000000"
};
private static Segment segment0;

View File

@ -29,6 +29,7 @@ import com.google.common.collect.Sets;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.guava.Sequences;
import io.druid.js.JavaScriptConfig;
import io.druid.query.Druids;
import io.druid.query.QueryRunner;
import io.druid.query.QueryRunnerTestHelper;
@ -37,6 +38,8 @@ import io.druid.query.TableDataSource;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.dimension.ExtractionDimensionSpec;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.JavaScriptExtractionFn;
import io.druid.query.extraction.MapLookupExtractor;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.DimFilter;
@ -44,6 +47,8 @@ import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.lookup.LookupExtractionFn;
import io.druid.query.spec.LegacySegmentSpec;
import io.druid.query.spec.QuerySegmentSpec;
import io.druid.segment.column.Column;
import io.druid.segment.column.ValueType;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.junit.Assert;
@ -64,36 +69,36 @@ import java.util.Set;
@RunWith(Parameterized.class)
public class SelectQueryRunnerTest
{
// copied from druid.sample.tsv
// copied from druid.sample.numeric.tsv
public static final String[] V_0112 = {
"2011-01-12T00:00:00.000Z spot automotive preferred apreferred 100.000000",
"2011-01-12T00:00:00.000Z spot business preferred bpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot entertainment preferred epreferred 100.000000",
"2011-01-12T00:00:00.000Z spot health preferred hpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot mezzanine preferred mpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot news preferred npreferred 100.000000",
"2011-01-12T00:00:00.000Z spot premium preferred ppreferred 100.000000",
"2011-01-12T00:00:00.000Z spot technology preferred tpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot travel preferred tpreferred 100.000000",
"2011-01-12T00:00:00.000Z total_market mezzanine preferred mpreferred 1000.000000",
"2011-01-12T00:00:00.000Z total_market premium preferred ppreferred 1000.000000",
"2011-01-12T00:00:00.000Z upfront mezzanine preferred mpreferred 800.000000 value",
"2011-01-12T00:00:00.000Z upfront premium preferred ppreferred 800.000000 value"
"2011-01-12T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 100.000000",
"2011-01-12T00:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 100.000000",
"2011-01-12T00:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 100.000000",
"2011-01-12T00:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 100.000000",
"2011-01-12T00:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 100.000000",
"2011-01-12T00:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 100.000000",
"2011-01-12T00:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1000.000000",
"2011-01-12T00:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1000.000000",
"2011-01-12T00:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 800.000000 value",
"2011-01-12T00:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 800.000000 value"
};
public static final String[] V_0113 = {
"2011-01-13T00:00:00.000Z spot automotive preferred apreferred 94.874713",
"2011-01-13T00:00:00.000Z spot business preferred bpreferred 103.629399",
"2011-01-13T00:00:00.000Z spot entertainment preferred epreferred 110.087299",
"2011-01-13T00:00:00.000Z spot health preferred hpreferred 114.947403",
"2011-01-13T00:00:00.000Z spot mezzanine preferred mpreferred 104.465767",
"2011-01-13T00:00:00.000Z spot news preferred npreferred 102.851683",
"2011-01-13T00:00:00.000Z spot premium preferred ppreferred 108.863011",
"2011-01-13T00:00:00.000Z spot technology preferred tpreferred 111.356672",
"2011-01-13T00:00:00.000Z spot travel preferred tpreferred 106.236928",
"2011-01-13T00:00:00.000Z total_market mezzanine preferred mpreferred 1040.945505",
"2011-01-13T00:00:00.000Z total_market premium preferred ppreferred 1689.012875",
"2011-01-13T00:00:00.000Z upfront mezzanine preferred mpreferred 826.060182 value",
"2011-01-13T00:00:00.000Z upfront premium preferred ppreferred 1564.617729 value"
"2011-01-13T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713",
"2011-01-13T00:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399",
"2011-01-13T00:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299",
"2011-01-13T00:00:00.000Z spot health\t1300\t13000.0\t130000\tpreferred hpreferred 114.947403",
"2011-01-13T00:00:00.000Z spot mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 104.465767",
"2011-01-13T00:00:00.000Z spot news\t1500\t15000.0\t150000\tpreferred npreferred 102.851683",
"2011-01-13T00:00:00.000Z spot premium\t1600\t16000.0\t160000\tpreferred ppreferred 108.863011",
"2011-01-13T00:00:00.000Z spot technology\t1700\t17000.0\t170000\tpreferred tpreferred 111.356672",
"2011-01-13T00:00:00.000Z spot travel\t1800\t18000.0\t180000\tpreferred tpreferred 106.236928",
"2011-01-13T00:00:00.000Z total_market mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 1040.945505",
"2011-01-13T00:00:00.000Z total_market premium\t1600\t16000.0\t160000\tpreferred ppreferred 1689.012875",
"2011-01-13T00:00:00.000Z upfront mezzanine\t1400\t14000.0\t140000\tpreferred mpreferred 826.060182 value",
"2011-01-13T00:00:00.000Z upfront premium\t1600\t16000.0\t160000\tpreferred ppreferred 1564.617729 value"
};
public static final QuerySegmentSpec I_0112_0114 = new LegacySegmentSpec(
@ -157,8 +162,8 @@ public class SelectQueryRunnerTest
PagingOffset offset = query.getPagingOffset(QueryRunnerTestHelper.segmentId);
List<Result<SelectResultValue>> expectedResults = toExpected(
toFullEvents(V_0112_0114),
Lists.newArrayList("market", "quality", "placement", "placementish", "partial_null_column", "null_column"),
Lists.newArrayList("index", "quality_uniques", "indexMin", "indexMaxPlusTen"),
Lists.newArrayList("market", "quality", "qualityNumericString", "placement", "placementish", "partial_null_column", "null_column"),
Lists.newArrayList("index", "quality_uniques", "qualityLong", "qualityFloat", "indexMin", "indexMaxPlusTen"),
offset.startOffset(),
offset.threshold()
);
@ -247,7 +252,7 @@ public class SelectQueryRunnerTest
new SelectResultValue(
ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2),
Sets.newHashSet("mar", "qual", "place"),
Sets.newHashSet("index", "quality_uniques", "indexMin", "indexMaxPlusTen"),
Sets.newHashSet("index", "quality_uniques", "qualityLong", "qualityFloat", "indexMin", "indexMaxPlusTen"),
Arrays.asList(
new EventHolder(
QueryRunnerTestHelper.segmentId,
@ -293,7 +298,7 @@ public class SelectQueryRunnerTest
new SelectResultValue(
ImmutableMap.of(QueryRunnerTestHelper.segmentId, -3),
Sets.newHashSet("mar", "qual", "place"),
Sets.newHashSet("index", "quality_uniques", "indexMin", "indexMaxPlusTen"),
Sets.newHashSet("index", "qualityLong", "qualityFloat", "quality_uniques", "indexMin", "indexMaxPlusTen"),
Arrays.asList(
new EventHolder(
QueryRunnerTestHelper.segmentId,
@ -360,6 +365,9 @@ public class SelectQueryRunnerTest
null,
null,
null,
null,
null,
null,
QueryRunnerTestHelper.indexMetric + ":FLOAT"
},
V_0112_0114
@ -553,8 +561,8 @@ public class SelectQueryRunnerTest
new DateTime("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
ImmutableMap.<String, Integer>of(),
Sets.newHashSet("market", "quality", "placement", "placementish", "partial_null_column", "null_column"),
Sets.newHashSet("index", "quality_uniques", "indexMin", "indexMaxPlusTen"),
Sets.newHashSet("market", "quality", "qualityNumericString", "placement", "placementish", "partial_null_column", "null_column"),
Sets.newHashSet("index", "quality_uniques", "qualityLong", "qualityFloat", "indexMin", "indexMaxPlusTen"),
Lists.<EventHolder>newArrayList()
)
)
@ -597,6 +605,239 @@ public class SelectQueryRunnerTest
verify(expectedResults, results);
}
@Test
public void testFullOnSelectWithLongAndFloat()
{
List<DimensionSpec> dimSpecs = Arrays.<DimensionSpec>asList(
new DefaultDimensionSpec(QueryRunnerTestHelper.indexMetric, "floatIndex", ValueType.FLOAT),
new DefaultDimensionSpec(Column.TIME_COLUMN_NAME, "longTime", ValueType.LONG)
);
SelectQuery query = newTestQuery()
.dimensionSpecs(dimSpecs)
.metrics(Arrays.asList(Column.TIME_COLUMN_NAME, "index"))
.intervals(I_0112_0114)
.build();
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, context),
Lists.<Result<SelectResultValue>>newArrayList()
);
List<Result<SelectResultValue>> expectedResultsAsc = Arrays.asList(
new Result<SelectResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2),
Sets.newHashSet("null_column", "floatIndex", "longTime"),
Sets.newHashSet("__time", "index"),
Arrays.asList(
new EventHolder(
QueryRunnerTestHelper.segmentId,
0,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put("longTime", 1294790400000L)
.put("floatIndex", 100.0f)
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.put(Column.TIME_COLUMN_NAME, 1294790400000L)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put("longTime", 1294790400000L)
.put("floatIndex", 100.0f)
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.put(Column.TIME_COLUMN_NAME, 1294790400000L)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put("longTime", 1294790400000L)
.put("floatIndex", 100.0f)
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.put(Column.TIME_COLUMN_NAME, 1294790400000L)
.build()
)
)
)
)
);
List<Result<SelectResultValue>> expectedResultsDsc = Arrays.asList(
new Result<SelectResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
ImmutableMap.of(QueryRunnerTestHelper.segmentId, -3),
Sets.newHashSet("null_column", "floatIndex", "longTime"),
Sets.newHashSet("__time", "index"),
Arrays.asList(
new EventHolder(
QueryRunnerTestHelper.segmentId,
-1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z"))
.put("longTime", 1294876800000L)
.put("floatIndex", 1564.6177f)
.put(QueryRunnerTestHelper.indexMetric, 1564.6177f)
.put(Column.TIME_COLUMN_NAME, 1294876800000L)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
-2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z"))
.put("longTime", 1294876800000L)
.put("floatIndex", 826.0602f)
.put(QueryRunnerTestHelper.indexMetric, 826.0602f)
.put(Column.TIME_COLUMN_NAME, 1294876800000L)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
-3,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z"))
.put("longTime", 1294876800000L)
.put("floatIndex", 1689.0128f)
.put(QueryRunnerTestHelper.indexMetric, 1689.0128f)
.put(Column.TIME_COLUMN_NAME, 1294876800000L)
.build()
)
)
)
)
);
verify(descending ? expectedResultsDsc : expectedResultsAsc, populateNullColumnAtLastForQueryableIndexCase(results, "null_column"));
}
@Test
public void testFullOnSelectWithLongAndFloatWithExFn()
{
String jsFn = "function(str) { return 'super-' + str; }";
ExtractionFn jsExtractionFn = new JavaScriptExtractionFn(jsFn, false, JavaScriptConfig.getDefault());
List<DimensionSpec> dimSpecs = Arrays.<DimensionSpec>asList(
new ExtractionDimensionSpec(QueryRunnerTestHelper.indexMetric, "floatIndex", jsExtractionFn),
new ExtractionDimensionSpec(Column.TIME_COLUMN_NAME, "longTime", jsExtractionFn)
);
SelectQuery query = newTestQuery()
.dimensionSpecs(dimSpecs)
.metrics(Arrays.asList(Column.TIME_COLUMN_NAME, "index"))
.intervals(I_0112_0114)
.build();
HashMap<String, Object> context = new HashMap<String, Object>();
Iterable<Result<SelectResultValue>> results = Sequences.toList(
runner.run(query, context),
Lists.<Result<SelectResultValue>>newArrayList()
);
List<Result<SelectResultValue>> expectedResultsAsc = Arrays.asList(
new Result<SelectResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
ImmutableMap.of(QueryRunnerTestHelper.segmentId, 2),
Sets.newHashSet("null_column", "floatIndex", "longTime"),
Sets.newHashSet("__time", "index"),
Arrays.asList(
new EventHolder(
QueryRunnerTestHelper.segmentId,
0,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put("longTime", "super-1294790400000")
.put("floatIndex", "super-100")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.put(Column.TIME_COLUMN_NAME, 1294790400000L)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put("longTime", "super-1294790400000")
.put("floatIndex", "super-100")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.put(Column.TIME_COLUMN_NAME, 1294790400000L)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-12T00:00:00.000Z"))
.put("longTime", "super-1294790400000")
.put("floatIndex", "super-100")
.put(QueryRunnerTestHelper.indexMetric, 100.000000F)
.put(Column.TIME_COLUMN_NAME, 1294790400000L)
.build()
)
)
)
)
);
List<Result<SelectResultValue>> expectedResultsDsc = Arrays.asList(
new Result<SelectResultValue>(
new DateTime("2011-01-12T00:00:00.000Z"),
new SelectResultValue(
ImmutableMap.of(QueryRunnerTestHelper.segmentId, -3),
Sets.newHashSet("null_column", "floatIndex", "longTime"),
Sets.newHashSet("__time", "index"),
Arrays.asList(
new EventHolder(
QueryRunnerTestHelper.segmentId,
-1,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z"))
.put("longTime", "super-1294876800000")
.put("floatIndex", "super-1564.61767578125")
.put(QueryRunnerTestHelper.indexMetric, 1564.6177f)
.put(Column.TIME_COLUMN_NAME, 1294876800000L)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
-2,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z"))
.put("longTime", "super-1294876800000")
.put("floatIndex", "super-826.0601806640625")
.put(QueryRunnerTestHelper.indexMetric, 826.0602f)
.put(Column.TIME_COLUMN_NAME, 1294876800000L)
.build()
),
new EventHolder(
QueryRunnerTestHelper.segmentId,
-3,
new ImmutableMap.Builder<String, Object>()
.put(EventHolder.timestampKey, new DateTime("2011-01-13T00:00:00.000Z"))
.put("longTime", "super-1294876800000")
.put("floatIndex", "super-1689.0128173828125")
.put(QueryRunnerTestHelper.indexMetric, 1689.0128f)
.put(Column.TIME_COLUMN_NAME, 1294876800000L)
.build()
)
)
)
)
);
verify(descending ? expectedResultsDsc : expectedResultsAsc, populateNullColumnAtLastForQueryableIndexCase(results, "null_column"));
}
private Map<String, Integer> toPagingIdentifier(int startDelta, boolean descending)
{
return ImmutableMap.of(
@ -610,6 +851,9 @@ public class SelectQueryRunnerTest
return toEvents(new String[]{EventHolder.timestampKey + ":TIME",
QueryRunnerTestHelper.marketDimension + ":STRING",
QueryRunnerTestHelper.qualityDimension + ":STRING",
"qualityLong" + ":LONG",
"qualityFloat" + ":FLOAT",
"qualityNumericString" + ":STRING",
QueryRunnerTestHelper.placementDimension + ":STRING",
QueryRunnerTestHelper.placementishDimension + ":STRINGS",
QueryRunnerTestHelper.indexMetric + ":FLOAT",

View File

@ -58,7 +58,9 @@ public class SelectQuerySpecTest
+ "\"descending\":true,"
+ "\"filter\":null,"
+ "\"granularity\":{\"type\":\"all\"},"
+ "\"dimensions\":[{\"type\":\"default\",\"dimension\":\"market\",\"outputName\":\"market\"},{\"type\":\"default\",\"dimension\":\"quality\",\"outputName\":\"quality\"}],"
+ "\"dimensions\":"
+ "[{\"type\":\"default\",\"dimension\":\"market\",\"outputName\":\"market\",\"outputType\":\"STRING\"},"
+ "{\"type\":\"default\",\"dimension\":\"quality\",\"outputName\":\"quality\",\"outputType\":\"STRING\"}],"
+ "\"metrics\":[\"index\"],"
+ "\"virtualColumns\":[],"
+ "\"pagingSpec\":{\"pagingIdentifiers\":{},\"threshold\":3,\"fromNext\":false},"

View File

@ -90,21 +90,21 @@ public class TimeBoundaryQueryRunnerTest
// Adapted from MultiSegmentSelectQueryTest, with modifications to make filtering meaningful
public static final String[] V_0112 = {
"2011-01-12T01:00:00.000Z spot business preferred bpreferred 100.000000",
"2011-01-12T02:00:00.000Z spot entertainment preferred epreferred 100.000000",
"2011-01-13T00:00:00.000Z spot automotive preferred apreferred 100.000000",
"2011-01-13T01:00:00.000Z spot business preferred bpreferred 100.000000",
"2011-01-12T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000",
"2011-01-12T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 100.000000",
"2011-01-13T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 100.000000",
"2011-01-13T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 100.000000",
};
public static final String[] V_0113 = {
"2011-01-14T00:00:00.000Z spot automotive preferred apreferred 94.874713",
"2011-01-14T02:00:00.000Z spot entertainment preferred epreferred 110.087299",
"2011-01-15T00:00:00.000Z spot automotive preferred apreferred 94.874713",
"2011-01-15T01:00:00.000Z spot business preferred bpreferred 103.629399",
"2011-01-16T00:00:00.000Z spot automotive preferred apreferred 94.874713",
"2011-01-16T01:00:00.000Z spot business preferred bpreferred 103.629399",
"2011-01-16T02:00:00.000Z spot entertainment preferred epreferred 110.087299",
"2011-01-17T01:00:00.000Z spot business preferred bpreferred 103.629399",
"2011-01-17T02:00:00.000Z spot entertainment preferred epreferred 110.087299",
"2011-01-14T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713",
"2011-01-14T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299",
"2011-01-15T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713",
"2011-01-15T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399",
"2011-01-16T00:00:00.000Z spot automotive\t1000\t10000.0\t100000\tpreferred apreferred 94.874713",
"2011-01-16T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399",
"2011-01-16T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299",
"2011-01-17T01:00:00.000Z spot business\t1100\t11000.0\t110000\tpreferred bpreferred 103.629399",
"2011-01-17T02:00:00.000Z spot entertainment\t1200\t12000.0\t120000\tpreferred epreferred 110.087299",
};
private static IncrementalIndex newIndex(String minTimeStamp)

View File

@ -94,8 +94,7 @@ public class TopNQueryTest
new ExtractionDimensionSpec(
marketDimension,
marketDimension,
new LookupExtractionFn(new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), true, null, false, false),
null
new LookupExtractionFn(new MapLookupExtractor(ImmutableMap.of("foo", "bar"), false), true, null, false, false)
)
)
.metric(new NumericTopNMetricSpec(indexMetric))

View File

@ -35,6 +35,7 @@ import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
import io.druid.query.aggregation.DoubleMinAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.aggregation.LongSumAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
import io.druid.segment.incremental.IncrementalIndex;
@ -59,6 +60,9 @@ public class TestIndex
"ts",
"market",
"quality",
"qualityLong",
"qualityFloat",
"qualityNumericString",
"placement",
"placementish",
"index",
@ -71,6 +75,7 @@ public class TestIndex
public static final String[] DIMENSIONS = new String[]{
"market",
"quality",
"qualityNumericString",
"placement",
"placementish",
"partial_null_column",
@ -88,7 +93,9 @@ public class TestIndex
new DoubleSumAggregatorFactory(METRICS[0], METRICS[0]),
new DoubleMinAggregatorFactory(METRICS[1], METRICS[0]),
new DoubleMaxAggregatorFactory(METRICS[2], VIRTUAL_COLUMNS.getVirtualColumns()[0].getOutputName()),
new HyperUniquesAggregatorFactory("quality_uniques", "quality")
new HyperUniquesAggregatorFactory("quality_uniques", "quality"),
new LongSumAggregatorFactory("qualityLong", "qualityLong"),
new DoubleSumAggregatorFactory("qualityFloat", "qualityFloat")
};
private static final IndexSpec indexSpec = new IndexSpec();
@ -115,7 +122,7 @@ public class TestIndex
}
}
return realtimeIndex = makeRealtimeIndex("druid.sample.tsv");
return realtimeIndex = makeRealtimeIndex("druid.sample.numeric.tsv");
}
public static IncrementalIndex getNoRollupIncrementalTestIndex()
@ -126,7 +133,7 @@ public class TestIndex
}
}
return noRollupRealtimeIndex = makeRealtimeIndex("druid.sample.tsv", false);
return noRollupRealtimeIndex = makeRealtimeIndex("druid.sample.numeric.tsv", false);
}
public static QueryableIndex getMMappedTestIndex()
@ -165,8 +172,8 @@ public class TestIndex
}
try {
IncrementalIndex top = makeRealtimeIndex("druid.sample.tsv.top");
IncrementalIndex bottom = makeRealtimeIndex("druid.sample.tsv.bottom");
IncrementalIndex top = makeRealtimeIndex("druid.sample.numeric.tsv.top");
IncrementalIndex bottom = makeRealtimeIndex("druid.sample.numeric.tsv.bottom");
File tmpFile = File.createTempFile("yay", "who");
tmpFile.delete();

View File

@ -37,6 +37,7 @@ import io.druid.query.extraction.JavaScriptExtractionFn;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.DruidFloatPredicate;
import io.druid.query.filter.DruidLongPredicate;
import io.druid.query.filter.DruidPredicateFactory;
import io.druid.query.filter.Filter;
@ -142,6 +143,20 @@ public class FilterPartitionTest extends BaseFilterTest
}
};
}
@Override
public DruidFloatPredicate makeFloatPredicate()
{
return new DruidFloatPredicate()
{
@Override
public boolean applyFloat(float input)
{
return Objects.equals(valueOrNull, String.valueOf(input));
}
};
}
};
return new NoBitmapDimensionPredicateFilter(dimension, predicateFactory, extractionFn);

View File

@ -0,0 +1,430 @@
/*
* 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.segment.filter;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import io.druid.data.input.InputRow;
import io.druid.data.input.impl.DimensionsSpec;
import io.druid.data.input.impl.InputRowParser;
import io.druid.data.input.impl.MapInputRowParser;
import io.druid.data.input.impl.TimeAndDimsParseSpec;
import io.druid.data.input.impl.TimestampSpec;
import io.druid.java.util.common.Pair;
import io.druid.js.JavaScriptConfig;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.extraction.MapLookupExtractor;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.InDimFilter;
import io.druid.query.filter.JavaScriptDimFilter;
import io.druid.query.filter.RegexDimFilter;
import io.druid.query.filter.SearchQueryDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.lookup.LookupExtractionFn;
import io.druid.query.lookup.LookupExtractor;
import io.druid.query.ordering.StringComparators;
import io.druid.query.search.search.ContainsSearchQuerySpec;
import io.druid.segment.IndexBuilder;
import io.druid.segment.StorageAdapter;
import io.druid.segment.incremental.IncrementalIndexSchema;
import org.joda.time.DateTime;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.Closeable;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
@RunWith(Parameterized.class)
public class FloatFilteringTest extends BaseFilterTest
{
private static final String FLOAT_COLUMN = "flt";
private static final String TIMESTAMP_COLUMN = "ts";
private static int EXECUTOR_NUM_THREADS = 16;
private static int EXECUTOR_NUM_TASKS = 2000;
private static final InputRowParser<Map<String, Object>> PARSER = new MapInputRowParser(
new TimeAndDimsParseSpec(
new TimestampSpec(TIMESTAMP_COLUMN, "millis", new DateTime("2000")),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")),
null,
null
)
)
);
private static final List<InputRow> ROWS = ImmutableList.of(
PARSER.parse(ImmutableMap.<String, Object>of("ts", 1L, "dim0", "1", "flt", 1.0f, "dim1", "", "dim2", ImmutableList.of("a", "b"))),
PARSER.parse(ImmutableMap.<String, Object>of("ts", 2L, "dim0", "2", "flt", 2.0f, "dim1", "10", "dim2", ImmutableList.of())),
PARSER.parse(ImmutableMap.<String, Object>of("ts", 3L, "dim0", "3", "flt", 3.0f, "dim1", "2", "dim2", ImmutableList.of(""))),
PARSER.parse(ImmutableMap.<String, Object>of("ts", 4L, "dim0", "4", "flt", 4.0f, "dim1", "1", "dim2", ImmutableList.of("a"))),
PARSER.parse(ImmutableMap.<String, Object>of("ts", 5L, "dim0", "5", "flt", 5.0f, "dim1", "def", "dim2", ImmutableList.of("c"))),
PARSER.parse(ImmutableMap.<String, Object>of("ts", 6L, "dim0", "6", "flt", 6.0f, "dim1", "abc"))
);
public FloatFilteringTest(
String testName,
IndexBuilder indexBuilder,
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
boolean cnf,
boolean optimize
)
{
super(
testName,
ROWS,
indexBuilder.schema(
new IncrementalIndexSchema.Builder()
.withMetrics(
new AggregatorFactory[]{
new DoubleSumAggregatorFactory(FLOAT_COLUMN, FLOAT_COLUMN)
}
).build()
),
finisher,
cnf,
optimize
);
}
@AfterClass
public static void tearDown() throws Exception
{
BaseFilterTest.tearDown(FloatFilteringTest.class.getName());
}
@Test
public void testFloatColumnFiltering()
{
assertFilterMatches(
new SelectorDimFilter(FLOAT_COLUMN, "0", null),
ImmutableList.<String>of()
);
assertFilterMatches(
new SelectorDimFilter(FLOAT_COLUMN, "0.0", null),
ImmutableList.<String>of()
);
assertFilterMatches(
new SelectorDimFilter(FLOAT_COLUMN, "3", null),
ImmutableList.<String>of("3")
);
assertFilterMatches(
new SelectorDimFilter(FLOAT_COLUMN, "3.0", null),
ImmutableList.<String>of("3")
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, "2", "5", false, false, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("2", "3", "4", "5")
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, "2.0", "5.0", false, false, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("2", "3", "4", "5")
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, "1", "4", true, true, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("2", "3")
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, "1.0", "4.0", true, true, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("2", "3")
);
assertFilterMatches(
new InDimFilter(FLOAT_COLUMN, Arrays.asList("2", "4", "8"), null),
ImmutableList.<String>of("2", "4")
);
assertFilterMatches(
new InDimFilter(FLOAT_COLUMN, Arrays.asList("2.0", "4.0", "8.0"), null),
ImmutableList.<String>of("2", "4")
);
// cross the hashing threshold to test hashset implementation, filter on even values
List<String> infilterValues = new ArrayList<>(InDimFilter.NUMERIC_HASHING_THRESHOLD * 2);
for (int i = 0; i < InDimFilter.NUMERIC_HASHING_THRESHOLD * 2; i++) {
infilterValues.add(String.valueOf(i * 2));
}
assertFilterMatches(
new InDimFilter(FLOAT_COLUMN, infilterValues, null),
ImmutableList.<String>of("2", "4", "6")
);
String jsFn = "function(x) { return(x === 3 || x === 5) }";
assertFilterMatches(
new JavaScriptDimFilter(FLOAT_COLUMN, jsFn, null, JavaScriptConfig.getDefault()),
ImmutableList.<String>of("3", "5")
);
String jsFn2 = "function(x) { return(x === 3.0 || x === 5.0) }";
assertFilterMatches(
new JavaScriptDimFilter(FLOAT_COLUMN, jsFn2, null, JavaScriptConfig.getDefault()),
ImmutableList.<String>of("3", "5")
);
assertFilterMatches(
new RegexDimFilter(FLOAT_COLUMN, "4", null),
ImmutableList.<String>of("4")
);
assertFilterMatches(
new RegexDimFilter(FLOAT_COLUMN, "4.0", null),
ImmutableList.<String>of("4")
);
assertFilterMatches(
new SearchQueryDimFilter(FLOAT_COLUMN, new ContainsSearchQuerySpec("2", true), null),
ImmutableList.<String>of("2")
);
assertFilterMatches(
new SearchQueryDimFilter(FLOAT_COLUMN, new ContainsSearchQuerySpec("2", true), null),
ImmutableList.<String>of("2")
);
}
@Test
public void testFloatColumnFilteringWithNonNumbers()
{
assertFilterMatches(
new SelectorDimFilter(FLOAT_COLUMN, "", null),
ImmutableList.<String>of()
);
assertFilterMatches(
new SelectorDimFilter(FLOAT_COLUMN, null, null),
ImmutableList.<String>of()
);
assertFilterMatches(
new SelectorDimFilter(FLOAT_COLUMN, "abc", null),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, "a", "b", false, false, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, " ", "4", false, false, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("1", "2", "3", "4")
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, " ", "4", false, false, null, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of("1", "2", "3")
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, " ", "4.0", false, false, null, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of("1", "2", "3", "4")
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, " ", "A", false, false, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, " ", "A", false, false, null, null, StringComparators.LEXICOGRAPHIC),
ImmutableList.<String>of("1", "2", "3", "4", "5", "6")
);
}
@Test
public void testFloatFilterWithExtractionFn()
{
final Map<String, String> stringMap = new HashMap<>();
stringMap.put("1.0", "Monday");
stringMap.put("2.0", "Tuesday");
stringMap.put("3.0", "Wednesday");
stringMap.put("4.0", "Thursday");
stringMap.put("5.0", "Friday");
stringMap.put("6.0", "Saturday");
LookupExtractor mapExtractor = new MapLookupExtractor(stringMap, false);
LookupExtractionFn exfn = new LookupExtractionFn(mapExtractor, false, "UNKNOWN", false, true);
assertFilterMatches(
new SelectorDimFilter(FLOAT_COLUMN, "Monday", exfn),
ImmutableList.<String>of("1")
);
assertFilterMatches(
new SelectorDimFilter(FLOAT_COLUMN, "Notaday", exfn),
ImmutableList.<String>of()
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, "Fridax", "Fridaz", false, false, null, exfn, StringComparators.ALPHANUMERIC),
ImmutableList.<String>of("5")
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, "Friday", "Friday", true, true, null, exfn, StringComparators.ALPHANUMERIC),
ImmutableList.<String>of()
);
assertFilterMatches(
new InDimFilter(FLOAT_COLUMN, Arrays.asList("Caturday", "Saturday", "Tuesday"), exfn),
ImmutableList.<String>of("2", "6")
);
// test InFilter HashSet implementation
List<String> bigList = Arrays.asList(
"Saturday", "Tuesday",
"Caturday", "Xanaday", "Vojuday", "Gribaday", "Kipoday", "Dheferday", "Fakeday", "Qeearaday",
"Hello", "World", "1", "2", "3", "4", "5", "6", "7"
);
assertFilterMatches(
new InDimFilter(FLOAT_COLUMN, bigList, exfn),
ImmutableList.<String>of("2", "6")
);
String jsFn = "function(x) { return(x === 'Wednesday' || x === 'Thursday') }";
assertFilterMatches(
new JavaScriptDimFilter(FLOAT_COLUMN, jsFn, exfn, JavaScriptConfig.getDefault()),
ImmutableList.<String>of("3", "4")
);
assertFilterMatches(
new RegexDimFilter(FLOAT_COLUMN, ".*day", exfn),
ImmutableList.<String>of("1", "2", "3", "4", "5", "6")
);
assertFilterMatches(
new SearchQueryDimFilter(FLOAT_COLUMN, new ContainsSearchQuerySpec("s", true), exfn),
ImmutableList.<String>of("2", "3", "4")
);
}
@Test
public void testMultithreaded()
{
assertFilterMatchesMultithreaded(
new SelectorDimFilter(FLOAT_COLUMN, "3", null),
ImmutableList.<String>of("3")
);
assertFilterMatchesMultithreaded(
new SelectorDimFilter(FLOAT_COLUMN, "3.0", null),
ImmutableList.<String>of("3")
);
assertFilterMatchesMultithreaded(
new InDimFilter(FLOAT_COLUMN, Arrays.asList("2", "4", "8"), null),
ImmutableList.<String>of("2", "4")
);
assertFilterMatchesMultithreaded(
new InDimFilter(FLOAT_COLUMN, Arrays.asList("2.0", "4.0", "8.0"), null),
ImmutableList.<String>of("2", "4")
);
// cross the hashing threshold to test hashset implementation, filter on even values
List<String> infilterValues = new ArrayList<>(InDimFilter.NUMERIC_HASHING_THRESHOLD * 2);
for (int i = 0; i < InDimFilter.NUMERIC_HASHING_THRESHOLD * 2; i++) {
infilterValues.add(String.valueOf(i * 2));
}
assertFilterMatchesMultithreaded(
new InDimFilter(FLOAT_COLUMN, infilterValues, null),
ImmutableList.<String>of("2", "4", "6")
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, "2", "5", false, false, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("2", "3", "4", "5")
);
assertFilterMatches(
new BoundDimFilter(FLOAT_COLUMN, "2.0", "5.0", false, false, null, null, StringComparators.NUMERIC),
ImmutableList.<String>of("2", "3", "4", "5")
);
}
private void assertFilterMatchesMultithreaded(
final DimFilter filter,
final List<String> expectedRows
)
{
testWithExecutor(filter, expectedRows);
}
private Runnable makeFilterRunner(
final DimFilter filter,
final List<String> expectedRows
)
{
return new Runnable()
{
@Override
public void run()
{
assertFilterMatches(filter, expectedRows);
}
};
}
private void testWithExecutor(
final DimFilter filter,
final List<String> expectedRows
)
{
ListeningExecutorService executor = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(EXECUTOR_NUM_THREADS));
List<ListenableFuture<?>> futures = new ArrayList<>();
for (int i = 0; i < EXECUTOR_NUM_TASKS; i++) {
Runnable runnable = makeFilterRunner(filter, expectedRows);
ListenableFuture fut = executor.submit(runnable);
futures.add(fut);
}
try {
Futures.allAsList(futures).get(60, TimeUnit.SECONDS);
}
catch (Exception ex) {
Assert.fail(ex.getMessage());
}
executor.shutdown();
}
}

View File

@ -124,16 +124,6 @@ public class InvalidFilteringTest extends BaseFilterTest
ImmutableList.<String>of("1", "2", "3", "4", "5", "6")
);
assertFilterMatches(
new SelectorDimFilter("dmax", "another string", null),
ImmutableList.<String>of()
);
assertFilterMatches(
new SelectorDimFilter("dmax", null, null),
ImmutableList.<String>of("1", "2", "3", "4", "5", "6")
);
// predicate based matching
assertFilterMatches(
new InDimFilter("hyperion", Arrays.asList("hello", "world"), null),
@ -144,15 +134,5 @@ public class InvalidFilteringTest extends BaseFilterTest
new InDimFilter("hyperion", Arrays.asList("hello", "world", null), null),
ImmutableList.<String>of("1", "2", "3", "4", "5", "6")
);
assertFilterMatches(
new InDimFilter("dmax", Arrays.asList("hello", "world"), null),
ImmutableList.<String>of()
);
assertFilterMatches(
new InDimFilter("dmax", Arrays.asList("hello", "world", null), null),
ImmutableList.<String>of("1", "2", "3", "4", "5", "6")
);
}
}

View File

@ -155,8 +155,8 @@ public class LongFilteringTest extends BaseFilterTest
);
// cross the hashing threshold to test hashset implementation, filter on even values
List<String> infilterValues = new ArrayList<>(InDimFilter.LONG_HASHING_THRESHOLD * 2);
for (int i = 0; i < InDimFilter.LONG_HASHING_THRESHOLD * 2; i++) {
List<String> infilterValues = new ArrayList<>(InDimFilter.NUMERIC_HASHING_THRESHOLD * 2);
for (int i = 0; i < InDimFilter.NUMERIC_HASHING_THRESHOLD * 2; i++) {
infilterValues.add(String.valueOf(i * 2));
}
assertFilterMatches(
@ -303,8 +303,8 @@ public class LongFilteringTest extends BaseFilterTest
);
// cross the hashing threshold to test hashset implementation, filter on even values
List<String> infilterValues = new ArrayList<>(InDimFilter.LONG_HASHING_THRESHOLD * 2);
for (int i = 0; i < InDimFilter.LONG_HASHING_THRESHOLD * 2; i++) {
List<String> infilterValues = new ArrayList<>(InDimFilter.NUMERIC_HASHING_THRESHOLD * 2);
for (int i = 0; i < InDimFilter.NUMERIC_HASHING_THRESHOLD * 2; i++) {
infilterValues.add(String.valueOf(i * 2));
}
assertFilterMatchesMultithreaded(

Some files were not shown because too many files have changed in this diff Show More