Query vectorization. (#6794)

* Benchmarks: New SqlBenchmark, add caching & vectorization to some others.

- Introduce a new SqlBenchmark geared towards benchmarking a wide
  variety of SQL queries. Rename the old SqlBenchmark to
  SqlVsNativeBenchmark.
- Add (optional) caching to SegmentGenerator to enable easier
  benchmarking of larger segments.
- Add vectorization to FilteredAggregatorBenchmark and GroupByBenchmark.

* Query vectorization.

This patch includes vectorized timeseries and groupBy engines, as well
as some analogs of your favorite Druid classes:

- VectorCursor is like Cursor. (It comes from StorageAdapter.makeVectorCursor.)
- VectorColumnSelectorFactory is like ColumnSelectorFactory, and it has
  methods to create analogs of the column selectors you know and love.
- VectorOffset and ReadableVectorOffset are like Offset and ReadableOffset.
- VectorAggregator is like BufferAggregator.
- VectorValueMatcher is like ValueMatcher.

There are some noticeable differences between vectorized and regular
execution:

- Unlike regular cursors, vector cursors do not understand time
  granularity. They expect query engines to handle this on their own,
  which a new VectorCursorGranularizer class helps with. This is to
  avoid too much batch-splitting and to respect the fact that vector
  selectors are somewhat more heavyweight than regular selectors.
- Unlike FilteredOffset, FilteredVectorOffset does not leverage indexes
  for filters that might partially support them (like an OR of one
  filter that supports indexing and another that doesn't). I'm not sure
  that this behavior is desirable anyway (it is potentially too eager)
  but, at any rate, it'd be better to harmonize it between the two
  classes. Potentially they should both do some different thing that
  is smarter than what either of them is doing right now.
- When vector cursors are created by QueryableIndexCursorSequenceBuilder,
  they use a morphing binary-then-linear search to find their start and
  end rows, rather than linear search.

Limitations in this patch are:

- Only timeseries and groupBy have vectorized engines.
- GroupBy doesn't handle multi-value dimensions yet.
- Vector cursors cannot handle virtual columns or descending order.
- Only some filters have vectorized matchers: "selector", "bound", "in",
  "like", "regex", "search", "and", "or", and "not".
- Only some aggregators have vectorized implementations: "count",
  "doubleSum", "floatSum", "longSum", "hyperUnique", and "filtered".
- Dimension specs other than "default" don't work yet (no extraction
  functions or filtered dimension specs).

Currently, the testing strategy includes adding vectorization-enabled
tests to TimeseriesQueryRunnerTest, GroupByQueryRunnerTest,
GroupByTimeseriesQueryRunnerTest, CalciteQueryTest, and all of the
filtering tests that extend BaseFilterTest. In all of those classes,
there are some test cases that don't support vectorization. They are
marked by special function calls like "cannotVectorize" or "skipVectorize"
that tell the test harness to either expect an exception or to skip the
test case.

Testing should be expanded in the future -- a project in and of itself.

Related to #3011.

* WIP

* Adjustments for unused things.

* Adjust javadocs.

* DimensionDictionarySelector adjustments.

* Add "clone" to BatchIteratorAdapter.

* ValueMatcher javadocs.

* Fix benchmark.

* Fixups post-merge.

* Expect exception on testGroupByWithStringVirtualColumn for IncrementalIndex.

* BloomDimFilterSqlTest: Tag two non-vectorizable tests.

* Minor adjustments.

* Update surefire, bump up Xmx in Travis.

* Some more adjustments.

* Javadoc adjustments

* AggregatorAdapters adjustments.

* Additional comments.

* Remove switching search.

* Only missiles.
This commit is contained in:
Gian Merlino 2019-07-12 12:54:07 -07:00 committed by GitHub
parent abf9843e2a
commit ffa25b7832
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
205 changed files with 11425 additions and 1939 deletions

View File

@ -60,7 +60,7 @@ matrix:
before_script: unset _JAVA_OPTIONS
script:
# Set MAVEN_OPTS for Surefire launcher
- MAVEN_OPTS='-Xmx512m' mvn test -B -pl processing
- MAVEN_OPTS='-Xmx800m' mvn test -B -pl processing
- sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0"
- free -m
@ -71,7 +71,7 @@ matrix:
before_script: unset _JAVA_OPTIONS
script:
# Set MAVEN_OPTS for Surefire launcher
- MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl processing
- MAVEN_OPTS='-Xmx800m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl processing
- sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0"
- free -m
@ -82,7 +82,7 @@ matrix:
before_script: unset _JAVA_OPTIONS
script:
# Set MAVEN_OPTS for Surefire launcher
- MAVEN_OPTS='-Xmx512m' mvn test -B -pl server
- MAVEN_OPTS='-Xmx800m' mvn test -B -pl server
# server module test with SQL Compatibility enabled
- env:
@ -91,7 +91,7 @@ matrix:
before_script: unset _JAVA_OPTIONS
script:
# Set MAVEN_OPTS for Surefire launcher
- MAVEN_OPTS='-Xmx512m' mvn test -B -pl server -Ddruid.generic.useDefaultValueForNull=false
- MAVEN_OPTS='-Xmx800m' mvn test -B -pl server -Ddruid.generic.useDefaultValueForNull=false
# other modules test
@ -101,7 +101,7 @@ matrix:
before_script: unset _JAVA_OPTIONS
script:
# Set MAVEN_OPTS for Surefire launcher
- MAVEN_OPTS='-Xmx512m' mvn test -B -pl '!processing,!server'
- MAVEN_OPTS='-Xmx800m' mvn test -B -pl '!processing,!server'
- sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0"
- free -m
@ -112,7 +112,7 @@ matrix:
before_script: unset _JAVA_OPTIONS
script:
# Set MAVEN_OPTS for Surefire launcher
- MAVEN_OPTS='-Xmx512m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl '!processing,!server'
- MAVEN_OPTS='-Xmx800m' mvn test -B -Ddruid.generic.useDefaultValueForNull=false -pl '!processing,!server'
- sh -c "dmesg | egrep -i '(oom|out of memory|kill process|killed).*' -C 1 || exit 0"
- free -m

View File

@ -83,12 +83,24 @@
<version>${project.parent.version}</version>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-core</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-processing</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-server</artifactId>
<version>${project.parent.version}</version>
<type>test-jar</type>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>
<artifactId>druid-sql</artifactId>

View File

@ -27,6 +27,7 @@ import org.apache.druid.benchmark.datagen.SegmentGenerator;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.js.JavaScriptConfig;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
@ -73,15 +74,17 @@ public class ExpressionAggregationBenchmark
@Param({"1000000"})
private int rowsPerSegment;
private SegmentGenerator segmentGenerator;
private QueryableIndex index;
private JavaScriptAggregatorFactory javaScriptAggregatorFactory;
private DoubleSumAggregatorFactory expressionAggregatorFactory;
private ByteBuffer aggregationBuffer = ByteBuffer.allocate(Double.BYTES);
private Closer closer;
@Setup(Level.Trial)
public void setup()
{
this.closer = Closer.create();
final BenchmarkSchemaInfo schemaInfo = new BenchmarkSchemaInfo(
ImmutableList.of(
BenchmarkColumnSchema.makeNormal("x", ValueType.FLOAT, false, 1, 0d, 0d, 10000d, false),
@ -99,8 +102,10 @@ public class ExpressionAggregationBenchmark
.shardSpec(new LinearShardSpec(0))
.build();
this.segmentGenerator = new SegmentGenerator();
this.index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator());
this.index = closer.register(
segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment)
);
this.javaScriptAggregatorFactory = new JavaScriptAggregatorFactory(
"name",
ImmutableList.of("x", "y"),
@ -120,15 +125,7 @@ public class ExpressionAggregationBenchmark
@TearDown(Level.Trial)
public void tearDown() throws Exception
{
if (index != null) {
index.close();
index = null;
}
if (segmentGenerator != null) {
segmentGenerator.close();
segmentGenerator = null;
}
closer.close();
}
@Benchmark

View File

@ -26,6 +26,7 @@ import org.apache.druid.benchmark.datagen.SegmentGenerator;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.ExtractionDimensionSpec;
import org.apache.druid.query.expression.TestExprMacroTable;
@ -72,12 +73,14 @@ public class ExpressionSelectorBenchmark
@Param({"1000000"})
private int rowsPerSegment;
private SegmentGenerator segmentGenerator;
private QueryableIndex index;
private Closer closer;
@Setup(Level.Trial)
public void setup()
{
this.closer = Closer.create();
final BenchmarkSchemaInfo schemaInfo = new BenchmarkSchemaInfo(
ImmutableList.of(
BenchmarkColumnSchema.makeZipf(
@ -113,22 +116,16 @@ public class ExpressionSelectorBenchmark
.shardSpec(new LinearShardSpec(0))
.build();
this.segmentGenerator = new SegmentGenerator();
this.index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.HOUR, rowsPerSegment);
final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator());
this.index = closer.register(
segmentGenerator.generate(dataSegment, schemaInfo, Granularities.HOUR, rowsPerSegment)
);
}
@TearDown(Level.Trial)
public void tearDown() throws Exception
{
if (index != null) {
index.close();
index = null;
}
if (segmentGenerator != null) {
segmentGenerator.close();
segmentGenerator = null;
}
closer.close();
}
@Benchmark

View File

@ -20,6 +20,7 @@
package org.apache.druid.benchmark;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.Files;
import org.apache.commons.io.FileUtils;
import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator;
@ -31,7 +32,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.js.JavaScriptConfig;
import org.apache.druid.query.Druids;
import org.apache.druid.query.FinalizeResultsQueryRunner;
import org.apache.druid.query.Query;
@ -47,7 +47,6 @@ import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
import org.apache.druid.query.filter.BoundDimFilter;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.filter.InDimFilter;
import org.apache.druid.query.filter.JavaScriptDimFilter;
import org.apache.druid.query.filter.OrDimFilter;
import org.apache.druid.query.filter.RegexDimFilter;
import org.apache.druid.query.filter.SearchQueryDimFilter;
@ -106,6 +105,9 @@ public class FilteredAggregatorBenchmark
@Param({"basic"})
private String schema;
@Param({"false", "true"})
private String vectorize;
private static final Logger log = new Logger(FilteredAggregatorBenchmark.class);
private static final int RNG_SEED = 9999;
private static final IndexMergerV9 INDEX_MERGER_V9;
@ -160,12 +162,6 @@ public class FilteredAggregatorBenchmark
filter = new OrDimFilter(
Arrays.asList(
new BoundDimFilter("dimSequential", "-1", "-1", true, true, null, null, StringComparators.ALPHANUMERIC),
new JavaScriptDimFilter(
"dimSequential",
"function(x) { return false }",
null,
JavaScriptConfig.getEnabledInstance()
),
new RegexDimFilter("dimSequential", "X", null),
new SearchQueryDimFilter("dimSequential", new ContainsSearchQuerySpec("X", false), null),
new InDimFilter("dimSequential", Collections.singletonList("X"), null)
@ -233,7 +229,7 @@ public class FilteredAggregatorBenchmark
.buildOnheap();
}
private static <T> List<T> runQuery(QueryRunnerFactory factory, QueryRunner runner, Query<T> query)
private static <T> List<T> runQuery(QueryRunnerFactory factory, QueryRunner runner, Query<T> query, String vectorize)
{
QueryToolChest toolChest = factory.getToolchest();
QueryRunner<T> theRunner = new FinalizeResultsQueryRunner<>(
@ -241,7 +237,10 @@ public class FilteredAggregatorBenchmark
toolChest
);
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
final QueryPlus<T> queryToRun = QueryPlus.wrap(
query.withOverriddenContext(ImmutableMap.of("vectorize", vectorize))
);
Sequence<T> queryResult = theRunner.run(queryToRun, new HashMap<>());
return queryResult.toList();
}
@ -268,7 +267,12 @@ public class FilteredAggregatorBenchmark
new IncrementalIndexSegment(incIndex, SegmentId.dummy("incIndex"))
);
List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(factory, runner, query);
List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(
factory,
runner,
query,
vectorize
);
for (Result<TimeseriesResultValue> result : results) {
blackhole.consume(result);
}
@ -285,7 +289,12 @@ public class FilteredAggregatorBenchmark
new QueryableIndexSegment(qIndex, SegmentId.dummy("qIndex"))
);
List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(factory, runner, query);
List<Result<TimeseriesResultValue>> results = FilteredAggregatorBenchmark.runQuery(
factory,
runner,
query,
vectorize
);
for (Result<TimeseriesResultValue> result : results) {
blackhole.consume(result);
}

View File

@ -424,4 +424,26 @@ public class BenchmarkColumnSchema
schema.enumeratedProbabilities = enumeratedProbabilities;
return schema;
}
@Override
public String toString()
{
return "BenchmarkColumnSchema{" +
"distributionType=" + distributionType +
", name='" + name + '\'' +
", type=" + type +
", isMetric=" + isMetric +
", rowSize=" + rowSize +
", nullProbability=" + nullProbability +
", enumeratedValues=" + enumeratedValues +
", enumeratedProbabilities=" + enumeratedProbabilities +
", startInt=" + startInt +
", endInt=" + endInt +
", startDouble=" + startDouble +
", endDouble=" + endDouble +
", zipfExponent=" + zipfExponent +
", mean=" + mean +
", standardDeviation=" + standardDeviation +
'}';
}
}

View File

@ -68,4 +68,15 @@ public class BenchmarkSchemaInfo
{
return withRollup;
}
@Override
public String toString()
{
return "BenchmarkSchemaInfo{" +
"columnSchemas=" + columnSchemas +
", aggs=" + aggs +
", dataInterval=" + dataInterval +
", withRollup=" + withRollup +
'}';
}
}

View File

@ -94,7 +94,7 @@ public class BenchmarkSchemas
basicSchemaIngestAggsExpression.add(new DoubleMinAggregatorFactory("minFloatZipf", "metFloatZipf"));
basicSchemaIngestAggsExpression.add(new HyperUniquesAggregatorFactory("hyper", "dimHyperUnique"));
Interval basicSchemaDataInterval = Intervals.utc(0, 1000000);
Interval basicSchemaDataInterval = Intervals.of("2000-01-01/P1D");
BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo(
basicSchemaColumns,

View File

@ -20,7 +20,7 @@
package org.apache.druid.benchmark.datagen;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.hash.Hashing;
import com.google.common.io.Files;
import org.apache.commons.io.FileUtils;
import org.apache.druid.data.input.InputRow;
@ -31,6 +31,7 @@ import org.apache.druid.data.input.impl.FloatDimensionSchema;
import org.apache.druid.data.input.impl.LongDimensionSchema;
import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.aggregation.AggregatorFactory;
@ -40,18 +41,20 @@ import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexIndexableAdapter;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.data.RoaringBitmapSerdeFactory;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
public class SegmentGenerator implements Closeable
@ -59,15 +62,43 @@ public class SegmentGenerator implements Closeable
private static final Logger log = new Logger(SegmentGenerator.class);
private static final int MAX_ROWS_IN_MEMORY = 200000;
private static final int STARTING_SEED = 9999; // Consistent seed for reproducibility
private final File tempDir;
private final AtomicInteger seed;
// Setup can take a long time due to the need to generate large segments.
// Allow users to specify a cache directory via a JVM property or an environment variable.
private static final String CACHE_DIR_PROPERTY = "druid.benchmark.cacheDir";
private static final String CACHE_DIR_ENV_VAR = "DRUID_BENCHMARK_CACHE_DIR";
private final File cacheDir;
private final boolean cleanupCacheDir;
public SegmentGenerator()
{
this.tempDir = Files.createTempDir();
this.seed = new AtomicInteger(STARTING_SEED);
this(null);
}
public SegmentGenerator(@Nullable final File cacheDir)
{
if (cacheDir != null) {
this.cacheDir = cacheDir;
this.cleanupCacheDir = false;
} else {
final String userConfiguredCacheDir = System.getProperty(CACHE_DIR_PROPERTY, System.getenv(CACHE_DIR_ENV_VAR));
if (userConfiguredCacheDir != null) {
this.cacheDir = new File(userConfiguredCacheDir);
this.cleanupCacheDir = false;
} else {
log.warn("No cache directory provided; benchmark data caching is disabled. "
+ "Set the 'druid.benchmark.cacheDir' property or 'DRUID_BENCHMARK_CACHE_DIR' environment variable "
+ "to use caching.");
this.cacheDir = Files.createTempDir();
this.cleanupCacheDir = true;
}
}
}
public File getCacheDir()
{
return cacheDir;
}
public QueryableIndex generate(
@ -80,9 +111,32 @@ public class SegmentGenerator implements Closeable
// In case we need to generate hyperUniques.
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde());
final String dataHash = Hashing.sha256()
.newHasher()
.putString(dataSegment.getId().toString(), StandardCharsets.UTF_8)
.putString(schemaInfo.toString(), StandardCharsets.UTF_8)
.putString(granularity.toString(), StandardCharsets.UTF_8)
.putInt(numRows)
.hash()
.toString();
final File outDir = new File(getSegmentDir(dataSegment.getId(), dataHash), "merged");
if (outDir.exists()) {
try {
log.info("Found segment with hash[%s] cached in directory[%s].", dataHash, outDir);
return TestHelper.getTestIndexIO().loadIndex(outDir);
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
log.info("Writing segment with hash[%s] to directory[%s].", dataHash, outDir);
final BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(
schemaInfo.getColumnSchemas(),
seed.getAndIncrement(),
dataSegment.getId().hashCode(), /* Use segment identifier hashCode as seed */
schemaInfo.getDataInterval(),
numRows
);
@ -124,61 +178,69 @@ public class SegmentGenerator implements Closeable
rows.add(row);
if ((i + 1) % 20000 == 0) {
log.info("%,d/%,d rows generated.", i + 1, numRows);
log.info("%,d/%,d rows generated for[%s].", i + 1, numRows, dataSegment);
}
if (rows.size() % MAX_ROWS_IN_MEMORY == 0) {
indexes.add(makeIndex(dataSegment.getId(), indexes.size(), rows, indexSchema));
indexes.add(makeIndex(dataSegment.getId(), dataHash, indexes.size(), rows, indexSchema));
rows.clear();
}
}
log.info("%,d/%,d rows generated.", numRows, numRows);
log.info("%,d/%,d rows generated for[%s].", numRows, numRows, dataSegment);
if (rows.size() > 0) {
indexes.add(makeIndex(dataSegment.getId(), indexes.size(), rows, indexSchema));
indexes.add(makeIndex(dataSegment.getId(), dataHash, indexes.size(), rows, indexSchema));
rows.clear();
}
final QueryableIndex retVal;
if (indexes.isEmpty()) {
throw new ISE("No rows to index?");
} else if (indexes.size() == 1) {
return Iterables.getOnlyElement(indexes);
} else {
try {
final QueryableIndex merged = TestHelper.getTestIndexIO().loadIndex(
TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()).merge(
indexes.stream().map(QueryableIndexIndexableAdapter::new).collect(Collectors.toList()),
false,
schemaInfo.getAggs()
.stream()
.map(AggregatorFactory::getCombiningFactory)
.toArray(AggregatorFactory[]::new),
new File(tempDir, "merged"),
new IndexSpec()
)
);
retVal = TestHelper
.getTestIndexIO()
.loadIndex(
TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.merge(
indexes.stream().map(QueryableIndexIndexableAdapter::new).collect(Collectors.toList()),
false,
schemaInfo.getAggs()
.stream()
.map(AggregatorFactory::getCombiningFactory)
.toArray(AggregatorFactory[]::new),
outDir,
new IndexSpec(new RoaringBitmapSerdeFactory(true), null, null, null)
)
);
for (QueryableIndex index : indexes) {
index.close();
}
return merged;
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
log.info("Finished writing segment[%s] to[%s]", dataSegment, outDir);
return retVal;
}
@Override
public void close() throws IOException
{
FileUtils.deleteDirectory(tempDir);
if (cleanupCacheDir) {
FileUtils.deleteDirectory(cacheDir);
}
}
private QueryableIndex makeIndex(
final SegmentId identifier,
final String dataHash,
final int indexNumber,
final List<InputRow> rows,
final IncrementalIndexSchema indexSchema
@ -187,9 +249,14 @@ public class SegmentGenerator implements Closeable
return IndexBuilder
.create()
.schema(indexSchema)
.tmpDir(new File(new File(tempDir, identifier.toString()), String.valueOf(indexNumber)))
.tmpDir(new File(getSegmentDir(identifier, dataHash), String.valueOf(indexNumber)))
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.rows(rows)
.buildMMappedIndex();
}
private File getSegmentDir(final SegmentId identifier, final String dataHash)
{
return new File(cacheDir, StringUtils.format("%s_%s", identifier, dataHash));
}
}

View File

@ -51,6 +51,7 @@ import org.apache.druid.query.QueryRunner;
import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryToolChest;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
@ -134,6 +135,9 @@ public class GroupByBenchmark
@Param({"all", "day"})
private String queryGranularity;
@Param({"force", "false"})
private String vectorize;
private static final Logger log = new Logger(GroupByBenchmark.class);
private static final int RNG_SEED = 9999;
private static final IndexMergerV9 INDEX_MERGER_V9;
@ -178,10 +182,8 @@ public class GroupByBenchmark
{ // basic.A
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval()));
List<AggregatorFactory> queryAggs = new ArrayList<>();
queryAggs.add(new LongSumAggregatorFactory(
"sumLongSequential",
"sumLongSequential"
));
queryAggs.add(new CountAggregatorFactory("cnt"));
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
GroupByQuery queryA = GroupByQuery
.builder()
.setDataSource("blah")
@ -189,6 +191,7 @@ public class GroupByBenchmark
.setDimensions(new DefaultDimensionSpec("dimSequential", null), new DefaultDimensionSpec("dimZipf", null))
.setAggregatorSpecs(queryAggs)
.setGranularity(Granularity.fromString(queryGranularity))
.setContext(ImmutableMap.of("vectorize", vectorize))
.build();
basicQueries.put("A", queryA);
@ -209,6 +212,7 @@ public class GroupByBenchmark
.setDimensions(new DefaultDimensionSpec("dimSequential", null), new DefaultDimensionSpec("dimZipf", null))
.setAggregatorSpecs(queryAggs)
.setGranularity(Granularities.DAY)
.setContext(ImmutableMap.of("vectorize", vectorize))
.build();
GroupByQuery queryA = GroupByQuery
@ -218,6 +222,7 @@ public class GroupByBenchmark
.setDimensions(new DefaultDimensionSpec("dimSequential", null))
.setAggregatorSpecs(queryAggs)
.setGranularity(Granularities.WEEK)
.setContext(ImmutableMap.of("vectorize", vectorize))
.build();
basicQueries.put("nested", queryA);
@ -242,6 +247,7 @@ public class GroupByBenchmark
.setAggregatorSpecs(queryAggs)
.setGranularity(Granularity.fromString(queryGranularity))
.setDimFilter(new BoundDimFilter("dimUniform", "0", "100", true, true, null, null, null))
.setContext(ImmutableMap.of("vectorize", vectorize))
.build();
basicQueries.put("filter", queryA);
@ -265,6 +271,7 @@ public class GroupByBenchmark
.setDimensions(new DefaultDimensionSpec("dimZipf", null))
.setAggregatorSpecs(queryAggs)
.setGranularity(Granularity.fromString(queryGranularity))
.setContext(ImmutableMap.of("vectorize", vectorize))
.build();
basicQueries.put("singleZipf", queryA);
@ -292,6 +299,7 @@ public class GroupByBenchmark
queryAggs
)
.setGranularity(Granularity.fromString(queryGranularity))
.setContext(ImmutableMap.of("vectorize", vectorize))
.build();
simpleQueries.put("A", queryA);
@ -317,6 +325,7 @@ public class GroupByBenchmark
queryAggs
)
.setGranularity(Granularity.fromString(queryGranularity))
.setContext(ImmutableMap.of("vectorize", vectorize))
.build();
simpleLongQueries.put("A", queryA);
@ -340,6 +349,7 @@ public class GroupByBenchmark
.setDimensions(new DefaultDimensionSpec("dimSequential", "dimSequential", ValueType.FLOAT))
.setAggregatorSpecs(queryAggs)
.setGranularity(Granularity.fromString(queryGranularity))
.setContext(ImmutableMap.of("vectorize", vectorize))
.build();
simpleFloatQueries.put("A", queryA);
@ -502,9 +512,9 @@ public class GroupByBenchmark
return new IncrementalIndex.Builder()
.setIndexSchema(
new IncrementalIndexSchema.Builder()
.withMetrics(schemaInfo.getAggsArray())
.withRollup(withRollup)
.build()
.withMetrics(schemaInfo.getAggsArray())
.withRollup(withRollup)
.build()
)
.setReportParseExceptions(false)
.setConcurrentEventAdd(true)
@ -536,7 +546,7 @@ public class GroupByBenchmark
}
}
private static <T> List<T> runQuery(QueryRunnerFactory factory, QueryRunner runner, Query<T> query)
private static <T> Sequence<T> runQuery(QueryRunnerFactory factory, QueryRunner runner, Query<T> query)
{
QueryToolChest toolChest = factory.getToolchest();
QueryRunner<T> theRunner = new FinalizeResultsQueryRunner<>(
@ -544,8 +554,7 @@ public class GroupByBenchmark
toolChest
);
Sequence<T> queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>());
return queryResult.toList();
return theRunner.run(QueryPlus.wrap(query), new HashMap<>());
}
@Benchmark
@ -559,8 +568,13 @@ public class GroupByBenchmark
new IncrementalIndexSegment(anIncrementalIndex, SegmentId.dummy("incIndex"))
);
List<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
blackhole.consume(results);
final Sequence<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
final Row lastRow = results.accumulate(
null,
(accumulated, in) -> in
);
blackhole.consume(lastRow);
}
@Benchmark
@ -574,8 +588,13 @@ public class GroupByBenchmark
new QueryableIndexSegment(queryableIndexes.get(0), SegmentId.dummy("qIndex"))
);
List<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
blackhole.consume(results);
final Sequence<Row> results = GroupByBenchmark.runQuery(factory, runner, query);
final Row lastRow = results.accumulate(
null,
(accumulated, in) -> in
);
blackhole.consume(lastRow);
}
@Benchmark

