mirror of https://github.com/apache/druid.git
tidy up group by engines after removal of v1 (#15665)
This commit is contained in:
parent
87fbe42218
commit
2118258b54
|
@ -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(
|
||||
query,
|
||||
storageAdapter,
|
||||
bufferPool,
|
||||
configSupplier.get().withOverrides(query),
|
||||
processingConfig,
|
||||
groupByQueryMetrics
|
||||
);
|
||||
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,
|
||||
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.
|
||||
*/
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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,
|
|
@ -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;
|
||||
}
|
||||
|
||||
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;
|
||||
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();
|
||||
}
|
||||
// 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;
|
|
@ -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;
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
{
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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,23 +135,29 @@ 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()
|
||||
.setDataSource("test")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
|
||||
.addDimension("billy")
|
||||
.addDimension("sally")
|
||||
.addAggregator(new LongSumAggregatorFactory("cnt", "cnt"))
|
||||
.addOrderByColumn("billy")
|
||||
.build(),
|
||||
final GroupByQuery query = GroupByQuery.builder()
|
||||
.setDataSource("test")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
|
||||
.addDimension("billy")
|
||||
.addDimension("sally")
|
||||
.addAggregator(new LongSumAggregatorFactory("cnt", "cnt"))
|
||||
.addOrderByColumn("billy")
|
||||
.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,35 +199,41 @@ 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()
|
||||
.setDataSource("test")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
|
||||
.addDimension("billy")
|
||||
.addDimension("sally")
|
||||
.addAggregator(
|
||||
new LongSumAggregatorFactory("cnt", "cnt")
|
||||
)
|
||||
.addAggregator(
|
||||
new JavaScriptAggregatorFactory(
|
||||
"fieldLength",
|
||||
Arrays.asList("sally", "billy"),
|
||||
"function(current, s, b) { return current + (s == null ? 0 : s.length) + (b == null ? 0 : b.length); }",
|
||||
"function() { return 0; }",
|
||||
"function(a,b) { return a + b; }",
|
||||
JavaScriptConfig.getEnabledInstance()
|
||||
)
|
||||
)
|
||||
.addOrderByColumn("billy")
|
||||
.build(),
|
||||
final GroupByQuery query = GroupByQuery.builder()
|
||||
.setDataSource("test")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
|
||||
.addDimension("billy")
|
||||
.addDimension("sally")
|
||||
.addAggregator(
|
||||
new LongSumAggregatorFactory("cnt", "cnt")
|
||||
)
|
||||
.addAggregator(
|
||||
new JavaScriptAggregatorFactory(
|
||||
"fieldLength",
|
||||
Arrays.asList("sally", "billy"),
|
||||
"function(current, s, b) { return current + (s == null ? 0 : s.length) + (b == null ? 0 : b.length); }",
|
||||
"function() { return 0; }",
|
||||
"function(a,b) { return a + b; }",
|
||||
JavaScriptConfig.getEnabledInstance()
|
||||
)
|
||||
)
|
||||
.addOrderByColumn("billy")
|
||||
.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,23 +378,31 @@ 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()
|
||||
.setDataSource("test")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
|
||||
.addDimension("billy")
|
||||
.addDimension("sally")
|
||||
.addAggregator(new LongSumAggregatorFactory("cnt", "cnt"))
|
||||
.setDimFilter(DimFilters.dimEquals("sally", (String) null))
|
||||
.build(),
|
||||
final GroupByQuery query = GroupByQuery.builder()
|
||||
.setDataSource("test")
|
||||
.setGranularity(Granularities.ALL)
|
||||
.setInterval(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))
|
||||
.addDimension("billy")
|
||||
.addDimension("sally")
|
||||
.addAggregator(new LongSumAggregatorFactory("cnt", "cnt"))
|
||||
.setDimFilter(DimFilters.dimEquals("sally", (String) null))
|
||||
.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
|
||||
);
|
||||
|
||||
|
|
Loading…
Reference in New Issue