mirror of
https://github.com/apache/druid.git
synced 2025-02-11 04:24:58 +00:00
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:
parent
e08cd0066b
commit
ca2b04f0fd
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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.
|
||||
|
@ -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.|
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
|
@ -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",
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
@ -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()
|
||||
{
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
@ -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);
|
||||
}
|
||||
|
@ -26,4 +26,6 @@ public interface DruidPredicateFactory
|
||||
public Predicate<String> makeStringPredicate();
|
||||
|
||||
public DruidLongPredicate makeLongPredicate();
|
||||
|
||||
public DruidFloatPredicate makeFloatPredicate();
|
||||
}
|
||||
|
@ -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());
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -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());
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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());
|
||||
}
|
||||
|
@ -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()) {
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -156,6 +156,7 @@ public class GroupByRowProcessor
|
||||
Pair<Grouper<RowBasedKey>, Accumulator<Grouper<RowBasedKey>, Row>> pair = RowBasedGrouperHelper.createGrouperAccumulatorPair(
|
||||
query,
|
||||
true,
|
||||
rowSignature,
|
||||
querySpecificConfig,
|
||||
mergeBufferHolder.get(),
|
||||
-1,
|
||||
|
@ -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(
|
||||
|
@ -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)
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
@ -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
|
||||
)
|
||||
|
@ -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,
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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);
|
||||
|
||||
/**
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
);
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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) {
|
||||
|
@ -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(
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -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)) {
|
||||
|
@ -28,7 +28,7 @@ import java.util.Iterator;
|
||||
public interface TopNResultBuilder
|
||||
{
|
||||
public TopNResultBuilder addEntry(
|
||||
String dimName,
|
||||
Comparable dimNameObj,
|
||||
Object dimValIndex,
|
||||
Object[] metricVals
|
||||
);
|
||||
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
);
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
@ -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();
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
@ -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));
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -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));
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
@ -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(),
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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
|
||||
);
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -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
|
||||
|
@ -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))
|
||||
);
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
@ -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(
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -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,
|
||||
|
@ -110,7 +110,7 @@ public class SegmentMetadataUnionQueryTest
|
||||
null
|
||||
)
|
||||
),
|
||||
mmap ? 374976 : 378068,
|
||||
mmap ? 495876 : 498656,
|
||||
4836,
|
||||
null,
|
||||
null,
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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",
|
||||
|
@ -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},"
|
||||
|
@ -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)
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -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))
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
@ -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")
|
||||
);
|
||||
}
|
||||
}
|
||||
|
@ -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
Loading…
x
Reference in New Issue
Block a user