mirror of https://github.com/apache/druid.git
Virtual column updates for exploiting base column internal structure (#7618)
* VirtualColumn updates for exploiting base column internal structure * unit tests for virtual column interface updates * groupBy needs to use VirtualizedColumnSelectorFactory if outer query in nested groupBy has virtual columns. * fix strict compile checks * fix teamcity build errors * add comment explaining useVirtualizedColumnSelectorFactory flag in RowBasedGrouperHelper.createGrouperAccumulatorPair(..)
This commit is contained in:
parent
c4a4223c9c
commit
fb0c846941
|
@ -201,7 +201,8 @@ public class GroupByMergingQueryRunnerV2 implements QueryRunner<Row>
|
||||||
priority,
|
priority,
|
||||||
hasTimeout,
|
hasTimeout,
|
||||||
timeoutAt,
|
timeoutAt,
|
||||||
mergeBufferSize
|
mergeBufferSize,
|
||||||
|
false
|
||||||
);
|
);
|
||||||
final Grouper<RowBasedKey> grouper = pair.lhs;
|
final Grouper<RowBasedKey> grouper = pair.lhs;
|
||||||
final Accumulator<AggregateResult, Row> accumulator = pair.rhs;
|
final Accumulator<AggregateResult, Row> accumulator = pair.rhs;
|
||||||
|
|
|
@ -65,7 +65,8 @@ public class GroupByRowProcessor
|
||||||
final String processingTmpDir,
|
final String processingTmpDir,
|
||||||
final int mergeBufferSize,
|
final int mergeBufferSize,
|
||||||
final List<Closeable> closeOnExit,
|
final List<Closeable> closeOnExit,
|
||||||
final boolean wasQueryPushedDown
|
final boolean wasQueryPushedDown,
|
||||||
|
final boolean useVirtualizedColumnSelectorFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final GroupByQuery query = (GroupByQuery) queryParam;
|
final GroupByQuery query = (GroupByQuery) queryParam;
|
||||||
|
@ -112,7 +113,8 @@ public class GroupByRowProcessor
|
||||||
temporaryStorage,
|
temporaryStorage,
|
||||||
spillMapper,
|
spillMapper,
|
||||||
aggregatorFactories,
|
aggregatorFactories,
|
||||||
mergeBufferSize
|
mergeBufferSize,
|
||||||
|
useVirtualizedColumnSelectorFactory
|
||||||
);
|
);
|
||||||
final Grouper<RowBasedKey> grouper = pair.lhs;
|
final Grouper<RowBasedKey> grouper = pair.lhs;
|
||||||
final Accumulator<AggregateResult, Row> accumulator = pair.rhs;
|
final Accumulator<AggregateResult, Row> accumulator = pair.rhs;
|
||||||
|
|
|
@ -105,7 +105,8 @@ public class RowBasedGrouperHelper
|
||||||
final LimitedTemporaryStorage temporaryStorage,
|
final LimitedTemporaryStorage temporaryStorage,
|
||||||
final ObjectMapper spillMapper,
|
final ObjectMapper spillMapper,
|
||||||
final AggregatorFactory[] aggregatorFactories,
|
final AggregatorFactory[] aggregatorFactories,
|
||||||
final int mergeBufferSize
|
final int mergeBufferSize,
|
||||||
|
final boolean useVirtualizedColumnSelectorFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return createGrouperAccumulatorPair(
|
return createGrouperAccumulatorPair(
|
||||||
|
@ -123,7 +124,8 @@ public class RowBasedGrouperHelper
|
||||||
UNKNOWN_THREAD_PRIORITY,
|
UNKNOWN_THREAD_PRIORITY,
|
||||||
false,
|
false,
|
||||||
UNKNOWN_TIMEOUT,
|
UNKNOWN_TIMEOUT,
|
||||||
mergeBufferSize
|
mergeBufferSize,
|
||||||
|
useVirtualizedColumnSelectorFactory
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -147,7 +149,8 @@ public class RowBasedGrouperHelper
|
||||||
final int priority,
|
final int priority,
|
||||||
final boolean hasQueryTimeout,
|
final boolean hasQueryTimeout,
|
||||||
final long queryTimeoutAt,
|
final long queryTimeoutAt,
|
||||||
final int mergeBufferSize
|
final int mergeBufferSize,
|
||||||
|
final boolean useVirtualizedColumnSelectorFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
// concurrencyHint >= 1 for concurrent groupers, -1 for single-threaded
|
// concurrencyHint >= 1 for concurrent groupers, -1 for single-threaded
|
||||||
|
@ -159,13 +162,23 @@ public class RowBasedGrouperHelper
|
||||||
final boolean includeTimestamp = GroupByStrategyV2.getUniversalTimestamp(query) == null;
|
final boolean includeTimestamp = GroupByStrategyV2.getUniversalTimestamp(query) == null;
|
||||||
|
|
||||||
final ThreadLocal<Row> columnSelectorRow = new ThreadLocal<>();
|
final ThreadLocal<Row> columnSelectorRow = new ThreadLocal<>();
|
||||||
final ColumnSelectorFactory columnSelectorFactory = query.getVirtualColumns().wrap(
|
|
||||||
RowBasedColumnSelectorFactory.create(
|
ColumnSelectorFactory columnSelectorFactory = RowBasedColumnSelectorFactory.create(
|
||||||
columnSelectorRow,
|
columnSelectorRow,
|
||||||
rawInputRowSignature
|
rawInputRowSignature
|
||||||
)
|
|
||||||
);
|
);
|
||||||
|
|
||||||
|
// Although queries would work fine if we always wrap the columnSelectorFactory into a
|
||||||
|
// VirtualizedColumnSelectorFactory. However, VirtualizedColumnSelectorFactory is incapable of using
|
||||||
|
// ColumnSelector based variants of makeXXX methods which are more efficient.
|
||||||
|
// this flag is set to true when it is essential to wrap e.g. a nested groupBy query with virtual columns in
|
||||||
|
// the outer query. Without this flag, groupBy query processing would never use more efficient ColumnSelector
|
||||||
|
// based methods in VirtualColumn interface.
|
||||||
|
// For more details, See https://github.com/apache/incubator-druid/issues/7574
|
||||||
|
if (useVirtualizedColumnSelectorFactory) {
|
||||||
|
columnSelectorFactory = query.getVirtualColumns().wrap(columnSelectorFactory);
|
||||||
|
}
|
||||||
|
|
||||||
final boolean willApplyLimitPushDown = query.isApplyLimitPushDown();
|
final boolean willApplyLimitPushDown = query.isApplyLimitPushDown();
|
||||||
final DefaultLimitSpec limitSpec = willApplyLimitPushDown ? (DefaultLimitSpec) query.getLimitSpec() : null;
|
final DefaultLimitSpec limitSpec = willApplyLimitPushDown ? (DefaultLimitSpec) query.getLimitSpec() : null;
|
||||||
boolean sortHasNonGroupingFields = false;
|
boolean sortHasNonGroupingFields = false;
|
||||||
|
|
|
@ -351,7 +351,8 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||||
processingConfig.getTmpDir(),
|
processingConfig.getTmpDir(),
|
||||||
processingConfig.intermediateComputeSizeBytes(),
|
processingConfig.intermediateComputeSizeBytes(),
|
||||||
closeOnExit,
|
closeOnExit,
|
||||||
wasQueryPushedDown
|
wasQueryPushedDown,
|
||||||
|
true
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -414,6 +415,7 @@ public class GroupByStrategyV2 implements GroupByStrategy
|
||||||
processingConfig.getTmpDir(),
|
processingConfig.getTmpDir(),
|
||||||
processingConfig.intermediateComputeSizeBytes(),
|
processingConfig.intermediateComputeSizeBytes(),
|
||||||
closeOnExit,
|
closeOnExit,
|
||||||
|
false,
|
||||||
false
|
false
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
|
@ -61,8 +61,49 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
||||||
public CloseableIndexed<String> getDimensionValues(String dimension)
|
public CloseableIndexed<String> getDimensionValues(String dimension)
|
||||||
{
|
{
|
||||||
if (isVirtualColumn(dimension)) {
|
if (isVirtualColumn(dimension)) {
|
||||||
// Virtual columns don't have dictionaries or indexes.
|
BitmapIndex bitmapIndex = virtualColumns.getBitmapIndex(dimension, index);
|
||||||
return null;
|
if (bitmapIndex == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
return new CloseableIndexed<String>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public int size()
|
||||||
|
{
|
||||||
|
return bitmapIndex.getCardinality();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String get(int index)
|
||||||
|
{
|
||||||
|
return bitmapIndex.getValue(index);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int indexOf(String value)
|
||||||
|
{
|
||||||
|
return bitmapIndex.getIndex(value);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Iterator<String> iterator()
|
||||||
|
{
|
||||||
|
return IndexedIterable.create(this).iterator();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
inspector.visit("column", bitmapIndex);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
final ColumnHolder columnHolder = index.getColumnHolder(dimension);
|
final ColumnHolder columnHolder = index.getColumnHolder(dimension);
|
||||||
|
@ -144,8 +185,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
||||||
public BitmapIndex getBitmapIndex(String dimension)
|
public BitmapIndex getBitmapIndex(String dimension)
|
||||||
{
|
{
|
||||||
if (isVirtualColumn(dimension)) {
|
if (isVirtualColumn(dimension)) {
|
||||||
// Virtual columns don't have dictionaries or indexes.
|
return virtualColumns.getBitmapIndex(dimension, index);
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
final ColumnHolder columnHolder = index.getColumnHolder(dimension);
|
final ColumnHolder columnHolder = index.getColumnHolder(dimension);
|
||||||
|
@ -214,8 +254,11 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
||||||
public ImmutableBitmap getBitmapIndex(String dimension, String value)
|
public ImmutableBitmap getBitmapIndex(String dimension, String value)
|
||||||
{
|
{
|
||||||
if (isVirtualColumn(dimension)) {
|
if (isVirtualColumn(dimension)) {
|
||||||
// Virtual columns don't have dictionaries or indexes.
|
BitmapIndex idx = virtualColumns.getBitmapIndex(dimension, index);
|
||||||
return null;
|
if (idx == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return idx.getBitmap(idx.getIndex(value));
|
||||||
}
|
}
|
||||||
|
|
||||||
final ColumnHolder columnHolder = index.getColumnHolder(dimension);
|
final ColumnHolder columnHolder = index.getColumnHolder(dimension);
|
||||||
|
|
|
@ -80,7 +80,12 @@ class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||||
dimensionSpec,
|
dimensionSpec,
|
||||||
spec -> {
|
spec -> {
|
||||||
if (virtualColumns.exists(spec.getDimension())) {
|
if (virtualColumns.exists(spec.getDimension())) {
|
||||||
return virtualColumns.makeDimensionSelector(spec, this);
|
DimensionSelector dimensionSelector = virtualColumns.makeDimensionSelector(dimensionSpec, index, offset);
|
||||||
|
if (dimensionSelector == null) {
|
||||||
|
return virtualColumns.makeDimensionSelector(dimensionSpec, this);
|
||||||
|
} else {
|
||||||
|
return dimensionSelector;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return spec.decorate(makeDimensionSelectorUndecorated(spec));
|
return spec.decorate(makeDimensionSelectorUndecorated(spec));
|
||||||
|
@ -123,7 +128,12 @@ class QueryableIndexColumnSelectorFactory implements ColumnSelectorFactory
|
||||||
columnName,
|
columnName,
|
||||||
name -> {
|
name -> {
|
||||||
if (virtualColumns.exists(columnName)) {
|
if (virtualColumns.exists(columnName)) {
|
||||||
return virtualColumns.makeColumnValueSelector(columnName, this);
|
ColumnValueSelector<?> selector = virtualColumns.makeColumnValueSelector(columnName, index, offset);
|
||||||
|
if (selector == null) {
|
||||||
|
return virtualColumns.makeColumnValueSelector(columnName, this);
|
||||||
|
} else {
|
||||||
|
return selector;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
BaseColumn column = getCachedColumn(columnName, BaseColumn.class);
|
BaseColumn column = getCachedColumn(columnName, BaseColumn.class);
|
||||||
|
|
|
@ -23,13 +23,16 @@ import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
import org.apache.druid.java.util.common.Cacheable;
|
import org.apache.druid.java.util.common.Cacheable;
|
||||||
import org.apache.druid.query.dimension.DimensionSpec;
|
import org.apache.druid.query.dimension.DimensionSpec;
|
||||||
|
import org.apache.druid.segment.column.BitmapIndex;
|
||||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
|
import org.apache.druid.segment.data.ReadableOffset;
|
||||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Virtual columns are "views" created over a ColumnSelectorFactory. They can potentially draw from multiple
|
* Virtual columns are "views" created over a ColumnSelectorFactory or ColumnSelector. They can potentially draw from multiple
|
||||||
* underlying columns, although they always present themselves as if they were a single column.
|
* underlying columns, although they always present themselves as if they were a single column.
|
||||||
*
|
*
|
||||||
* A virtual column object will be shared amongst threads and must be thread safe. The selectors returned
|
* A virtual column object will be shared amongst threads and must be thread safe. The selectors returned
|
||||||
|
@ -61,6 +64,25 @@ public interface VirtualColumn extends Cacheable
|
||||||
*/
|
*/
|
||||||
DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelectorFactory factory);
|
DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelectorFactory factory);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns similar DimensionSelector object as returned by {@link #makeDimensionSelector(DimensionSpec, ColumnSelectorFactory)}
|
||||||
|
* except this method has full access to underlying column and can potentially provide a more efficient implementation.
|
||||||
|
*
|
||||||
|
* Users of this interface must ensure to first call this method whenever possible. Typically this can not be called in
|
||||||
|
* query paths on top of IncrementalIndex which doesn't have columns as in persisted segments.
|
||||||
|
*
|
||||||
|
* @param dimensionSpec
|
||||||
|
* @param columnSelector
|
||||||
|
* @param offset
|
||||||
|
* @return the selector
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
@Nullable
|
||||||
|
default DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelector columnSelector, ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Build a selector corresponding to this virtual column. Also provides the name that the
|
* Build a selector corresponding to this virtual column. Also provides the name that the
|
||||||
* virtual column was referenced with, which is useful if this column uses dot notation.
|
* virtual column was referenced with, which is useful if this column uses dot notation.
|
||||||
|
@ -72,6 +94,24 @@ public interface VirtualColumn extends Cacheable
|
||||||
*/
|
*/
|
||||||
ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelectorFactory factory);
|
ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelectorFactory factory);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns similar ColumnValueSelector object as returned by {@link #makeColumnValueSelector(String, ColumnSelectorFactory)}
|
||||||
|
* except this method has full access to underlying column and can potentially provide a more efficient implementation.
|
||||||
|
*
|
||||||
|
* Users of this interface must ensure to first call this method whenever possible. Typically this can not be called in
|
||||||
|
* query paths on top of IncrementalIndex which doesn't have columns as in persisted segments.
|
||||||
|
*
|
||||||
|
* @param columnName
|
||||||
|
* @param columnSelector
|
||||||
|
* @param offset
|
||||||
|
* @return the selector
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
@Nullable
|
||||||
|
default ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelector columnSelector, ReadableOffset offset)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the capabilities of this virtual column, which includes a type that corresponds to the best
|
* Returns the capabilities of this virtual column, which includes a type that corresponds to the best
|
||||||
|
@ -107,4 +147,17 @@ public interface VirtualColumn extends Cacheable
|
||||||
* @return whether to use dot notation
|
* @return whether to use dot notation
|
||||||
*/
|
*/
|
||||||
boolean usesDotNotation();
|
boolean usesDotNotation();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the BitmapIndex for efficient filtering on columns that support it. This method is only used if
|
||||||
|
* {@link ColumnCapabilities} returned from {@link #capabilities(String)} has flag for BitmapIndex support.
|
||||||
|
* @param columnName
|
||||||
|
* @param selector
|
||||||
|
* @return BitmapIndex
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
default BitmapIndex getBitmapIndex(String columnName, ColumnSelector selector)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("not supported");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -31,8 +31,10 @@ import org.apache.druid.java.util.common.IAE;
|
||||||
import org.apache.druid.java.util.common.Pair;
|
import org.apache.druid.java.util.common.Pair;
|
||||||
import org.apache.druid.query.cache.CacheKeyBuilder;
|
import org.apache.druid.query.cache.CacheKeyBuilder;
|
||||||
import org.apache.druid.query.dimension.DimensionSpec;
|
import org.apache.druid.query.dimension.DimensionSpec;
|
||||||
|
import org.apache.druid.segment.column.BitmapIndex;
|
||||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
import org.apache.druid.segment.column.ColumnHolder;
|
import org.apache.druid.segment.column.ColumnHolder;
|
||||||
|
import org.apache.druid.segment.data.ReadableOffset;
|
||||||
import org.apache.druid.segment.virtual.VirtualizedColumnSelectorFactory;
|
import org.apache.druid.segment.virtual.VirtualizedColumnSelectorFactory;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -171,6 +173,36 @@ public class VirtualColumns implements Cacheable
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public BitmapIndex getBitmapIndex(String columnName, ColumnSelector columnSelector)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumn(columnName);
|
||||||
|
if (virtualColumn == null) {
|
||||||
|
throw new IAE("No such virtual column[%s]", columnName);
|
||||||
|
} else {
|
||||||
|
return virtualColumn.capabilities(columnName).hasBitmapIndexes() ? virtualColumn.getBitmapIndex(columnName, columnSelector) : null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelector columnSelector, ReadableOffset offset)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumn(dimensionSpec.getDimension());
|
||||||
|
if (virtualColumn == null) {
|
||||||
|
throw new IAE("No such virtual column[%s]", dimensionSpec.getDimension());
|
||||||
|
} else {
|
||||||
|
return virtualColumn.makeDimensionSelector(dimensionSpec, columnSelector, offset);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelector columnSelector, ReadableOffset offset)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumn(columnName);
|
||||||
|
if (virtualColumn == null) {
|
||||||
|
throw new IAE("No such virtual column[%s]", columnName);
|
||||||
|
} else {
|
||||||
|
return virtualColumn.makeColumnValueSelector(columnName, columnSelector, offset);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a column value selector.
|
* Create a column value selector.
|
||||||
*
|
*
|
||||||
|
|
|
@ -6316,9 +6316,11 @@ public class GroupByQueryRunnerTest
|
||||||
.builder()
|
.builder()
|
||||||
.setDataSource(QueryRunnerTestHelper.dataSource)
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
.setQuerySegmentSpec(QueryRunnerTestHelper.firstToThird)
|
||||||
|
.setVirtualColumns(new ExpressionVirtualColumn("alias", "quality", ValueType.STRING, TestExprMacroTable.INSTANCE))
|
||||||
.setDimensions(Lists.newArrayList(
|
.setDimensions(Lists.newArrayList(
|
||||||
new DefaultDimensionSpec("quality", "alias"),
|
new DefaultDimensionSpec("quality", "quality"),
|
||||||
new DefaultDimensionSpec("market", "market")
|
new DefaultDimensionSpec("market", "market"),
|
||||||
|
new DefaultDimensionSpec("alias", "alias")
|
||||||
))
|
))
|
||||||
.setAggregatorSpecs(
|
.setAggregatorSpecs(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
|
|
|
@ -0,0 +1,289 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.druid.segment.virtual;
|
||||||
|
|
||||||
|
import com.google.common.base.Predicate;
|
||||||
|
import org.apache.druid.java.util.common.IAE;
|
||||||
|
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||||
|
import org.apache.druid.query.dimension.DimensionSpec;
|
||||||
|
import org.apache.druid.query.filter.ValueMatcher;
|
||||||
|
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
|
import org.apache.druid.segment.ColumnSelector;
|
||||||
|
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||||
|
import org.apache.druid.segment.ColumnValueSelector;
|
||||||
|
import org.apache.druid.segment.DimensionSelector;
|
||||||
|
import org.apache.druid.segment.IdLookup;
|
||||||
|
import org.apache.druid.segment.NilColumnValueSelector;
|
||||||
|
import org.apache.druid.segment.VirtualColumn;
|
||||||
|
import org.apache.druid.segment.column.BaseColumn;
|
||||||
|
import org.apache.druid.segment.column.BitmapIndex;
|
||||||
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
|
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||||
|
import org.apache.druid.segment.column.ColumnHolder;
|
||||||
|
import org.apache.druid.segment.column.StringDictionaryEncodedColumn;
|
||||||
|
import org.apache.druid.segment.column.ValueType;
|
||||||
|
import org.apache.druid.segment.data.IndexedInts;
|
||||||
|
import org.apache.druid.segment.data.ReadableOffset;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A String column like VirtualColumn to test drive VirtualColumn interface.
|
||||||
|
*/
|
||||||
|
public class DummyStringVirtualColumn implements VirtualColumn
|
||||||
|
{
|
||||||
|
private final String baseColumnName;
|
||||||
|
private final String outputName;
|
||||||
|
|
||||||
|
private final boolean enableRowBasedMethods;
|
||||||
|
private final boolean enableColumnBasedMethods;
|
||||||
|
private final boolean enableBitmaps;
|
||||||
|
private final boolean disableValueMatchers;
|
||||||
|
|
||||||
|
public DummyStringVirtualColumn(
|
||||||
|
String baseColumnName,
|
||||||
|
String outputName,
|
||||||
|
boolean enableRowBasedMethods,
|
||||||
|
boolean enableColumnBasedMethods,
|
||||||
|
boolean enableBitmaps,
|
||||||
|
boolean disableValueMatchers
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.baseColumnName = baseColumnName;
|
||||||
|
this.outputName = outputName;
|
||||||
|
this.enableRowBasedMethods = enableRowBasedMethods;
|
||||||
|
this.enableColumnBasedMethods = enableColumnBasedMethods;
|
||||||
|
this.enableBitmaps = enableBitmaps;
|
||||||
|
this.disableValueMatchers = disableValueMatchers;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getOutputName()
|
||||||
|
{
|
||||||
|
return this.outputName;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DimensionSelector makeDimensionSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableOffset offset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (enableColumnBasedMethods) {
|
||||||
|
ColumnHolder holder = columnSelector.getColumnHolder(baseColumnName);
|
||||||
|
if (holder == null) {
|
||||||
|
return DimensionSelector.constant(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
StringDictionaryEncodedColumn stringCol = toStringDictionaryEncodedColumn(holder.getColumn());
|
||||||
|
|
||||||
|
DimensionSelector baseDimensionSelector = stringCol.makeDimensionSelector(
|
||||||
|
offset,
|
||||||
|
dimensionSpec.getExtractionFn()
|
||||||
|
);
|
||||||
|
if (disableValueMatchers) {
|
||||||
|
baseDimensionSelector = disableValueMatchers(baseDimensionSelector);
|
||||||
|
}
|
||||||
|
return dimensionSpec.decorate(baseDimensionSelector);
|
||||||
|
} else {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DimensionSelector makeDimensionSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
ColumnSelectorFactory factory
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (enableRowBasedMethods) {
|
||||||
|
DimensionSelector baseDimensionSelector = factory.makeDimensionSelector(new DefaultDimensionSpec(
|
||||||
|
baseColumnName,
|
||||||
|
baseColumnName,
|
||||||
|
null
|
||||||
|
));
|
||||||
|
|
||||||
|
if (disableValueMatchers) {
|
||||||
|
baseDimensionSelector = disableValueMatchers(baseDimensionSelector);
|
||||||
|
}
|
||||||
|
return dimensionSpec.decorate(baseDimensionSelector);
|
||||||
|
} else {
|
||||||
|
throw new UnsupportedOperationException("not supported");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ColumnValueSelector<?> makeColumnValueSelector(
|
||||||
|
String columnName,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableOffset offset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (enableColumnBasedMethods) {
|
||||||
|
ColumnHolder holder = columnSelector.getColumnHolder(baseColumnName);
|
||||||
|
if (holder == null) {
|
||||||
|
return NilColumnValueSelector.instance();
|
||||||
|
}
|
||||||
|
|
||||||
|
StringDictionaryEncodedColumn stringCol = toStringDictionaryEncodedColumn(holder.getColumn());
|
||||||
|
return stringCol.makeColumnValueSelector(offset);
|
||||||
|
} else {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ColumnValueSelector<?> makeColumnValueSelector(
|
||||||
|
String columnName,
|
||||||
|
ColumnSelectorFactory factory
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (enableRowBasedMethods) {
|
||||||
|
return factory.makeColumnValueSelector(baseColumnName);
|
||||||
|
} else {
|
||||||
|
throw new UnsupportedOperationException("not supported");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BitmapIndex getBitmapIndex(String columnName, ColumnSelector columnSelector)
|
||||||
|
{
|
||||||
|
if (enableBitmaps) {
|
||||||
|
ColumnHolder holder = columnSelector.getColumnHolder(baseColumnName);
|
||||||
|
if (holder == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
return holder.getBitmapIndex();
|
||||||
|
} else {
|
||||||
|
throw new UnsupportedOperationException("not supported");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ColumnCapabilities capabilities(String columnName)
|
||||||
|
{
|
||||||
|
ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl().setType(ValueType.STRING)
|
||||||
|
.setDictionaryEncoded(true);
|
||||||
|
if (enableBitmaps) {
|
||||||
|
capabilities.setHasBitmapIndexes(true);
|
||||||
|
}
|
||||||
|
return capabilities;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> requiredColumns()
|
||||||
|
{
|
||||||
|
return Collections.singletonList(baseColumnName);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean usesDotNotation()
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new byte[0];
|
||||||
|
}
|
||||||
|
|
||||||
|
private StringDictionaryEncodedColumn toStringDictionaryEncodedColumn(BaseColumn column)
|
||||||
|
{
|
||||||
|
if (!(column instanceof StringDictionaryEncodedColumn)) {
|
||||||
|
throw new IAE("I can only work with StringDictionaryEncodedColumn");
|
||||||
|
}
|
||||||
|
|
||||||
|
return (StringDictionaryEncodedColumn) column;
|
||||||
|
}
|
||||||
|
|
||||||
|
private DimensionSelector disableValueMatchers(DimensionSelector base)
|
||||||
|
{
|
||||||
|
return new DimensionSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public IndexedInts getRow()
|
||||||
|
{
|
||||||
|
return base.getRow();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(@Nullable String value)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ValueMatcher makeValueMatcher(Predicate<String> predicate)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getValueCardinality()
|
||||||
|
{
|
||||||
|
return base.getValueCardinality();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public String lookupName(int id)
|
||||||
|
{
|
||||||
|
return base.lookupName(id);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nameLookupPossibleInAdvance()
|
||||||
|
{
|
||||||
|
return base.nameLookupPossibleInAdvance();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return base.idLookup();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||||
|
{
|
||||||
|
base.inspectRuntimeShape(inspector);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public Object getObject()
|
||||||
|
{
|
||||||
|
return base.getObject();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Class<?> classOfObject()
|
||||||
|
{
|
||||||
|
return base.classOfObject();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,395 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
|
* or more contributor license agreements. See the NOTICE file
|
||||||
|
* distributed with this work for additional information
|
||||||
|
* regarding copyright ownership. The ASF licenses this file
|
||||||
|
* to you under the Apache License, Version 2.0 (the
|
||||||
|
* "License"); you may not use this file except in compliance
|
||||||
|
* with the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
|
||||||
|
package org.apache.druid.segment.virtual;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import org.apache.druid.data.input.Row;
|
||||||
|
import org.apache.druid.java.util.common.DateTimes;
|
||||||
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
|
import org.apache.druid.query.QueryRunnerTestHelper;
|
||||||
|
import org.apache.druid.query.Result;
|
||||||
|
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
||||||
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
|
import org.apache.druid.query.filter.RegexDimFilter;
|
||||||
|
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||||
|
import org.apache.druid.query.groupby.GroupByQuery;
|
||||||
|
import org.apache.druid.query.groupby.GroupByQueryConfig;
|
||||||
|
import org.apache.druid.query.groupby.GroupByQueryRunnerTestHelper;
|
||||||
|
import org.apache.druid.query.topn.TopNQuery;
|
||||||
|
import org.apache.druid.query.topn.TopNQueryBuilder;
|
||||||
|
import org.apache.druid.query.topn.TopNResultValue;
|
||||||
|
import org.apache.druid.segment.IncrementalIndexSegment;
|
||||||
|
import org.apache.druid.segment.QueryableIndexSegment;
|
||||||
|
import org.apache.druid.segment.Segment;
|
||||||
|
import org.apache.druid.segment.TestHelper;
|
||||||
|
import org.apache.druid.segment.TestIndex;
|
||||||
|
import org.apache.druid.timeline.SegmentId;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class DummyStringVirtualColumnTest
|
||||||
|
{
|
||||||
|
private static final String VSTRING_DIM = "vstring";
|
||||||
|
private static final String COUNT = "count";
|
||||||
|
|
||||||
|
private final List<Segment> mmappedSegments;
|
||||||
|
private final List<Segment> inMemorySegments;
|
||||||
|
private final List<Segment> mixedSegments;
|
||||||
|
|
||||||
|
private final AggregationTestHelper topNTestHelper;
|
||||||
|
private final AggregationTestHelper groupByTestHelper;
|
||||||
|
|
||||||
|
public DummyStringVirtualColumnTest()
|
||||||
|
{
|
||||||
|
QueryableIndexSegment queryableIndexSegment = new QueryableIndexSegment(
|
||||||
|
TestIndex.getMMappedTestIndex(),
|
||||||
|
SegmentId.dummy(QueryRunnerTestHelper.dataSource)
|
||||||
|
);
|
||||||
|
IncrementalIndexSegment incrementalIndexSegment = new IncrementalIndexSegment(
|
||||||
|
TestIndex.getIncrementalTestIndex(),
|
||||||
|
SegmentId.dummy(QueryRunnerTestHelper.dataSource)
|
||||||
|
);
|
||||||
|
|
||||||
|
mmappedSegments = Lists.newArrayList(queryableIndexSegment, queryableIndexSegment);
|
||||||
|
inMemorySegments = Lists.newArrayList(incrementalIndexSegment, incrementalIndexSegment);
|
||||||
|
mixedSegments = Lists.newArrayList(incrementalIndexSegment, queryableIndexSegment);
|
||||||
|
|
||||||
|
topNTestHelper = AggregationTestHelper.createTopNQueryAggregationTestHelper(
|
||||||
|
Collections.EMPTY_LIST,
|
||||||
|
null
|
||||||
|
);
|
||||||
|
|
||||||
|
groupByTestHelper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||||
|
Collections.EMPTY_LIST,
|
||||||
|
new GroupByQueryConfig(),
|
||||||
|
null
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByWithMMappedSegments()
|
||||||
|
{
|
||||||
|
testGroupBy(mmappedSegments, true, true);
|
||||||
|
testGroupBy(mmappedSegments, true, false);
|
||||||
|
testGroupBy(mmappedSegments, false, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByWithInMemorySegments()
|
||||||
|
{
|
||||||
|
testGroupBy(inMemorySegments, true, true);
|
||||||
|
testGroupBy(inMemorySegments, true, false);
|
||||||
|
|
||||||
|
try {
|
||||||
|
testGroupBy(inMemorySegments, false, true);
|
||||||
|
Assert.fail("must need row based methods");
|
||||||
|
}
|
||||||
|
catch (Exception ex) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByWithMixedSegments()
|
||||||
|
{
|
||||||
|
testGroupBy(mixedSegments, true, true);
|
||||||
|
testGroupBy(mixedSegments, true, false);
|
||||||
|
|
||||||
|
try {
|
||||||
|
testGroupBy(mixedSegments, false, true);
|
||||||
|
Assert.fail("must need row based methods");
|
||||||
|
}
|
||||||
|
catch (Exception ex) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByWithSelectFilterWithMMappedSegments()
|
||||||
|
{
|
||||||
|
testGroupByWithSelectFilter(mmappedSegments, true, false, false, false);
|
||||||
|
testGroupByWithSelectFilter(mmappedSegments, true, false, true, true);
|
||||||
|
testGroupByWithSelectFilter(mmappedSegments, false, true, true, true);
|
||||||
|
testGroupByWithSelectFilter(mmappedSegments, true, true, true, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByWithSelectFilterWithInMemorySegments()
|
||||||
|
{
|
||||||
|
testGroupByWithSelectFilter(inMemorySegments, true, false, false, false);
|
||||||
|
testGroupByWithSelectFilter(inMemorySegments, true, true, true, false);
|
||||||
|
|
||||||
|
try {
|
||||||
|
testGroupByWithSelectFilter(inMemorySegments, true, true, true, true);
|
||||||
|
Assert.fail("value matchers must be required");
|
||||||
|
}
|
||||||
|
catch (Exception ex) {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByWithSelectFilterWithMixedSegments()
|
||||||
|
{
|
||||||
|
testGroupByWithSelectFilter(mixedSegments, true, false, false, false);
|
||||||
|
testGroupByWithSelectFilter(mixedSegments, true, true, true, false);
|
||||||
|
|
||||||
|
try {
|
||||||
|
testGroupByWithSelectFilter(mixedSegments, true, true, true, true);
|
||||||
|
Assert.fail("value matchers must be required");
|
||||||
|
}
|
||||||
|
catch (Exception ex) {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByWithRegexFilterWithMMappedSegments()
|
||||||
|
{
|
||||||
|
testGroupByWithRegexFilter(mmappedSegments, true, false, false, false);
|
||||||
|
testGroupByWithRegexFilter(mmappedSegments, true, false, true, true);
|
||||||
|
testGroupByWithRegexFilter(mmappedSegments, false, true, true, true);
|
||||||
|
testGroupByWithRegexFilter(mmappedSegments, true, true, true, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByWithRegexFilterWithInMemorySegments()
|
||||||
|
{
|
||||||
|
testGroupByWithRegexFilter(inMemorySegments, true, false, false, false);
|
||||||
|
testGroupByWithRegexFilter(inMemorySegments, true, true, true, false);
|
||||||
|
|
||||||
|
try {
|
||||||
|
testGroupByWithRegexFilter(inMemorySegments, true, true, true, true);
|
||||||
|
Assert.fail("value matchers must be required");
|
||||||
|
}
|
||||||
|
catch (Exception ex) {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByWithRegexFilterWithMixedSegments()
|
||||||
|
{
|
||||||
|
testGroupByWithRegexFilter(mixedSegments, true, false, false, false);
|
||||||
|
testGroupByWithRegexFilter(mixedSegments, true, true, true, false);
|
||||||
|
|
||||||
|
try {
|
||||||
|
testGroupByWithRegexFilter(mixedSegments, true, true, true, true);
|
||||||
|
Assert.fail("value matchers must be required");
|
||||||
|
}
|
||||||
|
catch (Exception ex) {
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTopNWithMMappedSegments()
|
||||||
|
{
|
||||||
|
testTopN(mmappedSegments, true, true);
|
||||||
|
testTopN(mmappedSegments, true, false);
|
||||||
|
testTopN(mmappedSegments, false, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTopNWithInMemorySegments()
|
||||||
|
{
|
||||||
|
testTopN(inMemorySegments, true, true);
|
||||||
|
testTopN(inMemorySegments, true, false);
|
||||||
|
|
||||||
|
try {
|
||||||
|
testTopN(inMemorySegments, false, true);
|
||||||
|
Assert.fail("must need row based methods");
|
||||||
|
}
|
||||||
|
catch (Exception ex) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTopNWithMixedSegments()
|
||||||
|
{
|
||||||
|
testTopN(mixedSegments, true, true);
|
||||||
|
testTopN(mixedSegments, true, false);
|
||||||
|
|
||||||
|
try {
|
||||||
|
testTopN(mixedSegments, false, true);
|
||||||
|
Assert.fail("must need row based methods");
|
||||||
|
}
|
||||||
|
catch (Exception ex) {
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testGroupBy(List<Segment> segments, boolean enableRowBasedMethods, boolean enableColumnBasedMethods)
|
||||||
|
{
|
||||||
|
GroupByQuery query = new GroupByQuery.Builder()
|
||||||
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.setGranularity(Granularities.ALL)
|
||||||
|
.setVirtualColumns(
|
||||||
|
new DummyStringVirtualColumn(QueryRunnerTestHelper.marketDimension, VSTRING_DIM,
|
||||||
|
enableRowBasedMethods, enableColumnBasedMethods,
|
||||||
|
false, true
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.addDimension(VSTRING_DIM)
|
||||||
|
.setAggregatorSpecs(new CountAggregatorFactory(COUNT))
|
||||||
|
.setInterval("2000/2030")
|
||||||
|
.addOrderByColumn(VSTRING_DIM)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
List<Row> rows = groupByTestHelper.runQueryOnSegmentsObjs(segments, query).toList();
|
||||||
|
|
||||||
|
List<Row> expectedRows = Arrays.asList(
|
||||||
|
GroupByQueryRunnerTestHelper.createExpectedRow("2000-01-01T00:00:00.000Z", COUNT, 1674L, VSTRING_DIM, "spot"),
|
||||||
|
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||||
|
"2000-01-01T00:00:00.000Z",
|
||||||
|
COUNT,
|
||||||
|
372L,
|
||||||
|
VSTRING_DIM,
|
||||||
|
"total_market"
|
||||||
|
),
|
||||||
|
GroupByQueryRunnerTestHelper.createExpectedRow("2000-01-01T00:00:00.000Z", COUNT, 372L, VSTRING_DIM, "upfront")
|
||||||
|
);
|
||||||
|
|
||||||
|
TestHelper.assertExpectedObjects(expectedRows, rows, "failed");
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testGroupByWithSelectFilter(
|
||||||
|
List<Segment> segments,
|
||||||
|
boolean enableRowBasedMethods,
|
||||||
|
boolean enableColumnBasedMethods,
|
||||||
|
boolean enableBitmaps,
|
||||||
|
boolean disableValueMatchers
|
||||||
|
)
|
||||||
|
{
|
||||||
|
GroupByQuery query = new GroupByQuery.Builder()
|
||||||
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.setGranularity(Granularities.ALL)
|
||||||
|
.setVirtualColumns(
|
||||||
|
new DummyStringVirtualColumn(
|
||||||
|
QueryRunnerTestHelper.marketDimension,
|
||||||
|
VSTRING_DIM,
|
||||||
|
enableRowBasedMethods,
|
||||||
|
enableColumnBasedMethods,
|
||||||
|
enableBitmaps,
|
||||||
|
disableValueMatchers
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.addDimension(VSTRING_DIM)
|
||||||
|
.setAggregatorSpecs(new CountAggregatorFactory(COUNT))
|
||||||
|
.setInterval("2000/2030")
|
||||||
|
.addOrderByColumn(VSTRING_DIM)
|
||||||
|
.setDimFilter(new SelectorDimFilter(VSTRING_DIM, "spot", null))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
List<Row> rows = groupByTestHelper.runQueryOnSegmentsObjs(segments, query).toList();
|
||||||
|
|
||||||
|
List<Row> expectedRows = Collections.singletonList(
|
||||||
|
GroupByQueryRunnerTestHelper.createExpectedRow("2000-01-01T00:00:00.000Z", COUNT, 1674L, VSTRING_DIM, "spot")
|
||||||
|
);
|
||||||
|
|
||||||
|
TestHelper.assertExpectedObjects(expectedRows, rows, "failed");
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testGroupByWithRegexFilter(
|
||||||
|
List<Segment> segments,
|
||||||
|
boolean enableRowBasedMethods,
|
||||||
|
boolean enableColumnBasedMethods,
|
||||||
|
boolean enableBitmaps,
|
||||||
|
boolean disableValueMatchers
|
||||||
|
)
|
||||||
|
{
|
||||||
|
GroupByQuery query = new GroupByQuery.Builder()
|
||||||
|
.setDataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.setGranularity(Granularities.ALL)
|
||||||
|
.setVirtualColumns(
|
||||||
|
new DummyStringVirtualColumn(
|
||||||
|
QueryRunnerTestHelper.marketDimension,
|
||||||
|
VSTRING_DIM,
|
||||||
|
enableRowBasedMethods,
|
||||||
|
enableColumnBasedMethods,
|
||||||
|
enableBitmaps,
|
||||||
|
disableValueMatchers
|
||||||
|
)
|
||||||
|
)
|
||||||
|
.addDimension(VSTRING_DIM)
|
||||||
|
.setAggregatorSpecs(new CountAggregatorFactory(COUNT))
|
||||||
|
.setInterval("2000/2030")
|
||||||
|
.addOrderByColumn(VSTRING_DIM)
|
||||||
|
.setDimFilter(new RegexDimFilter(VSTRING_DIM, "(spot)|(upfront)", null))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
List<Row> rows = groupByTestHelper.runQueryOnSegmentsObjs(segments, query).toList();
|
||||||
|
|
||||||
|
List<Row> expectedRows = Arrays.asList(
|
||||||
|
GroupByQueryRunnerTestHelper.createExpectedRow("2000-01-01T00:00:00.000Z", COUNT, 1674L, VSTRING_DIM, "spot"),
|
||||||
|
GroupByQueryRunnerTestHelper.createExpectedRow("2000-01-01T00:00:00.000Z", COUNT, 372L, VSTRING_DIM, "upfront")
|
||||||
|
);
|
||||||
|
|
||||||
|
TestHelper.assertExpectedObjects(expectedRows, rows, "failed");
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testTopN(
|
||||||
|
List<Segment> segments,
|
||||||
|
boolean enableRowBasedMethods,
|
||||||
|
boolean enableColumnBasedMethods
|
||||||
|
)
|
||||||
|
{
|
||||||
|
TopNQuery query = new TopNQueryBuilder()
|
||||||
|
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||||
|
.granularity(Granularities.ALL)
|
||||||
|
.dimension(VSTRING_DIM)
|
||||||
|
.metric(COUNT)
|
||||||
|
.threshold(1)
|
||||||
|
.aggregators(
|
||||||
|
Collections.singletonList(new CountAggregatorFactory(COUNT))
|
||||||
|
)
|
||||||
|
.virtualColumns(new DummyStringVirtualColumn(
|
||||||
|
QueryRunnerTestHelper.marketDimension,
|
||||||
|
VSTRING_DIM,
|
||||||
|
enableRowBasedMethods,
|
||||||
|
enableColumnBasedMethods,
|
||||||
|
false,
|
||||||
|
true
|
||||||
|
))
|
||||||
|
.intervals("2000/2030")
|
||||||
|
.build();
|
||||||
|
|
||||||
|
List rows = topNTestHelper.runQueryOnSegmentsObjs(segments, query).toList();
|
||||||
|
|
||||||
|
List<Result<TopNResultValue>> expectedRows = Collections.singletonList(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2011-01-12T00:00:00.000Z"),
|
||||||
|
new TopNResultValue(
|
||||||
|
Collections.<Map<String, Object>>singletonList(
|
||||||
|
ImmutableMap.<String, Object>builder()
|
||||||
|
.put(COUNT, 1674L)
|
||||||
|
.put(VSTRING_DIM, "spot")
|
||||||
|
.build()
|
||||||
|
)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
TestHelper.assertExpectedResults(expectedRows, (List<Result<TopNResultValue>>) rows, "failed");
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue