tidy up group by engines after removal of v1 (#15665)

This commit is contained in:
Clint Wylie 2024-01-11 00:52:52 -08:00 committed by GitHub
parent 87fbe42218
commit 2118258b54
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 382 additions and 360 deletions

View File

@ -25,14 +25,20 @@ import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.inject.Inject;
import org.apache.druid.collections.BlockingPool;
import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.collections.ReferenceCountingResourceHolder;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.guice.annotations.Global;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.guice.annotations.Merging;
import org.apache.druid.guice.annotations.Smile;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.collect.Utils;
@ -57,18 +63,29 @@ import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.groupby.epinephelinae.GroupByBinaryFnV2;
import org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunnerV2;
import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.groupby.epinephelinae.BufferArrayGrouper;
import org.apache.druid.query.groupby.epinephelinae.GroupByMergingQueryRunner;
import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngine;
import org.apache.druid.query.groupby.epinephelinae.GroupByResultMergeFn;
import org.apache.druid.query.groupby.epinephelinae.GroupByRowProcessor;
import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine;
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
import org.apache.druid.query.groupby.orderby.LimitSpec;
import org.apache.druid.query.groupby.orderby.NoopLimitSpec;
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
import org.apache.druid.segment.DimensionHandlerUtils;
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.ColumnType;
import org.apache.druid.segment.column.Types;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.filter.Filters;
import org.apache.druid.segment.join.filter.AllNullColumnSelectorFactory;
import org.apache.druid.utils.CloseableUtils;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
@ -84,6 +101,9 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.BinaryOperator;
import java.util.stream.Collectors;
/**
* Common code for processing {@link GroupByQuery}.
*/
public class GroupingEngine
{
public static final String CTX_KEY_FUDGE_TIMESTAMP = "fudgeTimestamp";
@ -174,7 +194,7 @@ public class GroupingEngine
*/
public BinaryOperator<ResultRow> createMergeFn(Query<ResultRow> queryParam)
{
return new GroupByBinaryFnV2((GroupByQuery) queryParam);
return new GroupByResultMergeFn((GroupByQuery) queryParam);
}
public GroupByQuery prepareGroupByQuery(GroupByQuery query)
@ -398,7 +418,7 @@ public class GroupingEngine
final Iterable<QueryRunner<ResultRow>> queryRunners
)
{
return new GroupByMergingQueryRunnerV2(
return new GroupByMergingQueryRunner(
configSupplier.get(),
processingConfig,
queryProcessingPool,
@ -430,14 +450,71 @@ public class GroupingEngine
@Nullable GroupByQueryMetrics groupByQueryMetrics
)
{
return GroupByQueryEngineV2.process(
final GroupByQueryConfig querySpecificConfig = configSupplier.get().withOverrides(query);
if (storageAdapter == null) {
throw new ISE(
"Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped."
);
}
final List<Interval> intervals = query.getQuerySegmentSpec().getIntervals();
if (intervals.size() != 1) {
throw new IAE("Should only have one interval, got[%s]", intervals);
}
final ResourceHolder<ByteBuffer> bufferHolder = bufferPool.take();
try {
final String fudgeTimestampString = NullHandling.emptyToNullIfNeeded(
query.context().getString(GroupingEngine.CTX_KEY_FUDGE_TIMESTAMP)
);
final DateTime fudgeTimestamp = fudgeTimestampString == null
? 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 = query.context().getVectorize().shouldVectorize(
VectorGroupByEngine.canVectorize(query, storageAdapter, filter)
);
final Sequence<ResultRow> result;
if (doVectorize) {
result = VectorGroupByEngine.process(
query,
storageAdapter,
bufferPool,
configSupplier.get().withOverrides(query),
bufferHolder.get(),
fudgeTimestamp,
filter,
interval,
querySpecificConfig,
processingConfig,
groupByQueryMetrics
);
} else {
result = GroupByQueryEngine.process(
query,
storageAdapter,
bufferHolder.get(),
fudgeTimestamp,
querySpecificConfig,
processingConfig,
filter,
interval,
groupByQueryMetrics
);
}
return result.withBaggage(bufferHolder);
}
catch (Throwable e) {
bufferHolder.close();
throw e;
}
}
/**
@ -743,6 +820,89 @@ public class GroupingEngine
return aggsAndPostAggs;
}
/**
* 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,
StorageAdapter storageAdapter,
ByteBuffer buffer
)
{
if (querySpecificConfig.isForceHashAggregation()) {
return -1;
}
final List<DimensionSpec> dimensions = query.getDimensions();
final ColumnCapabilities columnCapabilities;
final int cardinality;
// Find cardinality
if (dimensions.isEmpty()) {
columnCapabilities = null;
cardinality = 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 (query.getVirtualColumns().exists(Iterables.getOnlyElement(dimensions).getDimension())) {
return -1;
}
// We cannot support array-based aggregation on array based grouping as we we donot have all the indexes up front
// to allocate appropriate values
if (dimensions.get(0).getOutputType().isArray()) {
return -1;
}
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 -1;
}
// Choose array-based aggregation if the grouping key is a single string dimension of a known cardinality
if (Types.is(columnCapabilities, 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
if (requiredBufferCapacity < 0 || requiredBufferCapacity > buffer.capacity()) {
return -1;
} else {
return cardinality;
}
} else {
return -1;
}
}
public static void convertRowTypesToOutputTypes(
final List<DimensionSpec> dimensionSpecs,
final ResultRow resultRow,
final int resultRowDimensionStart
)
{
for (int i = 0; i < dimensionSpecs.size(); i++) {
DimensionSpec dimSpec = dimensionSpecs.get(i);
final int resultRowIndex = resultRowDimensionStart + i;
final ColumnType outputType = dimSpec.getOutputType();
resultRow.set(
resultRowIndex,
DimensionHandlerUtils.convertObjectToType(resultRow.get(resultRowIndex), outputType)
);
}
}
/**
* Wraps the sequence around if for this query a summary row might be needed in case the input becomes empty.
*/

View File

@ -77,7 +77,7 @@ public class BufferArrayGrouper implements VectorGrouper, IntGrouper
* Returns -1 if cardinality + 1 (for null) > Integer.MAX_VALUE. Returns computed required buffer capacity
* otherwise.
*/
static long requiredBufferCapacity(int cardinality, AggregatorFactory[] aggregatorFactories)
public static long requiredBufferCapacity(int cardinality, AggregatorFactory[] aggregatorFactories)
{
final long cardinalityWithMissingValue = computeCardinalityWithMissingValue(cardinality);
// Cardinality should be in the integer range. See DimensionDictionarySelector.

View File

@ -73,7 +73,7 @@ import java.util.concurrent.TimeoutException;
* using a buffer provided by {@code mergeBufferPool} and a parallel executor provided by {@code exec}. Outputs a
* fully aggregated stream of {@link ResultRow} objects. Does not apply post-aggregators.
*
* The input {@code queryables} are expected to come from a {@link GroupByQueryEngineV2}. This code runs on data
* The input {@code queryables} are expected to come from a {@link GroupByQueryEngine}. This code runs on data
* servers, like Historicals.
*
* This class has some resemblance to {@link GroupByRowProcessor}. See the javadoc of that class for a discussion of
@ -82,9 +82,9 @@ import java.util.concurrent.TimeoutException;
* Used by
* {@link org.apache.druid.query.groupby.GroupingEngine#mergeRunners(QueryProcessingPool, Iterable)}
*/
public class GroupByMergingQueryRunnerV2 implements QueryRunner<ResultRow>
public class GroupByMergingQueryRunner implements QueryRunner<ResultRow>
{
private static final Logger log = new Logger(GroupByMergingQueryRunnerV2.class);
private static final Logger log = new Logger(GroupByMergingQueryRunner.class);
private static final String CTX_KEY_MERGE_RUNNERS_USING_CHAINED_EXECUTION = "mergeRunnersUsingChainedExecution";
private final GroupByQueryConfig config;
@ -98,7 +98,7 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<ResultRow>
private final String processingTmpDir;
private final int mergeBufferSize;
public GroupByMergingQueryRunnerV2(
public GroupByMergingQueryRunner(
GroupByQueryConfig config,
DruidProcessingConfig processingConfig,
QueryProcessingPool queryProcessingPool,

View File

@ -19,14 +19,11 @@
package org.apache.druid.query.groupby.epinephelinae;
import com.google.common.annotations.VisibleForTesting;
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 org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.guava.BaseSequence;
@ -55,7 +52,6 @@ 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.NullableNumericGroupByColumnSelectorStrategy;
import org.apache.druid.query.groupby.epinephelinae.column.StringGroupByColumnSelectorStrategy;
import org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine;
import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
import org.apache.druid.query.ordering.StringComparator;
@ -68,10 +64,8 @@ import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.Types;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.data.IndexedInts;
import org.apache.druid.segment.filter.Filters;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -84,113 +78,28 @@ import java.util.NoSuchElementException;
import java.util.stream.Stream;
/**
* Class that knows how to process a groupBy query on a single {@link StorageAdapter}. It returns a {@link Sequence}
* of {@link ResultRow} objects that are not guaranteed to be in any particular order, and may not even be fully
* grouped. It is expected that a downstream {@link GroupByMergingQueryRunnerV2} will finish grouping these results.
*
* This code runs on data servers, like Historicals.
*
* Used by
* Contains logic to process a groupBy query on a single {@link StorageAdapter} in a non-vectorized manner.
* Processing returns a {@link Sequence} of {@link ResultRow} objects that are not guaranteed to be in any particular
* order, and may not even be fully grouped. It is expected that a downstream {@link GroupByMergingQueryRunner} will
* finish grouping these results.
* <p>
* This code runs on anything that processes {@link StorageAdapter} directly, typically data servers like Historicals.
* <p>
* Used for non-vectorized processing by
* {@link GroupingEngine#process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)}.
*
* @see org.apache.druid.query.groupby.epinephelinae.vector.VectorGroupByEngine for vectorized version of this class
*/
public class GroupByQueryEngineV2
public class GroupByQueryEngine
{
public static final GroupByStrategyFactory STRATEGY_FACTORY = new GroupByStrategyFactory();
private static final GroupByStrategyFactory STRATEGY_FACTORY = new GroupByStrategyFactory();
public static GroupByColumnSelectorPlus[] createGroupBySelectorPlus(
ColumnSelectorPlus<GroupByColumnSelectorStrategy>[] baseSelectorPlus,
int dimensionStart
)
{
GroupByColumnSelectorPlus[] retInfo = new GroupByColumnSelectorPlus[baseSelectorPlus.length];
int curPos = 0;
for (int i = 0; i < retInfo.length; i++) {
retInfo[i] = new GroupByColumnSelectorPlus(baseSelectorPlus[i], curPos, dimensionStart + i);
curPos += retInfo[i].getColumnSelectorStrategy().getGroupingKeySize();
}
return retInfo;
}
private GroupByQueryEngineV2()
private GroupByQueryEngine()
{
// No instantiation
}
public static Sequence<ResultRow> process(
final GroupByQuery query,
@Nullable final StorageAdapter storageAdapter,
final NonBlockingPool<ByteBuffer> intermediateResultsBufferPool,
final GroupByQueryConfig querySpecificConfig,
final DruidProcessingConfig processingConfig,
@Nullable final GroupByQueryMetrics groupByQueryMetrics
)
{
if (storageAdapter == null) {
throw new ISE(
"Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped."
);
}
final List<Interval> intervals = query.getQuerySegmentSpec().getIntervals();
if (intervals.size() != 1) {
throw new IAE("Should only have one interval, got[%s]", intervals);
}
final ResourceHolder<ByteBuffer> bufferHolder = intermediateResultsBufferPool.take();
try {
final String fudgeTimestampString = NullHandling.emptyToNullIfNeeded(
query.context().getString(GroupingEngine.CTX_KEY_FUDGE_TIMESTAMP)
);
final DateTime fudgeTimestamp = fudgeTimestampString == null
? 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 = query.context().getVectorize().shouldVectorize(
VectorGroupByEngine.canVectorize(query, storageAdapter, filter)
);
final Sequence<ResultRow> result;
if (doVectorize) {
result = VectorGroupByEngine.process(
query,
storageAdapter,
bufferHolder.get(),
fudgeTimestamp,
filter,
interval,
querySpecificConfig,
processingConfig,
groupByQueryMetrics
);
} else {
result = processNonVectorized(
query,
storageAdapter,
bufferHolder.get(),
fudgeTimestamp,
querySpecificConfig,
processingConfig,
filter,
interval,
groupByQueryMetrics
);
}
return result.withBaggage(bufferHolder);
}
catch (Throwable e) {
bufferHolder.close();
throw e;
}
}
private static Sequence<ResultRow> processNonVectorized(
final GroupByQuery query,
final StorageAdapter storageAdapter,
final ByteBuffer processingBuffer,
@ -216,7 +125,7 @@ public class GroupByQueryEngineV2
new BaseSequence.IteratorMaker<ResultRow, GroupByEngineIterator<?>>()
{
@Override
public GroupByEngineIterator make()
public GroupByEngineIterator<?> make()
{
final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory();
final ColumnSelectorPlus<GroupByColumnSelectorStrategy>[] selectorPlus = DimensionHandlerUtils
@ -225,12 +134,18 @@ public class GroupByQueryEngineV2
query.getDimensions(),
columnSelectorFactory
);
final GroupByColumnSelectorPlus[] dims = createGroupBySelectorPlus(
selectorPlus,
query.getResultRowDimensionStart()
GroupByColumnSelectorPlus[] dims = new GroupByColumnSelectorPlus[selectorPlus.length];
int curPos = 0;
for (int i = 0; i < dims.length; i++) {
dims[i] = new GroupByColumnSelectorPlus(
selectorPlus[i],
curPos,
query.getResultRowDimensionStart() + i
);
curPos += dims[i].getColumnSelectorStrategy().getGroupingKeySize();
}
final int cardinalityForArrayAggregation = getCardinalityForArrayAggregation(
final int cardinalityForArrayAggregation = GroupingEngine.getCardinalityForArrayAggregation(
querySpecificConfig,
query,
storageAdapter,
@ -246,7 +161,7 @@ public class GroupByQueryEngineV2
processingBuffer,
fudgeTimestamp,
dims,
hasNoExplodingDimensions(columnSelectorFactory, query.getDimensions()),
hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions()),
cardinalityForArrayAggregation
);
} else {
@ -258,13 +173,13 @@ public class GroupByQueryEngineV2
processingBuffer,
fudgeTimestamp,
dims,
hasNoExplodingDimensions(columnSelectorFactory, query.getDimensions())
hasNoImplicitUnnestDimensions(columnSelectorFactory, query.getDimensions())
);
}
}
@Override
public void cleanup(GroupByEngineIterator iterFromMake)
public void cleanup(GroupByEngineIterator<?> iterFromMake)
{
iterFromMake.close();
}
@ -274,77 +189,32 @@ public class GroupByQueryEngineV2
}
/**
* Returns the cardinality of array needed to do array-based aggregation, or -1 if array-based aggregation
* is impossible.
* check if a column will operate correctly with {@link LimitedBufferHashGrouper} for query limit pushdown
*/
public static int getCardinalityForArrayAggregation(
GroupByQueryConfig querySpecificConfig,
GroupByQuery query,
StorageAdapter storageAdapter,
ByteBuffer buffer
)
@VisibleForTesting
public static boolean canPushDownLimit(ColumnSelectorFactory columnSelectorFactory, String columnName)
{
if (querySpecificConfig.isForceHashAggregation()) {
return -1;
ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(columnName);
if (capabilities != null) {
// strings can be pushed down if dictionaries are sorted and unique per id
if (capabilities.is(ValueType.STRING)) {
return capabilities.areDictionaryValuesSorted().and(capabilities.areDictionaryValuesUnique()).isTrue();
}
final List<DimensionSpec> dimensions = query.getDimensions();
final ColumnCapabilities columnCapabilities;
final int cardinality;
// Find cardinality
if (dimensions.isEmpty()) {
columnCapabilities = null;
cardinality = 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 (query.getVirtualColumns().exists(Iterables.getOnlyElement(dimensions).getDimension())) {
return -1;
}
// We cannot support array-based aggregation on array based grouping as we we donot have all the indexes up front
// to allocate appropriate values
if (dimensions.get(0).getOutputType().isArray()) {
return -1;
}
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 -1;
}
// Choose array-based aggregation if the grouping key is a single string dimension of a known cardinality
if (Types.is(columnCapabilities, 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
if (requiredBufferCapacity < 0 || requiredBufferCapacity > buffer.capacity()) {
return -1;
} else {
return cardinality;
}
} else {
return -1;
// party on
return true;
}
// we don't know what we don't know, don't assume otherwise
return false;
}
/**
* Checks whether all "dimensions" are either single-valued, or if the input column or output dimension spec has
* specified a type that {@link ColumnType#isArray()}. Both cases indicate we don't want to explode the under-lying
* specified a type that {@link ColumnType#isArray()}. Both cases indicate we don't want to unnest the under-lying
* multi value column. Since selectors on non-existent columns will show up as full of nulls, they are effectively
* single valued, however capabilites on columns can also be null, for example during broker merge with an 'inline'
* datasource subquery, so we only return true from this method when capabilities are fully known.
*/
public static boolean hasNoExplodingDimensions(
private static boolean hasNoImplicitUnnestDimensions(
final ColumnInspector inspector,
final List<DimensionSpec> dimensions
)
@ -369,44 +239,7 @@ public class GroupByQueryEngineV2
});
}
public static void convertRowTypesToOutputTypes(
final List<DimensionSpec> dimensionSpecs,
final ResultRow resultRow,
final int resultRowDimensionStart
)
{
for (int i = 0; i < dimensionSpecs.size(); i++) {
DimensionSpec dimSpec = dimensionSpecs.get(i);
final int resultRowIndex = resultRowDimensionStart + i;
final ColumnType outputType = dimSpec.getOutputType();
resultRow.set(
resultRowIndex,
DimensionHandlerUtils.convertObjectToType(resultRow.get(resultRowIndex), outputType)
);
}
}
/**
* check if a column will operate correctly with {@link LimitedBufferHashGrouper} for query limit pushdown
*/
public static boolean canPushDownLimit(ColumnSelectorFactory columnSelectorFactory, String columnName)
{
ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(columnName);
if (capabilities != null) {
// strings can be pushed down if dictionaries are sorted and unique per id
if (capabilities.is(ValueType.STRING)) {
return capabilities.areDictionaryValuesSorted().and(capabilities.areDictionaryValuesUnique()).isTrue();
}
// party on
return true;
}
// we don't know what we don't know, don't assume otherwise
return false;
}
private static class GroupByStrategyFactory
implements ColumnSelectorStrategyFactory<GroupByColumnSelectorStrategy>
private static class GroupByStrategyFactory implements ColumnSelectorStrategyFactory<GroupByColumnSelectorStrategy>
{
@Override
public GroupByColumnSelectorStrategy makeColumnSelectorStrategy(
@ -457,7 +290,7 @@ public class GroupByQueryEngineV2
}
}
public abstract static class GroupByEngineIterator<KeyType> implements Iterator<ResultRow>, Closeable
private abstract static class GroupByEngineIterator<KeyType> implements Iterator<ResultRow>, Closeable
{
protected final GroupByQuery query;
protected final GroupByQueryConfig querySpecificConfig;
@ -528,7 +361,7 @@ public class GroupByQueryEngineV2
// Add dimensions, and convert their types if necessary.
putToRow(entry.getKey(), resultRow);
convertRowTypesToOutputTypes(query.getDimensions(), resultRow, resultRowDimensionStart);
GroupingEngine.convertRowTypesToOutputTypes(query.getDimensions(), resultRow, resultRowDimensionStart);
// Add aggregations.
for (int i = 0; i < entry.getValues().length; i++) {
@ -623,7 +456,7 @@ public class GroupByQueryEngineV2
}
}
public static class HashAggregateIterator extends GroupByEngineIterator<ByteBuffer>
private static class HashAggregateIterator extends GroupByEngineIterator<ByteBuffer>
{
private static final Logger LOGGER = new Logger(HashAggregateIterator.class);
@ -637,7 +470,7 @@ public class GroupByQueryEngineV2
// Sum of internal state footprint across all "dims".
private long selectorInternalFootprint = 0;
public HashAggregateIterator(
private HashAggregateIterator(
GroupByQuery query,
GroupByQueryConfig querySpecificConfig,
DruidProcessingConfig processingConfig,
@ -673,7 +506,7 @@ public class GroupByQueryEngineV2
canDoLimitPushdown = Stream.concat(
query.getDimensions().stream().map(DimensionSpec::getDimension),
limitSpec.getColumns().stream().map(OrderByColumnSpec::getDimension)
).allMatch(col -> GroupByQueryEngineV2.canPushDownLimit(selectorFactory, col));
).allMatch(col -> canPushDownLimit(selectorFactory, col));
} else {
canDoLimitPushdown = false;
}
@ -876,7 +709,7 @@ public class GroupByQueryEngineV2
private IndexedInts multiValues;
private int nextValIndex;
public ArrayAggregateIterator(
private ArrayAggregateIterator(
GroupByQuery query,
GroupByQueryConfig querySpecificConfig,
DruidProcessingConfig processingConfig,
@ -1005,7 +838,7 @@ public class GroupByQueryEngineV2
private final GroupByColumnSelectorPlus[] dims;
private final GroupByQuery query;
public GroupByEngineKeySerde(final GroupByColumnSelectorPlus[] dims, GroupByQuery query)
private GroupByEngineKeySerde(final GroupByColumnSelectorPlus[] dims, GroupByQuery query)
{
this.dims = dims;
int keySize = 0;

View File

@ -36,11 +36,11 @@ import java.util.function.BinaryOperator;
* Used by
* {@link org.apache.druid.query.groupby.GroupingEngine#mergeResults}.
*/
public class GroupByBinaryFnV2 implements BinaryOperator<ResultRow>
public class GroupByResultMergeFn implements BinaryOperator<ResultRow>
{
private final GroupByQuery query;
public GroupByBinaryFnV2(GroupByQuery query)
public GroupByResultMergeFn(GroupByQuery query)
{
this.query = query;
}

View File

@ -53,7 +53,7 @@ import java.util.UUID;
*
* This class has two primary uses: processing nested groupBys, and processing subtotals.
*
* This class has some similarity to {@link GroupByMergingQueryRunnerV2}, but is different enough that it deserved to
* This class has some similarity to {@link GroupByMergingQueryRunner}, but is different enough that it deserved to
* be its own class. Some common code between the two classes is in {@link RowBasedGrouperHelper}.
*/
public class GroupByRowProcessor

View File

@ -98,7 +98,7 @@ import java.util.stream.Collectors;
import java.util.stream.IntStream;
/**
* This class contains shared code between {@link GroupByMergingQueryRunnerV2} and {@link GroupByRowProcessor}.
* This class contains shared code between {@link GroupByMergingQueryRunner} and {@link GroupByRowProcessor}.
*/
public class RowBasedGrouperHelper
{

View File

@ -34,11 +34,11 @@ 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.GroupByQueryMetrics;
import org.apache.druid.query.groupby.GroupingEngine;
import org.apache.druid.query.groupby.ResultRow;
import org.apache.druid.query.groupby.epinephelinae.AggregateResult;
import org.apache.druid.query.groupby.epinephelinae.BufferArrayGrouper;
import org.apache.druid.query.groupby.epinephelinae.CloseableGrouperIterator;
import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2;
import org.apache.druid.query.groupby.epinephelinae.HashVectorGrouper;
import org.apache.druid.query.groupby.epinephelinae.VectorGrouper;
import org.apache.druid.query.groupby.epinephelinae.collection.MemoryPointer;
@ -64,6 +64,15 @@ import java.util.List;
import java.util.NoSuchElementException;
import java.util.stream.Collectors;
/**
* Contains logic to process a groupBy query on a single {@link StorageAdapter} in a vectorized manner.
* This code runs on anything that processes {@link StorageAdapter} directly, typically data servers like Historicals.
* <p>
* Used for vectorized processing by
* {@link GroupingEngine#process(GroupByQuery, StorageAdapter, GroupByQueryMetrics)}.
*
* @see org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngine for non-vectorized version of this logic
*/
public class VectorGroupByEngine
{
private VectorGroupByEngine()
@ -71,58 +80,6 @@ public class VectorGroupByEngine
// No instantiation.
}
public static boolean canVectorize(
final GroupByQuery query,
final StorageAdapter adapter,
@Nullable final Filter filter
)
{
final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(adapter);
return adapter.canVectorize(filter, query.getVirtualColumns(), false)
&& canVectorizeDimensions(inspector, query.getDimensions())
&& VirtualColumns.shouldVectorize(query, query.getVirtualColumns(), adapter)
&& query.getAggregatorSpecs()
.stream()
.allMatch(aggregatorFactory -> aggregatorFactory.canVectorize(inspector));
}
public static boolean canVectorizeDimensions(
final ColumnInspector inspector,
final List<DimensionSpec> dimensions
)
{
return dimensions
.stream()
.allMatch(
dimension -> {
if (!dimension.canVectorize()) {
return false;
}
if (dimension.mustDecorate()) {
// group by on multi value dimensions are not currently supported
// DimensionSpecs that decorate may turn singly-valued columns into multi-valued selectors.
// To be safe, we must return false here.
return false;
}
if (dimension.getOutputType().isArray()) {
// group by on arrays is not currently supported in the vector processing engine
return false;
}
// Now check column capabilities.
final ColumnCapabilities columnCapabilities = inspector.getColumnCapabilities(dimension.getDimension());
// null here currently means the column does not exist, nil columns can be vectorized
if (columnCapabilities == null) {
return true;
}
// must be single valued
return columnCapabilities.hasMultipleValues().isFalse();
});
}
public static Sequence<ResultRow> process(
final GroupByQuery query,
final StorageAdapter storageAdapter,
@ -226,6 +183,58 @@ public class VectorGroupByEngine
);
}
public static boolean canVectorize(
final GroupByQuery query,
final StorageAdapter adapter,
@Nullable final Filter filter
)
{
final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(adapter);
return adapter.canVectorize(filter, query.getVirtualColumns(), false)
&& canVectorizeDimensions(inspector, query.getDimensions())
&& VirtualColumns.shouldVectorize(query, query.getVirtualColumns(), adapter)
&& query.getAggregatorSpecs()
.stream()
.allMatch(aggregatorFactory -> aggregatorFactory.canVectorize(inspector));
}
private static boolean canVectorizeDimensions(
final ColumnInspector inspector,
final List<DimensionSpec> dimensions
)
{
return dimensions
.stream()
.allMatch(
dimension -> {
if (!dimension.canVectorize()) {
return false;
}
if (dimension.mustDecorate()) {
// group by on multi value dimensions are not currently supported
// DimensionSpecs that decorate may turn singly-valued columns into multi-valued selectors.
// To be safe, we must return false here.
return false;
}
if (dimension.getOutputType().isArray()) {
// group by on arrays is not currently supported in the vector processing engine
return false;
}
// Now check column capabilities.
final ColumnCapabilities columnCapabilities = inspector.getColumnCapabilities(dimension.getDimension());
// null here currently means the column does not exist, nil columns can be vectorized
if (columnCapabilities == null) {
return true;
}
// must be single valued
return columnCapabilities.hasMultipleValues().isFalse();
});
}
@VisibleForTesting
static class VectorGroupByEngineIterator implements CloseableIterator<ResultRow>
{
@ -347,7 +356,7 @@ public class VectorGroupByEngine
{
final VectorGrouper grouper;
final int cardinalityForArrayAggregation = GroupByQueryEngineV2.getCardinalityForArrayAggregation(
final int cardinalityForArrayAggregation = GroupingEngine.getCardinalityForArrayAggregation(
querySpecificConfig,
query,
storageAdapter,
@ -474,7 +483,7 @@ public class VectorGroupByEngine
}
// Convert dimension values to desired output types, possibly.
GroupByQueryEngineV2.convertRowTypesToOutputTypes(
GroupingEngine.convertRowTypesToOutputTypes(
query.getDimensions(),
resultRow,
resultRowDimensionStart

View File

@ -58,9 +58,9 @@ import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
import org.apache.druid.query.groupby.GroupingEngine;
import org.apache.druid.query.groupby.ResultRow;
import org.apache.druid.query.groupby.TestGroupByBuffers;
import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2;
import org.apache.druid.query.scan.ScanQueryConfig;
import org.apache.druid.query.scan.ScanQueryEngine;
import org.apache.druid.query.scan.ScanQueryQueryToolChest;
@ -774,7 +774,7 @@ public class AggregationTestHelper implements Closeable
resultRows.stream()
.peek(row -> {
GroupByQuery query = (GroupByQuery) queryPlus.getQuery();
GroupByQueryEngineV2.convertRowTypesToOutputTypes(
GroupingEngine.convertRowTypesToOutputTypes(
query.getDimensions(),
row,
query.getResultRowDimensionStart()

View File

@ -32,7 +32,6 @@ import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.aggregation.AggregationTestHelper;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
@ -485,7 +484,7 @@ public class NestedGroupByArrayQueryTest
List<ResultRow> serdeAndBack =
results.stream()
.peek(
row -> GroupByQueryEngineV2.convertRowTypesToOutputTypes(
row -> GroupingEngine.convertRowTypesToOutputTypes(
query.getDimensions(),
row,
query.getResultRowDimensionStart()

View File

@ -29,7 +29,7 @@ import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
public class GroupByQueryEngineV2Test
public class GroupByQueryEngineTest
{
private static final String DIM = "d0";
ColumnSelectorFactory factory;
@ -51,7 +51,7 @@ public class GroupByQueryEngineV2Test
.setDictionaryValuesUnique(true);
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
EasyMock.replay(factory);
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
Assert.assertTrue(GroupByQueryEngine.canPushDownLimit(factory, DIM));
EasyMock.verify(factory);
}
@ -66,7 +66,7 @@ public class GroupByQueryEngineV2Test
.setDictionaryValuesUnique(true);
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
EasyMock.replay(factory);
Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
Assert.assertFalse(GroupByQueryEngine.canPushDownLimit(factory, DIM));
EasyMock.verify(factory);
}
@ -81,7 +81,7 @@ public class GroupByQueryEngineV2Test
.setDictionaryValuesUnique(false);
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
EasyMock.replay(factory);
Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
Assert.assertFalse(GroupByQueryEngine.canPushDownLimit(factory, DIM));
EasyMock.verify(factory);
}
@ -96,7 +96,7 @@ public class GroupByQueryEngineV2Test
.setDictionaryValuesUnique(false);
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
EasyMock.replay(factory);
Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
Assert.assertFalse(GroupByQueryEngine.canPushDownLimit(factory, DIM));
EasyMock.verify(factory);
}
@ -111,11 +111,11 @@ public class GroupByQueryEngineV2Test
.setDictionaryValuesUnique(false);
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).anyTimes();
EasyMock.replay(factory);
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
Assert.assertTrue(GroupByQueryEngine.canPushDownLimit(factory, DIM));
capabilities.setType(ColumnType.DOUBLE);
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
Assert.assertTrue(GroupByQueryEngine.canPushDownLimit(factory, DIM));
capabilities.setType(ColumnType.FLOAT);
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
Assert.assertTrue(GroupByQueryEngine.canPushDownLimit(factory, DIM));
EasyMock.verify(factory);
}
@ -130,7 +130,7 @@ public class GroupByQueryEngineV2Test
.setDictionaryValuesUnique(false);
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
EasyMock.replay(factory);
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
Assert.assertTrue(GroupByQueryEngine.canPushDownLimit(factory, DIM));
EasyMock.verify(factory);
}
}

View File

@ -434,7 +434,7 @@ public class LimitedBufferHashGrouperTest extends InitializedNullHandlingTest
}
/**
* key serde for more realistic ordering tests, similar to the {@link GroupByQueryEngineV2.GroupByEngineKeySerde} or
* key serde for more realistic ordering tests, similar to the {@link GroupByQueryEngine.GroupByEngineKeySerde} or
* {@link RowBasedGrouperHelper.RowBasedKeySerde} which are likely to be used in practice by the group-by engine,
* which also both use {@link GrouperBufferComparatorUtils} to make comparators
*/

View File

@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import org.apache.druid.collections.CloseableStupidPool;
import org.apache.druid.collections.ResourceHolder;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.guice.NestedDataModule;
@ -50,7 +51,7 @@ import org.apache.druid.query.filter.ValueMatcher;
import org.apache.druid.query.groupby.GroupByQuery;
import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.ResultRow;
import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngineV2;
import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngine;
import org.apache.druid.query.topn.TopNQueryBuilder;
import org.apache.druid.query.topn.TopNQueryEngine;
import org.apache.druid.query.topn.TopNResultValue;
@ -134,11 +135,10 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT
CloseableStupidPool<ByteBuffer> pool = new CloseableStupidPool<>(
"GroupByQueryEngine-bufferPool",
() -> ByteBuffer.allocate(50000)
)
);
ResourceHolder<ByteBuffer> processingBuffer = pool.take()
) {
final Sequence<ResultRow> rows = GroupByQueryEngineV2.process(
GroupByQuery.builder()
final GroupByQuery query = GroupByQuery.builder()
.setDataSource("test")
.setGranularity(Granularities.ALL)
.setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
@ -146,11 +146,18 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT
.addDimension("sally")
.addAggregator(new LongSumAggregatorFactory("cnt", "cnt"))
.addOrderByColumn("billy")
.build(),
.build();
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()));
final Interval interval = Iterables.getOnlyElement(query.getIntervals());
final Sequence<ResultRow> rows = GroupByQueryEngine.process(
query,
new IncrementalIndexStorageAdapter(index),
pool,
processingBuffer.get(),
null,
new GroupByQueryConfig(),
new DruidProcessingConfig(),
filter,
interval,
null
);
@ -192,11 +199,10 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT
CloseableStupidPool<ByteBuffer> pool = new CloseableStupidPool<>(
"GroupByQueryEngine-bufferPool",
() -> ByteBuffer.allocate(50000)
)
);
ResourceHolder<ByteBuffer> processingBuffer = pool.take();
) {
final Sequence<ResultRow> rows = GroupByQueryEngineV2.process(
GroupByQuery.builder()
final GroupByQuery query = GroupByQuery.builder()
.setDataSource("test")
.setGranularity(Granularities.ALL)
.setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
@ -216,11 +222,18 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT
)
)
.addOrderByColumn("billy")
.build(),
.build();
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()));
final Interval interval = Iterables.getOnlyElement(query.getIntervals());
final Sequence<ResultRow> rows = GroupByQueryEngine.process(
query,
new IncrementalIndexStorageAdapter(index),
pool,
processingBuffer.get(),
null,
new GroupByQueryConfig(),
new DruidProcessingConfig(),
filter,
interval,
null
);
@ -365,11 +378,11 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT
CloseableStupidPool<ByteBuffer> pool = new CloseableStupidPool<>(
"GroupByQueryEngine-bufferPool",
() -> ByteBuffer.allocate(50000)
)
);
ResourceHolder<ByteBuffer> processingBuffer = pool.take();
) {
final Sequence<ResultRow> rows = GroupByQueryEngineV2.process(
GroupByQuery.builder()
final GroupByQuery query = GroupByQuery.builder()
.setDataSource("test")
.setGranularity(Granularities.ALL)
.setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
@ -377,11 +390,19 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT
.addDimension("sally")
.addAggregator(new LongSumAggregatorFactory("cnt", "cnt"))
.setDimFilter(DimFilters.dimEquals("sally", (String) null))
.build(),
.build();
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter()));
final Interval interval = Iterables.getOnlyElement(query.getIntervals());
final Sequence<ResultRow> rows = GroupByQueryEngine.process(
query,
new IncrementalIndexStorageAdapter(index),
pool,
processingBuffer.get(),
null,
new GroupByQueryConfig(),
new DruidProcessingConfig(),
filter,
interval,
null
);