View File

@ -19,30 +19,26 @@
package org.apache.druid.benchmark.query;
import com.google.common.io.Files;
import org.apache.commons.io.FileUtils;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo;
import org.apache.druid.benchmark.datagen.BenchmarkSchemas;
import org.apache.druid.benchmark.datagen.SegmentGenerator;
import org.apache.druid.data.input.Row;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.server.security.AuthenticationResult;
import org.apache.druid.server.security.NoopEscalator;
import org.apache.druid.sql.SqlLifecycle;
import org.apache.druid.sql.SqlLifecycleFactory;
import org.apache.druid.sql.calcite.planner.Calcites;
import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
import org.apache.druid.sql.calcite.planner.PlannerResult;
import org.apache.druid.sql.calcite.schema.DruidSchema;
import org.apache.druid.sql.calcite.schema.SystemSchema;
import org.apache.druid.sql.calcite.util.CalciteTests;
@ -64,39 +60,112 @@ import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
import java.io.File;
import java.util.HashMap;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
/**
* Benchmark that compares the same groupBy query through the native query layer and through the SQL layer.
* Benchmark that tests various SQL queries.
*/
@State(Scope.Benchmark)
@Fork(value = 1)
@Warmup(iterations = 15)
@Measurement(iterations = 30)
@Measurement(iterations = 25)
public class SqlBenchmark
{
@Param({"200000", "1000000"})
private int rowsPerSegment;
static {
Calcites.setSystemProperties();
}
private static final Logger log = new Logger(SqlBenchmark.class);
private File tmpDir;
private SegmentGenerator segmentGenerator;
private SpecificSegmentsQuerySegmentWalker walker;
private SqlLifecycleFactory sqlLifecycleFactory;
private GroupByQuery groupByQuery;
private String sqlQuery;
private Closer resourceCloser;
private static final List<String> QUERIES = ImmutableList.of(
// 0, 1, 2, 3: Timeseries, unfiltered
"SELECT COUNT(*) FROM foo",
"SELECT COUNT(DISTINCT hyper) FROM foo",
"SELECT SUM(sumLongSequential), SUM(sumFloatNormal) FROM foo",
"SELECT FLOOR(__time TO MINUTE), SUM(sumLongSequential), SUM(sumFloatNormal) FROM foo GROUP BY 1",
// 4: Timeseries, low selectivity filter (90% of rows match)
"SELECT SUM(sumLongSequential), SUM(sumFloatNormal) FROM foo WHERE dimSequential NOT LIKE '%3'",
// 5: Timeseries, high selectivity filter (0.1% of rows match)
"SELECT SUM(sumLongSequential), SUM(sumFloatNormal) FROM foo WHERE dimSequential = '311'",
// 6: Timeseries, mixing low selectivity index-capable filter (90% of rows match) + cursor filter
"SELECT SUM(sumLongSequential), SUM(sumFloatNormal) FROM foo\n"
+ "WHERE dimSequential NOT LIKE '%3' AND maxLongUniform > 10",
// 7: Timeseries, low selectivity toplevel filter (90%), high selectivity filtered aggregator (0.1%)
"SELECT\n"
+ " SUM(sumLongSequential) FILTER(WHERE dimSequential = '311'),\n"
+ " SUM(sumFloatNormal)\n"
+ "FROM foo\n"
+ "WHERE dimSequential NOT LIKE '%3'",
// 8: Timeseries, no toplevel filter, various filtered aggregators with clauses repeated.
"SELECT\n"
+ " SUM(sumLongSequential) FILTER(WHERE dimSequential = '311'),\n"
+ " SUM(sumLongSequential) FILTER(WHERE dimSequential <> '311'),\n"
+ " SUM(sumLongSequential) FILTER(WHERE dimSequential LIKE '%3'),\n"
+ " SUM(sumLongSequential) FILTER(WHERE dimSequential NOT LIKE '%3'),\n"
+ " SUM(sumLongSequential),\n"
+ " SUM(sumFloatNormal) FILTER(WHERE dimSequential = '311'),\n"
+ " SUM(sumFloatNormal) FILTER(WHERE dimSequential <> '311'),\n"
+ " SUM(sumFloatNormal) FILTER(WHERE dimSequential LIKE '%3'),\n"
+ " SUM(sumFloatNormal) FILTER(WHERE dimSequential NOT LIKE '%3'),\n"
+ " SUM(sumFloatNormal),\n"
+ " COUNT(*) FILTER(WHERE dimSequential = '311'),\n"
+ " COUNT(*) FILTER(WHERE dimSequential <> '311'),\n"
+ " COUNT(*) FILTER(WHERE dimSequential LIKE '%3'),\n"
+ " COUNT(*) FILTER(WHERE dimSequential NOT LIKE '%3'),\n"
+ " COUNT(*)\n"
+ "FROM foo",
// 9: Timeseries, toplevel time filter, time-comparison filtered aggregators
"SELECT\n"
+ " SUM(sumLongSequential)\n"
+ " FILTER(WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2000-01-01 12:00:00'),\n"
+ " SUM(sumLongSequential)\n"
+ " FILTER(WHERE __time >= TIMESTAMP '2000-01-01 12:00:00' AND __time < TIMESTAMP '2000-01-02 00:00:00')\n"
+ "FROM foo\n"
+ "WHERE __time >= TIMESTAMP '2000-01-01 00:00:00' AND __time < TIMESTAMP '2000-01-02 00:00:00'",
// 10, 11: GroupBy two strings, unfiltered, unordered
"SELECT dimSequential, dimZipf, SUM(sumLongSequential) FROM foo GROUP BY 1, 2",
"SELECT dimSequential, dimZipf, SUM(sumLongSequential), COUNT(*) FROM foo GROUP BY 1, 2",
// 12, 13, 14: GroupBy one string, unfiltered, various aggregator configurations
"SELECT dimZipf FROM foo GROUP BY 1",
"SELECT dimZipf, COUNT(*) FROM foo GROUP BY 1 ORDER BY COUNT(*) DESC",
"SELECT dimZipf, SUM(sumLongSequential), COUNT(*) FROM foo GROUP BY 1 ORDER BY COUNT(*) DESC",
// 15, 16: GroupBy long, unfiltered, unordered; with and without aggregators
"SELECT maxLongUniform FROM foo GROUP BY 1",
"SELECT maxLongUniform, SUM(sumLongSequential), COUNT(*) FROM foo GROUP BY 1",
// 17, 18: GroupBy long, filter by long, unordered; with and without aggregators
"SELECT maxLongUniform FROM foo WHERE maxLongUniform > 10 GROUP BY 1",
"SELECT maxLongUniform, SUM(sumLongSequential), COUNT(*) FROM foo WHERE maxLongUniform > 10 GROUP BY 1"
);
@Param({"5000000"})
private int rowsPerSegment;
@Param({"false", "force"})
private String vectorize;
@Param({"10", "15"})
private String query;
@Nullable
private PlannerFactory plannerFactory;
private Closer closer = Closer.create();
@Setup(Level.Trial)
public void setup()
{
tmpDir = Files.createTempDir();
log.info("Starting benchmark setup using tmpDir[%s], rows[%,d].", tmpDir, rowsPerSegment);
final BenchmarkSchemaInfo schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get("basic");
final DataSegment dataSegment = DataSegment.builder()
@ -106,87 +175,55 @@ public class SqlBenchmark
.shardSpec(new LinearShardSpec(0))
.build();
this.segmentGenerator = new SegmentGenerator();
final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
.createQueryRunnerFactoryConglomerate();
final QueryRunnerFactoryConglomerate conglomerate = conglomerateCloserPair.lhs;
final PlannerConfig plannerConfig = new PlannerConfig();
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator());
log.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment);
final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerate = CalciteTests.createQueryRunnerFactoryConglomerate();
closer.register(conglomerate.rhs);
final SpecificSegmentsQuerySegmentWalker walker = new SpecificSegmentsQuerySegmentWalker(conglomerate.lhs).add(
dataSegment,
index
);
closer.register(walker);
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate.lhs, walker, plannerConfig);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
this.walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add(dataSegment, index);
final PlannerFactory plannerFactory = new PlannerFactory(
plannerFactory = new PlannerFactory(
druidSchema,
systemSchema,
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate.lhs),
CalciteTests.createOperatorTable(),
CalciteTests.createExprMacroTable(),
plannerConfig,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper()
);
this.sqlLifecycleFactory = CalciteTests.createSqlLifecycleFactory(plannerFactory);
groupByQuery = GroupByQuery
.builder()
.setDataSource("foo")
.setInterval(Intervals.ETERNITY)
.setDimensions(new DefaultDimensionSpec("dimZipf", "d0"), new DefaultDimensionSpec("dimSequential", "d1"))
.setAggregatorSpecs(new CountAggregatorFactory("c"))
.setGranularity(Granularities.ALL)
.build();
sqlQuery = "SELECT\n"
+ " dimZipf AS d0,"
+ " dimSequential AS d1,\n"
+ " COUNT(*) AS c\n"
+ "FROM druid.foo\n"
+ "GROUP BY dimZipf, dimSequential";
}
@TearDown(Level.Trial)
public void tearDown() throws Exception
{
if (walker != null) {
walker.close();
walker = null;
}
if (segmentGenerator != null) {
segmentGenerator.close();
segmentGenerator = null;
}
if (resourceCloser != null) {
resourceCloser.close();
}
if (tmpDir != null) {
FileUtils.deleteDirectory(tmpDir);
}
closer.close();
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryNative(Blackhole blackhole)
public void querySql(Blackhole blackhole) throws Exception
{
final Sequence<Row> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, new HashMap<>());
final List<Row> resultList = resultSequence.toList();
blackhole.consume(resultList);
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryPlanner(Blackhole blackhole) throws Exception
{
SqlLifecycle sqlLifecycle = sqlLifecycleFactory.factorize();
final List<Object[]> results = sqlLifecycle.runSimple(
sqlQuery,
null,
NoopEscalator.getInstance().createEscalatedAuthenticationResult()
).toList();
blackhole.consume(results);
final Map<String, Object> context = ImmutableMap.of("vectorize", vectorize);
final AuthenticationResult authenticationResult = NoopEscalator.getInstance()
.createEscalatedAuthenticationResult();
try (final DruidPlanner planner = plannerFactory.createPlanner(context, authenticationResult)) {
final PlannerResult plannerResult = planner.plan(QUERIES.get(Integer.parseInt(query)));
final Sequence<Object[]> resultSequence = plannerResult.run();
final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
blackhole.consume(lastRow);
}
}
}

View File

@ -0,0 +1,173 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.benchmark.query;
import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo;
import org.apache.druid.benchmark.datagen.BenchmarkSchemas;
import org.apache.druid.benchmark.datagen.SegmentGenerator;
import org.apache.druid.data.input.Row;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.server.security.AuthenticationResult;
import org.apache.druid.server.security.NoopEscalator;
import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
import org.apache.druid.sql.calcite.planner.PlannerResult;
import org.apache.druid.sql.calcite.schema.DruidSchema;
import org.apache.druid.sql.calcite.schema.SystemSchema;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
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.util.HashMap;
import java.util.concurrent.TimeUnit;
/**
* Benchmark that compares the same groupBy query through the native query layer and through the SQL layer.
*/
@State(Scope.Benchmark)
@Fork(value = 1)
@Warmup(iterations = 15)
@Measurement(iterations = 30)
public class SqlVsNativeBenchmark
{
@Param({"200000", "1000000"})
private int rowsPerSegment;
private static final Logger log = new Logger(SqlVsNativeBenchmark.class);
private SpecificSegmentsQuerySegmentWalker walker;
private PlannerFactory plannerFactory;
private GroupByQuery groupByQuery;
private String sqlQuery;
private Closer closer;
@Setup(Level.Trial)
public void setup()
{
this.closer = Closer.create();
final BenchmarkSchemaInfo schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get("basic");
final DataSegment dataSegment = DataSegment.builder()
.dataSource("foo")
.interval(schemaInfo.getDataInterval())
.version("1")
.shardSpec(new LinearShardSpec(0))
.build();
final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator());
log.info("Starting benchmark setup using tmpDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment);
final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment);
final Pair<QueryRunnerFactoryConglomerate, Closer> conglomerateCloserPair = CalciteTests
.createQueryRunnerFactoryConglomerate();
final QueryRunnerFactoryConglomerate conglomerate = conglomerateCloserPair.lhs;
final PlannerConfig plannerConfig = new PlannerConfig();
this.walker = closer.register(new SpecificSegmentsQuerySegmentWalker(conglomerate).add(dataSegment, index));
final DruidSchema druidSchema = CalciteTests.createMockSchema(conglomerate, walker, plannerConfig);
final SystemSchema systemSchema = CalciteTests.createMockSystemSchema(druidSchema, walker, plannerConfig);
plannerFactory = new PlannerFactory(
druidSchema,
systemSchema,
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
CalciteTests.createOperatorTable(),
CalciteTests.createExprMacroTable(),
plannerConfig,
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
CalciteTests.getJsonMapper()
);
groupByQuery = GroupByQuery
.builder()
.setDataSource("foo")
.setInterval(Intervals.ETERNITY)
.setDimensions(new DefaultDimensionSpec("dimZipf", "d0"), new DefaultDimensionSpec("dimSequential", "d1"))
.setAggregatorSpecs(new CountAggregatorFactory("c"))
.setGranularity(Granularities.ALL)
.build();
sqlQuery = "SELECT\n"
+ " dimZipf AS d0,"
+ " dimSequential AS d1,\n"
+ " COUNT(*) AS c\n"
+ "FROM druid.foo\n"
+ "GROUP BY dimZipf, dimSequential";
}
@TearDown(Level.Trial)
public void tearDown() throws Exception
{
closer.close();
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryNative(Blackhole blackhole)
{
final Sequence<Row> resultSequence = QueryPlus.wrap(groupByQuery).run(walker, new HashMap<>());
final Row lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
blackhole.consume(lastRow);
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void queryPlanner(Blackhole blackhole) throws Exception
{
final AuthenticationResult authenticationResult = NoopEscalator.getInstance()
.createEscalatedAuthenticationResult();
try (final DruidPlanner planner = plannerFactory.createPlanner(null, authenticationResult)) {
final PlannerResult plannerResult = planner.plan(sqlQuery);
final Sequence<Object[]> resultSequence = plannerResult.run();
final Object[] lastRow = resultSequence.accumulate(null, (accumulated, in) -> in);
blackhole.consume(lastRow);
}
}
}

View File

@ -0,0 +1,32 @@
<?xml version="1.0" encoding="UTF-8" ?>
<!--
~ Licensed to the Apache Software Foundation (ASF) under one
~ or more contributor license agreements. See the NOTICE file
~ distributed with this work for additional information
~ regarding copyright ownership. The ASF 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.
-->
<Configuration status="WARN">
<Appenders>
<Console name="Console" target="SYSTEM_OUT">
<PatternLayout pattern="%d{ISO8601} %p [%t] %c - %m%n"/>
</Console>
</Appenders>
<Loggers>
<Root level="info">
<AppenderRef ref="Console"/>
</Root>
</Loggers>
</Configuration>

View File

@ -28,11 +28,11 @@ import java.io.IOException;
public class FilteredSequence<T> implements Sequence<T>
{
private final Sequence<T> baseSequence;
private final Predicate<T> pred;
private final Predicate<? super T> pred;
public FilteredSequence(
Sequence<T> baseSequence,
Predicate<T> pred
Predicate<? super T> pred
)
{
this.baseSequence = baseSequence;

View File

@ -25,10 +25,10 @@ import com.google.common.base.Predicate;
*/
public class FilteringAccumulator<OutType, T> implements Accumulator<OutType, T>
{
private final Predicate<T> pred;
private final Predicate<? super T> pred;
private final Accumulator<OutType, T> accumulator;
public FilteringAccumulator(Predicate<T> pred, Accumulator<OutType, T> accumulator)
public FilteringAccumulator(Predicate<? super T> pred, Accumulator<OutType, T> accumulator)
{
this.pred = pred;
this.accumulator = accumulator;

View File

@ -25,12 +25,12 @@ import com.google.common.base.Predicate;
*/
public class FilteringYieldingAccumulator<OutType, T> extends YieldingAccumulator<OutType, T>
{
private final Predicate<T> pred;
private final Predicate<? super T> pred;
private final YieldingAccumulator<OutType, T> accumulator;
private volatile boolean didSomething = false;
public FilteringYieldingAccumulator(Predicate<T> pred, YieldingAccumulator<OutType, T> accumulator)
public FilteringYieldingAccumulator(Predicate<? super T> pred, YieldingAccumulator<OutType, T> accumulator)
{
this.pred = pred;
this.accumulator = accumulator;

View File

@ -19,6 +19,7 @@
package org.apache.druid.java.util.common.guava;
import com.google.common.base.Predicate;
import com.google.common.collect.Ordering;
import java.io.Closeable;
@ -53,7 +54,7 @@ public interface Sequence<T>
*/
<OutType> OutType accumulate(OutType initValue, Accumulator<OutType, T> accumulator);
/**
/**
* Return a Yielder for accumulated sequence.
*
* @param initValue the initial value to pass along to start the accumulation.
@ -71,11 +72,13 @@ public interface Sequence<T>
return new MappedSequence<>(this, mapper);
}
default Sequence<T> filter(Predicate<? super T> predicate)
{
return Sequences.filter(this, predicate);
}
/**
* This will materialize the entire sequence. Use at your own risk.
*
* Several benchmarks rely on this method to eagerly accumulate Sequences to ArrayLists. e.g.
* GroupByBenchmark.
*/
default List<T> toList()
{

View File

@ -84,7 +84,7 @@ public class Sequences
return new MappedSequence<>(sequence, fn::apply);
}
public static <T> Sequence<T> filter(Sequence<T> sequence, Predicate<T> pred)
public static <T> Sequence<T> filter(Sequence<T> sequence, Predicate<? super T> pred)
{
return new FilteredSequence<>(sequence, pred);
}

View File

@ -60,3 +60,31 @@ In addition, some query types offer context parameters specific to that query ty
### GroupBy queries
See [GroupBy query context](groupbyquery.html#query-context).
### Vectorizable queries
The GroupBy and Timeseries query types can run in _vectorized_ mode, which speeds up query execution by processing
batches of rows at a time. Not all queries can be vectorized. In particular, vectorization currently has the following
requirements:
- All query-level filters must either be able to run on bitmap indexes or must offer vectorized row-matchers. These
include "selector", "bound", "in", "like", "regex", "search", "and", "or", and "not".
- All filters in filtered aggregators must offer vectorized row-matchers.
- All aggregators must offer vectorized implementations. These include "count", "doubleSum", "floatSum", "longSum",
"hyperUnique", and "filtered".
- No virtual columns.
- For GroupBy: All dimension specs must be "default" (no extraction functions or filtered dimension specs).
- For GroupBy: No multi-value dimensions.
- For Timeseries: No "descending" order.
- Only immutable segments (not real-time).
Other query types (like TopN, Scan, Select, and Search) ignore the "vectorize" parameter, and will execute without
vectorization. These query types will ignore the "vectorize" parameter even if it is set to `"force"`.
Vectorization is an alpha-quality feature as of Druid #{DRUIDVERSION}. We heartily welcome any feedback and testing
from the community as we work to battle-test it.
|property|default| description|
|--------|-------|------------|
|vectorize|`false`|Enables or disables vectorized query execution. Possible values are `false` (disabled), `true` (enabled if possible, disabled otherwise, on a per-segment basis), and `force` (enabled, and groupBy or timeseries queries that cannot be vectorized will fail). The `"force"` setting is meant to aid in testing, and is not generally useful in production (since real-time segments can never be processed with vectorized execution, any queries on real-time data will fail).|
|vectorSize|`512`|Sets the row batching size for a particular query.|

View File

@ -20,12 +20,13 @@
package org.apache.druid.query.aggregation.datasketches.theta;
import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.yahoo.sketches.theta.Sketches;
import com.yahoo.sketches.theta.UpdateSketch;
import org.apache.druid.data.input.MapBasedRow;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorAdapters;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.groupby.epinephelinae.BufferHashGrouper;
import org.apache.druid.query.groupby.epinephelinae.Grouper;
@ -47,11 +48,13 @@ public class BufferHashGrouperUsingSketchMergeAggregatorFactoryTest
final BufferHashGrouper<Integer> grouper = new BufferHashGrouper<>(
Suppliers.ofInstance(ByteBuffer.allocate(bufferSize)),
GrouperTestUtil.intKeySerde(),
columnSelectorFactory,
new AggregatorFactory[]{
new SketchMergeAggregatorFactory("sketch", "sketch", 16, false, true, 2),
new CountAggregatorFactory("count")
},
AggregatorAdapters.factorizeBuffered(
columnSelectorFactory,
ImmutableList.of(
new SketchMergeAggregatorFactory("sketch", "sketch", 16, false, true, 2),
new CountAggregatorFactory("count")
)
),
Integer.MAX_VALUE,
0.75f,
initialBuckets,

View File

@ -246,21 +246,24 @@ public class BloomDimFilterTest extends BaseFilterTest
@Test
public void testExpressionVirtualColumn() throws IOException
{
assertFilterMatches(
assertFilterMatchesSkipVectorize(
new BloomDimFilter("expr", bloomKFilter(1000, 1.1F), null),
ImmutableList.of("0", "1", "2", "3", "4", "5")
);
assertFilterMatches(new BloomDimFilter("expr", bloomKFilter(1000, 1.2F), null), ImmutableList.of());
assertFilterMatches(
assertFilterMatchesSkipVectorize(new BloomDimFilter("expr", bloomKFilter(1000, 1.2F), null), ImmutableList.of());
assertFilterMatchesSkipVectorize(
new BloomDimFilter("exprDouble", bloomKFilter(1000, 2.1D), null),
ImmutableList.of("0", "1", "2", "3", "4", "5")
);
assertFilterMatches(new BloomDimFilter("exprDouble", bloomKFilter(1000, 2.2D), null), ImmutableList.of());
assertFilterMatches(
assertFilterMatchesSkipVectorize(
new BloomDimFilter("exprDouble", bloomKFilter(1000, 2.2D), null),
ImmutableList.of()
);
assertFilterMatchesSkipVectorize(
new BloomDimFilter("exprLong", bloomKFilter(1000, 3L), null),
ImmutableList.of("0", "1", "2", "3", "4", "5")
);
assertFilterMatches(new BloomDimFilter("exprLong", bloomKFilter(1000, 4L), null), ImmutableList.of());
assertFilterMatchesSkipVectorize(new BloomDimFilter("exprLong", bloomKFilter(1000, 4L), null), ImmutableList.of());
}
@Test

View File

@ -169,6 +169,9 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
@Test
public void testBloomFilterVirtualColumn() throws Exception
{
// Cannot vectorize due to expression virtual columns.
cannotVectorize();
BloomKFilter filter = new BloomKFilter(1500);
filter.addString("def-foo");
byte[] bytes = BloomFilterSerializersModule.bloomKFilterToBytes(filter);
@ -199,6 +202,9 @@ public class BloomDimFilterSqlTest extends BaseCalciteQueryTest
@Test
public void testBloomFilterVirtualColumnNumber() throws Exception
{
// Cannot vectorize due to expression virtual columns.
cannotVectorize();
BloomKFilter filter = new BloomKFilter(1500);
filter.addFloat(20.2f);
byte[] bytes = BloomFilterSerializersModule.bloomKFilterToBytes(filter);

View File

@ -45,6 +45,7 @@ import org.junit.runners.Parameterized;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
/**
*
@ -58,9 +59,22 @@ public class VarianceGroupByQueryTest
private final String testName;
@Parameterized.Parameters(name = "{0}")
public static Collection<?> constructorFeeder()
public static Collection<Object[]> constructorFeeder()
{
return GroupByQueryRunnerTest.constructorFeeder();
// Use GroupByQueryRunnerTest's constructorFeeder, but remove vectorized tests, since this aggregator
// can't vectorize yet.
return GroupByQueryRunnerTest.constructorFeeder().stream()
.filter(constructor -> !((boolean) constructor[4]) /* !vectorize */)
.map(
constructor ->
new Object[]{
constructor[0],
constructor[1],
constructor[2],
constructor[3]
}
)
.collect(Collectors.toList());
}
public VarianceGroupByQueryTest(

View File

@ -37,6 +37,8 @@ import org.junit.runners.Parameterized;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
@RunWith(Parameterized.class)
public class VarianceTimeseriesQueryTest
@ -44,13 +46,22 @@ public class VarianceTimeseriesQueryTest
@Parameterized.Parameters(name = "{0}:descending={1}")
public static Iterable<Object[]> constructorFeeder()
{
return TimeseriesQueryRunnerTest.constructorFeeder();
// Use TimeseriesQueryRunnerTest's constructorFeeder, but remove vectorized tests, since this aggregator
// can't vectorize yet.
return StreamSupport.stream(TimeseriesQueryRunnerTest.constructorFeeder().spliterator(), false)
.filter(constructor -> !((boolean) constructor[2]) /* !vectorize */)
.map(constructor -> new Object[]{constructor[0], constructor[1], constructor[3]})
.collect(Collectors.toList());
}
private final QueryRunner runner;
private final boolean descending;
public VarianceTimeseriesQueryTest(QueryRunner runner, boolean descending, List<AggregatorFactory> aggregatorFactories)
public VarianceTimeseriesQueryTest(
QueryRunner runner,
boolean descending,
List<AggregatorFactory> aggregatorFactories
)
{
this.runner = runner;
this.descending = descending;

View File

@ -1227,7 +1227,7 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<version>2.19.1</version>
<version>2.22.2</version>
<configuration>
<!-- locale settings must be set on the command line before startup -->
<!-- set default options -->

View File

@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.collections.bitmap;
import com.google.common.base.Preconditions;
import org.roaringbitmap.BatchIterator;
import org.roaringbitmap.IntIterator;
public final class BatchIteratorAdapter implements BatchIterator
{
private final IntIterator iterator;
public BatchIteratorAdapter(IntIterator iterator)
{
this.iterator = Preconditions.checkNotNull(iterator, "iterator");
}
@Override
public int nextBatch(int[] buffer)
{
int i;
for (i = 0; i < buffer.length && iterator.hasNext(); i++) {
buffer[i] = iterator.next();
}
return i;
}
@Override
public boolean hasNext()
{
return iterator.hasNext();
}
@Override
public BatchIterator clone()
{
// It's okay to make a "new BatchIteratorAdapter" instead of calling super.clone(), since this class is final.
return new BatchIteratorAdapter(iterator.clone());
}
}

View File

@ -19,6 +19,7 @@
package org.apache.druid.collections.bitmap;
import org.roaringbitmap.BatchIterator;
import org.roaringbitmap.IntIterator;
/**
@ -32,6 +33,14 @@ public interface ImmutableBitmap
*/
IntIterator iterator();
/**
* @return a batched iterator over the set bits of this bitmap
*/
default BatchIterator batchIterator()
{
return new BatchIteratorAdapter(iterator());
}
/**
* @return The number of bits set to true in this bitmap
*/

View File

@ -19,6 +19,7 @@
package org.apache.druid.collections.bitmap;
import org.roaringbitmap.BatchIterator;
import org.roaringbitmap.IntIterator;
import org.roaringbitmap.buffer.ImmutableRoaringBitmap;
@ -76,6 +77,12 @@ public class WrappedImmutableRoaringBitmap implements ImmutableBitmap
return bitmap.getIntIterator();
}
@Override
public BatchIterator batchIterator()
{
return bitmap.getBatchIterator();
}
@Override
public int size()
{

View File

@ -196,6 +196,12 @@ public class DefaultQueryMetrics<QueryType extends Query<?>> implements QueryMet
// Emit nothing by default.
}
@Override
public void vectorized(final boolean vectorized)
{
// Emit nothing by default.
}
@Override
public BitmapResultFactory<?> makeBitmapResultFactory(BitmapFactory factory)
{

View File

@ -23,7 +23,10 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Numbers;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import java.util.concurrent.TimeUnit;
@ -43,11 +46,44 @@ public class QueryContexts
public static final boolean DEFAULT_USE_CACHE = true;
public static final boolean DEFAULT_POPULATE_RESULTLEVEL_CACHE = true;
public static final boolean DEFAULT_USE_RESULTLEVEL_CACHE = true;
public static final Vectorize DEFAULT_VECTORIZE = Vectorize.FALSE;
public static final int DEFAULT_PRIORITY = 0;
public static final int DEFAULT_UNCOVERED_INTERVALS_LIMIT = 0;
public static final long DEFAULT_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5);
public static final long NO_TIMEOUT = 0;
@SuppressWarnings("unused") // Used by Jackson serialization
public enum Vectorize
{
FALSE {
@Override
public boolean shouldVectorize(final boolean canVectorize)
{
return false;
}
},
TRUE {
@Override
public boolean shouldVectorize(final boolean canVectorize)
{
return canVectorize;
}
},
FORCE {
@Override
public boolean shouldVectorize(final boolean canVectorize)
{
if (!canVectorize) {
throw new ISE("Cannot vectorize!");
}
return true;
}
};
public abstract boolean shouldVectorize(boolean canVectorize);
}
public static <T> boolean isBySegment(Query<T> query)
{
return isBySegment(query, DEFAULT_BY_SEGMENT);
@ -113,6 +149,16 @@ public class QueryContexts
return parseBoolean(query, "serializeDateTimeAsLongInner", defaultValue);
}
public static <T> Vectorize getVectorize(Query<T> query)
{
return parseEnum(query, "vectorize", Vectorize.class, DEFAULT_VECTORIZE);
}
public static <T> int getVectorSize(Query<T> query)
{
return parseInt(query, "vectorSize", QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE);
}
public static <T> int getUncoveredIntervalsLimit(Query<T> query)
{
return getUncoveredIntervalsLimit(query, DEFAULT_UNCOVERED_INTERVALS_LIMIT);
@ -239,4 +285,19 @@ public class QueryContexts
private QueryContexts()
{
}
static <T, E extends Enum<E>> E parseEnum(Query<T> query, String key, Class<E> clazz, E defaultValue)
{
Object val = query.getContextValue(key);
if (val == null) {
return defaultValue;
}
if (val instanceof String) {
return Enum.valueOf(clazz, StringUtils.toUpperCase((String) val));
} else if (val instanceof Boolean) {
return Enum.valueOf(clazz, StringUtils.toUpperCase(String.valueOf(val)));
} else {
throw new ISE("Unknown type [%s]. Cannot parse!", val.getClass());
}
}
}

View File

@ -236,6 +236,12 @@ public interface QueryMetrics<QueryType extends Query<?>>
*/
void identity(String identity);
/**
* Sets whether or not a segment scan has been vectorized. Generally expected to only be attached to segment-level
* metrics, since at whole-query level we might have a mix of vectorized and non-vectorized segment scans.
*/
void vectorized(boolean vectorized);
/**
* Creates a {@link BitmapResultFactory} which may record some information along bitmap construction from {@link
* #preFilters(List)}. The returned BitmapResultFactory may add some dimensions to this QueryMetrics from it's {@link

View File

@ -21,7 +21,6 @@ package org.apache.druid.query;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicates;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.java.util.common.guava.Sequences;
@ -34,19 +33,19 @@ import org.joda.time.Interval;
import java.io.Closeable;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
*/
public class QueryRunnerHelper
{
public static <T> Sequence<Result<T>> makeCursorBasedQuery(
final StorageAdapter adapter,
List<Interval> queryIntervals,
Filter filter,
VirtualColumns virtualColumns,
boolean descending,
Granularity granularity,
final List<Interval> queryIntervals,
final Filter filter,
final VirtualColumns virtualColumns,
final boolean descending,
final Granularity granularity,
final Function<Cursor, Result<T>> mapFn
)
{
@ -57,16 +56,9 @@ public class QueryRunnerHelper
return Sequences.filter(
Sequences.map(
adapter.makeCursors(filter, queryIntervals.get(0), virtualColumns, granularity, descending, null),
new Function<Cursor, Result<T>>()
{
@Override
public Result<T> apply(Cursor input)
{
return mapFn.apply(input);
}
}
mapFn
),
Predicates.notNull()
Objects::nonNull
);
}

View File

@ -0,0 +1,389 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation;
import com.google.common.primitives.Ints;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
/**
* A class that helps query engines use Buffer- or VectorAggregators in a consistent way.
*
* The two main benefits this class provides are:
*
* (1) Query engines can treat BufferAggregators and VectorAggregators the same for operations that are equivalent
* across them, like "init", "get", "relocate", and "close".
* (2) Query engines are freed from the need to manage how much space each individual aggregator needs. They only
* need to allocate a block of size "spaceNeeded".
*/
public class AggregatorAdapters implements Closeable
{
private static final Logger log = new Logger(AggregatorAdapters.class);
private final List<Adapter> adapters;
private final List<AggregatorFactory> factories;
private final int[] aggregatorPositions;
private final int spaceNeeded;
private AggregatorAdapters(final List<Adapter> adapters)
{
this.adapters = adapters;
this.factories = adapters.stream().map(Adapter::getFactory).collect(Collectors.toList());
this.aggregatorPositions = new int[adapters.size()];
long nextPosition = 0;
for (int i = 0; i < adapters.size(); i++) {
final AggregatorFactory aggregatorFactory = adapters.get(i).getFactory();
aggregatorPositions[i] = Ints.checkedCast(nextPosition);
nextPosition += aggregatorFactory.getMaxIntermediateSizeWithNulls();
}
this.spaceNeeded = Ints.checkedCast(nextPosition);
}
/**
* Create an adapters object based on {@link VectorAggregator}.
*/
public static AggregatorAdapters factorizeVector(
final VectorColumnSelectorFactory columnSelectorFactory,
final List<AggregatorFactory> aggregatorFactories
)
{
final Adapter[] adapters = new Adapter[aggregatorFactories.size()];
for (int i = 0; i < aggregatorFactories.size(); i++) {
final AggregatorFactory aggregatorFactory = aggregatorFactories.get(i);
adapters[i] = new VectorAggregatorAdapter(
aggregatorFactory,
aggregatorFactory.factorizeVector(columnSelectorFactory)
);
}
return new AggregatorAdapters(Arrays.asList(adapters));
}
/**
* Create an adapters object based on {@link BufferAggregator}.
*/
public static AggregatorAdapters factorizeBuffered(
final ColumnSelectorFactory columnSelectorFactory,
final List<AggregatorFactory> aggregatorFactories
)
{
final Adapter[] adapters = new Adapter[aggregatorFactories.size()];
for (int i = 0; i < aggregatorFactories.size(); i++) {
final AggregatorFactory aggregatorFactory = aggregatorFactories.get(i);
adapters[i] = new BufferAggregatorAdapter(
aggregatorFactory,
aggregatorFactory.factorizeBuffered(columnSelectorFactory)
);
}
return new AggregatorAdapters(Arrays.asList(adapters));
}
/**
* Return the amount of buffer bytes needed by all aggregators wrapped up in this object.
*/
public int spaceNeeded()
{
return spaceNeeded;
}
/**
* Return the {@link AggregatorFactory} objects that were used to create this object.
*/
public List<AggregatorFactory> factories()
{
return factories;
}
/**
* Return the individual positions of each aggregator within a hypothetical buffer of size {@link #spaceNeeded()}.
*/
public int[] aggregatorPositions()
{
return aggregatorPositions;
}
/**
* Return the number of aggregators in this object.
*/
public int size()
{
return adapters.size();
}
/**
* Initialize all aggregators.
*
* @param buf aggregation buffer
* @param position position in buffer where our block of size {@link #spaceNeeded()} starts
*/
public void init(final ByteBuffer buf, final int position)
{
for (int i = 0; i < adapters.size(); i++) {
adapters.get(i).init(buf, position + aggregatorPositions[i]);
}
}
/**
* Call {@link BufferAggregator#aggregate(ByteBuffer, int)} on all of our aggregators.
*
* This method is only valid if the underlying aggregators are {@link BufferAggregator}.
*/
public void aggregateBuffered(final ByteBuffer buf, final int position)
{
for (int i = 0; i < adapters.size(); i++) {
final Adapter adapter = adapters.get(i);
adapter.asBufferAggregator().aggregate(buf, position + aggregatorPositions[i]);
}
}
/**
* Call {@link VectorAggregator#aggregate(ByteBuffer, int, int, int)} on all of our aggregators.
*
* This method is only valid if the underlying aggregators are {@link VectorAggregator}.
*/
public void aggregateVector(
final ByteBuffer buf,
final int position,
final int start,
final int end
)
{
for (int i = 0; i < adapters.size(); i++) {
final Adapter adapter = adapters.get(i);
adapter.asVectorAggregator().aggregate(buf, position + aggregatorPositions[i], start, end);
}
}
/**
* Call {@link VectorAggregator#aggregate(ByteBuffer, int, int[], int[], int)} on all of our aggregators.
*
* This method is only valid if the underlying aggregators are {@link VectorAggregator}.
*/
public void aggregateVector(
final ByteBuffer buf,
final int numRows,
final int[] positions,
@Nullable final int[] rows
)
{
for (int i = 0; i < adapters.size(); i++) {
final Adapter adapter = adapters.get(i);
adapter.asVectorAggregator().aggregate(buf, numRows, positions, rows, aggregatorPositions[i]);
}
}
/**
* Retrieve aggregation state from one of our aggregators.
*
* @param buf aggregation buffer
* @param position position in buffer where our block of size {@link #spaceNeeded()} starts
* @param aggregatorNumber which aggregator to retrieve state, from 0 to {@link #size()} - 1
*/
@Nullable
public Object get(final ByteBuffer buf, final int position, final int aggregatorNumber)
{
return adapters.get(aggregatorNumber).get(buf, position + aggregatorPositions[aggregatorNumber]);
}
/**
* Inform all of our aggregators that they are being relocated.
*/
public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
{
for (int i = 0; i < adapters.size(); i++) {
adapters.get(i).relocate(
oldPosition + aggregatorPositions[i],
newPosition + aggregatorPositions[i],
oldBuffer,
newBuffer
);
}
}
/**
* Close all of our aggregators.
*/
@Override
public void close()
{
for (Adapter adapter : adapters) {
try {
adapter.close();
}
catch (Exception e) {
log.warn(e, "Could not close aggregator [%s], skipping.", adapter.getFactory().getName());
}
}
}
/**
* The interface that allows this class to achieve its goals of partially unifying handling of
* BufferAggregator and VectorAggregator. Private, since it doesn't escape this class and the
* only two implementations are private static classes below.
*/
private interface Adapter extends Closeable
{
void init(ByteBuffer buf, int position);
@Nullable
Object get(ByteBuffer buf, int position);
void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer);
@Override
void close();
AggregatorFactory getFactory();
BufferAggregator asBufferAggregator();
VectorAggregator asVectorAggregator();
}
private static class VectorAggregatorAdapter implements Adapter
{
private final AggregatorFactory factory;
private final VectorAggregator aggregator;
VectorAggregatorAdapter(final AggregatorFactory factory, final VectorAggregator aggregator)
{
this.factory = factory;
this.aggregator = aggregator;
}
@Override
public void init(final ByteBuffer buf, final int position)
{
aggregator.init(buf, position);
}
@Override
public Object get(final ByteBuffer buf, final int position)
{
return aggregator.get(buf, position);
}
@Override
public void close()
{
aggregator.close();
}
@Override
public void relocate(
final int oldPosition,
final int newPosition,
final ByteBuffer oldBuffer,
final ByteBuffer newBuffer
)
{
aggregator.relocate(oldPosition, newPosition, oldBuffer, newBuffer);
}
@Override
public AggregatorFactory getFactory()
{
return factory;
}
@Override
public BufferAggregator asBufferAggregator()
{
throw new ISE("Not a BufferAggregator!");
}
@Override
public VectorAggregator asVectorAggregator()
{
return aggregator;
}
}
private static class BufferAggregatorAdapter implements Adapter
{
private final AggregatorFactory factory;
private final BufferAggregator aggregator;
BufferAggregatorAdapter(final AggregatorFactory factory, final BufferAggregator aggregator)
{
this.factory = factory;
this.aggregator = aggregator;
}
@Override
public void init(final ByteBuffer buf, final int position)
{
aggregator.init(buf, position);
}
@Override
public Object get(final ByteBuffer buf, final int position)
{
return aggregator.get(buf, position);
}
@Override
public void close()
{
aggregator.close();
}
@Override
public void relocate(
final int oldPosition,
final int newPosition,
final ByteBuffer oldBuffer,
final ByteBuffer newBuffer
)
{
aggregator.relocate(oldPosition, newPosition, oldBuffer, newBuffer);
}
@Override
public AggregatorFactory getFactory()
{
return factory;
}
@Override
public BufferAggregator asBufferAggregator()
{
return aggregator;
}
@Override
public VectorAggregator asVectorAggregator()
{
throw new ISE("Not a VectorAggregator!");
}
}
}

View File

@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.PerSegmentQueryOptimizationContext;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import javax.annotation.Nullable;
import java.util.Arrays;
@ -49,6 +50,23 @@ public abstract class AggregatorFactory implements Cacheable
public abstract BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory);
/**
* Create a VectorAggregator based on the provided column selector factory. Will throw an exception if
* this aggregation class does not support vectorization: check "canVectorize" first.
*/
public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory)
{
throw new UOE("Aggregator[%s] cannot vectorize", getClass().getName());
}
/**
* Returns whether or not this aggregation class supports vectorization. The default implementation returns false.
*/
public boolean canVectorize()
{
return false;
}
public abstract Comparator getComparator();
/**

View File

@ -24,6 +24,7 @@ import org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop;
import org.apache.druid.query.monomorphicprocessing.HotLoopCallee;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
/**
@ -33,6 +34,8 @@ import java.nio.ByteBuffer;
*
* Thus, an Aggregator can be thought of as a closure over some other thing that is stateful and changes between calls
* to aggregate(...).
*
* @see VectorAggregator, the vectorized version
*/
@ExtensionPoint
public interface BufferAggregator extends HotLoopCallee
@ -87,6 +90,7 @@ public interface BufferAggregator extends HotLoopCallee
* @param position offset within the byte buffer at which the aggregate value is stored
* @return the Object representation of the aggregate
*/
@Nullable
Object get(ByteBuffer buf, int position);
/**

View File

@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import javax.annotation.Nullable;
import java.util.Collections;
@ -58,12 +59,24 @@ public class CountAggregatorFactory extends AggregatorFactory
return new CountBufferAggregator();
}
@Override
public VectorAggregator factorizeVector(final VectorColumnSelectorFactory selectorFactory)
{
return new CountVectorAggregator();
}
@Override
public Comparator getComparator()
{
return CountAggregator.COMPARATOR;
}
@Override
public boolean canVectorize()
{
return true;
}
@Override
public Object combine(Object lhs, Object rhs)
{

View File

@ -0,0 +1,66 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
public class CountVectorAggregator implements VectorAggregator
{
@Override
public void init(final ByteBuffer buf, final int position)
{
buf.putLong(position, 0);
}
@Override
public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
{
final int delta = endRow - startRow;
buf.putLong(position, buf.getLong(position) + delta);
}
@Override
public void aggregate(
final ByteBuffer buf,
final int numRows,
final int[] positions,
@Nullable final int[] rows,
final int positionOffset
)
{
for (int i = 0; i < numRows; i++) {
final int position = positions[i] + positionOffset;
buf.putLong(position, buf.getLong(position) + 1);
}
}
@Override
public Object get(final ByteBuffer buf, final int position)
{
return buf.getLong(position);
}
@Override
public void close()
{
// Nothing to close.
}
}

View File

@ -26,6 +26,8 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseDoubleColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
@ -61,12 +63,24 @@ public class DoubleSumAggregatorFactory extends SimpleDoubleAggregatorFactory
);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseDoubleColumnValueSelector selector)
{
return new DoubleSumAggregator(selector);
}
@Override
public boolean canVectorize()
{
return expression == null;
}
@Override
protected BufferAggregator factorizeBuffered(
ColumnSelectorFactory metricFactory,
@ -76,6 +90,15 @@ public class DoubleSumAggregatorFactory extends SimpleDoubleAggregatorFactory
return new DoubleSumBufferAggregator(selector);
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,
VectorValueSelector selector
)
{
return new DoubleSumVectorAggregator(selector);
}
@Override
@Nullable
public Object combine(@Nullable Object lhs, @Nullable Object rhs)

View File

@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
public class DoubleSumVectorAggregator implements VectorAggregator
{
private final VectorValueSelector selector;
public DoubleSumVectorAggregator(final VectorValueSelector selector)
{
this.selector = selector;
}
@Override
public void init(final ByteBuffer buf, final int position)
{
buf.putDouble(position, 0);
}
@Override
public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
{
final double[] vector = selector.getDoubleVector();
double sum = 0;
for (int i = startRow; i < endRow; i++) {
sum += vector[i];
}
buf.putDouble(position, buf.getDouble(position) + sum);
}
@Override
public void aggregate(
final ByteBuffer buf,
final int numRows,
final int[] positions,
@Nullable final int[] rows,
final int positionOffset
)
{
final double[] vector = selector.getDoubleVector();
for (int i = 0; i < numRows; i++) {
final int position = positions[i] + positionOffset;
buf.putDouble(position, buf.getDouble(position) + vector[rows != null ? rows[i] : i]);
}
}
@Override
public Object get(final ByteBuffer buf, final int position)
{
return buf.getDouble(position);
}
@Override
public void close()
{
// Nothing to close.
}
}

View File

@ -25,11 +25,13 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import org.apache.druid.query.PerSegmentQueryOptimizationContext;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.filter.IntervalDimFilter;
import org.apache.druid.query.filter.ValueMatcher;
import org.apache.druid.query.filter.vector.VectorValueMatcher;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.filter.Filters;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.joda.time.Interval;
import javax.annotation.Nullable;
@ -42,7 +44,10 @@ import java.util.Objects;
public class FilteredAggregatorFactory extends AggregatorFactory
{
private final AggregatorFactory delegate;
private final DimFilter filter;
private final DimFilter dimFilter;
private final Filter filter;
@Nullable
private final String name;
// Constructor for backwards compat only
@ -57,22 +62,23 @@ public class FilteredAggregatorFactory extends AggregatorFactory
@JsonCreator
public FilteredAggregatorFactory(
@JsonProperty("aggregator") AggregatorFactory delegate,
@JsonProperty("filter") DimFilter filter,
@JsonProperty("name") String name
@JsonProperty("filter") DimFilter dimFilter,
@Nullable @JsonProperty("name") String name
)
{
Preconditions.checkNotNull(delegate);
Preconditions.checkNotNull(filter);
Preconditions.checkNotNull(delegate, "aggregator");
Preconditions.checkNotNull(dimFilter, "filter");
this.delegate = delegate;
this.filter = filter;
this.dimFilter = dimFilter;
this.filter = dimFilter.toFilter();
this.name = name;
}
@Override
public Aggregator factorize(ColumnSelectorFactory columnSelectorFactory)
{
final ValueMatcher valueMatcher = Filters.toFilter(filter).makeMatcher(columnSelectorFactory);
final ValueMatcher valueMatcher = filter.makeMatcher(columnSelectorFactory);
return new FilteredAggregator(
valueMatcher,
delegate.factorize(columnSelectorFactory)
@ -82,13 +88,30 @@ public class FilteredAggregatorFactory extends AggregatorFactory
@Override
public BufferAggregator factorizeBuffered(ColumnSelectorFactory columnSelectorFactory)
{
final ValueMatcher valueMatcher = Filters.toFilter(filter).makeMatcher(columnSelectorFactory);
final ValueMatcher valueMatcher = filter.makeMatcher(columnSelectorFactory);
return new FilteredBufferAggregator(
valueMatcher,
delegate.factorizeBuffered(columnSelectorFactory)
);
}
@Override
public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelectorFactory)
{
Preconditions.checkState(canVectorize(), "Cannot vectorize");
final VectorValueMatcher valueMatcher = filter.makeVectorMatcher(columnSelectorFactory);
return new FilteredVectorAggregator(
valueMatcher,
delegate.factorizeVector(columnSelectorFactory)
);
}
@Override
public boolean canVectorize()
{
return delegate.canVectorize() && filter.canVectorizeMatcher();
}
@Override
public Comparator getComparator()
{
@ -147,7 +170,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
@Override
public byte[] getCacheKey()
{
byte[] filterCacheKey = filter.getCacheKey();
byte[] filterCacheKey = dimFilter.getCacheKey();
byte[] aggregatorCacheKey = delegate.getCacheKey();
return ByteBuffer.allocate(1 + filterCacheKey.length + aggregatorCacheKey.length)
.put(AggregatorUtil.FILTERED_AGG_CACHE_TYPE_ID)
@ -171,8 +194,8 @@ public class FilteredAggregatorFactory extends AggregatorFactory
@Override
public AggregatorFactory optimizeForSegment(PerSegmentQueryOptimizationContext optimizationContext)
{
if (filter instanceof IntervalDimFilter) {
IntervalDimFilter intervalDimFilter = ((IntervalDimFilter) filter);
if (dimFilter instanceof IntervalDimFilter) {
IntervalDimFilter intervalDimFilter = ((IntervalDimFilter) dimFilter);
if (intervalDimFilter.getExtractionFn() != null) {
// no support for extraction functions right now
return this;
@ -238,7 +261,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
@JsonProperty
public DimFilter getFilter()
{
return filter;
return dimFilter;
}
@Override
@ -248,7 +271,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
}
@Override
public boolean equals(Object o)
public boolean equals(final Object o)
{
if (this == o) {
return true;
@ -256,16 +279,17 @@ public class FilteredAggregatorFactory extends AggregatorFactory
if (o == null || getClass() != o.getClass()) {
return false;
}
FilteredAggregatorFactory that = (FilteredAggregatorFactory) o;
final FilteredAggregatorFactory that = (FilteredAggregatorFactory) o;
return Objects.equals(delegate, that.delegate) &&
Objects.equals(filter, that.filter) &&
Objects.equals(dimFilter, that.dimFilter) &&
Objects.equals(name, that.name);
}
@Override
public int hashCode()
{
return Objects.hash(delegate, filter, name);
return Objects.hash(delegate, dimFilter, name);
}
@Override
@ -273,7 +297,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
{
return "FilteredAggregatorFactory{" +
"delegate=" + delegate +
", filter=" + filter +
", dimFilter=" + dimFilter +
", name='" + name + '\'' +
'}';
}

View File

@ -0,0 +1,153 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.query.filter.vector.ReadableVectorMatch;
import org.apache.druid.query.filter.vector.VectorMatch;
import org.apache.druid.query.filter.vector.VectorValueMatcher;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Arrays;
public class FilteredVectorAggregator implements VectorAggregator
{
private final VectorValueMatcher matcher;
private final VectorAggregator delegate;
private final int[] delegatePositions;
@Nullable
private VectorMatch maskScratch = null;
public FilteredVectorAggregator(
final VectorValueMatcher matcher,
final VectorAggregator delegate
)
{
this.matcher = matcher;
this.delegate = delegate;
this.delegatePositions = new int[matcher.getMaxVectorSize()];
}
@Override
public void init(final ByteBuffer buf, final int position)
{
delegate.init(buf, position);
}
@Override
public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
{
final ReadableVectorMatch mask;
if (startRow == 0) {
mask = VectorMatch.allTrue(endRow);
} else {
if (maskScratch == null) {
maskScratch = VectorMatch.wrap(new int[matcher.getMaxVectorSize()]);
}
final int maskSize = endRow - startRow;
final int[] maskArray = maskScratch.getSelection();
for (int i = 0; i < maskSize; i++) {
maskArray[i] = startRow + i;
}
maskScratch.setSelectionSize(maskSize);
mask = maskScratch;
}
final ReadableVectorMatch match = matcher.match(mask);
if (match.isAllTrue(matcher.getCurrentVectorSize())) {
delegate.aggregate(buf, position, startRow, endRow);
} else if (!match.isAllFalse()) {
Arrays.fill(delegatePositions, 0, match.getSelectionSize(), position);
delegate.aggregate(buf, match.getSelectionSize(), delegatePositions, match.getSelection(), 0);
}
}
@Override
public void aggregate(
final ByteBuffer buf,
final int numRows,
final int[] positions,
@Nullable final int[] rows,
final int positionOffset
)
{
final ReadableVectorMatch match0;
if (rows == null) {
match0 = VectorMatch.allTrue(numRows);
} else {
match0 = VectorMatch.wrap(rows).setSelectionSize(numRows);
}
final ReadableVectorMatch match = matcher.match(match0);
final int[] selection = match.getSelection();
if (rows == null) {
for (int i = 0; i < match.getSelectionSize(); i++) {
delegatePositions[i] = positions[selection[i]];
}
} else {
// i iterates over the match; j iterates over the "rows" array
for (int i = 0, j = 0; i < match.getSelectionSize(); i++) {
for (; rows[j] < selection[i]; j++) {
// Do nothing; the for loop is doing the work of incrementing j.
}
if (rows[j] != selection[i]) {
throw new ISE("Selection contained phantom row[%d]", selection[i]);
}
delegatePositions[i] = positions[j];
}
}
delegate.aggregate(buf, match.getSelectionSize(), delegatePositions, selection, positionOffset);
}
@Override
public Object get(final ByteBuffer buf, final int position)
{
return delegate.get(buf, position);
}
@Override
public void close()
{
delegate.close();
maskScratch = null;
}
@Override
public void relocate(
final int oldPosition,
final int newPosition,
final ByteBuffer oldBuffer,
final ByteBuffer newBuffer
)
{
delegate.relocate(oldPosition, newPosition, oldBuffer, newBuffer);
}
}

View File

@ -26,6 +26,8 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
@ -61,12 +63,24 @@ public class FloatSumAggregatorFactory extends SimpleFloatAggregatorFactory
);
}
@Override
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseFloatColumnValueSelector selector)
{
return new FloatSumAggregator(selector);
}
@Override
public boolean canVectorize()
{
return expression == null;
}
@Override
protected BufferAggregator factorizeBuffered(
ColumnSelectorFactory metricFactory,
@ -76,6 +90,15 @@ public class FloatSumAggregatorFactory extends SimpleFloatAggregatorFactory
return new FloatSumBufferAggregator(selector);
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,
VectorValueSelector selector
)
{
return new FloatSumVectorAggregator(selector);
}
@Override
@Nullable
public Object combine(@Nullable Object lhs, @Nullable Object rhs)

View File

@ -0,0 +1,84 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
public class FloatSumVectorAggregator implements VectorAggregator
{
private final VectorValueSelector selector;
public FloatSumVectorAggregator(final VectorValueSelector selector)
{
this.selector = selector;
}
@Override
public void init(final ByteBuffer buf, final int position)
{
buf.putFloat(position, 0);
}
@Override
public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
{
final float[] vector = selector.getFloatVector();
float sum = 0;
for (int i = startRow; i < endRow; i++) {
sum += vector[i];
}
buf.putFloat(position, buf.getFloat(position) + sum);
}
@Override
public void aggregate(
final ByteBuffer buf,
final int numPositions,
final int[] positions,
@Nullable final int[] rows,
final int positionOffset
)
{
final float[] vector = selector.getFloatVector();
for (int i = 0; i < numPositions; i++) {
final int position = positions[i] + positionOffset;
buf.putFloat(position, buf.getFloat(position) + vector[rows != null ? rows[i] : i]);
}
}
@Override
public Object get(final ByteBuffer buf, final int position)
{
return buf.getFloat(position);
}
@Override
public void close()
{
// Nothing to close.
}
}

View File

@ -26,6 +26,8 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
@ -62,9 +64,9 @@ public class LongSumAggregatorFactory extends SimpleLongAggregatorFactory
}
@Override
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseLongColumnValueSelector selector)
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
return new LongSumAggregator(selector);
return columnSelectorFactory.makeValueSelector(fieldName);
}
@Override
@ -76,6 +78,27 @@ public class LongSumAggregatorFactory extends SimpleLongAggregatorFactory
return new LongSumBufferAggregator(selector);
}
@Override
protected VectorAggregator factorizeVector(
VectorColumnSelectorFactory columnSelectorFactory,
VectorValueSelector selector
)
{
return new LongSumVectorAggregator(selector);
}
@Override
protected Aggregator factorize(ColumnSelectorFactory metricFactory, BaseLongColumnValueSelector selector)
{
return new LongSumAggregator(selector);
}
@Override
public boolean canVectorize()
{
return expression == null;
}
@Override
@Nullable
public Object combine(@Nullable Object lhs, @Nullable Object rhs)

View File

@ -0,0 +1,83 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
public class LongSumVectorAggregator implements VectorAggregator
{
private final VectorValueSelector selector;
public LongSumVectorAggregator(final VectorValueSelector selector)
{
this.selector = selector;
}
@Override
public void init(final ByteBuffer buf, final int position)
{
buf.putLong(position, 0);
}
@Override
public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
{
final long[] vector = selector.getLongVector();
long sum = 0;
for (int i = startRow; i < endRow; i++) {
sum += vector[i];
}
buf.putLong(position, buf.getLong(position) + sum);
}
@Override
public void aggregate(
final ByteBuffer buf,
final int numRows,
final int[] positions,
@Nullable final int[] rows,
final int positionOffset
)
{
final long[] vector = selector.getLongVector();
for (int i = 0; i < numRows; i++) {
final int position = positions[i] + positionOffset;
buf.putLong(position, buf.getLong(position) + vector[rows != null ? rows[i] : i]);
}
}
@Override
public Object get(final ByteBuffer buf, final int position)
{
return buf.getLong(position);
}
@Override
public void close()
{
// Nothing to close.
}
}

View File

@ -0,0 +1,74 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
public class NoopVectorAggregator implements VectorAggregator
{
private static final NoopVectorAggregator INSTANCE = new NoopVectorAggregator();
public static NoopVectorAggregator instance()
{
return INSTANCE;
}
private NoopVectorAggregator()
{
// Singleton.
}
@Override
public void init(ByteBuffer buf, int position)
{
// Do nothing.
}
@Override
public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
{
// Do nothing.
}
@Override
public void aggregate(
final ByteBuffer buf,
final int numRows,
final int[] positions,
@Nullable final int[] rows,
final int positionOffset
)
{
// Do nothing.
}
@Override
public Object get(ByteBuffer buf, int position)
{
return null;
}
@Override
public void close()
{
// Do nothing.
}
}

View File

@ -20,11 +20,14 @@
package org.apache.druid.query.aggregation;
import com.google.common.base.Preconditions;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.guice.annotations.ExtensionPoint;
import org.apache.druid.segment.BaseNullableColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorValueSelector;
/**
* Abstract class with functionality to wrap {@link Aggregator}, {@link BufferAggregator} and {@link AggregateCombiner}
@ -35,21 +38,30 @@ import org.apache.druid.segment.ColumnValueSelector;
public abstract class NullableAggregatorFactory<T extends BaseNullableColumnValueSelector> extends AggregatorFactory
{
@Override
public final Aggregator factorize(ColumnSelectorFactory metricFactory)
public final Aggregator factorize(ColumnSelectorFactory columnSelectorFactory)
{
T selector = selector(metricFactory);
Aggregator aggregator = factorize(metricFactory, selector);
T selector = selector(columnSelectorFactory);
Aggregator aggregator = factorize(columnSelectorFactory, selector);
return NullHandling.replaceWithDefault() ? aggregator : new NullableAggregator(aggregator, selector);
}
@Override
public final BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
public final BufferAggregator factorizeBuffered(ColumnSelectorFactory columnSelectorFactory)
{
T selector = selector(metricFactory);
BufferAggregator aggregator = factorizeBuffered(metricFactory, selector);
T selector = selector(columnSelectorFactory);
BufferAggregator aggregator = factorizeBuffered(columnSelectorFactory, selector);
return NullHandling.replaceWithDefault() ? aggregator : new NullableBufferAggregator(aggregator, selector);
}
@Override
public final VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelectorFactory)
{
Preconditions.checkState(canVectorize(), "Cannot vectorize");
VectorValueSelector selector = vectorSelector(columnSelectorFactory);
VectorAggregator aggregator = factorizeVector(columnSelectorFactory, selector);
return NullHandling.replaceWithDefault() ? aggregator : new NullableVectorAggregator(aggregator, selector);
}
@Override
public final AggregateCombiner makeNullableAggregateCombiner()
{
@ -70,26 +82,59 @@ public abstract class NullableAggregatorFactory<T extends BaseNullableColumnValu
*
* @see ColumnValueSelector
*/
protected abstract T selector(ColumnSelectorFactory metricFactory);
protected abstract T selector(ColumnSelectorFactory columnSelectorFactory);
/**
* Creates a {@link VectorValueSelector} for the aggregated column.
*
* @see VectorValueSelector
*/
protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnSelectorFactory)
{
throw new UnsupportedOperationException("Cannot vectorize");
}
/**
* Creates an {@link Aggregator} to aggregate values from several rows, by using the provided selector.
* @param metricFactory metricFactory
* @param selector {@link ColumnValueSelector} for the column to aggregate.
*
* @param columnSelectorFactory metricFactory
* @param selector {@link ColumnValueSelector} for the column to aggregate.
*
* @see Aggregator
*/
protected abstract Aggregator factorize(ColumnSelectorFactory metricFactory, T selector);
protected abstract Aggregator factorize(ColumnSelectorFactory columnSelectorFactory, T selector);
/**
* Creates an {@link BufferAggregator} to aggregate values from several rows into a ByteBuffer.
* @param metricFactory metricFactory
* @param selector {@link ColumnValueSelector} for the column to aggregate.
*
* @param columnSelectorFactory columnSelectorFactory in case any other columns are needed.
* @param selector {@link ColumnValueSelector} for the column to aggregate.
*
* @see BufferAggregator
*/
protected abstract BufferAggregator factorizeBuffered(
ColumnSelectorFactory metricFactory,
ColumnSelectorFactory columnSelectorFactory,
T selector
);
/**
* Creates a {@link VectorAggregator} to aggregate values from several rows into a ByteBuffer.
*
* @param columnSelectorFactory columnSelectorFactory in case any other columns are needed.
* @param selector {@link VectorValueSelector} for the column to aggregate.
*
* @see BufferAggregator
*/
protected VectorAggregator factorizeVector(
// Not used by current aggregators, but here for parity with "factorizeBuffered".
@SuppressWarnings("unused") VectorColumnSelectorFactory columnSelectorFactory,
VectorValueSelector selector
)
{
if (!canVectorize()) {
throw new UnsupportedOperationException("Cannot vectorize");
} else {
throw new UnsupportedOperationException("canVectorize returned true but 'factorizeVector' is not implemented");
}
}
}

View File

@ -28,14 +28,18 @@ import javax.annotation.Nullable;
import java.nio.ByteBuffer;
/**
* The result of a NullableBufferAggregator will be null if all the values to be aggregated are null values or no values
* are aggregated at all. If any of the value is non-null, the result would be the aggregated value of the delegate
* aggregator. Note that the delegate aggregator is not required to perform check for
* {@link BaseNullableColumnValueSelector#isNull()} on the selector as only non-null values will be passed to the
* delegate aggregator. This class is only used when SQL compatible null handling is enabled.
* When writing aggregated result to buffer, it will write an additional byte to store the nullability of the
* aggregated result.
* Buffer Layout - 1 byte for storing nullability + delegate storage bytes.
* A wrapper around a non-null-aware BufferAggregator that makes it null-aware. This removes the need for each
* aggregator class to handle nulls on its own.
*
* The result of this aggregator will be null if all the values to be aggregated are null values or no values are
* aggregated at all. If any of the values are non-null, the result will be the aggregated value of the delegate
* aggregator.
*
* When wrapped by this class, the underlying aggregator's required storage space is increased by one byte. The extra
* byte is a boolean that stores whether or not any non-null values have been seen. The extra byte is placed before
* the underlying aggregator's normal state. (Buffer layout = [nullability byte] [delegate storage bytes])
*
* @see NullableVectorAggregator, the vectorized version.
*/
@PublicApi
public final class NullableBufferAggregator implements BufferAggregator
@ -111,6 +115,12 @@ public final class NullableBufferAggregator implements BufferAggregator
return buf.get(position) == NullHandling.IS_NULL_BYTE || delegate.isNull(buf, position + Byte.BYTES);
}
@Override
public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
{
delegate.relocate(oldPosition + Byte.BYTES, newPosition + Byte.BYTES, oldBuffer, newBuffer);
}
@Override
public void close()
{

View File

@ -0,0 +1,165 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.util.Arrays;
/**
* A wrapper around a non-null-aware VectorAggregator that makes it null-aware. This removes the need for each
* aggregator class to handle nulls on its own. This class only makes sense as a wrapper for "primitive" aggregators,
* i.e., ones that take {@link VectorValueSelector} as input.
*
* The result of this aggregator will be null if all the values to be aggregated are null values or no values are
* aggregated at all. If any of the values are non-null, the result will be the aggregated value of the delegate
* aggregator.
*
* When wrapped by this class, the underlying aggregator's required storage space is increased by one byte. The extra
* byte is a boolean that stores whether or not any non-null values have been seen. The extra byte is placed before
* the underlying aggregator's normal state. (Buffer layout = [nullability byte] [delegate storage bytes])
*
* @see NullableBufferAggregator, the vectorized version.
*/
public class NullableVectorAggregator implements VectorAggregator
{
private final VectorAggregator delegate;
private final VectorValueSelector selector;
@Nullable
private int[] vAggregationPositions = null;
@Nullable
private int[] vAggregationRows = null;
NullableVectorAggregator(VectorAggregator delegate, VectorValueSelector selector)
{
this.delegate = delegate;
this.selector = selector;
}
@Override
public void init(ByteBuffer buf, int position)
{
buf.put(position, NullHandling.IS_NULL_BYTE);
delegate.init(buf, position + Byte.BYTES);
}
@Override
public void aggregate(ByteBuffer buf, int position, int startRow, int endRow)
{
final boolean[] nullVector = selector.getNullVector();
if (nullVector != null) {
// Deferred initialization, since vAggregationPositions and vAggregationRows are only needed if nulls
// actually occur.
if (vAggregationPositions == null) {
vAggregationPositions = new int[selector.getMaxVectorSize()];
vAggregationRows = new int[selector.getMaxVectorSize()];
}
int j = 0;
for (int i = startRow; i < endRow; i++) {
if (!nullVector[i]) {
vAggregationRows[j++] = i;
}
}
Arrays.fill(vAggregationPositions, 0, j, position);
doAggregate(buf, j, vAggregationPositions, vAggregationRows, 0);
} else {
doAggregate(buf, position, startRow, endRow);
}
}
@Override
public void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset)
{
final boolean[] nullVector = selector.getNullVector();
if (nullVector != null) {
// Deferred initialization, since vAggregationPositions and vAggregationRows are only needed if nulls
// actually occur.
if (vAggregationPositions == null) {
vAggregationPositions = new int[selector.getMaxVectorSize()];
vAggregationRows = new int[selector.getMaxVectorSize()];
}
int j = 0;
for (int i = 0; i < numRows; i++) {
final int rowNum = rows == null ? i : rows[i];
if (!nullVector[rowNum]) {
vAggregationPositions[j] = positions[i];
vAggregationRows[j] = rowNum;
j++;
}
}
doAggregate(buf, j, vAggregationPositions, vAggregationRows, positionOffset);
} else {
doAggregate(buf, numRows, positions, rows, positionOffset);
}
}
@Override
@Nullable
public Object get(ByteBuffer buf, int position)
{
switch (buf.get(position)) {
case NullHandling.IS_NULL_BYTE:
return null;
case NullHandling.IS_NOT_NULL_BYTE:
return delegate.get(buf, position + Byte.BYTES);
default:
// Corrupted byte?
throw new ISE("Bad null-marker byte, delegate class[%s]", delegate.getClass().getName());
}
}
@Override
public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
{
delegate.relocate(oldPosition + Byte.BYTES, newPosition + Byte.BYTES, oldBuffer, newBuffer);
}
@Override
public void close()
{
delegate.close();
}
private void doAggregate(ByteBuffer buf, int position, int start, int end)
{
buf.put(position, NullHandling.IS_NOT_NULL_BYTE);
delegate.aggregate(buf, position + Byte.BYTES, start, end);
}
private void doAggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset)
{
for (int i = 0; i < numRows; i++) {
buf.put(positions[i] + positionOffset, NullHandling.IS_NOT_NULL_BYTE);
}
delegate.aggregate(buf, numRows, positions, rows, positionOffset + Byte.BYTES);
}
}

View File

@ -23,6 +23,7 @@ import org.apache.druid.query.PerSegmentQueryOptimizationContext;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
@ -62,6 +63,18 @@ public class SuppressedAggregatorFactory extends AggregatorFactory
return new SuppressedBufferAggregator(delegate.factorizeBuffered(metricFactory));
}
@Override
public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelectorFactory)
{
return new SuppressedVectorAggregator(delegate.factorizeVector(columnSelectorFactory));
}
@Override
public boolean canVectorize()
{
return delegate.canVectorize();
}
@Override
public Comparator getComparator()
{
@ -135,6 +148,12 @@ public class SuppressedAggregatorFactory extends AggregatorFactory
return delegate.getMaxIntermediateSize();
}
@Override
public int getMaxIntermediateSizeWithNulls()
{
return delegate.getMaxIntermediateSizeWithNulls();
}
@Override
public AggregatorFactory optimizeForSegment(PerSegmentQueryOptimizationContext optimizationContext)
{
@ -372,4 +391,78 @@ public class SuppressedAggregatorFactory extends AggregatorFactory
return delegate;
}
}
public static class SuppressedVectorAggregator implements VectorAggregator
{
private final VectorAggregator delegate;
public SuppressedVectorAggregator(VectorAggregator delegate)
{
this.delegate = delegate;
}
@Override
public void init(ByteBuffer buf, int position)
{
delegate.init(buf, position);
}
@Override
public void aggregate(ByteBuffer buf, int position, int startRow, int endRow)
{
// no-op
}
@Override
public void aggregate(ByteBuffer buf, int numRows, int[] positions, @Nullable int[] rows, int positionOffset)
{
// no-op
}
@Nullable
@Override
public Object get(ByteBuffer buf, int position)
{
return delegate.get(buf, position);
}
@Override
public void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
{
delegate.relocate(oldPosition, newPosition, oldBuffer, newBuffer);
}
@Override
public void close()
{
delegate.close();
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SuppressedVectorAggregator that = (SuppressedVectorAggregator) o;
return Objects.equals(delegate, that.delegate);
}
@Override
public int hashCode()
{
return Objects.hash(delegate);
}
@Override
public String toString()
{
return "SuppressedVectorAggregator{" +
"delegate=" + delegate +
'}';
}
}
}

View File

@ -0,0 +1,86 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
/**
* An object that can aggregate metrics into a ByteBuffer, from vectorized column selectors. Its aggregation-related
* methods (namely, "aggregate" and "get") do not take the actual input values to aggregate, because it is assumed that
* the VectorAggregator was given something that it can use to get at the current batch of data.
*
* None of the methods in this class are annotated with
* {@link org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop} because vectorized query engines do not use
* monomorphic-processing-style specialization.
*
* @see BufferAggregator, the vectorized version.
*/
public interface VectorAggregator
{
/**
* Same as {@link BufferAggregator#init}.
*/
void init(ByteBuffer buf, int position);
/**
* Aggregate a range of rows into a single aggregation slot.
*
* <b>Implementations must not change the position, limit or mark of the given buffer</b>
*
* @param buf byte buffer storing the byte array representation of the aggregate
* @param position offset within the byte buffer at which the current aggregate value is stored
* @param startRow first row of the range within the current batch to aggregate (inclusive)
* @param endRow end row of the range (exclusive)
*/
void aggregate(ByteBuffer buf, int position, int startRow, int endRow);
/**
* Aggregate a list of rows ("rows") into a list of aggregation slots ("positions").
*
* <b>Implementations must not change the position, limit or mark of the given buffer</b>
*
* @param buf byte buffer storing the byte array representation of the aggregate
* @param numRows number of rows to aggregate
* @param positions array of aggregate value positions within the buffer; must be at least as long as "numRows"
* @param rows array of row numbers within the current row batch; must be at least as long as "numRows". If
* null, the aggregator will aggregate rows from 0 (inclusive) to numRows (exclusive).
* @param positionOffset an offset to apply to each value from "positions"
*/
void aggregate(ByteBuffer buf, int numRows, int positions[], @Nullable int[] rows, int positionOffset);
/**
* Same as {@link BufferAggregator#get}.
*/
@Nullable
Object get(ByteBuffer buf, int position);
/**
* Same as {@link BufferAggregator#relocate}.
*/
default void relocate(int oldPosition, int newPosition, ByteBuffer oldBuffer, ByteBuffer newBuffer)
{
}
/**
* Release any resources used by the aggregator.
*/
void close();
}

View File

@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.query.ColumnSelectorPlus;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.cardinality.types.CardinalityAggregatorColumnSelectorStrategy;
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesBufferAggregator;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import java.nio.ByteBuffer;
@ -33,8 +34,6 @@ public class CardinalityBufferAggregator implements BufferAggregator
private final ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>[] selectorPluses;
private final boolean byRow;
private static final byte[] EMPTY_BYTES = HyperLogLogCollector.makeEmptyVersionedByteArray();
CardinalityBufferAggregator(
ColumnSelectorPlus<CardinalityAggregatorColumnSelectorStrategy>[] selectorPluses,
boolean byRow
@ -47,9 +46,7 @@ public class CardinalityBufferAggregator implements BufferAggregator
@Override
public void init(ByteBuffer buf, int position)
{
final ByteBuffer mutationBuffer = buf.duplicate();
mutationBuffer.position(position);
mutationBuffer.put(EMPTY_BYTES);
HyperUniquesBufferAggregator.doInit(buf, position);
}
@Override
@ -78,11 +75,7 @@ public class CardinalityBufferAggregator implements BufferAggregator
@Override
public Object get(ByteBuffer buf, int position)
{
ByteBuffer dataCopyBuffer = ByteBuffer.allocate(HyperLogLogCollector.getLatestNumBytesForDenseStorage());
ByteBuffer mutationBuffer = buf.duplicate();
mutationBuffer.position(position);
mutationBuffer.get(dataCopyBuffer.array());
return HyperLogLogCollector.makeCollector(dataCopyBuffer);
return HyperUniquesBufferAggregator.doGet(buf, position);
}
@Override

View File

@ -33,11 +33,16 @@ import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.NoopAggregator;
import org.apache.druid.query.aggregation.NoopBufferAggregator;
import org.apache.druid.query.aggregation.NoopVectorAggregator;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.query.aggregation.cardinality.HyperLogLogCollectorAggregateCombiner;
import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.NilColumnValueSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
@ -123,6 +128,23 @@ public class HyperUniquesAggregatorFactory extends AggregatorFactory
throw new IAE("Incompatible type for metric[%s], expected a HyperUnique, got a %s", fieldName, classOfObject);
}
@Override
public VectorAggregator factorizeVector(final VectorColumnSelectorFactory selectorFactory)
{
final ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(fieldName);
if (capabilities == null || capabilities.getType() != ValueType.COMPLEX) {
return NoopVectorAggregator.instance();
} else {
return new HyperUniquesVectorAggregator(selectorFactory.makeObjectSelector(fieldName));
}
}
@Override
public boolean canVectorize()
{
return true;
}
@Override
public Comparator getComparator()
{

View File

@ -38,14 +38,31 @@ public class HyperUniquesBufferAggregator implements BufferAggregator
this.selector = selector;
}
@Override
public void init(ByteBuffer buf, int position)
public static void doInit(ByteBuffer buf, int position)
{
final ByteBuffer mutationBuffer = buf.duplicate();
mutationBuffer.position(position);
mutationBuffer.put(EMPTY_BYTES);
}
public static HyperLogLogCollector doGet(ByteBuffer buf, int position)
{
final int size = HyperLogLogCollector.getLatestNumBytesForDenseStorage();
ByteBuffer dataCopyBuffer = ByteBuffer.allocate(size);
ByteBuffer mutationBuffer = buf.duplicate();
mutationBuffer.position(position);
mutationBuffer.limit(position + size);
dataCopyBuffer.put(mutationBuffer);
dataCopyBuffer.rewind();
return HyperLogLogCollector.makeCollector(dataCopyBuffer);
}
@Override
public void init(ByteBuffer buf, int position)
{
doInit(buf, position);
}
@Override
public void aggregate(ByteBuffer buf, int position)
{
@ -73,14 +90,7 @@ public class HyperUniquesBufferAggregator implements BufferAggregator
@Override
public Object get(ByteBuffer buf, int position)
{
final int size = HyperLogLogCollector.getLatestNumBytesForDenseStorage();
ByteBuffer dataCopyBuffer = ByteBuffer.allocate(size);
ByteBuffer mutationBuffer = buf.duplicate();
mutationBuffer.position(position);
mutationBuffer.limit(position + size);
dataCopyBuffer.put(mutationBuffer);
dataCopyBuffer.rewind();
return HyperLogLogCollector.makeCollector(dataCopyBuffer);
return doGet(buf, position);
}
@Override

View File

@ -0,0 +1,116 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.aggregation.hyperloglog;
import com.google.common.base.Preconditions;
import org.apache.druid.hll.HyperLogLogCollector;
import org.apache.druid.query.aggregation.VectorAggregator;
import org.apache.druid.segment.vector.VectorObjectSelector;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
public class HyperUniquesVectorAggregator implements VectorAggregator
{
private final VectorObjectSelector selector;
public HyperUniquesVectorAggregator(final VectorObjectSelector selector)
{
this.selector = Preconditions.checkNotNull(selector, "selector");
}
@Override
public void init(final ByteBuffer buf, final int position)
{
HyperUniquesBufferAggregator.doInit(buf, position);
}
@Override
public void aggregate(final ByteBuffer buf, final int position, final int startRow, final int endRow)
{
// Save position, limit and restore later instead of allocating a new ByteBuffer object
final int oldPosition = buf.position();
final int oldLimit = buf.limit();
buf.limit(position + HyperLogLogCollector.getLatestNumBytesForDenseStorage());
buf.position(position);
try {
final HyperLogLogCollector collector = HyperLogLogCollector.makeCollector(buf);
final Object[] vector = selector.getObjectVector();
for (int i = startRow; i < endRow; i++) {
final HyperLogLogCollector otherCollector = (HyperLogLogCollector) vector[i];
if (otherCollector != null) {
collector.fold(otherCollector);
}
}
}
finally {
buf.limit(oldLimit);
buf.position(oldPosition);
}
}
@Override
public void aggregate(
final ByteBuffer buf,
final int numRows,
final int[] positions,
@Nullable final int[] rows,
final int positionOffset
)
{
final Object[] vector = selector.getObjectVector();
for (int i = 0; i < numRows; i++) {
final HyperLogLogCollector otherCollector = (HyperLogLogCollector) vector[rows != null ? rows[i] : i];
if (otherCollector == null) {
continue;
}
final int position = positions[i] + positionOffset;
// Save position, limit and restore later instead of allocating a new ByteBuffer object
final int oldPosition = buf.position();
final int oldLimit = buf.limit();
buf.limit(position + HyperLogLogCollector.getLatestNumBytesForDenseStorage());
buf.position(position);
try {
HyperLogLogCollector.makeCollector(buf).fold(otherCollector);
}
finally {
buf.limit(oldLimit);
buf.position(oldPosition);
}
}
}
@Override
public Object get(final ByteBuffer buf, final int position)
{
return HyperUniquesBufferAggregator.doGet(buf, position);
}
@Override
public void close()
{
// Nothing to close.
}
}

View File

@ -25,6 +25,8 @@ import org.apache.druid.query.cache.CacheKeyBuilder;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
import javax.annotation.Nullable;
import java.util.Arrays;
@ -110,6 +112,24 @@ public class DefaultDimensionSpec implements DimensionSpec
return selector;
}
@Override
public SingleValueDimensionVectorSelector decorate(final SingleValueDimensionVectorSelector selector)
{
return selector;
}
@Override
public MultiValueDimensionVectorSelector decorate(final MultiValueDimensionVectorSelector selector)
{
return selector;
}
@Override
public boolean canVectorize()
{
return true;
}
@Override
public boolean mustDecorate()
{

View File

@ -22,9 +22,12 @@ package org.apache.druid.query.dimension;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import org.apache.druid.java.util.common.Cacheable;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
import javax.annotation.Nullable;
@ -55,10 +58,29 @@ public interface DimensionSpec extends Cacheable
DimensionSelector decorate(DimensionSelector selector);
default SingleValueDimensionVectorSelector decorate(SingleValueDimensionVectorSelector selector)
{
throw new UOE("DimensionSpec[%s] cannot vectorize", getClass().getName());
}
default MultiValueDimensionVectorSelector decorate(MultiValueDimensionVectorSelector selector)
{
throw new UOE("DimensionSpec[%s] cannot vectorize", getClass().getName());
}
/**
* Does this DimensionSpec require that decorate() be called to produce correct results?
*/
boolean mustDecorate();
/**
* Does this DimensionSpec have working {@link #decorate(SingleValueDimensionVectorSelector)} and
* {@link #decorate(MultiValueDimensionVectorSelector)} methods?
*/
default boolean canVectorize()
{
return false;
}
boolean preservesOrdering();
}

View File

@ -0,0 +1,41 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.dimension;
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
/**
* Class that encapsulates knowledge about how to create vector column processors. Used by
* {@link org.apache.druid.segment.DimensionHandlerUtils#makeVectorProcessor}.
*/
public interface VectorColumnStrategizer<T>
{
T makeSingleValueDimensionStrategy(SingleValueDimensionVectorSelector selector);
T makeMultiValueDimensionStrategy(MultiValueDimensionVectorSelector selector);
T makeFloatStrategy(VectorValueSelector selector);
T makeDoubleStrategy(VectorValueSelector selector);
T makeLongStrategy(VectorValueSelector selector);
}

View File

@ -20,10 +20,13 @@
package org.apache.druid.query.filter;
import org.apache.druid.collections.bitmap.ImmutableBitmap;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.BitmapResultFactory;
import org.apache.druid.query.DefaultBitmapResultFactory;
import org.apache.druid.query.filter.vector.VectorValueMatcher;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
public interface Filter
{
@ -84,6 +87,17 @@ public interface Filter
*/
ValueMatcher makeMatcher(ColumnSelectorFactory factory);
/**
* Get a VectorValueMatcher that applies this filter to row vectors.
*
* @param factory Object used to create ValueMatchers
*
* @return VectorValueMatcher that applies this filter to row vectors.
*/
default VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory)
{
throw new UOE("Filter[%s] cannot vectorize", getClass().getName());
}
/**
* Indicates whether this filter can return a bitmap index for filtering, based on
@ -107,4 +121,12 @@ public interface Filter
* @return true if this Filter supports selectivity estimation, false otherwise.
*/
boolean supportsSelectivityEstimation(ColumnSelector columnSelector, BitmapIndexSelector indexSelector);
/**
* Returns true if this filter can produce a vectorized matcher from its "makeVectorMatcher" method.
*/
default boolean canVectorizeMatcher()
{
return false;
}
}

View File

@ -163,6 +163,12 @@ public class IntervalDimFilter implements DimFilter
return result;
}
@Override
public String toString()
{
return convertedFilter.toString();
}
private List<Pair<Long, Long>> makeIntervalLongs()
{
List<Pair<Long, Long>> intervalLongs = new ArrayList<>();

View File

@ -20,20 +20,72 @@
package org.apache.druid.query.filter;
import com.google.common.base.Predicate;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.segment.DimensionDictionarySelector;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.data.IndexedInts;
import org.apache.druid.segment.filter.BooleanValueMatcher;
import javax.annotation.Nullable;
import java.util.Objects;
public class StringValueMatcherColumnSelectorStrategy implements ValueMatcherColumnSelectorStrategy<DimensionSelector>
{
private static final String[] NULL_VALUE = new String[]{null};
private static final ValueGetter NULL_VALUE_GETTER = () -> NULL_VALUE;
@Override
public ValueMatcher makeValueMatcher(final DimensionSelector selector, String value)
private final boolean hasMultipleValues;
public StringValueMatcherColumnSelectorStrategy(final boolean hasMultipleValues)
{
this.hasMultipleValues = hasMultipleValues;
}
@Nullable
public static Boolean toBooleanIfPossible(
final DimensionDictionarySelector selector,
final boolean hasMultipleValues,
final Predicate<String> predicate
)
{
if (selector.getValueCardinality() == 0) {
return BooleanValueMatcher.of(value == null);
// Column has no values (it doesn't exist, or it's all empty arrays).
// Match if and only if "predicate" matches null.
return predicate.apply(null);
} else if (!hasMultipleValues && selector.getValueCardinality() == 1 && selector.nameLookupPossibleInAdvance()) {
// Every row has the same value. Match if and only if "predicate" matches the possible value.
return predicate.apply(selector.lookupName(0));
} else {
return null;
}
}
@Nullable
private static ValueMatcher toBooleanMatcherIfPossible(
final DimensionSelector selector,
final boolean hasMultipleValues,
final Predicate<String> predicate
)
{
final Boolean booleanValue = StringValueMatcherColumnSelectorStrategy.toBooleanIfPossible(
selector,
hasMultipleValues,
predicate
);
return booleanValue == null ? null : BooleanValueMatcher.of(booleanValue);
}
@Override
public ValueMatcher makeValueMatcher(final DimensionSelector selector, final String value)
{
final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(
selector,
hasMultipleValues,
s -> Objects.equals(s, NullHandling.emptyToNullIfNeeded(value))
);
if (booleanMatcher != null) {
return booleanMatcher;
} else {
return selector.makeValueMatcher(value);
}
@ -46,8 +98,10 @@ public class StringValueMatcherColumnSelectorStrategy implements ValueMatcherCol
)
{
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
if (selector.getValueCardinality() == 0) {
return BooleanValueMatcher.of(predicate.apply(null));
final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, hasMultipleValues, predicate);
if (booleanMatcher != null) {
return booleanMatcher;
} else {
return selector.makeValueMatcher(predicate);
}

View File

@ -25,6 +25,11 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseNullableColumnValueSelector;
/**
* An object that returns a boolean indicating if the "current" row should be selected or not. The most prominent use
* of this interface is that it is returned by the {@link Filter} "makeMatcher" method, where it is used to identify
* selected rows for filtered cursors and filtered aggregators.
*
* @see org.apache.druid.query.filter.vector.VectorValueMatcher, the vectorized version
*/
public interface ValueMatcher extends HotLoopCallee
{

View File

@ -49,7 +49,7 @@ public class ValueMatcherColumnSelectorStrategyFactory
ValueType type = capabilities.getType();
switch (type) {
case STRING:
return new StringValueMatcherColumnSelectorStrategy();
return new StringValueMatcherColumnSelectorStrategy(capabilities.hasMultipleValues());
case LONG:
return new LongValueMatcherColumnSelectorStrategy();
case FLOAT:

View File

@ -0,0 +1,44 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import org.apache.druid.segment.vector.VectorSizeInspector;
public abstract class BaseVectorValueMatcher implements VectorValueMatcher
{
private final VectorSizeInspector selector;
public BaseVectorValueMatcher(final VectorSizeInspector selector)
{
this.selector = selector;
}
@Override
public int getCurrentVectorSize()
{
return selector.getCurrentVectorSize();
}
@Override
public int getMaxVectorSize()
{
return selector.getMaxVectorSize();
}
}

View File

@ -0,0 +1,63 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import org.apache.druid.segment.vector.VectorSizeInspector;
public class BooleanVectorValueMatcher extends BaseVectorValueMatcher
{
private final VectorSizeInspector selector;
private final boolean matches;
private BooleanVectorValueMatcher(final VectorSizeInspector selector, final boolean matches)
{
super(selector);
this.selector = selector;
this.matches = matches;
}
public static BooleanVectorValueMatcher of(final VectorSizeInspector selector, final boolean matches)
{
return new BooleanVectorValueMatcher(selector, matches);
}
@Override
public int getCurrentVectorSize()
{
return selector.getCurrentVectorSize();
}
@Override
public int getMaxVectorSize()
{
return selector.getCurrentVectorSize();
}
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
if (matches) {
assert mask.isValid(mask);
return mask;
} else {
return VectorMatch.allFalse();
}
}
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import org.apache.druid.query.filter.DruidDoublePredicate;
import org.apache.druid.query.filter.DruidPredicateFactory;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
public class DoubleVectorValueMatcher implements VectorValueMatcherFactory
{
private final VectorValueSelector selector;
public DoubleVectorValueMatcher(final VectorValueSelector selector)
{
this.selector = selector;
}
@Override
public VectorValueMatcher makeMatcher(@Nullable final String value)
{
final Double matchVal = DimensionHandlerUtils.convertObjectToDouble(value);
if (matchVal == null) {
return BooleanVectorValueMatcher.of(selector, false);
}
final double matchValDouble = matchVal;
return new BaseVectorValueMatcher(selector)
{
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final double[] vector = selector.getDoubleVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
if (vector[rowNum] == matchValDouble) {
selection[numRows++] = rowNum;
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
}
@Override
public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
{
final DruidDoublePredicate predicate = predicateFactory.makeDoublePredicate();
return new BaseVectorValueMatcher(selector)
{
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final double[] vector = selector.getDoubleVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
if (predicate.applyDouble(vector[rowNum])) {
selection[numRows++] = rowNum;
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
}
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import org.apache.druid.query.filter.DruidFloatPredicate;
import org.apache.druid.query.filter.DruidPredicateFactory;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
public class FloatVectorValueMatcher implements VectorValueMatcherFactory
{
private final VectorValueSelector selector;
public FloatVectorValueMatcher(final VectorValueSelector selector)
{
this.selector = selector;
}
@Override
public VectorValueMatcher makeMatcher(@Nullable final String value)
{
final Float matchVal = DimensionHandlerUtils.convertObjectToFloat(value);
if (matchVal == null) {
return BooleanVectorValueMatcher.of(selector, false);
}
final float matchValFloat = matchVal;
return new BaseVectorValueMatcher(selector)
{
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final float[] vector = selector.getFloatVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
if (vector[rowNum] == matchValFloat) {
selection[numRows++] = rowNum;
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
}
@Override
public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
{
final DruidFloatPredicate predicate = predicateFactory.makeFloatPredicate();
return new BaseVectorValueMatcher(selector)
{
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final float[] vector = selector.getFloatVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
if (predicate.applyFloat(vector[rowNum])) {
selection[numRows++] = rowNum;
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
}
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import org.apache.druid.query.filter.DruidLongPredicate;
import org.apache.druid.query.filter.DruidPredicateFactory;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.vector.VectorValueSelector;
import javax.annotation.Nullable;
public class LongVectorValueMatcher implements VectorValueMatcherFactory
{
private final VectorValueSelector selector;
public LongVectorValueMatcher(final VectorValueSelector selector)
{
this.selector = selector;
}
@Override
public VectorValueMatcher makeMatcher(@Nullable final String value)
{
final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value);
if (matchVal == null) {
return BooleanVectorValueMatcher.of(selector, false);
}
final long matchValLong = matchVal;
return new BaseVectorValueMatcher(selector)
{
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final long[] vector = selector.getLongVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
if (vector[rowNum] == matchValLong) {
selection[numRows++] = rowNum;
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
}
@Override
public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
{
final DruidLongPredicate predicate = predicateFactory.makeLongPredicate();
return new BaseVectorValueMatcher(selector)
{
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final long[] vector = selector.getLongVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
if (predicate.applyLong(vector[rowNum])) {
selection[numRows++] = rowNum;
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
}
}

View File

@ -0,0 +1,208 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import com.google.common.base.Predicate;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.filter.DruidPredicateFactory;
import org.apache.druid.segment.IdLookup;
import org.apache.druid.segment.data.IndexedInts;
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
import javax.annotation.Nullable;
import java.util.BitSet;
import java.util.Objects;
public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFactory
{
private final MultiValueDimensionVectorSelector selector;
public MultiValueStringVectorValueMatcher(final MultiValueDimensionVectorSelector selector)
{
this.selector = selector;
}
@Override
public VectorValueMatcher makeMatcher(@Nullable final String value)
{
final String etnValue = NullHandling.emptyToNullIfNeeded(NullHandling.emptyToNullIfNeeded(value));
final IdLookup idLookup = selector.idLookup();
final int id;
if (idLookup != null) {
// Optimization when names can be looked up to IDs ahead of time.
id = idLookup.lookupId(etnValue);
if (id < 0) {
// Value doesn't exist in this column.
return BooleanVectorValueMatcher.of(selector, false);
}
// Check for "id".
return new BaseVectorValueMatcher(selector)
{
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final IndexedInts[] vector = selector.getRowVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
final IndexedInts ints = vector[rowNum];
final int n = ints.size();
if (n == 0) {
// null should match empty rows in multi-value columns
if (etnValue == null) {
selection[numRows++] = rowNum;
}
} else {
for (int j = 0; j < n; j++) {
if (ints.get(j) == id) {
selection[numRows++] = rowNum;
break;
}
}
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
} else {
return makeMatcher(s -> Objects.equals(s, etnValue));
}
}
@Override
public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
{
return makeMatcher(predicateFactory.makeStringPredicate());
}
private VectorValueMatcher makeMatcher(final Predicate<String> predicate)
{
final boolean matchNull = predicate.apply(null);
if (selector.getValueCardinality() > 0) {
final BitSet checkedIds = new BitSet(selector.getValueCardinality());
final BitSet matchingIds = new BitSet(selector.getValueCardinality());
// Lazy matcher; only check an id if matches() is called.
return new BaseVectorValueMatcher(selector)
{
private final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final IndexedInts[] vector = selector.getRowVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
final IndexedInts ints = vector[rowNum];
final int n = ints.size();
if (n == 0) {
// null should match empty rows in multi-value columns
if (matchNull) {
selection[numRows++] = rowNum;
}
} else {
for (int j = 0; j < n; j++) {
final int id = ints.get(j);
final boolean matches;
if (checkedIds.get(id)) {
matches = matchingIds.get(id);
} else {
matches = predicate.apply(selector.lookupName(id));
checkedIds.set(id);
if (matches) {
matchingIds.set(id);
}
}
if (matches) {
selection[numRows++] = rowNum;
break;
}
}
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
} else {
// Evaluate "lookupName" and "predicate" on every row.
return new BaseVectorValueMatcher(selector)
{
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final IndexedInts[] vector = selector.getRowVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
final IndexedInts ints = vector[rowNum];
final int n = ints.size();
if (n == 0) {
// null should match empty rows in multi-value columns
if (matchNull) {
selection[numRows++] = rowNum;
}
} else {
for (int j = 0; j < n; j++) {
final int id = ints.get(j);
if (predicate.apply(selector.lookupName(id))) {
selection[numRows++] = rowNum;
break;
}
}
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
}
}
}

View File

@ -0,0 +1,68 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import javax.annotation.Nullable;
/**
* The result of calling {@link VectorValueMatcher#match}.
*
* @see VectorMatch, the implementation, which also adds some extra mutation methods.
*/
public interface ReadableVectorMatch
{
/**
* Returns an array of indexes into the current batch. Only the first "getSelectionSize" are valid.
*
* Even though this array is technically mutable, it is very poor form to mutate it if you are not the owner of the
* VectorMatch object. The reason we use a mutable array here instead of positional getter methods, by the way, is in
* the hopes of keeping access to the selection vector as low-level and optimizable as possible. Potential
* optimizations could include making it easier for the JVM to use CPU-level vectorization, avoid method calls, etc.
*/
int[] getSelection();
/**
* Returns the number of valid values in the array from "getSelection".
*/
int getSelectionSize();
/**
* Checks if this match has accepted every row in the vector.
*
* @param vectorSize the current vector size; must be passed in since VectorMatch objects do not "know" the size
* of the vector they came from.
*/
boolean isAllTrue(int vectorSize);
/**
* Checks if this match has accepted *nothing*.
*/
boolean isAllFalse();
/**
* Checks if this match is valid (increasing row numbers, no out-of-range row numbers). Can additionally verify
* that the match is a subset of a provided "mask".
*
* Used by assertions and tests.
*
* @param mask if provided, checks if this match is a subset of the mask.
*/
boolean isValid(@Nullable ReadableVectorMatch mask);
}

View File

@ -0,0 +1,192 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import com.google.common.base.Predicate;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.query.filter.DruidPredicateFactory;
import org.apache.druid.query.filter.StringValueMatcherColumnSelectorStrategy;
import org.apache.druid.segment.IdLookup;
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
import javax.annotation.Nullable;
import java.util.BitSet;
import java.util.Objects;
public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFactory
{
private final SingleValueDimensionVectorSelector selector;
public SingleValueStringVectorValueMatcher(final SingleValueDimensionVectorSelector selector)
{
this.selector = selector;
}
@Nullable
private static BooleanVectorValueMatcher toBooleanMatcherIfPossible(
final SingleValueDimensionVectorSelector selector,
final Predicate<String> predicate
)
{
final Boolean booleanValue = StringValueMatcherColumnSelectorStrategy.toBooleanIfPossible(
selector,
false,
predicate
);
return booleanValue == null ? null : BooleanVectorValueMatcher.of(selector, booleanValue);
}
@Override
public VectorValueMatcher makeMatcher(@Nullable final String value)
{
final String etnValue = NullHandling.emptyToNullIfNeeded(value);
final VectorValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, s -> Objects.equals(s, etnValue));
if (booleanMatcher != null) {
return booleanMatcher;
}
final IdLookup idLookup = selector.idLookup();
final int id;
if (idLookup != null) {
// Optimization when names can be looked up to IDs ahead of time.
id = idLookup.lookupId(etnValue);
if (id < 0) {
// Value doesn't exist in this column.
return BooleanVectorValueMatcher.of(selector, false);
}
// Check for "id".
return new BaseVectorValueMatcher(selector)
{
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final int[] vector = selector.getRowVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
if (vector[rowNum] == id) {
selection[numRows++] = rowNum;
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
} else {
return makeMatcher(s -> Objects.equals(s, etnValue));
}
}
@Override
public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory)
{
return makeMatcher(predicateFactory.makeStringPredicate());
}
private VectorValueMatcher makeMatcher(final Predicate<String> predicate)
{
final VectorValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, predicate);
if (booleanMatcher != null) {
return booleanMatcher;
}
if (selector.getValueCardinality() > 0) {
final BitSet checkedIds = new BitSet(selector.getValueCardinality());
final BitSet matchingIds = new BitSet(selector.getValueCardinality());
// Lazy matcher; only check an id if matches() is called.
return new BaseVectorValueMatcher(selector)
{
private final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final int[] vector = selector.getRowVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
final int id = vector[rowNum];
final boolean matches;
if (checkedIds.get(id)) {
matches = matchingIds.get(id);
} else {
matches = predicate.apply(selector.lookupName(id));
checkedIds.set(id);
if (matches) {
matchingIds.set(id);
}
}
if (matches) {
selection[numRows++] = rowNum;
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
} else {
// Evaluate "lookupName" and "predicate" on every row.
return new BaseVectorValueMatcher(selector)
{
final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]);
@Override
public ReadableVectorMatch match(final ReadableVectorMatch mask)
{
final int[] vector = selector.getRowVector();
final int[] selection = match.getSelection();
int numRows = 0;
for (int i = 0; i < mask.getSelectionSize(); i++) {
final int rowNum = mask.getSelection()[i];
if (predicate.apply(selector.lookupName(vector[rowNum]))) {
selection[numRows++] = rowNum;
}
}
match.setSelectionSize(numRows);
assert match.isValid(mask);
return match;
}
};
}
}
}

View File

@ -0,0 +1,267 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import com.google.common.base.Preconditions;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import javax.annotation.Nullable;
/**
* Implementation class for ReadableVectorMatch.
*
* Also adds some useful methods, like "addAll", "removeAll", and "copyFrom".
*/
public class VectorMatch implements ReadableVectorMatch
{
private static final int[] DEFAULT_ALL_TRUE_VECTOR = new int[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE];
private static final VectorMatch ALL_FALSE = new VectorMatch(new int[0], 0);
static {
for (int i = 0; i < DEFAULT_ALL_TRUE_VECTOR.length; i++) {
DEFAULT_ALL_TRUE_VECTOR[i] = i;
}
}
private final int[] selection;
private int selectionSize;
private VectorMatch(final int[] selection, final int selectionSize)
{
this.selection = selection;
this.selectionSize = selectionSize;
}
/**
* Creates a match that matches everything up to "numRows". This will often be the current vector size, but
* does not necessarily have to be.
*/
public static ReadableVectorMatch allTrue(final int numRows)
{
if (numRows <= DEFAULT_ALL_TRUE_VECTOR.length) {
return new VectorMatch(DEFAULT_ALL_TRUE_VECTOR, numRows);
} else {
final int[] selection = new int[numRows];
for (int i = 0; i < numRows; i++) {
selection[i] = i;
}
return new VectorMatch(selection, numRows);
}
}
/**
* Creates a match that matches nothing.
*/
public static ReadableVectorMatch allFalse()
{
return ALL_FALSE;
}
/**
* Creates a new match object with selectionSize = 0, and the provided array as a backing array.
*/
public static VectorMatch wrap(final int[] selection)
{
return new VectorMatch(selection, 0);
}
@Override
public boolean isAllTrue(final int vectorSize)
{
return selectionSize == vectorSize;
}
@Override
public boolean isAllFalse()
{
return selectionSize == 0;
}
@Override
public boolean isValid(@Nullable final ReadableVectorMatch mask)
{
if (mask != null && !mask.isValid(null)) {
// Invalid mask.
return false;
}
// row numbers must be increasing.
int rowNum = -1;
for (int i = 0; i < selectionSize; i++) {
if (selection[i] > rowNum) {
rowNum = selection[i];
} else {
return false;
}
}
// row number cannot be larger than the max length of the selection vector.
if (rowNum > selection.length) {
return false;
}
// row numbers must all be present in the mask, if it exists.
if (mask != null) {
final int[] maskArray = mask.getSelection();
for (int i = 0, j = 0; i < selectionSize; i++) {
while (j < mask.getSelectionSize() && selection[i] > maskArray[j]) {
j++;
}
if (j >= mask.getSelectionSize() || selection[i] != maskArray[j]) {
return false;
}
}
}
return true;
}
/**
* Removes all rows from this object that occur in "other", in place, and returns a reference to this object. Does
* not modify "other".
*/
public VectorMatch removeAll(final ReadableVectorMatch other)
{
//noinspection ObjectEquality
Preconditions.checkState(this != other, "'other' must be a different instance from 'this'");
int i = 0; // reading position in this.selection
int j = 0; // writing position in this.selection
int p = 0; // position in otherSelection
final int[] otherSelection = other.getSelection();
for (; i < selectionSize; i++) {
while (p < other.getSelectionSize() && otherSelection[p] < selection[i]) {
// Other value < selection[i], keep reading in other so we can see if selection[i] should be preserved or not.
p++;
}
if (!(p < other.getSelectionSize() && otherSelection[p] == selection[i])) {
// Preserve selection[i].
selection[j++] = selection[i];
}
}
selectionSize = j;
assert isValid(null);
return this;
}
/**
* Adds all rows from "other" to this object, using "scratch" as scratch space if needed. Does not modify "other".
* Returns a reference to this object.
*/
public VectorMatch addAll(final ReadableVectorMatch other, final VectorMatch scratch)
{
//noinspection ObjectEquality
Preconditions.checkState(this != scratch, "'scratch' must be a different instance from 'this'");
//noinspection ObjectEquality
Preconditions.checkState(other != scratch, "'scratch' must be a different instance from 'other'");
final int[] scratchSelection = scratch.getSelection();
final int[] otherSelection = other.getSelection();
int i = 0; // this.selection pointer
int j = 0; // otherSelection pointer
int k = 0; // scratchSelection pointer
for (; i < selectionSize; i++) {
while (j < other.getSelectionSize() && otherSelection[j] < selection[i]) {
scratchSelection[k++] = otherSelection[j++];
}
scratchSelection[k++] = selection[i];
if (j < other.getSelectionSize() && otherSelection[j] == selection[i]) {
j++;
}
}
while (j < other.getSelectionSize()) {
scratchSelection[k++] = otherSelection[j++];
}
scratch.setSelectionSize(k);
copyFrom(scratch);
assert isValid(null);
return this;
}
/**
* Copies "other" into this object, and returns a reference to this object. Does not modify "other".
*/
public VectorMatch copyFrom(final ReadableVectorMatch other)
{
Preconditions.checkState(
selection.length >= other.getSelectionSize(),
"Capacity[%s] cannot fit other match's selectionSize[%s]",
selection.length,
other.getSelectionSize()
);
System.arraycopy(other.getSelection(), 0, selection, 0, other.getSelectionSize());
selectionSize = other.getSelectionSize();
assert isValid(null);
return this;
}
@Override
public int[] getSelection()
{
return selection;
}
@Override
public int getSelectionSize()
{
return selectionSize;
}
/**
* Sets the valid selectionSize, and returns a reference to this object.
*/
public VectorMatch setSelectionSize(final int newSelectionSize)
{
Preconditions.checkArgument(
newSelectionSize <= selection.length,
"Oops! Cannot setSelectionSize[%s] > selection.length[%s].",
newSelectionSize,
selection.length
);
this.selectionSize = newSelectionSize;
assert isValid(null);
return this;
}
@Override
public String toString()
{
final StringBuilder retVal = new StringBuilder("[");
for (int i = 0; i < selectionSize; i++) {
if (i > 0) {
retVal.append(", ");
}
retVal.append(selection[i]);
}
retVal.append("]");
return retVal.toString();
}
}

View File

@ -0,0 +1,42 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.segment.vector.VectorSizeInspector;
/**
* An object that returns a boolean indicating if the "current" row should be selected or not. The most prominent use
* of this interface is that it is returned by the {@link Filter} "makeVectorMatcher" method, where it is used to
* identify selected rows for filtered cursors and filtered aggregators.
*
* @see org.apache.druid.query.filter.ValueMatcher, the non-vectorized version
*/
public interface VectorValueMatcher extends VectorSizeInspector
{
/**
* Examine the current vector and return a match indicating what is accepted.
*
* @param mask must not be null; use {@link VectorMatch#allTrue} if you don't need a mask.
*
* @return the subset of "mask" that this value matcher accepts
*/
ReadableVectorMatch match(ReadableVectorMatch mask);
}

View File

@ -0,0 +1,74 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import org.apache.druid.query.dimension.VectorColumnStrategizer;
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
public class VectorValueMatcherColumnStrategizer implements VectorColumnStrategizer<VectorValueMatcherFactory>
{
private static final VectorValueMatcherColumnStrategizer INSTANCE = new VectorValueMatcherColumnStrategizer();
private VectorValueMatcherColumnStrategizer()
{
// Singleton.
}
public static VectorValueMatcherColumnStrategizer instance()
{
return INSTANCE;
}
@Override
public VectorValueMatcherFactory makeSingleValueDimensionStrategy(
final SingleValueDimensionVectorSelector selector
)
{
return new SingleValueStringVectorValueMatcher(selector);
}
@Override
public VectorValueMatcherFactory makeMultiValueDimensionStrategy(
final MultiValueDimensionVectorSelector selector
)
{
return new MultiValueStringVectorValueMatcher(selector);
}
@Override
public VectorValueMatcherFactory makeFloatStrategy(final VectorValueSelector selector)
{
return new FloatVectorValueMatcher(selector);
}
@Override
public VectorValueMatcherFactory makeDoubleStrategy(final VectorValueSelector selector)
{
return new DoubleVectorValueMatcher(selector);
}
@Override
public VectorValueMatcherFactory makeLongStrategy(final VectorValueSelector selector)
{
return new LongVectorValueMatcher(selector);
}
}

View File

@ -0,0 +1,31 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.filter.vector;
import org.apache.druid.query.filter.DruidPredicateFactory;
import javax.annotation.Nullable;
public interface VectorValueMatcherFactory
{
VectorValueMatcher makeMatcher(@Nullable String value);
VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory);
}

View File

@ -1077,6 +1077,7 @@ public class GroupByQuery extends BaseQuery<Row>
", aggregatorSpecs=" + aggregatorSpecs +
", postAggregatorSpecs=" + postAggregatorSpecs +
", havingSpec=" + havingSpec +
", context=" + getContext() +
'}';
}

View File

@ -42,6 +42,7 @@ public class GroupByQueryConfig
private static final String CTX_KEY_FORCE_HASH_AGGREGATION = "forceHashAggregation";
private static final String CTX_KEY_INTERMEDIATE_COMBINE_DEGREE = "intermediateCombineDegree";
private static final String CTX_KEY_NUM_PARALLEL_COMBINE_THREADS = "numParallelCombineThreads";
private static final String CTX_KEY_VECTORIZE = "vectorize";
@JsonProperty
private String defaultStrategy = GroupByStrategySelector.STRATEGY_V2;
@ -88,6 +89,9 @@ public class GroupByQueryConfig
@JsonProperty
private int numParallelCombineThreads = 1;
@JsonProperty
private boolean vectorize = false;
public String getDefaultStrategy()
{
return defaultStrategy;
@ -168,6 +172,11 @@ public class GroupByQueryConfig
return numParallelCombineThreads;
}
public boolean isVectorize()
{
return vectorize;
}
public boolean isForcePushDownNestedQuery()
{
return forcePushDownNestedQuery;
@ -203,7 +212,10 @@ public class GroupByQueryConfig
getMaxOnDiskStorage()
);
newConfig.maxMergingDictionarySize = Math.min(
((Number) query.getContextValue(CTX_KEY_MAX_MERGING_DICTIONARY_SIZE, getMaxMergingDictionarySize())).longValue(),
((Number) query.getContextValue(
CTX_KEY_MAX_MERGING_DICTIONARY_SIZE,
getMaxMergingDictionarySize()
)).longValue(),
getMaxMergingDictionarySize()
);
newConfig.forcePushDownLimit = query.getContextBoolean(CTX_KEY_FORCE_LIMIT_PUSH_DOWN, isForcePushDownLimit());
@ -217,6 +229,7 @@ public class GroupByQueryConfig
CTX_KEY_NUM_PARALLEL_COMBINE_THREADS,
getNumParallelCombineThreads()
);
newConfig.vectorize = query.getContextBoolean(CTX_KEY_VECTORIZE, isVectorize());
return newConfig;
}
@ -237,6 +250,7 @@ public class GroupByQueryConfig
", forceHashAggregation=" + forceHashAggregation +
", intermediateCombineDegree=" + intermediateCombineDegree +
", numParallelCombineThreads=" + numParallelCombineThreads +
", vectorize=" + vectorize +
", forcePushDownNestedQuery=" + forcePushDownNestedQuery +
'}';
}

View File

@ -44,6 +44,7 @@ import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionDictionarySelector;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.column.ValueType;
@ -331,7 +332,7 @@ public class GroupByQueryEngine
}
final DimensionSelector selector = cursor.getColumnSelectorFactory().makeDimensionSelector(dimSpec);
if (selector.getValueCardinality() == DimensionSelector.CARDINALITY_UNKNOWN) {
if (selector.getValueCardinality() == DimensionDictionarySelector.CARDINALITY_UNKNOWN) {
throw new UnsupportedOperationException(
"GroupBy v1 does not support dimension selectors with unknown cardinality.");
}

View File

@ -31,6 +31,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.BaseSingleValueDimensionSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.DimensionDictionarySelector;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.IdLookup;
@ -242,7 +243,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory
@Override
public int getValueCardinality()
{
return DimensionSelector.CARDINALITY_UNKNOWN;
return DimensionDictionarySelector.CARDINALITY_UNKNOWN;
}
@Override

View File

@ -22,8 +22,7 @@ package org.apache.druid.query.groupby.epinephelinae;
import com.google.common.base.Supplier;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.aggregation.AggregatorAdapters;
import java.nio.ByteBuffer;
@ -35,8 +34,8 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
protected final Supplier<ByteBuffer> bufferSupplier;
protected final KeySerde<KeyType> keySerde;
protected final int keySize;
protected final BufferAggregator[] aggregators;
protected final int[] aggregatorOffsets;
protected final AggregatorAdapters aggregators;
protected final int baseAggregatorOffset;
protected final int bufferGrouperMaxSize; // Integer.MAX_VALUE in production, only used for unit tests
// The load factor and bucket configurations are not final, to allow subclasses to set their own values
@ -53,15 +52,16 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
// the buffer returned from the below supplier can have dirty bits and should be cleared during initialization
final Supplier<ByteBuffer> bufferSupplier,
final KeySerde<KeyType> keySerde,
final AggregatorFactory[] aggregatorFactories,
final AggregatorAdapters aggregators,
final int baseAggregatorOffset,
final int bufferGrouperMaxSize
)
{
this.bufferSupplier = bufferSupplier;
this.keySerde = keySerde;
this.keySize = keySerde.keySize();
this.aggregators = new BufferAggregator[aggregatorFactories.length];
this.aggregatorOffsets = new int[aggregatorFactories.length];
this.aggregators = aggregators;
this.baseAggregatorOffset = baseAggregatorOffset;
this.bufferGrouperMaxSize = bufferGrouperMaxSize;
}
@ -77,8 +77,9 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
* Called to check if it's possible to skip aggregation for a row.
*
* @param bucketWasUsed Was the row a new entry in the hash table?
* @param bucketOffset Offset of the bucket containing this row's entry in the hash table,
* within the buffer returned by hashTable.getTableBuffer()
* @param bucketOffset Offset of the bucket containing this row's entry in the hash table,
* within the buffer returned by hashTable.getTableBuffer()
*
* @return true if aggregation can be skipped, false otherwise.
*/
public abstract boolean canSkipAggregate(boolean bucketWasUsed, int bucketOffset);
@ -123,7 +124,7 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
if (keyBuffer == null) {
// This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will
// be correct.
return Groupers.DICTIONARY_FULL;
return Groupers.dictionaryFull(0);
}
if (keyBuffer.remaining() != keySize) {
@ -135,11 +136,11 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
}
// find and try to expand if table is full and find again
int bucket = hashTable.findBucketWithAutoGrowth(keyBuffer, keyHash);
int bucket = hashTable.findBucketWithAutoGrowth(keyBuffer, keyHash, () -> {});
if (bucket < 0) {
// This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will
// be correct.
return Groupers.HASH_TABLE_FULL;
return Groupers.hashTableFull(0);
}
final int bucketStartOffset = hashTable.getOffsetForBucket(bucket);
@ -149,10 +150,7 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
// Set up key and initialize the aggs if this is a new bucket.
if (!bucketWasUsed) {
hashTable.initializeNewBucketKey(bucket, keyBuffer, keyHash);
for (int i = 0; i < aggregators.length; i++) {
aggregators[i].init(tableBuffer, bucketStartOffset + aggregatorOffsets[i]);
}
aggregators.init(tableBuffer, bucketStartOffset + baseAggregatorOffset);
newBucketHook(bucketStartOffset);
}
@ -161,9 +159,7 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
}
// Aggregate the current row.
for (int i = 0; i < aggregators.length; i++) {
aggregators[i].aggregate(tableBuffer, bucketStartOffset + aggregatorOffsets[i]);
}
aggregators.aggregateBuffered(tableBuffer, bucketStartOffset + baseAggregatorOffset);
afterAggregateHook(bucketStartOffset);
@ -173,23 +169,16 @@ public abstract class AbstractBufferHashGrouper<KeyType> implements Grouper<KeyT
@Override
public void close()
{
for (BufferAggregator aggregator : aggregators) {
try {
aggregator.close();
}
catch (Exception e) {
log.warn(e, "Could not close aggregator [%s], skipping.", aggregator);
}
}
aggregators.close();
}
protected Entry<KeyType> bucketEntryForOffset(final int bucketOffset)
{
final ByteBuffer tableBuffer = hashTable.getTableBuffer();
final KeyType key = keySerde.fromByteBuffer(tableBuffer, bucketOffset + HASH_SIZE);
final Object[] values = new Object[aggregators.length];
for (int i = 0; i < aggregators.length; i++) {
values[i] = aggregators[i].get(tableBuffer, bucketOffset + aggregatorOffsets[i]);
final Object[] values = new Object[aggregators.size()];
for (int i = 0; i < aggregators.size(); i++) {
values[i] = aggregators.get(tableBuffer, bucketOffset + baseAggregatorOffset, i);
}
return new Entry<>(key, values);

View File

@ -19,13 +19,19 @@
package org.apache.druid.query.groupby.epinephelinae;
import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.ISE;
import javax.annotation.Nullable;
import java.util.Objects;
public class AggregateResult
{
private static final AggregateResult OK = new AggregateResult(true, null);
private static final AggregateResult OK = new AggregateResult(0, null);
private final boolean ok;
private final int count;
@Nullable
private final String reason;
public static AggregateResult ok()
@ -33,29 +39,47 @@ public class AggregateResult
return OK;
}
public static AggregateResult failure(final String reason)
public static AggregateResult partial(final int count, final String reason)
{
return new AggregateResult(false, reason);
return new AggregateResult(count, Preconditions.checkNotNull(reason, "reason"));
}
private AggregateResult(final boolean ok, final String reason)
private AggregateResult(final int count, @Nullable final String reason)
{
this.ok = ok;
Preconditions.checkArgument(count >= 0, "count >= 0");
this.count = count;
this.reason = reason;
}
/**
* True if all rows have been processed.
*/
public boolean isOk()
{
return ok;
return reason == null;
}
public int getCount()
{
if (isOk()) {
throw new ISE("Cannot call getCount when isOk = true");
}
return count;
}
@Nullable
public String getReason()
{
if (isOk()) {
throw new ISE("Cannot call getReason when isOk = true");
}
return reason;
}
@Override
public boolean equals(final Object o)
public boolean equals(Object o)
{
if (this == o) {
return true;
@ -63,22 +87,22 @@ public class AggregateResult
if (o == null || getClass() != o.getClass()) {
return false;
}
final AggregateResult that = (AggregateResult) o;
return ok == that.ok &&
AggregateResult that = (AggregateResult) o;
return count == that.count &&
Objects.equals(reason, that.reason);
}
@Override
public int hashCode()
{
return Objects.hash(ok, reason);
return Objects.hash(count, reason);
}
@Override
public String toString()
{
return "AggregateResult{" +
"ok=" + ok +
"count=" + count +
", reason='" + reason + '\'' +
'}';
}

View File

@ -21,14 +21,14 @@ package org.apache.druid.query.groupby.epinephelinae;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.query.aggregation.AggregatorAdapters;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.BufferAggregator;
import org.apache.druid.query.groupby.epinephelinae.column.GroupByColumnSelectorStrategy;
import org.apache.druid.segment.ColumnSelectorFactory;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.NoSuchElementException;
@ -47,13 +47,10 @@ import java.util.NoSuchElementException;
* different segments cannot be currently retrieved, this grouper can be used only when performing per-segment query
* execution.
*/
public class BufferArrayGrouper implements IntGrouper
public class BufferArrayGrouper implements VectorGrouper, IntGrouper
{
private static final Logger LOG = new Logger(BufferArrayGrouper.class);
private final Supplier<ByteBuffer> bufferSupplier;
private final BufferAggregator[] aggregators;
private final int[] aggregatorOffsets;
private final AggregatorAdapters aggregators;
private final int cardinalityWithMissingValue;
private final int recordSize; // size of all aggregated values
@ -61,6 +58,10 @@ public class BufferArrayGrouper implements IntGrouper
private ByteBuffer usedFlagBuffer;
private ByteBuffer valBuffer;
// Scratch objects used by aggregateVector(). Only set if initVectorized() is called.
private int[] vAggregationPositions = null;
private int[] vAggregationRows = null;
static long requiredBufferCapacity(
int cardinality,
AggregatorFactory[] aggregatorFactories
@ -72,7 +73,7 @@ public class BufferArrayGrouper implements IntGrouper
.sum();
return getUsedFlagBufferCapacity(cardinalityWithMissingValue) + // total used flags size
(long) cardinalityWithMissingValue * recordSize; // total values size
(long) cardinalityWithMissingValue * recordSize; // total values size
}
/**
@ -86,26 +87,17 @@ public class BufferArrayGrouper implements IntGrouper
public BufferArrayGrouper(
// the buffer returned from the below supplier can have dirty bits and should be cleared during initialization
final Supplier<ByteBuffer> bufferSupplier,
final ColumnSelectorFactory columnSelectorFactory,
final AggregatorFactory[] aggregatorFactories,
final AggregatorAdapters aggregators,
final int cardinality
)
{
Preconditions.checkNotNull(aggregatorFactories, "aggregatorFactories");
Preconditions.checkNotNull(aggregators, "aggregators");
Preconditions.checkArgument(cardinality > 0, "Cardinality must a non-zero positive number");
this.bufferSupplier = Preconditions.checkNotNull(bufferSupplier, "bufferSupplier");
this.aggregators = new BufferAggregator[aggregatorFactories.length];
this.aggregatorOffsets = new int[aggregatorFactories.length];
this.aggregators = aggregators;
this.cardinalityWithMissingValue = cardinality + 1;
int offset = 0;
for (int i = 0; i < aggregatorFactories.length; i++) {
aggregators[i] = aggregatorFactories[i].factorizeBuffered(columnSelectorFactory);
aggregatorOffsets[i] = offset;
offset += aggregatorFactories[i].getMaxIntermediateSizeWithNulls();
}
recordSize = offset;
this.recordSize = aggregators.spaceNeeded();
}
@Override
@ -115,6 +107,20 @@ public class BufferArrayGrouper implements IntGrouper
final ByteBuffer buffer = bufferSupplier.get();
final int usedFlagBufferEnd = getUsedFlagBufferCapacity(cardinalityWithMissingValue);
// Sanity check on buffer capacity.
if (usedFlagBufferEnd + (long) cardinalityWithMissingValue * recordSize > buffer.capacity()) {
// Should not happen in production, since we should only select array-based aggregation if we have
// enough scratch space.
throw new ISE(
"Records of size[%,d] and possible cardinality[%,d] exceeds the buffer capacity[%,d].",
recordSize,
cardinalityWithMissingValue,
valBuffer.capacity()
);
}
// Slice up the buffer.
buffer.position(0);
buffer.limit(usedFlagBufferEnd);
usedFlagBuffer = buffer.slice();
@ -129,6 +135,15 @@ public class BufferArrayGrouper implements IntGrouper
}
}
@Override
public void initVectorized(final int maxVectorSize)
{
init();
this.vAggregationPositions = new int[maxVectorSize];
this.vAggregationRows = new int[maxVectorSize];
}
@Override
public boolean isInitialized()
{
@ -136,7 +151,7 @@ public class BufferArrayGrouper implements IntGrouper
}
@Override
public AggregateResult aggregateKeyHash(int dimIndex)
public AggregateResult aggregateKeyHash(final int dimIndex)
{
Preconditions.checkArgument(
dimIndex >= 0 && dimIndex < cardinalityWithMissingValue,
@ -144,39 +159,62 @@ public class BufferArrayGrouper implements IntGrouper
dimIndex
);
final int recordOffset = dimIndex * recordSize;
initializeSlotIfNeeded(dimIndex);
aggregators.aggregateBuffered(valBuffer, dimIndex * recordSize);
return AggregateResult.ok();
}
if (recordOffset + recordSize > valBuffer.capacity()) {
// This error cannot be recoverd, and the query must fail
throw new ISE(
"A record of size [%d] cannot be written to the array buffer at offset[%d] "
+ "because it exceeds the buffer capacity[%d]. Try increasing druid.processing.buffer.sizeBytes",
recordSize,
recordOffset,
valBuffer.capacity()
@Override
public AggregateResult aggregateVector(int[] keySpace, int startRow, int endRow)
{
if (keySpace.length == 0) {
// Empty key space, assume keys are all zeroes.
final int dimIndex = 1;
initializeSlotIfNeeded(dimIndex);
aggregators.aggregateVector(
valBuffer,
dimIndex * recordSize,
startRow,
endRow
);
}
} else {
final int numRows = endRow - startRow;
if (!isUsedSlot(dimIndex)) {
initializeSlot(dimIndex);
}
for (int i = 0; i < numRows; i++) {
// +1 matches what hashFunction() would do.
final int dimIndex = keySpace[i] + 1;
for (int i = 0; i < aggregators.length; i++) {
aggregators[i].aggregate(valBuffer, recordOffset + aggregatorOffsets[i]);
if (dimIndex < 0 || dimIndex >= cardinalityWithMissingValue) {
throw new IAE("Invalid dimIndex[%s]", dimIndex);
}
vAggregationPositions[i] = dimIndex * recordSize;
initializeSlotIfNeeded(dimIndex);
}
aggregators.aggregateVector(
valBuffer,
numRows,
vAggregationPositions,
Groupers.writeAggregationRows(vAggregationRows, startRow, endRow)
);
}
return AggregateResult.ok();
}
private void initializeSlot(int dimIndex)
private void initializeSlotIfNeeded(int dimIndex)
{
final int index = dimIndex / Byte.SIZE;
final int extraIndex = dimIndex % Byte.SIZE;
usedFlagBuffer.put(index, (byte) (usedFlagBuffer.get(index) | (1 << extraIndex)));
final int usedFlagByte = 1 << extraIndex;
final int recordOffset = dimIndex * recordSize;
for (int i = 0; i < aggregators.length; i++) {
aggregators[i].init(valBuffer, recordOffset + aggregatorOffsets[i]);
if ((usedFlagBuffer.get(index) & usedFlagByte) == 0) {
usedFlagBuffer.put(index, (byte) (usedFlagBuffer.get(index) | (1 << extraIndex)));
aggregators.init(valBuffer, dimIndex * recordSize);
}
}
@ -185,6 +223,7 @@ public class BufferArrayGrouper implements IntGrouper
final int index = dimIndex / Byte.SIZE;
final int extraIndex = dimIndex % Byte.SIZE;
final int usedFlagByte = 1 << extraIndex;
return (usedFlagBuffer.get(index) & usedFlagByte) != 0;
}
@ -214,14 +253,36 @@ public class BufferArrayGrouper implements IntGrouper
@Override
public void close()
{
for (BufferAggregator aggregator : aggregators) {
try {
aggregator.close();
aggregators.close();
}
@Override
public CloseableIterator<Entry<ByteBuffer>> iterator()
{
final CloseableIterator<Entry<Integer>> iterator = iterator(false);
final ByteBuffer keyBuffer = ByteBuffer.allocate(Integer.BYTES);
return new CloseableIterator<Entry<ByteBuffer>>()
{
@Override
public boolean hasNext()
{
return iterator.hasNext();
}
catch (Exception e) {
LOG.warn(e, "Could not close aggregator [%s], skipping.", aggregator);
@Override
public Entry<ByteBuffer> next()
{
final Entry<Integer> integerEntry = iterator.next();
keyBuffer.putInt(0, integerEntry.getKey());
return new Entry<>(keyBuffer, integerEntry.getValues());
}
}
@Override
public void close() throws IOException
{
iterator.close();
}
};
}
@Override
@ -252,10 +313,10 @@ public class BufferArrayGrouper implements IntGrouper
final int current = next;
next = findNext(current);
final Object[] values = new Object[aggregators.length];
final Object[] values = new Object[aggregators.size()];
final int recordOffset = current * recordSize;
for (int i = 0; i < aggregators.length; i++) {
values[i] = aggregators[i].get(valBuffer, recordOffset + aggregatorOffsets[i]);
for (int i = 0; i < aggregators.size(); i++) {
values[i] = aggregators.get(valBuffer, recordOffset, i);
}
// shift by -1 since values are initially shifted by +1 so they are all positive and
// GroupByColumnSelectorStrategy.GROUP_BY_MISSING_VALUE is -1

View File

@ -20,11 +20,13 @@
package org.apache.druid.query.groupby.epinephelinae;
import com.google.common.base.Supplier;
import org.apache.commons.lang.mutable.MutableInt;
import org.apache.druid.java.util.common.CloseableIterators;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.query.aggregation.AggregatorAdapters;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.ColumnSelectorFactory;
import java.nio.ByteBuffer;
import java.util.AbstractList;
@ -32,14 +34,14 @@ import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.function.ToIntFunction;
public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyType>
public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyType> implements VectorGrouper
{
private static final int MIN_INITIAL_BUCKETS = 4;
private static final int DEFAULT_INITIAL_BUCKETS = 1024;
private static final float DEFAULT_MAX_LOAD_FACTOR = 0.7f;
private final AggregatorFactory[] aggregatorFactories;
private ByteBuffer buffer;
private boolean initialized = false;
@ -58,19 +60,23 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
private ByteBuffer offsetListBuffer;
private ByteBufferIntList offsetList;
// Scratch objects used by aggregateVector(). Only set if initVectorized() is called.
private ByteBuffer vKeyBuffer = null;
private int[] vKeyHashCodes = null;
private int[] vAggregationPositions = null;
private int[] vAggregationRows = null;
public BufferHashGrouper(
final Supplier<ByteBuffer> bufferSupplier,
final KeySerde<KeyType> keySerde,
final ColumnSelectorFactory columnSelectorFactory,
final AggregatorFactory[] aggregatorFactories,
final AggregatorAdapters aggregators,
final int bufferGrouperMaxSize,
final float maxLoadFactor,
final int initialBuckets,
final boolean useDefaultSorting
)
{
super(bufferSupplier, keySerde, aggregatorFactories, bufferGrouperMaxSize);
this.aggregatorFactories = aggregatorFactories;
super(bufferSupplier, keySerde, aggregators, HASH_SIZE + keySerde.keySize(), bufferGrouperMaxSize);
this.maxLoadFactor = maxLoadFactor > 0 ? maxLoadFactor : DEFAULT_MAX_LOAD_FACTOR;
this.initialBuckets = initialBuckets > 0 ? Math.max(MIN_INITIAL_BUCKETS, initialBuckets) : DEFAULT_INITIAL_BUCKETS;
@ -79,14 +85,7 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
throw new IAE("Invalid maxLoadFactor[%f], must be < 1.0", maxLoadFactor);
}
int offset = HASH_SIZE + keySize;
for (int i = 0; i < aggregatorFactories.length; i++) {
aggregators[i] = aggregatorFactories[i].factorizeBuffered(columnSelectorFactory);
aggregatorOffsets[i] = offset;
offset += aggregatorFactories[i].getMaxIntermediateSizeWithNulls();
}
this.bucketSize = offset;
this.bucketSize = HASH_SIZE + keySerde.keySize() + aggregators.spaceNeeded();
this.useDefaultSorting = useDefaultSorting;
}
@ -132,15 +131,112 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
}
}
@Override
public void initVectorized(final int maxVectorSize)
{
if (!ByteBuffer.class.equals(keySerde.keyClazz())) {
throw new ISE("keyClazz[%s] must be ByteBuffer", keySerde.keyClazz());
}
if (keySize % Integer.BYTES != 0) {
throw new ISE("keySize[%s] must be a multiple of[%s]", keySize, Integer.BYTES);
}
init();
this.vKeyBuffer = ByteBuffer.allocate(keySize);
this.vKeyHashCodes = new int[maxVectorSize];
this.vAggregationPositions = new int[maxVectorSize];
this.vAggregationRows = new int[maxVectorSize];
}
@Override
public AggregateResult aggregateVector(final int[] keySpace, final int startRow, final int endRow)
{
final int keyIntSize = keySize / Integer.BYTES;
final int numRows = endRow - startRow;
// Initialize vKeyHashCodes: one int per key.
// Does *not* use hashFunction(). This is okay because the API of VectorGrouper does not expose any way of messing
// about with hash codes.
for (int i = 0, rowStart = 0; i < numRows; i++, rowStart += keyIntSize) {
vKeyHashCodes[i] = Groupers.hashIntArray(keySpace, rowStart, keyIntSize);
}
final MutableInt aggregationStartRow = new MutableInt(startRow);
final MutableInt aggregationNumRows = new MutableInt(0);
for (int rowNum = 0, keySpacePosition = 0; rowNum < numRows; rowNum++, keySpacePosition += keyIntSize) {
// Copy current key into keyBuffer.
vKeyBuffer.rewind();
for (int i = 0; i < keyIntSize; i++) {
vKeyBuffer.putInt(keySpace[keySpacePosition + i]);
}
vKeyBuffer.rewind();
// Find, and if the table is full, expand and find again.
int bucket = hashTable.findBucketWithAutoGrowth(
vKeyBuffer,
vKeyHashCodes[rowNum],
() -> {
if (aggregationNumRows.intValue() > 0) {
doAggregateVector(aggregationStartRow.intValue(), aggregationNumRows.intValue());
aggregationStartRow.setValue(aggregationStartRow.intValue() + aggregationNumRows.intValue());
aggregationNumRows.setValue(0);
}
}
);
if (bucket < 0) {
// This may just trigger a spill and get ignored, which is ok. If it bubbles up to the user, the message will
// be correct.
// Aggregate any remaining rows.
if (aggregationNumRows.intValue() > 0) {
doAggregateVector(aggregationStartRow.intValue(), aggregationNumRows.intValue());
}
return Groupers.hashTableFull(rowNum);
}
final int bucketStartOffset = hashTable.getOffsetForBucket(bucket);
final boolean bucketWasUsed = hashTable.isBucketUsed(bucket);
// Set up key and initialize the aggs if this is a new bucket.
if (!bucketWasUsed) {
hashTable.initializeNewBucketKey(bucket, vKeyBuffer, vKeyHashCodes[rowNum]);
aggregators.init(hashTable.getTableBuffer(), bucketStartOffset + baseAggregatorOffset);
}
// Schedule the current row for aggregation.
vAggregationPositions[aggregationNumRows.intValue()] = bucketStartOffset + Integer.BYTES + keySize;
aggregationNumRows.increment();
}
// Aggregate any remaining rows.
if (aggregationNumRows.intValue() > 0) {
doAggregateVector(aggregationStartRow.intValue(), aggregationNumRows.intValue());
}
return AggregateResult.ok();
}
@Override
public boolean isInitialized()
{
return initialized;
}
@Override
public ToIntFunction<KeyType> hashFunction()
{
return Groupers::hashObject;
}
@Override
public void newBucketHook(int bucketOffset)
{
// Nothing needed.
}
@Override
@ -152,7 +248,7 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
@Override
public void afterAggregateHook(int bucketOffset)
{
// Nothing needed.
}
@Override
@ -163,6 +259,15 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
keySerde.reset();
}
@Override
@SuppressWarnings("unchecked")
public CloseableIterator<Entry<ByteBuffer>> iterator()
{
// Unchecked cast, since this method is only called through the VectorGrouper interface, which uses
// ByteBuffer keys (and this is verified in initVectorized).
return (CloseableIterator) iterator(false);
}
@Override
public CloseableIterator<Entry<KeyType>> iterator(boolean sorted)
{
@ -201,7 +306,10 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
if (useDefaultSorting) {
comparator = keySerde.bufferComparator();
} else {
comparator = keySerde.bufferComparatorWithAggregators(aggregatorFactories, aggregatorOffsets);
comparator = keySerde.bufferComparatorWithAggregators(
aggregators.factories().toArray(new AggregatorFactory[0]),
aggregators.aggregatorPositions()
);
}
// Sort offsets in-place.
@ -296,6 +404,16 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
}
}
private void doAggregateVector(final int startRow, final int numRows)
{
aggregators.aggregateVector(
hashTable.getTableBuffer(),
numRows,
vAggregationPositions,
Groupers.writeAggregationRows(vAggregationRows, startRow, startRow + numRows)
);
}
private class BufferGrouperBucketUpdateHandler implements ByteBufferHashTable.BucketUpdateHandler
{
@Override
@ -314,14 +432,12 @@ public class BufferHashGrouper<KeyType> extends AbstractBufferHashGrouper<KeyTyp
public void handleBucketMove(int oldBucketOffset, int newBucketOffset, ByteBuffer oldBuffer, ByteBuffer newBuffer)
{
// relocate aggregators (see https://github.com/apache/incubator-druid/pull/4071)
for (int i = 0; i < aggregators.length; i++) {
aggregators[i].relocate(
oldBucketOffset + aggregatorOffsets[i],
newBucketOffset + aggregatorOffsets[i],
oldBuffer,
newBuffer
);
}
aggregators.relocate(
oldBucketOffset + baseAggregatorOffset,
newBucketOffset + baseAggregatorOffset,
oldBuffer,
newBuffer
);
offsetList.add(newBucketOffset);
}

View File

@ -75,7 +75,6 @@ public class ByteBufferHashTable
protected int growthCount;
protected BucketUpdateHandler bucketUpdateHandler;
public ByteBufferHashTable(
@ -251,21 +250,25 @@ public class ByteBufferHashTable
}
/**
* Find a bucket for a key, attempting to resize the table with adjustTableWhenFull() if possible.
* Find a bucket for a key, attempting to grow the table with adjustTableWhenFull() if possible.
*
* @param keyBuffer buffer containing the key
* @param keyHash hash of the key
* @param preTableGrowthRunnable runnable that executes before the table grows
*
* @param keyBuffer buffer containing the key
* @param keyHash hash of the key
* @return bucket number of the found bucket or -1 if a bucket could not be allocated after resizing.
*/
protected int findBucketWithAutoGrowth(
final ByteBuffer keyBuffer,
final int keyHash
final int keyHash,
final Runnable preTableGrowthRunnable
)
{
int bucket = findBucket(canAllowNewBucket(), maxBuckets, tableBuffer, keyBuffer, keyHash);
if (bucket < 0) {
if (size < maxSizeForTesting) {
preTableGrowthRunnable.run();
adjustTableWhenFull();
bucket = findBucket(size < regrowthThreshold, maxBuckets, tableBuffer, keyBuffer, keyHash);
}
@ -277,7 +280,7 @@ public class ByteBufferHashTable
/**
* Finds the bucket into which we should insert a key.
*
* @param keyBuffer key, must have exactly keySize bytes remaining. Will not be modified.
* @param keyBuffer key, must have exactly keySize bytes remaining. Will not be modified.
* @param targetTableBuffer Need selectable buffer, since when resizing hash table,
* findBucket() is used on the newly allocated table buffer
*
@ -379,7 +382,9 @@ public class ByteBufferHashTable
public interface BucketUpdateHandler
{
void handleNewBucket(int bucketOffset);
void handlePreTableSwap();
void handleBucketMove(int oldBucketOffset, int newBucketOffset, ByteBuffer oldBuffer, ByteBuffer newBuffer);
}
}

View File

@ -0,0 +1,91 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.groupby.epinephelinae;
import com.google.common.collect.ImmutableList;
import org.apache.druid.query.aggregation.AggregatorFactory;
import java.nio.ByteBuffer;
import java.util.List;
public class ByteBufferKeySerde implements Grouper.KeySerde<ByteBuffer>
{
private final int keySize;
public ByteBufferKeySerde(final int keySize)
{
this.keySize = keySize;
}
@Override
public int keySize()
{
return keySize;
}
@Override
public Class<ByteBuffer> keyClazz()
{
return ByteBuffer.class;
}
@Override
public List<String> getDictionary()
{
return ImmutableList.of();
}
@Override
public ByteBuffer toByteBuffer(ByteBuffer key)
{
return key;
}
@Override
public ByteBuffer fromByteBuffer(ByteBuffer buffer, int position)
{
final ByteBuffer dup = buffer.duplicate();
dup.position(position).limit(position + keySize);
return dup.slice();
}
@Override
public Grouper.BufferComparator bufferComparator()
{
// This class is used by segment processing engines, where bufferComparator will not be called.
throw new UnsupportedOperationException();
}
@Override
public Grouper.BufferComparator bufferComparatorWithAggregators(
AggregatorFactory[] aggregatorFactories,
int[] aggregatorOffsets
)
{
// This class is used by segment processing engines, where bufferComparatorWithAggregators will not be called.
throw new UnsupportedOperationException();
}
@Override
public void reset()
{
// No state, nothing to reset
}
}

View File

@ -34,14 +34,13 @@ public class CloseableGrouperIterator<KeyType, T> implements CloseableIterator<T
private final Closer closer;
public CloseableGrouperIterator(
final Grouper<KeyType> grouper,
final boolean sorted,
final CloseableIterator<Entry<KeyType>> iterator,
final Function<Grouper.Entry<KeyType>, T> transformer,
final Closeable closeable
)
{
this.transformer = transformer;
this.iterator = grouper.iterator(sorted);
this.iterator = iterator;
this.closer = Closer.create();
closer.register(iterator);

View File

@ -264,9 +264,12 @@ public class ConcurrentGrouper<KeyType> implements Grouper<KeyType>
synchronized (hashBasedGrouper) {
if (!spilling) {
if (hashBasedGrouper.aggregate(key, keyHash).isOk()) {
final AggregateResult aggregateResult = hashBasedGrouper.aggregate(key, keyHash);
if (aggregateResult.isOk()) {
return AggregateResult.ok();
} else {
// Expecting all-or-nothing behavior.
assert aggregateResult.getCount() == 0;
spilling = true;
}
}

View File

@ -22,6 +22,7 @@ package org.apache.druid.query.groupby.epinephelinae;
import com.google.common.base.Preconditions;
import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.collections.ResourceHolder;
@ -34,9 +35,12 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.guava.BaseSequence;
import org.apache.druid.java.util.common.guava.Sequence;
import org.apache.druid.query.ColumnSelectorPlus;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.aggregation.AggregatorAdapters;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.epinephelinae.column.DictionaryBuildingStringGroupByColumnSelectorStrategy;
@ -47,6 +51,7 @@ import org.apache.druid.query.groupby.epinephelinae.column.GroupByColumnSelector
import org.apache.druid.query.groupby.epinephelinae.column.LongGroupByColumnSelectorStrategy;
import org.apache.druid.query.groupby.epinephelinae.column.NullableValueGroupByColumnSelectorStrategy;
import org.apache.druid.query.groupby.epinephelinae.column.StringGroupByColumnSelectorStrategy;
import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine;
import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
@ -54,7 +59,6 @@ import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.data.IndexedInts;
@ -69,6 +73,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.function.Function;
public class GroupByQueryEngineV2
{
@ -108,15 +113,6 @@ public class GroupByQueryEngineV2
throw new IAE("Should only have one interval, got[%s]", intervals);
}
final Sequence<Cursor> cursors = storageAdapter.makeCursors(
Filters.toFilter(query.getDimFilter()),
intervals.get(0),
query.getVirtualColumns(),
query.getGranularity(),
false,
null
);
final ResourceHolder<ByteBuffer> bufferHolder = intermediateResultsBufferPool.take();
final String fudgeTimestampString = NullHandling.emptyToNullIfNeeded(
@ -127,6 +123,59 @@ public class GroupByQueryEngineV2
? null
: DateTimes.utc(Long.parseLong(fudgeTimestampString));
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()));
final Interval interval = Iterables.getOnlyElement(query.getIntervals());
final boolean doVectorize = QueryContexts.getVectorize(query).shouldVectorize(
VectorGroupByEngine.canVectorize(query, storageAdapter, filter)
);
final Sequence<Row> result;
if (doVectorize) {
result = VectorGroupByEngine.process(
query,
storageAdapter,
bufferHolder.get(),
fudgeTimestamp,
filter,
interval,
querySpecificConfig
);
} else {
result = processNonVectorized(
query,
storageAdapter,
bufferHolder.get(),
fudgeTimestamp,
querySpecificConfig,
filter,
interval
);
}
return result.withBaggage(bufferHolder);
}
private static Sequence<Row> processNonVectorized(
final GroupByQuery query,
final StorageAdapter storageAdapter,
final ByteBuffer processingBuffer,
@Nullable final DateTime fudgeTimestamp,
final GroupByQueryConfig querySpecificConfig,
@Nullable final Filter filter,
final Interval interval
)
{
final Sequence<Cursor> cursors = storageAdapter.makeCursors(
filter,
interval,
query.getVirtualColumns(),
query.getGranularity(),
false,
null
);
return cursors.flatMap(
cursor -> new BaseSequence<>(
new BaseSequence.IteratorMaker<Row, GroupByEngineIterator<?>>()
@ -135,57 +184,42 @@ public class GroupByQueryEngineV2
public GroupByEngineIterator make()
{
final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
final boolean allSingleValueDims = query
.getDimensions()
.stream()
.allMatch(dimension -> {
final ColumnCapabilities columnCapabilities = columnSelectorFactory.getColumnCapabilities(
dimension.getDimension()
);
return columnCapabilities != null && !columnCapabilities.hasMultipleValues();
});
ColumnSelectorPlus<GroupByColumnSelectorStrategy>[] selectorPlus = DimensionHandlerUtils
final ColumnSelectorPlus<GroupByColumnSelectorStrategy>[] selectorPlus = DimensionHandlerUtils
.createColumnSelectorPluses(
STRATEGY_FACTORY,
query.getDimensions(),
columnSelectorFactory
);
GroupByColumnSelectorPlus[] dims = createGroupBySelectorPlus(selectorPlus);
final ByteBuffer buffer = bufferHolder.get();
final GroupByColumnSelectorPlus[] dims = createGroupBySelectorPlus(selectorPlus);
// Check if array-based aggregation is applicable
final boolean useArrayAggregation = isArrayAggregateApplicable(
final int cardinalityForArrayAggregation = getCardinalityForArrayAggregation(
querySpecificConfig,
query,
dims,
storageAdapter,
query.getVirtualColumns(),
buffer
processingBuffer
);
if (useArrayAggregation) {
if (cardinalityForArrayAggregation >= 0) {
return new ArrayAggregateIterator(
query,
querySpecificConfig,
cursor,
buffer,
processingBuffer,
fudgeTimestamp,
dims,
allSingleValueDims,
// There must be 0 or 1 dimension if isArrayAggregateApplicable() is true
dims.length == 0 ? 1 : storageAdapter.getDimensionCardinality(dims[0].getName())
isAllSingleValueDims(columnSelectorFactory::getColumnCapabilities, query.getDimensions()),
cardinalityForArrayAggregation
);
} else {
return new HashAggregateIterator(
query,
querySpecificConfig,
cursor,
buffer,
processingBuffer,
fudgeTimestamp,
dims,
allSingleValueDims
isAllSingleValueDims(columnSelectorFactory::getColumnCapabilities, query.getDimensions())
);
}
}
@ -197,65 +231,91 @@ public class GroupByQueryEngineV2
}
}
)
).withBaggage(bufferHolder);
);
}
private static boolean isArrayAggregateApplicable(
/**
* Returns the cardinality of array needed to do array-based aggregation, or -1 if array-based aggregation
* is impossible.
*/
public static int getCardinalityForArrayAggregation(
GroupByQueryConfig querySpecificConfig,
GroupByQuery query,
GroupByColumnSelectorPlus[] dims,
StorageAdapter storageAdapter,
VirtualColumns virtualColumns,
ByteBuffer buffer
)
{
if (querySpecificConfig.isForceHashAggregation()) {
return false;
return -1;
}
final List<DimensionSpec> dimensions = query.getDimensions();
final ColumnCapabilities columnCapabilities;
final int cardinality;
// Find cardinality
if (dims.length == 0) {
if (dimensions.isEmpty()) {
columnCapabilities = null;
cardinality = 1;
} else if (dims.length == 1) {
} else if (dimensions.size() == 1) {
// Only real columns can use array-based aggregation, since virtual columns cannot currently report their
// cardinality. We need to check if a virtual column exists with the same name, since virtual columns can shadow
// real columns, and we might miss that since we're going directly to the StorageAdapter (which only knows about
// real columns).
if (virtualColumns.exists(dims[0].getName())) {
return false;
if (query.getVirtualColumns().exists(Iterables.getOnlyElement(dimensions).getDimension())) {
return -1;
}
columnCapabilities = storageAdapter.getColumnCapabilities(dims[0].getName());
cardinality = storageAdapter.getDimensionCardinality(dims[0].getName());
final String columnName = Iterables.getOnlyElement(dimensions).getDimension();
columnCapabilities = storageAdapter.getColumnCapabilities(columnName);
cardinality = storageAdapter.getDimensionCardinality(columnName);
} else {
// Cannot use array-based aggregation with more than one dimension.
return false;
return -1;
}
// Choose array-based aggregation if the grouping key is a single string dimension of a
// known cardinality
if ((columnCapabilities == null || columnCapabilities.getType().equals(ValueType.STRING))
&& cardinality > 0) {
final AggregatorFactory[] aggregatorFactories = query
.getAggregatorSpecs()
.toArray(new AggregatorFactory[0]);
// Choose array-based aggregation if the grouping key is a single string dimension of a known cardinality
if (columnCapabilities != null && columnCapabilities.getType().equals(ValueType.STRING) && cardinality > 0) {
final AggregatorFactory[] aggregatorFactories = query.getAggregatorSpecs().toArray(new AggregatorFactory[0]);
final long requiredBufferCapacity = BufferArrayGrouper.requiredBufferCapacity(
cardinality,
aggregatorFactories
);
// Check that all keys and aggregated values can be contained in the buffer
return requiredBufferCapacity <= buffer.capacity();
return requiredBufferCapacity <= buffer.capacity() ? cardinality : -1;
} else {
return false;
return -1;
}
}
private static class GroupByStrategyFactory implements ColumnSelectorStrategyFactory<GroupByColumnSelectorStrategy>
/**
* Checks whether all "dimensions" are either single-valued or nonexistent (which is just as good as single-valued,
* since their selectors will show up as full of nulls).
*/
public static boolean isAllSingleValueDims(
final Function<String, ColumnCapabilities> capabilitiesFunction,
final List<DimensionSpec> dimensions
)
{
return dimensions
.stream()
.allMatch(
dimension -> {
if (dimension.mustDecorate()) {
// DimensionSpecs that decorate may turn singly-valued columns into multi-valued selectors.
// To be safe, we must return false here.
return false;
}
// Now check column capabilities.
final ColumnCapabilities columnCapabilities = capabilitiesFunction.apply(dimension.getDimension());
return columnCapabilities == null || !columnCapabilities.hasMultipleValues();
});
}
private static class GroupByStrategyFactory
implements ColumnSelectorStrategyFactory<GroupByColumnSelectorStrategy>
{
@Override
public GroupByColumnSelectorStrategy makeColumnSelectorStrategy(
@ -311,7 +371,7 @@ public class GroupByQueryEngineV2
final GroupByQueryConfig querySpecificConfig,
final Cursor cursor,
final ByteBuffer buffer,
final DateTime fudgeTimestamp,
@Nullable final DateTime fudgeTimestamp,
final GroupByColumnSelectorPlus[] dims,
final boolean allSingleValueDims
)
@ -340,8 +400,7 @@ public class GroupByQueryEngineV2
}
return new CloseableGrouperIterator<>(
grouper,
false,
grouper.iterator(false),
entry -> {
Map<String, Object> theMap = Maps.newLinkedHashMap();
@ -448,7 +507,7 @@ public class GroupByQueryEngineV2
GroupByQueryConfig querySpecificConfig,
Cursor cursor,
ByteBuffer buffer,
DateTime fudgeTimestamp,
@Nullable DateTime fudgeTimestamp,
GroupByColumnSelectorPlus[] dims,
boolean allSingleValueDims
)
@ -467,9 +526,10 @@ public class GroupByQueryEngineV2
return new BufferHashGrouper<>(
Suppliers.ofInstance(buffer),
keySerde,
cursor.getColumnSelectorFactory(),
query.getAggregatorSpecs()
.toArray(new AggregatorFactory[0]),
AggregatorAdapters.factorizeBuffered(
cursor.getColumnSelectorFactory(),
query.getAggregatorSpecs()
),
querySpecificConfig.getBufferGrouperMaxSize(),
querySpecificConfig.getBufferGrouperMaxLoadFactor(),
querySpecificConfig.getBufferGrouperInitialBuckets(),
@ -600,7 +660,7 @@ public class GroupByQueryEngineV2
GroupByQueryConfig querySpecificConfig,
Cursor cursor,
ByteBuffer buffer,
DateTime fudgeTimestamp,
@Nullable DateTime fudgeTimestamp,
GroupByColumnSelectorPlus[] dims,
boolean allSingleValueDims,
int cardinality
@ -622,9 +682,7 @@ public class GroupByQueryEngineV2
{
return new BufferArrayGrouper(
Suppliers.ofInstance(buffer),
cursor.getColumnSelectorFactory(),
query.getAggregatorSpecs()
.toArray(new AggregatorFactory[0]),
AggregatorAdapters.factorizeBuffered(cursor.getColumnSelectorFactory(), query.getAggregatorSpecs()),
cardinality
);
}
@ -698,7 +756,7 @@ public class GroupByQueryEngineV2
protected void putToMap(Integer key, Map<String, Object> map)
{
if (dim != null) {
if (key != -1) {
if (key != GroupByColumnSelectorStrategy.GROUP_BY_MISSING_VALUE) {
map.put(
dim.getOutputName(),
((DimensionSelector) dim.getSelector()).lookupName(key)
@ -710,7 +768,7 @@ public class GroupByQueryEngineV2
}
}
private static void convertRowTypesToOutputTypes(List<DimensionSpec> dimensionSpecs, Map<String, Object> rowMap)
public static void convertRowTypesToOutputTypes(List<DimensionSpec> dimensionSpecs, Map<String, Object> rowMap)
{
for (DimensionSpec dimSpec : dimensionSpecs) {
final ValueType outputType = dimSpec.getOutputType();

View File

@ -25,6 +25,7 @@ import com.google.common.base.Preconditions;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.query.aggregation.AggregatorFactory;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.nio.ByteBuffer;
import java.util.Arrays;
@ -36,10 +37,12 @@ import java.util.function.ToIntFunction;
* Groupers aggregate metrics from rows that they typically get from a ColumnSelectorFactory, under
* grouping keys that some outside driver is passing in. They can also iterate over the grouped
* rows after the aggregation is done.
* <p>
*
* They work sort of like a map of KeyType to aggregated values, except they don't support
* random lookups.
*
* See {@link VectorGrouper} for a vectorized version.
*
* @param <KeyType> type of the key that will be passed in
*/
public interface Grouper<KeyType> extends Closeable
@ -89,7 +92,7 @@ public interface Grouper<KeyType> extends Closeable
default ToIntFunction<KeyType> hashFunction()
{
return Groupers::hash;
return Groupers::hashObject;
}
/**
@ -247,6 +250,7 @@ public interface Grouper<KeyType> extends Closeable
*
* @return serialized key, or null if we are unable to serialize more keys due to resource limits
*/
@Nullable
ByteBuffer toByteBuffer(T key);
/**

View File

@ -19,6 +19,7 @@
package org.apache.druid.query.groupby.epinephelinae;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
public class Groupers
@ -28,17 +29,22 @@ public class Groupers
// No instantiation
}
static final AggregateResult DICTIONARY_FULL = AggregateResult.failure(
private static final AggregateResult DICTIONARY_FULL_ZERO_COUNT = AggregateResult.partial(
0,
"Not enough dictionary space to execute this query. Try increasing "
+ "druid.query.groupBy.maxMergingDictionarySize or enable disk spilling by setting "
+ "druid.query.groupBy.maxOnDiskStorage to a positive number."
);
static final AggregateResult HASH_TABLE_FULL = AggregateResult.failure(
private static final AggregateResult HASH_TABLE_FULL_ZERO_COUNT = AggregateResult.partial(
0,
"Not enough aggregation buffer space to execute this query. Try increasing "
+ "druid.processing.buffer.sizeBytes or enable disk spilling by setting "
+ "druid.query.groupBy.maxOnDiskStorage to a positive number."
);
private static final int USED_FLAG_MASK = 0x7fffffff;
private static final int C1 = 0xcc9e2d51;
private static final int C2 = 0x1b873593;
@ -50,18 +56,46 @@ public class Groupers
* 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)
private static int smear(int hashCode)
{
return C2 * Integer.rotateLeft(hashCode * C1, 15);
}
public static int hash(final Object obj)
public static AggregateResult dictionaryFull(final int count)
{
if (count == 0) {
return DICTIONARY_FULL_ZERO_COUNT;
} else {
return AggregateResult.partial(count, DICTIONARY_FULL_ZERO_COUNT.getReason());
}
}
public static AggregateResult hashTableFull(final int count)
{
if (count == 0) {
return HASH_TABLE_FULL_ZERO_COUNT;
} else {
return AggregateResult.partial(count, HASH_TABLE_FULL_ZERO_COUNT.getReason());
}
}
public static int hashObject(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 smear function, to improve distribution.
final int code = obj.hashCode();
return smear(code) & 0x7fffffff;
// Also apply the "smear" function, to improve distribution.
return smear(obj.hashCode()) & USED_FLAG_MASK;
}
public static int hashIntArray(final int[] ints, final int start, final int length)
{
// Similar to what Arrays.hashCode would do.
// Also apply the "smear" function, to improve distribution.
int hashCode = 1;
for (int i = 0; i < length; i++) {
hashCode = 31 * hashCode + ints[start + i];
}
return smear(hashCode) & USED_FLAG_MASK;
}
static int getUsedFlag(int keyHash)
@ -76,4 +110,22 @@ public class Groupers
slice.limit(slice.position() + sliceSize);
return slice.slice();
}
/**
* Write ints from "start" to "end" into "scratch", if start != 0. Otherwise, return null.
*/
@Nullable
public static int[] writeAggregationRows(final int[] scratch, final int start, final int end)
{
if (start == 0) {
return null;
} else {
final int numRows = end - start;
for (int i = 0; i < numRows; i++) {
scratch[i] = start + i;
}
return scratch;
}
}
}

View File

@ -24,8 +24,8 @@ import org.apache.druid.java.util.common.CloseableIterators;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.query.aggregation.AggregatorAdapters;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.ColumnSelectorFactory;
import java.nio.ByteBuffer;
import java.util.AbstractList;
@ -40,8 +40,6 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
private static final int DEFAULT_INITIAL_BUCKETS = 1024;
private static final float DEFAULT_MAX_LOAD_FACTOR = 0.7f;
private final AggregatorFactory[] aggregatorFactories;
// Limit to apply to results.
private int limit;
@ -66,8 +64,7 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
public LimitedBufferHashGrouper(
final Supplier<ByteBuffer> bufferSupplier,
final Grouper.KeySerde<KeyType> keySerde,
final ColumnSelectorFactory columnSelectorFactory,
final AggregatorFactory[] aggregatorFactories,
final AggregatorAdapters aggregators,
final int bufferGrouperMaxSize,
final float maxLoadFactor,
final int initialBuckets,
@ -75,7 +72,7 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
final boolean sortHasNonGroupingFields
)
{
super(bufferSupplier, keySerde, aggregatorFactories, bufferGrouperMaxSize);
super(bufferSupplier, keySerde, aggregators, HASH_SIZE + keySerde.keySize(), bufferGrouperMaxSize);
this.maxLoadFactor = maxLoadFactor > 0 ? maxLoadFactor : DEFAULT_MAX_LOAD_FACTOR;
this.initialBuckets = initialBuckets > 0 ? Math.max(MIN_INITIAL_BUCKETS, initialBuckets) : DEFAULT_INITIAL_BUCKETS;
this.limit = limit;
@ -85,18 +82,9 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
throw new IAE("Invalid maxLoadFactor[%f], must be < 1.0", maxLoadFactor);
}
int offset = HASH_SIZE + keySize;
this.aggregatorFactories = aggregatorFactories;
for (int i = 0; i < aggregatorFactories.length; i++) {
aggregators[i] = aggregatorFactories[i].factorizeBuffered(columnSelectorFactory);
aggregatorOffsets[i] = offset;
offset += aggregatorFactories[i].getMaxIntermediateSizeWithNulls();
}
// For each bucket, store an extra field indicating the bucket's current index within the heap when
// pushing down limits
offset += Integer.BYTES;
this.bucketSize = offset;
// pushing down limits (size Integer.BYTES).
this.bucketSize = HASH_SIZE + keySerde.keySize() + Integer.BYTES + aggregators.spaceNeeded();
}
@Override
@ -374,8 +362,8 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
return new Comparator<Integer>()
{
final BufferComparator bufferComparator = keySerde.bufferComparatorWithAggregators(
aggregatorFactories,
aggregatorOffsets
aggregators.factories().toArray(new AggregatorFactory[0]),
aggregators.aggregatorPositions()
);
@Override
@ -511,14 +499,12 @@ public class LimitedBufferHashGrouper<KeyType> extends AbstractBufferHashGrouper
offsetHeap.setAt(i, newBucketOffset);
// relocate aggregators (see https://github.com/apache/incubator-druid/pull/4071)
for (int j = 0; j < aggregators.length; j++) {
aggregators[j].relocate(
oldBucketOffset + aggregatorOffsets[j],
newBucketOffset + aggregatorOffsets[j],
tableBuffer,
newTableBuffer
);
}
aggregators.relocate(
oldBucketOffset + baseAggregatorOffset,
newBucketOffset + baseAggregatorOffset,
tableBuffer,
newTableBuffer
);
}
}

View File

@ -436,8 +436,7 @@ public class RowBasedGrouperHelper
final boolean includeTimestamp = GroupByStrategyV2.getUniversalTimestamp(query) == null;
return new CloseableGrouperIterator<>(
grouper,
true,
grouper.iterator(true),
new Function<Grouper.Entry<RowBasedKey>, Row>()
{
@Override
@ -833,7 +832,10 @@ public class RowBasedGrouperHelper
@Override
public int compare(Grouper.Entry<RowBasedKey> entry1, Grouper.Entry<RowBasedKey> entry2)
{
final int timeCompare = Longs.compare((long) entry1.getKey().getKey()[0], (long) entry2.getKey().getKey()[0]);
final int timeCompare = Longs.compare(
(long) entry1.getKey().getKey()[0],
(long) entry2.getKey().getKey()[0]
);
if (timeCompare != 0) {
return timeCompare;
@ -930,8 +932,10 @@ public class RowBasedGrouperHelper
// use natural comparison
cmp = Comparators.<Comparable>naturalNullsFirst().compare(lhs, rhs);
} else {
cmp = comparator.compare(DimensionHandlerUtils.convertObjectToString(lhs),
DimensionHandlerUtils.convertObjectToString(rhs));
cmp = comparator.compare(
DimensionHandlerUtils.convertObjectToString(lhs),
DimensionHandlerUtils.convertObjectToString(rhs)
);
}
if (cmp != 0) {
@ -1637,7 +1641,8 @@ public class RowBasedGrouperHelper
FloatRowBasedKeySerdeHelper(
int keyBufferPosition,
boolean pushLimitDown,
@Nullable StringComparator stringComparator)
@Nullable StringComparator stringComparator
)
{
this.keyBufferPosition = keyBufferPosition;
if (isPrimitiveComparable(pushLimitDown, stringComparator)) {

View File

@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.io.Closer;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import org.apache.druid.query.BaseQuery;
import org.apache.druid.query.aggregation.AggregatorAdapters;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
import org.apache.druid.segment.ColumnSelectorFactory;
@ -41,6 +42,7 @@ import java.io.FileInputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashSet;
import java.util.Iterator;
@ -55,11 +57,12 @@ import java.util.Set;
public class SpillingGrouper<KeyType> implements Grouper<KeyType>
{
private static final Logger log = new Logger(SpillingGrouper.class);
private final Grouper<KeyType> grouper;
private static final AggregateResult DISK_FULL = AggregateResult.failure(
private static final AggregateResult DISK_FULL = AggregateResult.partial(
0,
"Not enough disk space to execute this query. Try raising druid.query.groupBy.maxOnDiskStorage."
);
private final Grouper<KeyType> grouper;
private final KeySerde<KeyType> keySerde;
private final LimitedTemporaryStorage temporaryStorage;
private final ObjectMapper spillMapper;
@ -96,8 +99,7 @@ public class SpillingGrouper<KeyType> implements Grouper<KeyType>
LimitedBufferHashGrouper<KeyType> limitGrouper = new LimitedBufferHashGrouper<>(
bufferSupplier,
keySerde,
columnSelectorFactory,
aggregatorFactories,
AggregatorAdapters.factorizeBuffered(columnSelectorFactory, Arrays.asList(aggregatorFactories)),
bufferGrouperMaxSize,
bufferGrouperMaxLoadFactor,
bufferGrouperInitialBuckets,
@ -119,8 +121,7 @@ public class SpillingGrouper<KeyType> implements Grouper<KeyType>
this.grouper = new BufferHashGrouper<>(
bufferSupplier,
keySerde,
columnSelectorFactory,
aggregatorFactories,
AggregatorAdapters.factorizeBuffered(columnSelectorFactory, Arrays.asList(aggregatorFactories)),
bufferGrouperMaxSize,
bufferGrouperMaxLoadFactor,
bufferGrouperInitialBuckets,
@ -133,8 +134,7 @@ public class SpillingGrouper<KeyType> implements Grouper<KeyType>
this.grouper = new BufferHashGrouper<>(
bufferSupplier,
keySerde,
columnSelectorFactory,
aggregatorFactories,
AggregatorAdapters.factorizeBuffered(columnSelectorFactory, Arrays.asList(aggregatorFactories)),
bufferGrouperMaxSize,
bufferGrouperMaxLoadFactor,
bufferGrouperInitialBuckets,
@ -168,6 +168,9 @@ public class SpillingGrouper<KeyType> implements Grouper<KeyType>
if (result.isOk() || !spillingAllowed || temporaryStorage.maxSize() <= 0) {
return result;
} else {
// Expecting all-or-nothing behavior.
assert result.getCount() == 0;
// Warning: this can potentially block up a processing thread for a while.
try {
spill();

View File

@ -0,0 +1,77 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.groupby.epinephelinae;
import org.apache.druid.java.util.common.parsers.CloseableIterator;
import java.io.Closeable;
import java.nio.ByteBuffer;
/**
* Like a {@link Grouper}, but vectorized. Keys are always int arrays, so there is no generic type parameter KeyType.
* <p>
* This interface is designed such that an implementation can implement both Grouper and VectorGrouper. Of course,
* it would generally only make sense for a particular instance to be called with one set of functionality or the
* other.
*/
public interface VectorGrouper extends Closeable
{
/**
* Initialize the grouper. This method needs to be called before calling {@link #aggregateVector}.
*/
void initVectorized(int maxVectorSize);
/**
* Aggregate the current vector of rows from "startVectorOffset" to "endVectorOffset" using the provided keys.
*
* @param keySpace array holding keys, chunked into ints. First (endVectorOffset - startVectorOffset) keys
* must be valid.
* @param startRow row to start at (inclusive).
* @param endRow row to end at (exclusive).
*
* @return result that indicates how many keys were aggregated (may be partial due to resource limits)
*/
AggregateResult aggregateVector(int[] keySpace, int startRow, int endRow);
/**
* Reset the grouper to its initial state.
*/
void reset();
/**
* Close the grouper and release associated resources.
*/
@Override
void close();
/**
* Iterate through entries.
* <p>
* Some implementations allow writes even after this method is called. After you are done with the iterator
* returned by this method, you should either call {@link #close()} (if you are done with the VectorGrouper) or
* {@link #reset()} (if you want to reuse it).
* <p>
* Callers must process and discard the returned {@link Grouper.Entry}s immediately, because the keys may
* be reused.
*
* @return entry iterator
*/
CloseableIterator<Grouper.Entry<ByteBuffer>> iterator();
}

View File

@ -0,0 +1,71 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.groupby.epinephelinae.vector;
import org.apache.druid.segment.vector.VectorValueSelector;
import java.nio.ByteBuffer;
import java.util.Map;
public class DoubleGroupByVectorColumnSelector implements GroupByVectorColumnSelector
{
private final VectorValueSelector selector;
DoubleGroupByVectorColumnSelector(final VectorValueSelector selector)
{
this.selector = selector;
}
@Override
public int getGroupingKeySize()
{
return 2;
}
@Override
public void writeKeys(
final int[] keySpace,
final int keySize,
final int keyOffset,
final int startRow,
final int endRow
)
{
final double[] vector = selector.getDoubleVector();
for (int i = startRow, j = keyOffset; i < endRow; i++, j += keySize) {
final long longValue = Double.doubleToLongBits(vector[i]);
keySpace[j] = (int) (longValue >>> 32);
keySpace[j + 1] = (int) (longValue & 0xffffffffL);
}
}
@Override
public void writeKeyToResultRow(
final String outputName,
final ByteBuffer keyBuffer,
final int keyOffset,
final Map<String, Object> resultMap
)
{
final double value = keyBuffer.getDouble(keyOffset * Integer.BYTES);
resultMap.put(outputName, value);
}
}

View File

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.groupby.epinephelinae.vector;
import org.apache.druid.segment.vector.VectorValueSelector;
import java.nio.ByteBuffer;
import java.util.Map;
public class FloatGroupByVectorColumnSelector implements GroupByVectorColumnSelector
{
private final VectorValueSelector selector;
FloatGroupByVectorColumnSelector(final VectorValueSelector selector)
{
this.selector = selector;
}
@Override
public int getGroupingKeySize()
{
return 1;
}
@Override
public void writeKeys(
final int[] keySpace,
final int keySize,
final int keyOffset,
final int startRow,
final int endRow
)
{
final float[] vector = selector.getFloatVector();
for (int i = startRow, j = keyOffset; i < endRow; i++, j += keySize) {
keySpace[j] = Float.floatToIntBits(vector[i]);
}
}
@Override
public void writeKeyToResultRow(
final String outputName,
final ByteBuffer keyBuffer,
final int keyOffset,
final Map<String, Object> resultMap
)
{
final float value = Float.intBitsToFloat(keyBuffer.getInt(keyOffset * Integer.BYTES));
resultMap.put(outputName, value);
}
}

View File

@ -0,0 +1,37 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.groupby.epinephelinae.vector;
import java.nio.ByteBuffer;
import java.util.Map;
public interface GroupByVectorColumnSelector
{
int getGroupingKeySize();
void writeKeys(int[] keySpace, int keySize, int keyOffset, int startRow, int endRow);
void writeKeyToResultRow(
String outputName,
ByteBuffer keyBuffer,
int keyOffset,
Map<String, Object> resultMap
);
}

View File

@ -0,0 +1,70 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.groupby.epinephelinae.vector;
import org.apache.druid.query.dimension.VectorColumnStrategizer;
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
import org.apache.druid.segment.vector.VectorValueSelector;
public class GroupByVectorColumnStrategizer implements VectorColumnStrategizer<GroupByVectorColumnSelector>
{
private static final GroupByVectorColumnStrategizer INSTANCE = new GroupByVectorColumnStrategizer();
private GroupByVectorColumnStrategizer()
{
// Singleton.
}
public static GroupByVectorColumnStrategizer instance()
{
return INSTANCE;
}
@Override
public GroupByVectorColumnSelector makeSingleValueDimensionStrategy(final SingleValueDimensionVectorSelector selector)
{
return new SingleValueStringGroupByVectorColumnSelector(selector);
}
@Override
public GroupByVectorColumnSelector makeMultiValueDimensionStrategy(final MultiValueDimensionVectorSelector selector)
{
throw new UnsupportedOperationException("Multi-value dimensions not yet implemented for vectorized groupBys");
}
@Override
public GroupByVectorColumnSelector makeFloatStrategy(final VectorValueSelector selector)
{
return new FloatGroupByVectorColumnSelector(selector);
}
@Override
public GroupByVectorColumnSelector makeDoubleStrategy(final VectorValueSelector selector)
{
return new DoubleGroupByVectorColumnSelector(selector);
}
@Override
public GroupByVectorColumnSelector makeLongStrategy(final VectorValueSelector selector)
{
return new LongGroupByVectorColumnSelector(selector);
}
}

View File

@ -0,0 +1,70 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.groupby.epinephelinae.vector;
import org.apache.druid.segment.vector.VectorValueSelector;
import java.nio.ByteBuffer;
import java.util.Map;
public class LongGroupByVectorColumnSelector implements GroupByVectorColumnSelector
{
private final VectorValueSelector selector;
LongGroupByVectorColumnSelector(final VectorValueSelector selector)
{
this.selector = selector;
}
@Override
public int getGroupingKeySize()
{
return 2;
}
@Override
public void writeKeys(
final int[] keySpace,
final int keySize,
final int keyOffset,
final int startRow,
final int endRow
)
{
final long[] vector = selector.getLongVector();
for (int i = startRow, j = keyOffset; i < endRow; i++, j += keySize) {
keySpace[j] = (int) (vector[i] >>> 32);
keySpace[j + 1] = (int) (vector[i] & 0xffffffffL);
}
}
@Override
public void writeKeyToResultRow(
final String outputName,
final ByteBuffer keyBuffer,
final int keyOffset,
final Map<String, Object> resultMap
)
{
final long value = keyBuffer.getLong(keyOffset * Integer.BYTES);
resultMap.put(outputName, value);
}
}

View File

@ -0,0 +1,69 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF 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 org.apache.druid.query.groupby.epinephelinae.vector;
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
import java.nio.ByteBuffer;
import java.util.Map;
public class SingleValueStringGroupByVectorColumnSelector implements GroupByVectorColumnSelector
{
private final SingleValueDimensionVectorSelector selector;
SingleValueStringGroupByVectorColumnSelector(final SingleValueDimensionVectorSelector selector)
{
this.selector = selector;
}
@Override
public int getGroupingKeySize()
{
return 1;
}
@Override
public void writeKeys(
final int[] keySpace,
final int keySize,
final int keyOffset,
final int startRow,
final int endRow
)
{
final int[] rowVector = selector.getRowVector();
for (int i = startRow, j = keyOffset; i < endRow; i++, j += keySize) {
keySpace[j] = rowVector[i];
}
}
@Override
public void writeKeyToResultRow(
final String outputName,
final ByteBuffer keyBuffer,
final int keyOffset,
final Map<String, Object> resultMap
)
{
final int id = keyBuffer.getInt(keyOffset * Integer.BYTES);
resultMap.put(outputName, selector.lookupName(id));
}
}

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