mirror of https://github.com/apache/druid.git
allow vectorized query engines to utilize vectorized virtual columns (#10388)
* allow vectorized query engines to utilize vectorized virtual column implementations * javadoc, refactor, checkstyle * intellij inspection and more javadoc * better * review stuffs * fix incorrect refactor, thanks tests * minor adjustments
This commit is contained in:
parent
184b202411
commit
e012d5c41b
|
@ -55,6 +55,7 @@ import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
|
||||||
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
|
import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
|
||||||
import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
|
import org.apache.druid.query.groupby.strategy.GroupByStrategyV2;
|
||||||
import org.apache.druid.query.ordering.StringComparator;
|
import org.apache.druid.query.ordering.StringComparator;
|
||||||
|
import org.apache.druid.segment.ColumnInspector;
|
||||||
import org.apache.druid.segment.ColumnSelectorFactory;
|
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||||
import org.apache.druid.segment.ColumnValueSelector;
|
import org.apache.druid.segment.ColumnValueSelector;
|
||||||
import org.apache.druid.segment.Cursor;
|
import org.apache.druid.segment.Cursor;
|
||||||
|
@ -74,7 +75,6 @@ import java.nio.ByteBuffer;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.NoSuchElementException;
|
import java.util.NoSuchElementException;
|
||||||
import java.util.function.Function;
|
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -231,7 +231,7 @@ public class GroupByQueryEngineV2
|
||||||
processingBuffer,
|
processingBuffer,
|
||||||
fudgeTimestamp,
|
fudgeTimestamp,
|
||||||
dims,
|
dims,
|
||||||
isAllSingleValueDims(columnSelectorFactory::getColumnCapabilities, query.getDimensions(), false),
|
isAllSingleValueDims(columnSelectorFactory, query.getDimensions()),
|
||||||
cardinalityForArrayAggregation
|
cardinalityForArrayAggregation
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
|
@ -242,7 +242,7 @@ public class GroupByQueryEngineV2
|
||||||
processingBuffer,
|
processingBuffer,
|
||||||
fudgeTimestamp,
|
fudgeTimestamp,
|
||||||
dims,
|
dims,
|
||||||
isAllSingleValueDims(columnSelectorFactory::getColumnCapabilities, query.getDimensions(), false)
|
isAllSingleValueDims(columnSelectorFactory, query.getDimensions())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -319,13 +319,11 @@ public class GroupByQueryEngineV2
|
||||||
/**
|
/**
|
||||||
* Checks whether all "dimensions" are either single-valued, or if allowed, nonexistent. Since non-existent column
|
* Checks whether all "dimensions" are either single-valued, or if allowed, nonexistent. Since non-existent column
|
||||||
* selectors will show up as full of nulls they are effectively single valued, however they can also be null during
|
* selectors will show up as full of nulls they are effectively single valued, however they can also be null during
|
||||||
* broker merge, for example with an 'inline' datasource subquery. 'missingMeansNonExistent' is sort of a hack to let
|
* broker merge, for example with an 'inline' datasource subquery.
|
||||||
* the vectorized engine, which only operates on actual segments, to still work in this case for non-existent columns.
|
|
||||||
*/
|
*/
|
||||||
public static boolean isAllSingleValueDims(
|
public static boolean isAllSingleValueDims(
|
||||||
final Function<String, ColumnCapabilities> capabilitiesFunction,
|
final ColumnInspector inspector,
|
||||||
final List<DimensionSpec> dimensions,
|
final List<DimensionSpec> dimensions
|
||||||
final boolean missingMeansNonExistent
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return dimensions
|
return dimensions
|
||||||
|
@ -338,10 +336,9 @@ public class GroupByQueryEngineV2
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
// Now check column capabilities.
|
// Now check column capabilities, which must be present and explicitly not multi-valued
|
||||||
final ColumnCapabilities columnCapabilities = capabilitiesFunction.apply(dimension.getDimension());
|
final ColumnCapabilities columnCapabilities = inspector.getColumnCapabilities(dimension.getDimension());
|
||||||
return (columnCapabilities != null && columnCapabilities.hasMultipleValues().isFalse()) ||
|
return columnCapabilities != null && columnCapabilities.hasMultipleValues().isFalse();
|
||||||
(missingMeansNonExistent && columnCapabilities == null);
|
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -42,6 +42,8 @@ import org.apache.druid.query.groupby.epinephelinae.VectorGrouper;
|
||||||
import org.apache.druid.query.vector.VectorCursorGranularizer;
|
import org.apache.druid.query.vector.VectorCursorGranularizer;
|
||||||
import org.apache.druid.segment.DimensionHandlerUtils;
|
import org.apache.druid.segment.DimensionHandlerUtils;
|
||||||
import org.apache.druid.segment.StorageAdapter;
|
import org.apache.druid.segment.StorageAdapter;
|
||||||
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
|
import org.apache.druid.segment.column.ValueType;
|
||||||
import org.apache.druid.segment.filter.Filters;
|
import org.apache.druid.segment.filter.Filters;
|
||||||
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||||
import org.apache.druid.segment.vector.VectorCursor;
|
import org.apache.druid.segment.vector.VectorCursor;
|
||||||
|
@ -55,6 +57,7 @@ import java.util.Collections;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.NoSuchElementException;
|
import java.util.NoSuchElementException;
|
||||||
|
import java.util.function.Function;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class VectorGroupByEngine
|
public class VectorGroupByEngine
|
||||||
|
@ -70,24 +73,47 @@ public class VectorGroupByEngine
|
||||||
@Nullable final Filter filter
|
@Nullable final Filter filter
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
// Multi-value dimensions are not yet supported.
|
Function<String, ColumnCapabilities> capabilitiesFunction = name ->
|
||||||
//
|
query.getVirtualColumns().getColumnCapabilitiesWithFallback(adapter, name);
|
||||||
// Two notes here about how we're handling this check:
|
|
||||||
// 1) After multi-value dimensions are supported, we could alter "GroupByQueryEngineV2.isAllSingleValueDims"
|
|
||||||
// to accept a ColumnSelectorFactory, which makes more sense than using a StorageAdapter (see #8013).
|
|
||||||
// 2) Technically using StorageAdapter here is bad since it only looks at real columns, but they might
|
|
||||||
// be shadowed by virtual columns (again, see #8013). But it's fine for now since adapter.canVectorize
|
|
||||||
// always returns false if there are any virtual columns.
|
|
||||||
//
|
|
||||||
// This situation should sort itself out pretty well once this engine supports multi-valued columns. Then we
|
|
||||||
// won't have to worry about having this all-single-value-dims check here.
|
|
||||||
|
|
||||||
return GroupByQueryEngineV2.isAllSingleValueDims(adapter::getColumnCapabilities, query.getDimensions(), true)
|
return canVectorizeDimensions(capabilitiesFunction, query.getDimensions())
|
||||||
&& query.getDimensions().stream().allMatch(DimensionSpec::canVectorize)
|
&& query.getDimensions().stream().allMatch(DimensionSpec::canVectorize)
|
||||||
&& query.getAggregatorSpecs().stream().allMatch(aggregatorFactory -> aggregatorFactory.canVectorize(adapter))
|
&& query.getAggregatorSpecs().stream().allMatch(aggregatorFactory -> aggregatorFactory.canVectorize(adapter))
|
||||||
&& adapter.canVectorize(filter, query.getVirtualColumns(), false);
|
&& adapter.canVectorize(filter, query.getVirtualColumns(), false);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static boolean canVectorizeDimensions(
|
||||||
|
final Function<String, ColumnCapabilities> capabilitiesFunction,
|
||||||
|
final List<DimensionSpec> dimensions
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return dimensions
|
||||||
|
.stream()
|
||||||
|
.allMatch(
|
||||||
|
dimension -> {
|
||||||
|
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;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Now check column capabilities.
|
||||||
|
final ColumnCapabilities columnCapabilities = capabilitiesFunction.apply(dimension.getDimension());
|
||||||
|
// null here currently means the column does not exist, nil columns can be vectorized
|
||||||
|
if (columnCapabilities == null) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
// strings must be single valued, dictionary encoded, and have unique dictionary entries
|
||||||
|
if (ValueType.STRING.equals(columnCapabilities.getType())) {
|
||||||
|
return columnCapabilities.hasMultipleValues().isFalse() &&
|
||||||
|
columnCapabilities.isDictionaryEncoded().isTrue() &&
|
||||||
|
columnCapabilities.areDictionaryValuesUnique().isTrue();
|
||||||
|
}
|
||||||
|
return columnCapabilities.hasMultipleValues().isFalse();
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
public static Sequence<ResultRow> process(
|
public static Sequence<ResultRow> process(
|
||||||
final GroupByQuery query,
|
final GroupByQuery query,
|
||||||
final StorageAdapter storageAdapter,
|
final StorageAdapter storageAdapter,
|
||||||
|
|
|
@ -193,7 +193,6 @@ public class QueryableIndexCursorSequenceBuilder
|
||||||
public VectorCursor buildVectorized(final int vectorSize)
|
public VectorCursor buildVectorized(final int vectorSize)
|
||||||
{
|
{
|
||||||
// Sanity check - matches QueryableIndexStorageAdapter.canVectorize
|
// Sanity check - matches QueryableIndexStorageAdapter.canVectorize
|
||||||
Preconditions.checkState(virtualColumns.size() == 0, "virtualColumns.size == 0");
|
|
||||||
Preconditions.checkState(!descending, "!descending");
|
Preconditions.checkState(!descending, "!descending");
|
||||||
|
|
||||||
final Map<String, BaseColumn> columnCache = new HashMap<>();
|
final Map<String, BaseColumn> columnCache = new HashMap<>();
|
||||||
|
@ -229,17 +228,15 @@ public class QueryableIndexCursorSequenceBuilder
|
||||||
? new NoFilterVectorOffset(vectorSize, startOffset, endOffset)
|
? new NoFilterVectorOffset(vectorSize, startOffset, endOffset)
|
||||||
: new BitmapVectorOffset(vectorSize, filterBitmap, startOffset, endOffset);
|
: new BitmapVectorOffset(vectorSize, filterBitmap, startOffset, endOffset);
|
||||||
|
|
||||||
|
// baseColumnSelectorFactory using baseOffset is the column selector for filtering.
|
||||||
|
final VectorColumnSelectorFactory baseColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset(
|
||||||
|
columnCache,
|
||||||
|
baseOffset,
|
||||||
|
closer
|
||||||
|
);
|
||||||
if (postFilter == null) {
|
if (postFilter == null) {
|
||||||
return new QueryableIndexVectorCursor(index, baseOffset, closer, columnCache, vectorSize);
|
return new QueryableIndexVectorCursor(baseColumnSelectorFactory, baseOffset, vectorSize, closer);
|
||||||
} else {
|
} else {
|
||||||
// baseColumnSelectorFactory using baseOffset is the column selector for filtering.
|
|
||||||
final VectorColumnSelectorFactory baseColumnSelectorFactory = new QueryableIndexVectorColumnSelectorFactory(
|
|
||||||
index,
|
|
||||||
baseOffset,
|
|
||||||
closer,
|
|
||||||
columnCache
|
|
||||||
);
|
|
||||||
|
|
||||||
final VectorOffset filteredOffset = FilteredVectorOffset.create(
|
final VectorOffset filteredOffset = FilteredVectorOffset.create(
|
||||||
baseOffset,
|
baseOffset,
|
||||||
baseColumnSelectorFactory,
|
baseColumnSelectorFactory,
|
||||||
|
@ -254,10 +251,31 @@ public class QueryableIndexCursorSequenceBuilder
|
||||||
// object will get hit twice for some of the values (anything that matched the filter). This is probably most
|
// object will get hit twice for some of the values (anything that matched the filter). This is probably most
|
||||||
// noticeable if it causes thrashing of decompression buffers due to out-of-order reads. I haven't observed
|
// noticeable if it causes thrashing of decompression buffers due to out-of-order reads. I haven't observed
|
||||||
// this directly but it seems possible in principle.
|
// this directly but it seems possible in principle.
|
||||||
return new QueryableIndexVectorCursor(index, filteredOffset, closer, columnCache, vectorSize);
|
// baseColumnSelectorFactory using baseOffset is the column selector for filtering.
|
||||||
|
final VectorColumnSelectorFactory filteredColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset(
|
||||||
|
columnCache,
|
||||||
|
filteredOffset,
|
||||||
|
closer
|
||||||
|
);
|
||||||
|
return new QueryableIndexVectorCursor(filteredColumnSelectorFactory, filteredOffset, vectorSize, closer);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset(
|
||||||
|
Map<String, BaseColumn> columnCache,
|
||||||
|
VectorOffset baseOffset,
|
||||||
|
Closer closer
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new QueryableIndexVectorColumnSelectorFactory(
|
||||||
|
index,
|
||||||
|
baseOffset,
|
||||||
|
closer,
|
||||||
|
columnCache,
|
||||||
|
virtualColumns
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Search the time column using binary search. Benchmarks on various other approaches (linear search, binary
|
* Search the time column using binary search. Benchmarks on various other approaches (linear search, binary
|
||||||
* search that switches to linear at various closeness thresholds) indicated that a pure binary search worked best.
|
* search that switches to linear at various closeness thresholds) indicated that a pure binary search worked best.
|
||||||
|
@ -318,17 +336,16 @@ public class QueryableIndexCursorSequenceBuilder
|
||||||
private final VectorColumnSelectorFactory columnSelectorFactory;
|
private final VectorColumnSelectorFactory columnSelectorFactory;
|
||||||
|
|
||||||
public QueryableIndexVectorCursor(
|
public QueryableIndexVectorCursor(
|
||||||
final QueryableIndex index,
|
final VectorColumnSelectorFactory vectorColumnSelectorFactory,
|
||||||
final VectorOffset offset,
|
final VectorOffset offset,
|
||||||
final Closer closer,
|
final int vectorSize,
|
||||||
final Map<String, BaseColumn> columnCache,
|
final Closer closer
|
||||||
final int vectorSize
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
this.columnSelectorFactory = vectorColumnSelectorFactory;
|
||||||
|
this.vectorSize = vectorSize;
|
||||||
this.offset = offset;
|
this.offset = offset;
|
||||||
this.closer = closer;
|
this.closer = closer;
|
||||||
this.vectorSize = vectorSize;
|
|
||||||
this.columnSelectorFactory = new QueryableIndexVectorColumnSelectorFactory(index, offset, closer, columnCache);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -219,9 +219,8 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// 1) Virtual columns can't vectorize yet
|
// vector cursors can't iterate backwards yet
|
||||||
// 2) Vector cursors can't iterate backwards yet
|
return virtualColumns.canVectorize(this) && !descending;
|
||||||
return virtualColumns.size() == 0 && !descending;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -26,14 +26,21 @@ import org.apache.druid.query.dimension.DimensionSpec;
|
||||||
import org.apache.druid.segment.column.BitmapIndex;
|
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.data.ReadableOffset;
|
||||||
|
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
||||||
|
import org.apache.druid.segment.vector.ReadableVectorOffset;
|
||||||
|
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||||
|
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||||
|
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||||
|
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||||
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
import org.apache.druid.segment.virtual.ExpressionVirtualColumn;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Virtual columns are "views" created over a ColumnSelectorFactory or ColumnSelector. They can potentially draw from multiple
|
* Virtual columns are "views" created over a {@link ColumnSelectorFactory} or {@link ColumnSelector}. They can
|
||||||
* underlying columns, although they always present themselves as if they were a single column.
|
* potentially draw from multiple 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
|
||||||
* from the various makeXXXSelector methods need not be thread safe.
|
* from the various makeXXXSelector methods need not be thread safe.
|
||||||
|
@ -56,59 +63,173 @@ public interface VirtualColumn extends Cacheable
|
||||||
* virtual column was referenced with (through {@link DimensionSpec#getDimension()}, which
|
* virtual column was referenced with (through {@link DimensionSpec#getDimension()}, which
|
||||||
* is useful if this column uses dot notation. The virtual column is expected to apply any
|
* is useful if this column uses dot notation. The virtual column is expected to apply any
|
||||||
* necessary decoration from the dimensionSpec.
|
* necessary decoration from the dimensionSpec.
|
||||||
*
|
|
||||||
* @param dimensionSpec the dimensionSpec this column was referenced with
|
|
||||||
* @param factory column selector factory
|
|
||||||
*
|
|
||||||
* @return the selector, must not be null
|
|
||||||
*/
|
*/
|
||||||
DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelectorFactory factory);
|
DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelectorFactory factory);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns similar DimensionSelector object as returned by {@link #makeDimensionSelector(DimensionSpec, ColumnSelectorFactory)}
|
* Returns similar {@link DimensionSelector} object as returned by
|
||||||
* except this method has full access to underlying column and can potentially provide a more efficient implementation.
|
* {@link #makeDimensionSelector(DimensionSpec, ColumnSelectorFactory)} except this method has full access to the
|
||||||
|
* 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
|
* Users of this interface must ensure to first call this method whenever possible. Typically this can not be called
|
||||||
* query paths on top of IncrementalIndex which doesn't have columns as in persisted segments.
|
* 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")
|
@SuppressWarnings("unused")
|
||||||
@Nullable
|
@Nullable
|
||||||
default DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelector columnSelector, ReadableOffset offset)
|
default DimensionSelector makeDimensionSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableOffset offset
|
||||||
|
)
|
||||||
{
|
{
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Build a selector corresponding to this virtual column. Also provides the name that the
|
* Build a {@link ColumnValueSelector} 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.
|
||||||
*
|
|
||||||
* @param columnName the name this virtual column was referenced with
|
|
||||||
* @param factory column selector factory
|
|
||||||
*
|
|
||||||
* @return the selector, must not be null
|
|
||||||
*/
|
*/
|
||||||
ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelectorFactory factory);
|
ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelectorFactory factory);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns similar ColumnValueSelector object as returned by {@link #makeColumnValueSelector(String, ColumnSelectorFactory)}
|
* Returns similar {@link ColumnValueSelector} object as returned by
|
||||||
* except this method has full access to underlying column and can potentially provide a more efficient implementation.
|
* {@link #makeColumnValueSelector(String, ColumnSelectorFactory)} except this method has full access to the
|
||||||
|
* 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
|
* Users of this interface must ensure to first call this method whenever possible. Typically this can not be called
|
||||||
* query paths on top of IncrementalIndex which doesn't have columns as in persisted segments.
|
* 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")
|
@SuppressWarnings("unused")
|
||||||
@Nullable
|
@Nullable
|
||||||
default ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelector columnSelector, ReadableOffset offset)
|
default ColumnValueSelector<?> makeColumnValueSelector(
|
||||||
|
String columnName,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableOffset offset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
default boolean canVectorize(ColumnInspector inspector)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Build a {@link SingleValueDimensionVectorSelector} corresponding to this virtual column. Also provides the name
|
||||||
|
* that the virtual column was referenced with (through {@link DimensionSpec#getDimension()}, which is useful if this
|
||||||
|
* column uses dot notation. The virtual column is expected to apply any necessary decoration from the
|
||||||
|
* {@link DimensionSpec}.
|
||||||
|
*/
|
||||||
|
default SingleValueDimensionVectorSelector makeSingleValueVectorDimensionSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
VectorColumnSelectorFactory factory
|
||||||
|
)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns similar {@link SingleValueDimensionVectorSelector} object as returned by
|
||||||
|
* {@link #makeSingleValueVectorDimensionSelector(DimensionSpec, ColumnSelector, ReadableVectorOffset)} except this
|
||||||
|
* method has full access to the underlying column and can potentially provide a more efficient implementation.
|
||||||
|
*
|
||||||
|
* Users of this interface must ensure to first call this method whenever possible.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
@Nullable
|
||||||
|
default SingleValueDimensionVectorSelector makeSingleValueVectorDimensionSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableVectorOffset offset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Build a {@link MultiValueDimensionVectorSelector} corresponding to this virtual column. Also provides
|
||||||
|
* the name that the virtual column was referenced with (through {@link DimensionSpec#getDimension()}, which is useful
|
||||||
|
* if this column uses dot notation. The virtual column is expected to apply any necessary decoration from the
|
||||||
|
* {@link DimensionSpec}.
|
||||||
|
*/
|
||||||
|
default MultiValueDimensionVectorSelector makeMultiValueVectorDimensionSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
VectorColumnSelectorFactory factory
|
||||||
|
)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns similar {@link SingleValueDimensionVectorSelector} object as returned by
|
||||||
|
* {@link #makeSingleValueVectorDimensionSelector(DimensionSpec, ColumnSelector, ReadableVectorOffset)} except this
|
||||||
|
* method has full access to the underlying column and can potentially provide a more efficient implementation.
|
||||||
|
*
|
||||||
|
* Users of this interface must ensure to first call this method whenever possible.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
@Nullable
|
||||||
|
default MultiValueDimensionVectorSelector makeMultiValueVectorDimensionSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableVectorOffset offset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Build a {@link VectorValueSelector} 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.
|
||||||
|
*/
|
||||||
|
default VectorValueSelector makeVectorValueSelector(String columnName, VectorColumnSelectorFactory factory)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns similar {@link VectorValueSelector} object as returned by
|
||||||
|
* {@link #makeVectorValueSelector(String, VectorColumnSelectorFactory)} except this method has full access to the
|
||||||
|
* underlying column and can potentially provide a more efficient implementation.
|
||||||
|
*
|
||||||
|
* Users of this interface must ensure to first call this method whenever possible.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
@Nullable
|
||||||
|
default VectorValueSelector makeVectorValueSelector(
|
||||||
|
String columnName,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableVectorOffset offset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Build a {@link VectorObjectSelector} 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.
|
||||||
|
*/
|
||||||
|
default VectorObjectSelector makeVectorObjectSelector(String columnName, VectorColumnSelectorFactory factory)
|
||||||
|
{
|
||||||
|
throw new UnsupportedOperationException("not supported");
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns similar {@link VectorObjectSelector} object as returned by
|
||||||
|
* {@link #makeVectorObjectSelector(String, VectorColumnSelectorFactory)} except this method has full access to the
|
||||||
|
* underlying column and can potentially provide a more efficient implementation.
|
||||||
|
*
|
||||||
|
* Users of this interface must ensure to first call this method whenever possible.
|
||||||
|
*/
|
||||||
|
@SuppressWarnings("unused")
|
||||||
|
@Nullable
|
||||||
|
default VectorObjectSelector makeVectorObjectSelector(
|
||||||
|
String columnName,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableVectorOffset offset
|
||||||
|
)
|
||||||
{
|
{
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
|
@ -35,6 +35,12 @@ 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.data.ReadableOffset;
|
||||||
|
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
||||||
|
import org.apache.druid.segment.vector.ReadableVectorOffset;
|
||||||
|
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||||
|
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||||
|
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||||
|
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||||
import org.apache.druid.segment.virtual.VirtualizedColumnSelectorFactory;
|
import org.apache.druid.segment.virtual.VirtualizedColumnSelectorFactory;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
@ -152,42 +158,36 @@ public class VirtualColumns implements Cacheable
|
||||||
return withDotSupport.get(baseColumnName);
|
return withDotSupport.get(baseColumnName);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public BitmapIndex getBitmapIndex(String columnName, ColumnSelector columnSelector)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(columnName);
|
||||||
|
return virtualColumn.capabilities(columnName).hasBitmapIndexes() ? virtualColumn.getBitmapIndex(
|
||||||
|
columnName,
|
||||||
|
columnSelector
|
||||||
|
) : null;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a dimension (string) selector.
|
* Create a dimension (string) selector.
|
||||||
*
|
*
|
||||||
* @param dimensionSpec the dimensionSpec for this selector
|
|
||||||
* @param factory base column selector factory
|
|
||||||
*
|
|
||||||
* @return selector
|
|
||||||
*
|
|
||||||
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
*/
|
*/
|
||||||
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelectorFactory factory)
|
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelectorFactory factory)
|
||||||
{
|
{
|
||||||
final VirtualColumn virtualColumn = getVirtualColumn(dimensionSpec.getDimension());
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(dimensionSpec.getDimension());
|
||||||
if (virtualColumn == null) {
|
final DimensionSelector selector = virtualColumn.makeDimensionSelector(dimensionSpec, factory);
|
||||||
throw new IAE("No such virtual column[%s]", dimensionSpec.getDimension());
|
Preconditions.checkNotNull(selector, "selector");
|
||||||
} else {
|
return selector;
|
||||||
final DimensionSelector selector = virtualColumn.makeDimensionSelector(dimensionSpec, factory);
|
|
||||||
Preconditions.checkNotNull(selector, "selector");
|
|
||||||
return selector;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
|
||||||
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;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Try to create an optimized dimension (string) selector directly from a {@link ColumnSelector}. If this method
|
||||||
|
* returns null, callers should try to fallback to
|
||||||
|
* {@link #makeDimensionSelector(DimensionSpec, ColumnSelectorFactory)} instead.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
|
*/
|
||||||
@Nullable
|
@Nullable
|
||||||
public DimensionSelector makeDimensionSelector(
|
public DimensionSelector makeDimensionSelector(
|
||||||
DimensionSpec dimensionSpec,
|
DimensionSpec dimensionSpec,
|
||||||
|
@ -195,14 +195,29 @@ public class VirtualColumns implements Cacheable
|
||||||
ReadableOffset offset
|
ReadableOffset offset
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final VirtualColumn virtualColumn = getVirtualColumn(dimensionSpec.getDimension());
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(dimensionSpec.getDimension());
|
||||||
if (virtualColumn == null) {
|
return virtualColumn.makeDimensionSelector(dimensionSpec, columnSelector, offset);
|
||||||
throw new IAE("No such virtual column[%s]", dimensionSpec.getDimension());
|
|
||||||
} else {
|
|
||||||
return virtualColumn.makeDimensionSelector(dimensionSpec, columnSelector, offset);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a column value selector.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
|
*/
|
||||||
|
public ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelectorFactory factory)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(columnName);
|
||||||
|
final ColumnValueSelector<?> selector = virtualColumn.makeColumnValueSelector(columnName, factory);
|
||||||
|
Preconditions.checkNotNull(selector, "selector");
|
||||||
|
return selector;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Try to create an optimized value selector directly from a {@link ColumnSelector}. If this method returns null,
|
||||||
|
* callers should try to fallback to {@link #makeColumnValueSelector(String, ColumnSelectorFactory)} instead.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
|
*/
|
||||||
@Nullable
|
@Nullable
|
||||||
public ColumnValueSelector<?> makeColumnValueSelector(
|
public ColumnValueSelector<?> makeColumnValueSelector(
|
||||||
String columnName,
|
String columnName,
|
||||||
|
@ -210,34 +225,149 @@ public class VirtualColumns implements Cacheable
|
||||||
ReadableOffset offset
|
ReadableOffset offset
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
final VirtualColumn virtualColumn = getVirtualColumn(columnName);
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(columnName);
|
||||||
if (virtualColumn == null) {
|
return virtualColumn.makeColumnValueSelector(columnName, columnSelector, offset);
|
||||||
throw new IAE("No such virtual column[%s]", columnName);
|
}
|
||||||
} else {
|
|
||||||
return virtualColumn.makeColumnValueSelector(columnName, columnSelector, offset);
|
public boolean canVectorize(ColumnInspector columnInspector)
|
||||||
}
|
{
|
||||||
|
return virtualColumns.stream().allMatch(virtualColumn -> virtualColumn.canVectorize(columnInspector));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a column value selector.
|
* Create a single value dimension vector (string) selector.
|
||||||
*
|
|
||||||
* @param columnName column mame
|
|
||||||
* @param factory base column selector factory
|
|
||||||
*
|
|
||||||
* @return selector
|
|
||||||
*
|
*
|
||||||
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
*/
|
*/
|
||||||
public ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelectorFactory factory)
|
public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
VectorColumnSelectorFactory factory
|
||||||
|
)
|
||||||
{
|
{
|
||||||
final VirtualColumn virtualColumn = getVirtualColumn(columnName);
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(dimensionSpec.getDimension());
|
||||||
if (virtualColumn == null) {
|
final SingleValueDimensionVectorSelector selector = virtualColumn.makeSingleValueVectorDimensionSelector(
|
||||||
throw new IAE("No such virtual column[%s]", columnName);
|
dimensionSpec,
|
||||||
} else {
|
factory
|
||||||
final ColumnValueSelector<?> selector = virtualColumn.makeColumnValueSelector(columnName, factory);
|
);
|
||||||
Preconditions.checkNotNull(selector, "selector");
|
Preconditions.checkNotNull(selector, "selector");
|
||||||
return selector;
|
return selector;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Try to create an optimized single value dimension (string) vector selector, directly from a
|
||||||
|
* {@link ColumnSelector}. If this method returns null, callers should try to fallback to
|
||||||
|
* {@link #makeSingleValueDimensionVectorSelector(DimensionSpec, VectorColumnSelectorFactory)} instead.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public SingleValueDimensionVectorSelector makeSingleValueDimensionVectorSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableVectorOffset offset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(dimensionSpec.getDimension());
|
||||||
|
return virtualColumn.makeSingleValueVectorDimensionSelector(dimensionSpec, columnSelector, offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a multi value dimension vector (string) selector.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
|
*/
|
||||||
|
public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
VectorColumnSelectorFactory factory
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(dimensionSpec.getDimension());
|
||||||
|
final MultiValueDimensionVectorSelector selector = virtualColumn.makeMultiValueVectorDimensionSelector(
|
||||||
|
dimensionSpec,
|
||||||
|
factory
|
||||||
|
);
|
||||||
|
Preconditions.checkNotNull(selector, "selector");
|
||||||
|
return selector;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Try to create an optimized multi value dimension (string) vector selector, directly from a
|
||||||
|
* {@link ColumnSelector}. If this method returns null, callers should try to fallback to
|
||||||
|
* {@link #makeMultiValueDimensionVectorSelector(DimensionSpec, VectorColumnSelectorFactory)} instead.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public MultiValueDimensionVectorSelector makeMultiValueDimensionVectorSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableVectorOffset offset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(dimensionSpec.getDimension());
|
||||||
|
return virtualColumn.makeMultiValueVectorDimensionSelector(dimensionSpec, columnSelector, offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a column vector value selector.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
|
*/
|
||||||
|
public VectorValueSelector makeVectorValueSelector(String columnName, VectorColumnSelectorFactory factory)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(columnName);
|
||||||
|
final VectorValueSelector selector = virtualColumn.makeVectorValueSelector(columnName, factory);
|
||||||
|
Preconditions.checkNotNull(selector, "selector");
|
||||||
|
return selector;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Try to create an optimized vector value selector directly from a {@link ColumnSelector}. If this method returns
|
||||||
|
* null, callers should try to fallback to {@link #makeVectorValueSelector(String, VectorColumnSelectorFactory)}
|
||||||
|
* instead.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public VectorValueSelector makeVectorValueSelector(
|
||||||
|
String columnName,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableVectorOffset offset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(columnName);
|
||||||
|
return virtualColumn.makeVectorValueSelector(columnName, columnSelector, offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a column vector object selector.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
|
*/
|
||||||
|
public VectorObjectSelector makeVectorObjectSelector(String columnName, VectorColumnSelectorFactory factory)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(columnName);
|
||||||
|
final VectorObjectSelector selector = virtualColumn.makeVectorObjectSelector(columnName, factory);
|
||||||
|
Preconditions.checkNotNull(selector, "selector");
|
||||||
|
return selector;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Try to create an optimized vector object selector directly from a {@link ColumnSelector}.If this method returns
|
||||||
|
* null, callers should try to fallback to {@link #makeVectorObjectSelector(String, VectorColumnSelectorFactory)}
|
||||||
|
* instead.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if the virtual column does not exist (see {@link #exists(String)}
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
public VectorObjectSelector makeVectorObjectSelector(
|
||||||
|
String columnName,
|
||||||
|
ColumnSelector columnSelector,
|
||||||
|
ReadableVectorOffset offset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final VirtualColumn virtualColumn = getVirtualColumnForSelector(columnName);
|
||||||
|
return virtualColumn.makeVectorObjectSelector(columnName, columnSelector, offset);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
|
@ -273,11 +403,6 @@ public class VirtualColumns implements Cacheable
|
||||||
return virtualColumns.toArray(new VirtualColumn[0]);
|
return virtualColumns.toArray(new VirtualColumn[0]);
|
||||||
}
|
}
|
||||||
|
|
||||||
public int size()
|
|
||||||
{
|
|
||||||
return virtualColumns.size();
|
|
||||||
}
|
|
||||||
|
|
||||||
public ColumnSelectorFactory wrap(final ColumnSelectorFactory baseFactory)
|
public ColumnSelectorFactory wrap(final ColumnSelectorFactory baseFactory)
|
||||||
{
|
{
|
||||||
if (virtualColumns.isEmpty()) {
|
if (virtualColumns.isEmpty()) {
|
||||||
|
@ -294,6 +419,15 @@ public class VirtualColumns implements Cacheable
|
||||||
return new CacheKeyBuilder((byte) 0).appendCacheablesIgnoringOrder(virtualColumns).build();
|
return new CacheKeyBuilder((byte) 0).appendCacheablesIgnoringOrder(virtualColumns).build();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private VirtualColumn getVirtualColumnForSelector(String columnName)
|
||||||
|
{
|
||||||
|
VirtualColumn virtualColumn = getVirtualColumn(columnName);
|
||||||
|
if (virtualColumn == null) {
|
||||||
|
throw new IAE("No such virtual column[%s]", columnName);
|
||||||
|
}
|
||||||
|
return virtualColumn;
|
||||||
|
}
|
||||||
|
|
||||||
private void detectCycles(VirtualColumn virtualColumn, @Nullable Set<String> columnNames)
|
private void detectCycles(VirtualColumn virtualColumn, @Nullable Set<String> columnNames)
|
||||||
{
|
{
|
||||||
// Copy columnNames to avoid modifying it
|
// Copy columnNames to avoid modifying it
|
||||||
|
@ -339,4 +473,5 @@ public class VirtualColumns implements Cacheable
|
||||||
{
|
{
|
||||||
return virtualColumns.toString();
|
return virtualColumns.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.io.Closer;
|
||||||
import org.apache.druid.query.dimension.DimensionSpec;
|
import org.apache.druid.query.dimension.DimensionSpec;
|
||||||
import org.apache.druid.segment.QueryableIndex;
|
import org.apache.druid.segment.QueryableIndex;
|
||||||
import org.apache.druid.segment.QueryableIndexStorageAdapter;
|
import org.apache.druid.segment.QueryableIndexStorageAdapter;
|
||||||
|
import org.apache.druid.segment.VirtualColumns;
|
||||||
import org.apache.druid.segment.column.BaseColumn;
|
import org.apache.druid.segment.column.BaseColumn;
|
||||||
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;
|
||||||
|
@ -33,9 +34,11 @@ import org.apache.druid.segment.column.ValueType;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.function.Function;
|
||||||
|
|
||||||
public class QueryableIndexVectorColumnSelectorFactory implements VectorColumnSelectorFactory
|
public class QueryableIndexVectorColumnSelectorFactory implements VectorColumnSelectorFactory
|
||||||
{
|
{
|
||||||
|
private final VirtualColumns virtualColumns;
|
||||||
private final QueryableIndex index;
|
private final QueryableIndex index;
|
||||||
private final ReadableVectorOffset offset;
|
private final ReadableVectorOffset offset;
|
||||||
private final Closer closer;
|
private final Closer closer;
|
||||||
|
@ -52,12 +55,14 @@ public class QueryableIndexVectorColumnSelectorFactory implements VectorColumnSe
|
||||||
final QueryableIndex index,
|
final QueryableIndex index,
|
||||||
final ReadableVectorOffset offset,
|
final ReadableVectorOffset offset,
|
||||||
final Closer closer,
|
final Closer closer,
|
||||||
final Map<String, BaseColumn> columnCache
|
final Map<String, BaseColumn> columnCache,
|
||||||
|
final VirtualColumns virtualColumns
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.index = index;
|
this.index = index;
|
||||||
this.offset = offset;
|
this.offset = offset;
|
||||||
this.closer = closer;
|
this.closer = closer;
|
||||||
|
this.virtualColumns = virtualColumns;
|
||||||
this.columnCache = columnCache;
|
this.columnCache = columnCache;
|
||||||
this.singleValueDimensionSelectorCache = new HashMap<>();
|
this.singleValueDimensionSelectorCache = new HashMap<>();
|
||||||
this.multiValueDimensionSelectorCache = new HashMap<>();
|
this.multiValueDimensionSelectorCache = new HashMap<>();
|
||||||
|
@ -77,34 +82,53 @@ public class QueryableIndexVectorColumnSelectorFactory implements VectorColumnSe
|
||||||
if (!dimensionSpec.canVectorize()) {
|
if (!dimensionSpec.canVectorize()) {
|
||||||
throw new ISE("DimensionSpec[%s] cannot be vectorized", dimensionSpec);
|
throw new ISE("DimensionSpec[%s] cannot be vectorized", dimensionSpec);
|
||||||
}
|
}
|
||||||
|
Function<DimensionSpec, MultiValueDimensionVectorSelector> mappingFunction = spec -> {
|
||||||
return multiValueDimensionSelectorCache.computeIfAbsent(
|
if (virtualColumns.exists(spec.getDimension())) {
|
||||||
dimensionSpec,
|
MultiValueDimensionVectorSelector dimensionSelector = virtualColumns.makeMultiValueDimensionVectorSelector(
|
||||||
spec -> {
|
dimensionSpec,
|
||||||
final ColumnHolder holder = index.getColumnHolder(spec.getDimension());
|
index,
|
||||||
if (holder == null
|
offset
|
||||||
|| holder.getCapabilities().isDictionaryEncoded().isFalse()
|
);
|
||||||
|| holder.getCapabilities().getType() != ValueType.STRING
|
if (dimensionSelector == null) {
|
||||||
|| holder.getCapabilities().hasMultipleValues().isFalse()) {
|
return virtualColumns.makeMultiValueDimensionVectorSelector(dimensionSpec, this);
|
||||||
throw new ISE(
|
} else {
|
||||||
"Column[%s] is not a multi-value string column, do not ask for a multi-value selector",
|
return dimensionSelector;
|
||||||
spec.getDimension()
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
final DictionaryEncodedColumn<String> dictionaryEncodedColumn = (DictionaryEncodedColumn<String>)
|
|
||||||
getCachedColumn(spec.getDimension());
|
|
||||||
|
|
||||||
// dictionaryEncodedColumn is not null because of holder null check above
|
|
||||||
assert dictionaryEncodedColumn != null;
|
|
||||||
final MultiValueDimensionVectorSelector selector = dictionaryEncodedColumn.makeMultiValueDimensionVectorSelector(
|
|
||||||
offset
|
|
||||||
);
|
|
||||||
|
|
||||||
return spec.decorate(selector);
|
|
||||||
}
|
}
|
||||||
);
|
}
|
||||||
|
|
||||||
|
final ColumnHolder holder = index.getColumnHolder(spec.getDimension());
|
||||||
|
if (holder == null
|
||||||
|
|| holder.getCapabilities().isDictionaryEncoded().isFalse()
|
||||||
|
|| holder.getCapabilities().getType() != ValueType.STRING
|
||||||
|
|| holder.getCapabilities().hasMultipleValues().isFalse()) {
|
||||||
|
throw new ISE(
|
||||||
|
"Column[%s] is not a multi-value string column, do not ask for a multi-value selector",
|
||||||
|
spec.getDimension()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
final DictionaryEncodedColumn<String> dictionaryEncodedColumn = (DictionaryEncodedColumn<String>)
|
||||||
|
getCachedColumn(spec.getDimension());
|
||||||
|
|
||||||
|
// dictionaryEncodedColumn is not null because of holder null check above
|
||||||
|
assert dictionaryEncodedColumn != null;
|
||||||
|
final MultiValueDimensionVectorSelector selector = dictionaryEncodedColumn.makeMultiValueDimensionVectorSelector(
|
||||||
|
offset
|
||||||
|
);
|
||||||
|
|
||||||
|
return spec.decorate(selector);
|
||||||
|
};
|
||||||
|
|
||||||
|
// We cannot use computeIfAbsent() here since the function being applied may modify the cache itself through
|
||||||
|
// virtual column references, triggering a ConcurrentModificationException in JDK 9 and above.
|
||||||
|
MultiValueDimensionVectorSelector selector = multiValueDimensionSelectorCache.get(dimensionSpec);
|
||||||
|
if (selector == null) {
|
||||||
|
selector = mappingFunction.apply(dimensionSpec);
|
||||||
|
multiValueDimensionSelectorCache.put(dimensionSpec, selector);
|
||||||
|
}
|
||||||
|
|
||||||
|
return selector;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -114,66 +138,114 @@ public class QueryableIndexVectorColumnSelectorFactory implements VectorColumnSe
|
||||||
throw new ISE("DimensionSpec[%s] cannot be vectorized", dimensionSpec);
|
throw new ISE("DimensionSpec[%s] cannot be vectorized", dimensionSpec);
|
||||||
}
|
}
|
||||||
|
|
||||||
return singleValueDimensionSelectorCache.computeIfAbsent(
|
Function<DimensionSpec, SingleValueDimensionVectorSelector> mappingFunction = spec -> {
|
||||||
dimensionSpec,
|
if (virtualColumns.exists(spec.getDimension())) {
|
||||||
spec -> {
|
SingleValueDimensionVectorSelector dimensionSelector = virtualColumns.makeSingleValueDimensionVectorSelector(
|
||||||
final ColumnHolder holder = index.getColumnHolder(spec.getDimension());
|
dimensionSpec,
|
||||||
if (holder == null
|
index,
|
||||||
|| !holder.getCapabilities().isDictionaryEncoded().isTrue()
|
offset
|
||||||
|| holder.getCapabilities().getType() != ValueType.STRING) {
|
);
|
||||||
// Asking for a single-value dimension selector on a non-string column gets you a bunch of nulls.
|
if (dimensionSelector == null) {
|
||||||
return NilVectorSelector.create(offset);
|
return virtualColumns.makeSingleValueDimensionVectorSelector(dimensionSpec, this);
|
||||||
}
|
} else {
|
||||||
|
return dimensionSelector;
|
||||||
if (holder.getCapabilities().hasMultipleValues().isMaybeTrue()) {
|
|
||||||
// Asking for a single-value dimension selector on a multi-value column gets you an error.
|
|
||||||
throw new ISE("Column[%s] is multi-value, do not ask for a single-value selector", spec.getDimension());
|
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
final DictionaryEncodedColumn<String> dictionaryEncodedColumn = (DictionaryEncodedColumn<String>)
|
|
||||||
getCachedColumn(spec.getDimension());
|
|
||||||
|
|
||||||
// dictionaryEncodedColumn is not null because of holder null check above
|
|
||||||
assert dictionaryEncodedColumn != null;
|
|
||||||
final SingleValueDimensionVectorSelector selector =
|
|
||||||
dictionaryEncodedColumn.makeSingleValueDimensionVectorSelector(offset);
|
|
||||||
|
|
||||||
return spec.decorate(selector);
|
|
||||||
}
|
}
|
||||||
);
|
}
|
||||||
|
|
||||||
|
final ColumnHolder holder = index.getColumnHolder(spec.getDimension());
|
||||||
|
if (holder == null
|
||||||
|
|| !holder.getCapabilities().isDictionaryEncoded().isTrue()
|
||||||
|
|| holder.getCapabilities().getType() != ValueType.STRING) {
|
||||||
|
// Asking for a single-value dimension selector on a non-string column gets you a bunch of nulls.
|
||||||
|
return NilVectorSelector.create(offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (holder.getCapabilities().hasMultipleValues().isMaybeTrue()) {
|
||||||
|
// Asking for a single-value dimension selector on a multi-value column gets you an error.
|
||||||
|
throw new ISE("Column[%s] is multi-value, do not ask for a single-value selector", spec.getDimension());
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
final DictionaryEncodedColumn<String> dictionaryEncodedColumn = (DictionaryEncodedColumn<String>)
|
||||||
|
getCachedColumn(spec.getDimension());
|
||||||
|
|
||||||
|
// dictionaryEncodedColumn is not null because of holder null check above
|
||||||
|
assert dictionaryEncodedColumn != null;
|
||||||
|
final SingleValueDimensionVectorSelector selector =
|
||||||
|
dictionaryEncodedColumn.makeSingleValueDimensionVectorSelector(offset);
|
||||||
|
|
||||||
|
return spec.decorate(selector);
|
||||||
|
};
|
||||||
|
|
||||||
|
// We cannot use computeIfAbsent() here since the function being applied may modify the cache itself through
|
||||||
|
// virtual column references, triggering a ConcurrentModificationException in JDK 9 and above.
|
||||||
|
SingleValueDimensionVectorSelector selector = singleValueDimensionSelectorCache.get(dimensionSpec);
|
||||||
|
if (selector == null) {
|
||||||
|
selector = mappingFunction.apply(dimensionSpec);
|
||||||
|
singleValueDimensionSelectorCache.put(dimensionSpec, selector);
|
||||||
|
}
|
||||||
|
|
||||||
|
return selector;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VectorValueSelector makeValueSelector(final String columnName)
|
public VectorValueSelector makeValueSelector(final String columnName)
|
||||||
{
|
{
|
||||||
return valueSelectorCache.computeIfAbsent(
|
Function<String, VectorValueSelector> mappingFunction = name -> {
|
||||||
columnName,
|
if (virtualColumns.exists(columnName)) {
|
||||||
name -> {
|
VectorValueSelector selector = virtualColumns.makeVectorValueSelector(columnName, index, offset);
|
||||||
final BaseColumn column = getCachedColumn(name);
|
if (selector == null) {
|
||||||
if (column == null) {
|
return virtualColumns.makeVectorValueSelector(columnName, this);
|
||||||
return NilVectorSelector.create(offset);
|
} else {
|
||||||
} else {
|
return selector;
|
||||||
return column.makeVectorValueSelector(offset);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
);
|
}
|
||||||
|
final BaseColumn column = getCachedColumn(name);
|
||||||
|
if (column == null) {
|
||||||
|
return NilVectorSelector.create(offset);
|
||||||
|
} else {
|
||||||
|
return column.makeVectorValueSelector(offset);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
// We cannot use computeIfAbsent() here since the function being applied may modify the cache itself through
|
||||||
|
// virtual column references, triggering a ConcurrentModificationException in JDK 9 and above.
|
||||||
|
VectorValueSelector columnValueSelector = valueSelectorCache.get(columnName);
|
||||||
|
if (columnValueSelector == null) {
|
||||||
|
columnValueSelector = mappingFunction.apply(columnName);
|
||||||
|
valueSelectorCache.put(columnName, columnValueSelector);
|
||||||
|
}
|
||||||
|
|
||||||
|
return columnValueSelector;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public VectorObjectSelector makeObjectSelector(final String columnName)
|
public VectorObjectSelector makeObjectSelector(final String columnName)
|
||||||
{
|
{
|
||||||
return objectSelectorCache.computeIfAbsent(
|
Function<String, VectorObjectSelector> mappingFunction = name -> {
|
||||||
columnName,
|
if (virtualColumns.exists(columnName)) {
|
||||||
name -> {
|
VectorObjectSelector selector = virtualColumns.makeVectorObjectSelector(columnName, index, offset);
|
||||||
final BaseColumn column = getCachedColumn(name);
|
if (selector == null) {
|
||||||
if (column == null) {
|
return virtualColumns.makeVectorObjectSelector(columnName, this);
|
||||||
return NilVectorSelector.create(offset);
|
} else {
|
||||||
} else {
|
return selector;
|
||||||
return column.makeVectorObjectSelector(offset);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
);
|
}
|
||||||
|
final BaseColumn column = getCachedColumn(name);
|
||||||
|
if (column == null) {
|
||||||
|
return NilVectorSelector.create(offset);
|
||||||
|
} else {
|
||||||
|
return column.makeVectorObjectSelector(offset);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
// We cannot use computeIfAbsent() here since the function being applied may modify the cache itself through
|
||||||
|
// virtual column references, triggering a ConcurrentModificationException in JDK 9 and above.
|
||||||
|
VectorObjectSelector columnValueSelector = objectSelectorCache.get(columnName);
|
||||||
|
if (columnValueSelector == null) {
|
||||||
|
columnValueSelector = mappingFunction.apply(columnName);
|
||||||
|
objectSelectorCache.put(columnName, columnValueSelector);
|
||||||
|
}
|
||||||
|
|
||||||
|
return columnValueSelector;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
|
@ -193,6 +265,9 @@ public class QueryableIndexVectorColumnSelectorFactory implements VectorColumnSe
|
||||||
@Override
|
@Override
|
||||||
public ColumnCapabilities getColumnCapabilities(final String columnName)
|
public ColumnCapabilities getColumnCapabilities(final String columnName)
|
||||||
{
|
{
|
||||||
|
if (virtualColumns.exists(columnName)) {
|
||||||
|
return virtualColumns.getColumnCapabilities(columnName);
|
||||||
|
}
|
||||||
return QueryableIndexStorageAdapter.getColumnCapabilities(index, columnName);
|
return QueryableIndexStorageAdapter.getColumnCapabilities(index, columnName);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,257 @@
|
||||||
|
/*
|
||||||
|
* 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.ImmutableList;
|
||||||
|
import org.apache.druid.query.aggregation.Aggregator;
|
||||||
|
import org.apache.druid.query.aggregation.BufferAggregator;
|
||||||
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
|
import org.apache.druid.query.aggregation.CountVectorAggregator;
|
||||||
|
import org.apache.druid.query.aggregation.VectorAggregator;
|
||||||
|
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||||
|
import org.apache.druid.segment.ColumnSelectorFactory;
|
||||||
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
|
import org.apache.druid.segment.data.IndexedInts;
|
||||||
|
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
||||||
|
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||||
|
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||||
|
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||||
|
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||||
|
import org.junit.Assert;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Specialized aggregator factory for testing the selectors produced by {@link AlwaysTwoVectorizedVirtualColumn}, used
|
||||||
|
* for counting the number of values read so that tests can ensure the correct number of values have been processed. A
|
||||||
|
* {@link AlwaysTwoCounterVectorAggregator} will be produced, backed by a type appropriate selector for a given
|
||||||
|
* {@link ColumnCapabilities}.
|
||||||
|
*/
|
||||||
|
public class AlwaysTwoCounterAggregatorFactory extends CountAggregatorFactory
|
||||||
|
{
|
||||||
|
private final String fieldName;
|
||||||
|
|
||||||
|
public AlwaysTwoCounterAggregatorFactory(String name, String field)
|
||||||
|
{
|
||||||
|
super(name);
|
||||||
|
this.fieldName = field;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Aggregator factorize(ColumnSelectorFactory metricFactory)
|
||||||
|
{
|
||||||
|
throw new IllegalStateException("don't call this");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory)
|
||||||
|
{
|
||||||
|
throw new IllegalStateException("don't call this");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFactory)
|
||||||
|
{
|
||||||
|
ColumnCapabilities capabilities = selectorFactory.getColumnCapabilities(fieldName);
|
||||||
|
switch (capabilities.getType()) {
|
||||||
|
case LONG:
|
||||||
|
case DOUBLE:
|
||||||
|
case FLOAT:
|
||||||
|
return new AlwaysTwoCounterVectorAggregator(selectorFactory.makeValueSelector(fieldName));
|
||||||
|
case STRING:
|
||||||
|
if (capabilities.isDictionaryEncoded().isTrue()) {
|
||||||
|
if (capabilities.hasMultipleValues().isTrue()) {
|
||||||
|
return new AlwaysTwoCounterVectorAggregator(selectorFactory.makeMultiValueDimensionSelector(
|
||||||
|
DefaultDimensionSpec.of(fieldName)));
|
||||||
|
}
|
||||||
|
return new AlwaysTwoCounterVectorAggregator(selectorFactory.makeSingleValueDimensionSelector(DefaultDimensionSpec.of(fieldName)));
|
||||||
|
}
|
||||||
|
return new AlwaysTwoCounterVectorAggregator(selectorFactory.makeObjectSelector(fieldName));
|
||||||
|
default:
|
||||||
|
throw new IllegalStateException("how did this happen");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class AlwaysTwoCounterVectorAggregator extends CountVectorAggregator
|
||||||
|
{
|
||||||
|
@Nullable
|
||||||
|
private final VectorValueSelector valueSelector;
|
||||||
|
@Nullable
|
||||||
|
private final VectorObjectSelector objectSelector;
|
||||||
|
@Nullable
|
||||||
|
private final SingleValueDimensionVectorSelector singleValueDimensionSelector;
|
||||||
|
@Nullable
|
||||||
|
private final MultiValueDimensionVectorSelector multiValueDimensionSelector;
|
||||||
|
|
||||||
|
AlwaysTwoCounterVectorAggregator(VectorValueSelector valueSelector)
|
||||||
|
{
|
||||||
|
this(valueSelector, null, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
AlwaysTwoCounterVectorAggregator(VectorObjectSelector objectSelector)
|
||||||
|
{
|
||||||
|
this(null, objectSelector, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
AlwaysTwoCounterVectorAggregator(SingleValueDimensionVectorSelector dimSelector)
|
||||||
|
{
|
||||||
|
this(null, null, dimSelector, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
AlwaysTwoCounterVectorAggregator(MultiValueDimensionVectorSelector dimSelector)
|
||||||
|
{
|
||||||
|
this(null, null, null, dimSelector);
|
||||||
|
}
|
||||||
|
|
||||||
|
AlwaysTwoCounterVectorAggregator(
|
||||||
|
@Nullable VectorValueSelector valueSelector,
|
||||||
|
@Nullable VectorObjectSelector objectSelector,
|
||||||
|
@Nullable SingleValueDimensionVectorSelector singleValueDimensionSelector,
|
||||||
|
@Nullable MultiValueDimensionVectorSelector multiValueDimensionSelector
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.valueSelector = valueSelector;
|
||||||
|
this.objectSelector = objectSelector;
|
||||||
|
this.singleValueDimensionSelector = singleValueDimensionSelector;
|
||||||
|
this.multiValueDimensionSelector = multiValueDimensionSelector;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void aggregate(ByteBuffer buf, int position, int startRow, int endRow)
|
||||||
|
{
|
||||||
|
if (valueSelector != null) {
|
||||||
|
final long[] vector = valueSelector.getLongVector();
|
||||||
|
|
||||||
|
long count = 0;
|
||||||
|
for (int i = startRow; i < endRow; i++) {
|
||||||
|
Assert.assertEquals(2L, vector[i]);
|
||||||
|
count += 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
buf.putLong(position, buf.getLong(position) + count);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (objectSelector != null) {
|
||||||
|
final Object[] vector = objectSelector.getObjectVector();
|
||||||
|
|
||||||
|
long count = 0;
|
||||||
|
for (int i = startRow; i < endRow; i++) {
|
||||||
|
if (vector[i] instanceof List) {
|
||||||
|
Assert.assertEquals(ImmutableList.of("2", "2"), vector[i]);
|
||||||
|
count += 2;
|
||||||
|
} else {
|
||||||
|
Assert.assertEquals("2", vector[i]);
|
||||||
|
count += 1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
buf.putLong(position, buf.getLong(position) + count);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (singleValueDimensionSelector != null) {
|
||||||
|
final int[] rowVector = singleValueDimensionSelector.getRowVector();
|
||||||
|
|
||||||
|
long count = 0;
|
||||||
|
for (int i = startRow; i < endRow; i++) {
|
||||||
|
Assert.assertEquals("2", singleValueDimensionSelector.lookupName(rowVector[i]));
|
||||||
|
count += 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
buf.putLong(position, buf.getLong(position) + count);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (multiValueDimensionSelector != null) {
|
||||||
|
final IndexedInts[] rowVector = multiValueDimensionSelector.getRowVector();
|
||||||
|
|
||||||
|
long count = 0;
|
||||||
|
for (int i = startRow; i < endRow; i++) {
|
||||||
|
//noinspection SSBasedInspection
|
||||||
|
for (int j = 0; j < rowVector[i].size(); j++) {
|
||||||
|
Assert.assertEquals("2", multiValueDimensionSelector.lookupName(rowVector[i].get(j)));
|
||||||
|
count += 1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
buf.putLong(position, buf.getLong(position) + count);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
throw new IllegalStateException("how did this happen");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void aggregate(
|
||||||
|
ByteBuffer buf,
|
||||||
|
int numRows,
|
||||||
|
int[] positions,
|
||||||
|
@Nullable int[] rows,
|
||||||
|
int positionOffset
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (valueSelector != null) {
|
||||||
|
final long[] vector = valueSelector.getLongVector();
|
||||||
|
|
||||||
|
for (int i = 0; i < numRows; i++) {
|
||||||
|
final int position = positions[i] + positionOffset;
|
||||||
|
Assert.assertEquals(2L, vector[i]);
|
||||||
|
buf.putLong(position, buf.getLong(position) + 1);
|
||||||
|
}
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (objectSelector != null) {
|
||||||
|
final Object[] vector = objectSelector.getObjectVector();
|
||||||
|
for (int i = 0; i < numRows; i++) {
|
||||||
|
final int position = positions[i] + positionOffset;
|
||||||
|
if (vector[i] instanceof List) {
|
||||||
|
Assert.assertEquals(ImmutableList.of("2", "2"), vector[i]);
|
||||||
|
buf.putLong(position, buf.getLong(position) + 2);
|
||||||
|
} else {
|
||||||
|
Assert.assertEquals("2", vector[i]);
|
||||||
|
buf.putLong(position, buf.getLong(position) + 1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (singleValueDimensionSelector != null) {
|
||||||
|
final int[] rowVector = singleValueDimensionSelector.getRowVector();
|
||||||
|
for (int i = 0; i < numRows; i++) {
|
||||||
|
final int position = positions[i] + positionOffset;
|
||||||
|
Assert.assertEquals("2", singleValueDimensionSelector.lookupName(rowVector[i]));
|
||||||
|
buf.putLong(position, buf.getLong(position) + 1);
|
||||||
|
}
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
if (multiValueDimensionSelector != null) {
|
||||||
|
final IndexedInts[] rowVector = multiValueDimensionSelector.getRowVector();
|
||||||
|
for (int i = 0; i < numRows; i++) {
|
||||||
|
final int position = positions[i] + positionOffset;
|
||||||
|
//noinspection SSBasedInspection
|
||||||
|
for (int j = 0; j < rowVector[i].size(); j++) {
|
||||||
|
Assert.assertEquals("2", multiValueDimensionSelector.lookupName(rowVector[i].get(j)));
|
||||||
|
buf.putLong(position, buf.getLong(position) + 1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
throw new IllegalStateException("how did this happen");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,321 @@
|
||||||
|
/*
|
||||||
|
* 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.ImmutableList;
|
||||||
|
import org.apache.druid.query.dimension.DimensionSpec;
|
||||||
|
import org.apache.druid.segment.ColumnInspector;
|
||||||
|
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.VirtualColumn;
|
||||||
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
|
import org.apache.druid.segment.data.ArrayBasedIndexedInts;
|
||||||
|
import org.apache.druid.segment.data.IndexedInts;
|
||||||
|
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
|
||||||
|
import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector;
|
||||||
|
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
|
||||||
|
import org.apache.druid.segment.vector.VectorObjectSelector;
|
||||||
|
import org.apache.druid.segment.vector.VectorSizeInspector;
|
||||||
|
import org.apache.druid.segment.vector.VectorValueSelector;
|
||||||
|
import org.junit.Assert;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* {@link VirtualColumn} which only can produce all kinds of vector selectors and report any type of
|
||||||
|
* {@link ColumnCapabilities}
|
||||||
|
*/
|
||||||
|
public class AlwaysTwoVectorizedVirtualColumn implements VirtualColumn
|
||||||
|
{
|
||||||
|
private final String outputName;
|
||||||
|
private final ColumnCapabilities capabilities;
|
||||||
|
private final boolean dictionaryEncoded;
|
||||||
|
|
||||||
|
public AlwaysTwoVectorizedVirtualColumn(
|
||||||
|
String name,
|
||||||
|
ColumnCapabilities capabilites
|
||||||
|
)
|
||||||
|
{
|
||||||
|
this.outputName = name;
|
||||||
|
this.capabilities = capabilites;
|
||||||
|
this.dictionaryEncoded = capabilites.isDictionaryEncoded().isTrue() &&
|
||||||
|
capabilites.areDictionaryValuesUnique().isTrue();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean canVectorize(ColumnInspector inspector)
|
||||||
|
{
|
||||||
|
Assert.assertNotNull(inspector);
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getOutputName()
|
||||||
|
{
|
||||||
|
return outputName;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec, ColumnSelectorFactory factory)
|
||||||
|
{
|
||||||
|
throw new IllegalStateException("don't call this");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ColumnValueSelector<?> makeColumnValueSelector(String columnName, ColumnSelectorFactory factory)
|
||||||
|
{
|
||||||
|
throw new IllegalStateException("don't call this");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SingleValueDimensionVectorSelector makeSingleValueVectorDimensionSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
VectorColumnSelectorFactory factory
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Assert.assertEquals(outputName, dimensionSpec.getOutputName());
|
||||||
|
return new SingleValueDimensionVectorSelector()
|
||||||
|
{
|
||||||
|
private final VectorSizeInspector inspector = factory.getVectorSizeInspector();
|
||||||
|
private final int[] rowVector = new int[inspector.getMaxVectorSize()];
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int[] getRowVector()
|
||||||
|
{
|
||||||
|
|
||||||
|
return rowVector;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getValueCardinality()
|
||||||
|
{
|
||||||
|
return dictionaryEncoded ? 1 : CARDINALITY_UNKNOWN;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public String lookupName(int id)
|
||||||
|
{
|
||||||
|
return "2";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nameLookupPossibleInAdvance()
|
||||||
|
{
|
||||||
|
return dictionaryEncoded;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getMaxVectorSize()
|
||||||
|
{
|
||||||
|
return inspector.getMaxVectorSize();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getCurrentVectorSize()
|
||||||
|
{
|
||||||
|
return inspector.getCurrentVectorSize();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public MultiValueDimensionVectorSelector makeMultiValueVectorDimensionSelector(
|
||||||
|
DimensionSpec dimensionSpec,
|
||||||
|
VectorColumnSelectorFactory factory
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Assert.assertEquals(outputName, dimensionSpec.getOutputName());
|
||||||
|
final IndexedInts[] rowVector = new IndexedInts[factory.getVectorSizeInspector().getMaxVectorSize()];
|
||||||
|
Arrays.fill(rowVector, new ArrayBasedIndexedInts(new int[]{0, 0}));
|
||||||
|
return new MultiValueDimensionVectorSelector()
|
||||||
|
{
|
||||||
|
private final VectorSizeInspector inspector = factory.getVectorSizeInspector();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public IndexedInts[] getRowVector()
|
||||||
|
{
|
||||||
|
return rowVector;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getValueCardinality()
|
||||||
|
{
|
||||||
|
return dictionaryEncoded ? 1 : CARDINALITY_UNKNOWN;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public String lookupName(int id)
|
||||||
|
{
|
||||||
|
return "2";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean nameLookupPossibleInAdvance()
|
||||||
|
{
|
||||||
|
return dictionaryEncoded;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public IdLookup idLookup()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getMaxVectorSize()
|
||||||
|
{
|
||||||
|
return inspector.getMaxVectorSize();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getCurrentVectorSize()
|
||||||
|
{
|
||||||
|
return inspector.getCurrentVectorSize();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public VectorValueSelector makeVectorValueSelector(
|
||||||
|
String columnName,
|
||||||
|
VectorColumnSelectorFactory factory
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Assert.assertEquals(outputName, columnName);
|
||||||
|
final long[] longs = new long[factory.getVectorSizeInspector().getMaxVectorSize()];
|
||||||
|
final double[] doubles = new double[factory.getVectorSizeInspector().getMaxVectorSize()];
|
||||||
|
final float[] floats = new float[factory.getVectorSizeInspector().getMaxVectorSize()];
|
||||||
|
Arrays.fill(longs, 2L);
|
||||||
|
Arrays.fill(doubles, 2.0);
|
||||||
|
Arrays.fill(floats, 2.0f);
|
||||||
|
return new VectorValueSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public long[] getLongVector()
|
||||||
|
{
|
||||||
|
return longs;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public float[] getFloatVector()
|
||||||
|
{
|
||||||
|
return floats;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public double[] getDoubleVector()
|
||||||
|
{
|
||||||
|
return doubles;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public boolean[] getNullVector()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getMaxVectorSize()
|
||||||
|
{
|
||||||
|
return factory.getVectorSizeInspector().getMaxVectorSize();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getCurrentVectorSize()
|
||||||
|
{
|
||||||
|
return factory.getVectorSizeInspector().getCurrentVectorSize();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public VectorObjectSelector makeVectorObjectSelector(
|
||||||
|
String columnName,
|
||||||
|
VectorColumnSelectorFactory factory
|
||||||
|
)
|
||||||
|
{
|
||||||
|
Assert.assertEquals(outputName, columnName);
|
||||||
|
final Object[] objects = new Object[factory.getVectorSizeInspector().getMaxVectorSize()];
|
||||||
|
if (capabilities.hasMultipleValues().isTrue()) {
|
||||||
|
Arrays.fill(objects, ImmutableList.of("2", "2"));
|
||||||
|
} else {
|
||||||
|
Arrays.fill(objects, "2");
|
||||||
|
}
|
||||||
|
return new VectorObjectSelector()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public int getMaxVectorSize()
|
||||||
|
{
|
||||||
|
return factory.getVectorSizeInspector().getMaxVectorSize();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getCurrentVectorSize()
|
||||||
|
{
|
||||||
|
return factory.getVectorSizeInspector().getCurrentVectorSize();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Object[] getObjectVector()
|
||||||
|
{
|
||||||
|
return objects;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ColumnCapabilities capabilities(String columnName)
|
||||||
|
{
|
||||||
|
return capabilities;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<String> requiredColumns()
|
||||||
|
{
|
||||||
|
return ImmutableList.of();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean usesDotNotation()
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte[] getCacheKey()
|
||||||
|
{
|
||||||
|
return new byte[0];
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,334 @@
|
||||||
|
/*
|
||||||
|
* 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.ImmutableList;
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import com.google.common.collect.Lists;
|
||||||
|
import org.apache.druid.java.util.common.DateTimes;
|
||||||
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
|
import org.apache.druid.java.util.common.guava.Sequence;
|
||||||
|
import org.apache.druid.query.Druids;
|
||||||
|
import org.apache.druid.query.QueryContexts;
|
||||||
|
import org.apache.druid.query.QueryRunnerTestHelper;
|
||||||
|
import org.apache.druid.query.Result;
|
||||||
|
import org.apache.druid.query.aggregation.AggregationTestHelper;
|
||||||
|
import org.apache.druid.query.dimension.DefaultDimensionSpec;
|
||||||
|
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.groupby.ResultRow;
|
||||||
|
import org.apache.druid.query.timeseries.TimeseriesQuery;
|
||||||
|
import org.apache.druid.query.timeseries.TimeseriesResultValue;
|
||||||
|
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.segment.column.ColumnCapabilities;
|
||||||
|
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||||
|
import org.apache.druid.segment.column.ValueType;
|
||||||
|
import org.apache.druid.timeline.SegmentId;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Rule;
|
||||||
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
public class VectorizedVirtualColumnTest
|
||||||
|
{
|
||||||
|
private static final String ALWAYS_TWO = "two";
|
||||||
|
private static final String COUNT = "count";
|
||||||
|
private static final Map<String, Object> CONTEXT = ImmutableMap.of(QueryContexts.VECTORIZE_KEY, "force");
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public final TemporaryFolder tmpFolder = new TemporaryFolder();
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public ExpectedException expectedException = ExpectedException.none();
|
||||||
|
|
||||||
|
private AggregationTestHelper groupByTestHelper;
|
||||||
|
private AggregationTestHelper timeseriesTestHelper;
|
||||||
|
private List<Segment> segments = null;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setup()
|
||||||
|
{
|
||||||
|
groupByTestHelper = AggregationTestHelper.createGroupByQueryAggregationTestHelper(
|
||||||
|
Collections.emptyList(),
|
||||||
|
new GroupByQueryConfig(),
|
||||||
|
tmpFolder
|
||||||
|
);
|
||||||
|
timeseriesTestHelper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper(
|
||||||
|
Collections.emptyList(),
|
||||||
|
tmpFolder
|
||||||
|
);
|
||||||
|
QueryableIndexSegment queryableIndexSegment = new QueryableIndexSegment(
|
||||||
|
TestIndex.getMMappedTestIndex(),
|
||||||
|
SegmentId.dummy(QueryRunnerTestHelper.DATA_SOURCE)
|
||||||
|
);
|
||||||
|
|
||||||
|
segments = Lists.newArrayList(queryableIndexSegment, queryableIndexSegment);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupBySingleValueString()
|
||||||
|
{
|
||||||
|
testGroupBy(new ColumnCapabilitiesImpl()
|
||||||
|
.setType(ValueType.STRING)
|
||||||
|
.setDictionaryEncoded(true)
|
||||||
|
.setDictionaryValuesUnique(true)
|
||||||
|
.setHasMultipleValues(false)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByMultiValueString()
|
||||||
|
{
|
||||||
|
// cannot currently group by string columns that might be multi valued
|
||||||
|
cannotVectorize();
|
||||||
|
testGroupBy(new ColumnCapabilitiesImpl()
|
||||||
|
.setType(ValueType.STRING)
|
||||||
|
.setDictionaryEncoded(true)
|
||||||
|
.setDictionaryValuesUnique(true)
|
||||||
|
.setHasMultipleValues(true)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByMultiValueStringUnknown()
|
||||||
|
{
|
||||||
|
// cannot currently group by string columns that might be multi valued
|
||||||
|
cannotVectorize();
|
||||||
|
testGroupBy(new ColumnCapabilitiesImpl()
|
||||||
|
.setType(ValueType.STRING)
|
||||||
|
.setDictionaryEncoded(true)
|
||||||
|
.setDictionaryValuesUnique(true)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupBySingleValueStringNotDictionaryEncoded()
|
||||||
|
{
|
||||||
|
// cannot currently group by string columns that are not dictionary encoded
|
||||||
|
cannotVectorize();
|
||||||
|
testGroupBy(new ColumnCapabilitiesImpl()
|
||||||
|
.setType(ValueType.STRING)
|
||||||
|
.setDictionaryEncoded(false)
|
||||||
|
.setDictionaryValuesUnique(false)
|
||||||
|
.setHasMultipleValues(false)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByMultiValueStringNotDictionaryEncoded()
|
||||||
|
{
|
||||||
|
// cannot currently group by string columns that might be multi valued
|
||||||
|
cannotVectorize();
|
||||||
|
testGroupBy(new ColumnCapabilitiesImpl()
|
||||||
|
.setType(ValueType.STRING)
|
||||||
|
.setDictionaryEncoded(false)
|
||||||
|
.setDictionaryValuesUnique(false)
|
||||||
|
.setHasMultipleValues(true)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByLong()
|
||||||
|
{
|
||||||
|
testGroupBy(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByDouble()
|
||||||
|
{
|
||||||
|
testGroupBy(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGroupByFloat()
|
||||||
|
{
|
||||||
|
testGroupBy(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTimeseriesSingleValueString()
|
||||||
|
{
|
||||||
|
testTimeseries(new ColumnCapabilitiesImpl()
|
||||||
|
.setType(ValueType.STRING)
|
||||||
|
.setDictionaryEncoded(true)
|
||||||
|
.setDictionaryValuesUnique(true)
|
||||||
|
.setHasMultipleValues(false)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTimeseriesMultiValueString()
|
||||||
|
{
|
||||||
|
testTimeseries(new ColumnCapabilitiesImpl()
|
||||||
|
.setType(ValueType.STRING)
|
||||||
|
.setDictionaryEncoded(true)
|
||||||
|
.setDictionaryValuesUnique(true)
|
||||||
|
.setHasMultipleValues(true)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTimeseriesMultiValueStringUnknown()
|
||||||
|
{
|
||||||
|
testTimeseries(new ColumnCapabilitiesImpl()
|
||||||
|
.setType(ValueType.STRING)
|
||||||
|
.setDictionaryEncoded(true)
|
||||||
|
.setDictionaryValuesUnique(true)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTimeseriesSingleValueStringNotDictionaryEncoded()
|
||||||
|
{
|
||||||
|
testTimeseries(new ColumnCapabilitiesImpl()
|
||||||
|
.setType(ValueType.STRING)
|
||||||
|
.setDictionaryEncoded(false)
|
||||||
|
.setDictionaryValuesUnique(false)
|
||||||
|
.setHasMultipleValues(false)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTimeseriesMultiValueStringNotDictionaryEncoded()
|
||||||
|
{
|
||||||
|
testTimeseries(new ColumnCapabilitiesImpl()
|
||||||
|
.setType(ValueType.STRING)
|
||||||
|
.setDictionaryEncoded(false)
|
||||||
|
.setDictionaryValuesUnique(false)
|
||||||
|
.setHasMultipleValues(true)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTimeseriesLong()
|
||||||
|
{
|
||||||
|
testTimeseries(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTimeseriesDouble()
|
||||||
|
{
|
||||||
|
testTimeseries(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTimeseriesFloat()
|
||||||
|
{
|
||||||
|
testTimeseries(ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT));
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testTimeseries(ColumnCapabilities capabilities)
|
||||||
|
{
|
||||||
|
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||||
|
.intervals("2000/2030")
|
||||||
|
.dataSource(QueryRunnerTestHelper.DATA_SOURCE)
|
||||||
|
.granularity(Granularities.ALL)
|
||||||
|
.virtualColumns(new AlwaysTwoVectorizedVirtualColumn(ALWAYS_TWO, capabilities))
|
||||||
|
.aggregators(new AlwaysTwoCounterAggregatorFactory(COUNT, ALWAYS_TWO))
|
||||||
|
.context(CONTEXT)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
Sequence seq = timeseriesTestHelper.runQueryOnSegmentsObjs(segments, query);
|
||||||
|
|
||||||
|
List<Result<TimeseriesResultValue>> expectedResults = ImmutableList.of(
|
||||||
|
new Result<>(
|
||||||
|
DateTimes.of("2011-01-12T00:00:00.000Z"),
|
||||||
|
new TimeseriesResultValue(
|
||||||
|
ImmutableMap.of(COUNT, getCount(capabilities))
|
||||||
|
)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
TestHelper.assertExpectedObjects(expectedResults, seq.toList(), "failed");
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testGroupBy(ColumnCapabilities capabilities)
|
||||||
|
{
|
||||||
|
GroupByQuery query = new GroupByQuery.Builder()
|
||||||
|
.setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
|
||||||
|
.setGranularity(Granularities.ALL)
|
||||||
|
.setVirtualColumns(
|
||||||
|
new AlwaysTwoVectorizedVirtualColumn(ALWAYS_TWO, capabilities)
|
||||||
|
)
|
||||||
|
.addDimension(new DefaultDimensionSpec(ALWAYS_TWO, ALWAYS_TWO, capabilities.getType()))
|
||||||
|
.setAggregatorSpecs(new AlwaysTwoCounterAggregatorFactory(COUNT, ALWAYS_TWO))
|
||||||
|
.setInterval("2000/2030")
|
||||||
|
.setContext(CONTEXT)
|
||||||
|
.addOrderByColumn(ALWAYS_TWO)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
List<ResultRow> rows = groupByTestHelper.runQueryOnSegmentsObjs(segments, query).toList();
|
||||||
|
|
||||||
|
List<ResultRow> expectedRows = Collections.singletonList(
|
||||||
|
GroupByQueryRunnerTestHelper.createExpectedRow(
|
||||||
|
query,
|
||||||
|
"2000",
|
||||||
|
COUNT,
|
||||||
|
getCount(capabilities),
|
||||||
|
ALWAYS_TWO,
|
||||||
|
getTwo(capabilities)
|
||||||
|
)
|
||||||
|
);
|
||||||
|
|
||||||
|
TestHelper.assertExpectedObjects(expectedRows, rows, "failed");
|
||||||
|
}
|
||||||
|
|
||||||
|
private long getCount(ColumnCapabilities capabilities)
|
||||||
|
{
|
||||||
|
long modifier = 1L;
|
||||||
|
if (capabilities.hasMultipleValues().isTrue()) {
|
||||||
|
modifier = 2L;
|
||||||
|
}
|
||||||
|
return 2418L * modifier;
|
||||||
|
}
|
||||||
|
|
||||||
|
private Object getTwo(ColumnCapabilities capabilities)
|
||||||
|
{
|
||||||
|
switch (capabilities.getType()) {
|
||||||
|
case LONG:
|
||||||
|
return 2L;
|
||||||
|
case DOUBLE:
|
||||||
|
return 2.0;
|
||||||
|
case FLOAT:
|
||||||
|
return 2.0f;
|
||||||
|
case STRING:
|
||||||
|
default:
|
||||||
|
if (capabilities.hasMultipleValues().isTrue()) {
|
||||||
|
return ImmutableList.of("2", "2");
|
||||||
|
}
|
||||||
|
return "2";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void cannotVectorize()
|
||||||
|
{
|
||||||
|
expectedException.expect(RuntimeException.class);
|
||||||
|
expectedException.expectMessage("Cannot vectorize!");
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue