mirror of https://github.com/apache/druid.git
ColumnCapabilities.hasMultipleValues refactor (#9731)
* transition ColumnCapabilities.hasMultipleValues to Capable enum, remove ColumnCapabilities.isComplete * remove artifical, always multi-value capabilities from IncrementalIndexStorageAdapter and fix up fallout from that, fix ColumnCapabilities merge in index merger * fix typo * remove unused method * review stuffs, revert IncrementalIndexStorageAdapater capabilities change, plumb lame workaround to SegmentAnalyzer * more comment * use volatile booleans * fix line length * correctly handle missing columns for vector processors * return ColumnCapabilities.Capable for BitmapIndexSelector.hasMultipleValues, fix vector processor selection for complex * false on non-existent
This commit is contained in:
parent
e72f490be0
commit
77dd5b06ae
|
@ -33,6 +33,7 @@ import org.apache.druid.query.filter.BitmapIndexSelector;
|
|||
import org.apache.druid.query.filter.BoundDimFilter;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.segment.column.BitmapIndex;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.CloseableIndexed;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
|
@ -195,7 +196,7 @@ public class BoundFilterBenchmark
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues(final String dimension)
|
||||
public ColumnCapabilities.Capable hasMultipleValues(final String dimension)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.druid.query.filter.DruidFloatPredicate;
|
|||
import org.apache.druid.query.filter.DruidLongPredicate;
|
||||
import org.apache.druid.query.filter.DruidPredicateFactory;
|
||||
import org.apache.druid.segment.column.BitmapIndex;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.CloseableIndexed;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
|
@ -166,7 +167,7 @@ public class DimensionPredicateFilterBenchmark
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues(final String dimension)
|
||||
public ColumnCapabilities.Capable hasMultipleValues(final String dimension)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.druid.query.filter.RegexDimFilter;
|
|||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.ordering.StringComparators;
|
||||
import org.apache.druid.segment.column.BitmapIndex;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.CloseableIndexed;
|
||||
import org.apache.druid.segment.data.GenericIndexed;
|
||||
|
@ -166,7 +167,7 @@ public class LikeFilterBenchmark
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues(final String dimension)
|
||||
public ColumnCapabilities.Capable hasMultipleValues(final String dimension)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.druid.collections.bitmap.BitmapFactory;
|
|||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.collections.spatial.ImmutableRTree;
|
||||
import org.apache.druid.segment.column.BitmapIndex;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.data.CloseableIndexed;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -35,7 +36,7 @@ public interface BitmapIndexSelector
|
|||
@MustBeClosed
|
||||
@Nullable
|
||||
CloseableIndexed<String> getDimensionValues(String dimension);
|
||||
boolean hasMultipleValues(String dimension);
|
||||
ColumnCapabilities.Capable hasMultipleValues(String dimension);
|
||||
int getNumRows();
|
||||
BitmapFactory getBitmapFactory();
|
||||
@Nullable
|
||||
|
|
|
@ -321,13 +321,13 @@ public class GroupByQueryEngineV2
|
|||
/**
|
||||
* 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
|
||||
* 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. 'missingMeansNonExistent' is sort of a hack to let
|
||||
* the vectorized engine, which only operates on actual segments, to still work in this case for non-existent columns.
|
||||
*/
|
||||
public static boolean isAllSingleValueDims(
|
||||
final Function<String, ColumnCapabilities> capabilitiesFunction,
|
||||
final List<DimensionSpec> dimensions,
|
||||
final boolean missingMeansNonexistent
|
||||
final boolean missingMeansNonExistent
|
||||
)
|
||||
{
|
||||
return dimensions
|
||||
|
@ -342,8 +342,8 @@ public class GroupByQueryEngineV2
|
|||
|
||||
// Now check column capabilities.
|
||||
final ColumnCapabilities columnCapabilities = capabilitiesFunction.apply(dimension.getDimension());
|
||||
return (columnCapabilities != null && !columnCapabilities.hasMultipleValues()) ||
|
||||
(missingMeansNonexistent && columnCapabilities == null);
|
||||
return (columnCapabilities != null && !columnCapabilities.hasMultipleValues().isMaybeTrue()) ||
|
||||
(missingMeansNonExistent && columnCapabilities == null);
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.druid.segment.column.ComplexColumn;
|
|||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.IndexedInts;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||
import org.apache.druid.segment.serde.ComplexMetricSerde;
|
||||
import org.apache.druid.segment.serde.ComplexMetrics;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -101,9 +102,18 @@ public class SegmentAnalyzer
|
|||
|
||||
for (String columnName : columnNames) {
|
||||
final ColumnHolder columnHolder = index == null ? null : index.getColumnHolder(columnName);
|
||||
final ColumnCapabilities capabilities = columnHolder != null
|
||||
? columnHolder.getCapabilities()
|
||||
: storageAdapter.getColumnCapabilities(columnName);
|
||||
final ColumnCapabilities capabilities;
|
||||
if (columnHolder != null) {
|
||||
capabilities = columnHolder.getCapabilities();
|
||||
} else {
|
||||
// this can be removed if we get to the point where IncrementalIndexStorageAdapter.getColumnCapabilities
|
||||
// accurately reports the capabilities
|
||||
if (storageAdapter instanceof IncrementalIndexStorageAdapter) {
|
||||
capabilities = ((IncrementalIndexStorageAdapter) storageAdapter).getSnapshotColumnCapabilities(columnName);
|
||||
} else {
|
||||
capabilities = storageAdapter.getColumnCapabilities(columnName);
|
||||
}
|
||||
}
|
||||
|
||||
final ColumnAnalysis analysis;
|
||||
final ValueType type = capabilities.getType();
|
||||
|
@ -138,7 +148,7 @@ public class SegmentAnalyzer
|
|||
// Add time column too
|
||||
ColumnCapabilities timeCapabilities = storageAdapter.getColumnCapabilities(ColumnHolder.TIME_COLUMN_NAME);
|
||||
if (timeCapabilities == null) {
|
||||
timeCapabilities = new ColumnCapabilitiesImpl().setType(ValueType.LONG).setHasMultipleValues(false);
|
||||
timeCapabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG);
|
||||
}
|
||||
columns.put(
|
||||
ColumnHolder.TIME_COLUMN_NAME,
|
||||
|
@ -172,7 +182,7 @@ public class SegmentAnalyzer
|
|||
long size = 0;
|
||||
|
||||
if (analyzingSize()) {
|
||||
if (capabilities.hasMultipleValues()) {
|
||||
if (capabilities.hasMultipleValues().isTrue()) {
|
||||
return ColumnAnalysis.error("multi_value");
|
||||
}
|
||||
|
||||
|
@ -181,7 +191,7 @@ public class SegmentAnalyzer
|
|||
|
||||
return new ColumnAnalysis(
|
||||
capabilities.getType().name(),
|
||||
capabilities.hasMultipleValues(),
|
||||
capabilities.hasMultipleValues().isTrue(),
|
||||
size,
|
||||
null,
|
||||
null,
|
||||
|
@ -231,7 +241,7 @@ public class SegmentAnalyzer
|
|||
|
||||
return new ColumnAnalysis(
|
||||
capabilities.getType().name(),
|
||||
capabilities.hasMultipleValues(),
|
||||
capabilities.hasMultipleValues().isTrue(),
|
||||
size,
|
||||
analyzingCardinality() ? cardinality : 0,
|
||||
min,
|
||||
|
@ -308,7 +318,7 @@ public class SegmentAnalyzer
|
|||
|
||||
return new ColumnAnalysis(
|
||||
capabilities.getType().name(),
|
||||
capabilities.hasMultipleValues(),
|
||||
capabilities.hasMultipleValues().isTrue(),
|
||||
size,
|
||||
cardinality,
|
||||
min,
|
||||
|
@ -324,7 +334,7 @@ public class SegmentAnalyzer
|
|||
)
|
||||
{
|
||||
try (final ComplexColumn complexColumn = columnHolder != null ? (ComplexColumn) columnHolder.getColumn() : null) {
|
||||
final boolean hasMultipleValues = capabilities != null && capabilities.hasMultipleValues();
|
||||
final boolean hasMultipleValues = capabilities != null && capabilities.hasMultipleValues().isTrue();
|
||||
long size = 0;
|
||||
|
||||
if (analyzingSize() && complexColumn != null) {
|
||||
|
|
|
@ -197,6 +197,6 @@ public class ColumnProcessors
|
|||
*/
|
||||
private static boolean mayBeMultiValue(@Nullable final ColumnCapabilities capabilities)
|
||||
{
|
||||
return capabilities == null || !capabilities.isComplete() || capabilities.hasMultipleValues();
|
||||
return capabilities == null || capabilities.hasMultipleValues().isMaybeTrue();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.druid.query.filter.BitmapIndexSelector;
|
|||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.column.BaseColumn;
|
||||
import org.apache.druid.segment.column.BitmapIndex;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.DictionaryEncodedColumn;
|
||||
import org.apache.druid.segment.column.NumericColumn;
|
||||
|
@ -157,14 +158,18 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues(final String dimension)
|
||||
public ColumnCapabilities.Capable hasMultipleValues(final String dimension)
|
||||
{
|
||||
if (isVirtualColumn(dimension)) {
|
||||
return virtualColumns.getVirtualColumn(dimension).capabilities(dimension).hasMultipleValues();
|
||||
}
|
||||
|
||||
final ColumnHolder columnHolder = index.getColumnHolder(dimension);
|
||||
return columnHolder != null && columnHolder.getCapabilities().hasMultipleValues();
|
||||
// if ColumnHolder is null, the column doesn't exist, but report as not having multiple values so that
|
||||
// the empty bitmap will be used
|
||||
return columnHolder != null
|
||||
? columnHolder.getCapabilities().hasMultipleValues()
|
||||
: ColumnCapabilities.Capable.FALSE;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -289,15 +289,23 @@ public final class DimensionHandlerUtils
|
|||
final VectorColumnSelectorFactory selectorFactory
|
||||
)
|
||||
{
|
||||
final ColumnCapabilities capabilities = getEffectiveCapabilities(
|
||||
final ColumnCapabilities originalCapabilities =
|
||||
selectorFactory.getColumnCapabilities(dimensionSpec.getDimension());
|
||||
|
||||
final ColumnCapabilities effectiveCapabilites = getEffectiveCapabilities(
|
||||
dimensionSpec,
|
||||
selectorFactory.getColumnCapabilities(dimensionSpec.getDimension())
|
||||
originalCapabilities
|
||||
);
|
||||
|
||||
final ValueType type = capabilities.getType();
|
||||
final ValueType type = effectiveCapabilites.getType();
|
||||
|
||||
// vector selectors should never have null column capabilities, these signify a non-existent column, and complex
|
||||
// columns should never be treated as a multi-value column, so always use single value string processor
|
||||
final boolean forceSingleValue =
|
||||
originalCapabilities == null || ValueType.COMPLEX.equals(originalCapabilities.getType());
|
||||
|
||||
if (type == ValueType.STRING) {
|
||||
if (capabilities.hasMultipleValues()) {
|
||||
if (!forceSingleValue && effectiveCapabilites.hasMultipleValues().isMaybeTrue()) {
|
||||
return strategyFactory.makeMultiValueDimensionProcessor(
|
||||
selectorFactory.makeMultiValueDimensionSelector(dimensionSpec)
|
||||
);
|
||||
|
@ -328,7 +336,7 @@ public final class DimensionHandlerUtils
|
|||
selectorFactory.makeValueSelector(dimensionSpec.getDimension())
|
||||
);
|
||||
} else {
|
||||
throw new ISE("Unsupported type[%s]", capabilities.getType());
|
||||
throw new ISE("Unsupported type[%s]", effectiveCapabilites.getType());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -127,6 +127,19 @@ public interface DimensionIndexer
|
|||
*/
|
||||
EncodedKeyComponentType processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions);
|
||||
|
||||
/**
|
||||
* This method will be called while building an {@link IncrementalIndex} whenever a known dimension column (either
|
||||
* through an explicit schema on the ingestion spec, or auto-discovered while processing rows) is absent in any row
|
||||
* that is processed, to allow an indexer to account for any missing rows if necessary. Useful so that a string
|
||||
* {@link DimensionSelector} built on top of an {@link IncrementalIndex} may accurately report
|
||||
* {@link DimensionSelector#nameLookupPossibleInAdvance()} by allowing it to track if it has any implicit null valued
|
||||
* rows.
|
||||
*
|
||||
* At index persist/merge time all missing columns for a row will be explicitly replaced with the value appropriate
|
||||
* null or default value.
|
||||
*/
|
||||
void setSparseIndexed();
|
||||
|
||||
/**
|
||||
* Gives the estimated size in bytes for the given key
|
||||
*
|
||||
|
|
|
@ -47,6 +47,12 @@ public class DoubleDimensionIndexer implements DimensionIndexer<Double, Double,
|
|||
return DimensionHandlerUtils.convertObjectToDouble(dimValues, reportParseExceptions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setSparseIndexed()
|
||||
{
|
||||
// no-op, double columns do not have a dictionary to track null values
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimateEncodedKeyComponentSize(Double key)
|
||||
{
|
||||
|
|
|
@ -48,6 +48,12 @@ public class FloatDimensionIndexer implements DimensionIndexer<Float, Float, Flo
|
|||
return DimensionHandlerUtils.convertObjectToFloat(dimValues, reportParseExceptions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setSparseIndexed()
|
||||
{
|
||||
// no-op, float columns do not have a dictionary to track null values
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimateEncodedKeyComponentSize(Float key)
|
||||
{
|
||||
|
|
|
@ -165,7 +165,7 @@ public class IndexMergerV9 implements IndexMerger
|
|||
progress.progress();
|
||||
final Map<String, ValueType> metricsValueTypes = new TreeMap<>(Comparators.naturalNullsFirst());
|
||||
final Map<String, String> metricTypeNames = new TreeMap<>(Comparators.naturalNullsFirst());
|
||||
final List<ColumnCapabilitiesImpl> dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
||||
final List<ColumnCapabilities> dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size());
|
||||
mergeCapabilities(adapters, mergedDimensions, metricsValueTypes, metricTypeNames, dimCapabilities);
|
||||
|
||||
final Map<String, DimensionHandler> handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities);
|
||||
|
@ -716,18 +716,22 @@ public class IndexMergerV9 implements IndexMerger
|
|||
final List<String> mergedDimensions,
|
||||
final Map<String, ValueType> metricsValueTypes,
|
||||
final Map<String, String> metricTypeNames,
|
||||
final List<ColumnCapabilitiesImpl> dimCapabilities
|
||||
final List<ColumnCapabilities> dimCapabilities
|
||||
)
|
||||
{
|
||||
final Map<String, ColumnCapabilitiesImpl> capabilitiesMap = new HashMap<>();
|
||||
final Map<String, ColumnCapabilities> capabilitiesMap = new HashMap<>();
|
||||
for (IndexableAdapter adapter : adapters) {
|
||||
for (String dimension : adapter.getDimensionNames()) {
|
||||
ColumnCapabilities capabilities = adapter.getCapabilities(dimension);
|
||||
capabilitiesMap.computeIfAbsent(dimension, d -> new ColumnCapabilitiesImpl().setIsComplete(true)).merge(capabilities);
|
||||
capabilitiesMap.compute(dimension, (d, existingCapabilities) ->
|
||||
ColumnCapabilitiesImpl.snapshot(capabilities)
|
||||
.merge(ColumnCapabilitiesImpl.snapshot(existingCapabilities)));
|
||||
}
|
||||
for (String metric : adapter.getMetricNames()) {
|
||||
ColumnCapabilities capabilities = adapter.getCapabilities(metric);
|
||||
capabilitiesMap.computeIfAbsent(metric, m -> new ColumnCapabilitiesImpl().setIsComplete(true)).merge(capabilities);
|
||||
capabilitiesMap.compute(metric, (m, existingCapabilities) ->
|
||||
ColumnCapabilitiesImpl.snapshot(capabilities)
|
||||
.merge(ColumnCapabilitiesImpl.snapshot(existingCapabilities)));
|
||||
metricsValueTypes.put(metric, capabilities.getType());
|
||||
metricTypeNames.put(metric, adapter.getMetricType(metric));
|
||||
}
|
||||
|
@ -1002,7 +1006,7 @@ public class IndexMergerV9 implements IndexMerger
|
|||
|
||||
private Map<String, DimensionHandler> makeDimensionHandlers(
|
||||
final List<String> mergedDimensions,
|
||||
final List<ColumnCapabilitiesImpl> dimCapabilities
|
||||
final List<ColumnCapabilities> dimCapabilities
|
||||
)
|
||||
{
|
||||
Map<String, DimensionHandler> handlers = new LinkedHashMap<>();
|
||||
|
|
|
@ -48,6 +48,12 @@ public class LongDimensionIndexer implements DimensionIndexer<Long, Long, Long>
|
|||
return DimensionHandlerUtils.convertObjectToLong(dimValues, reportParseExceptions);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setSparseIndexed()
|
||||
{
|
||||
// no-op, long columns do not have a dictionary to track null values
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimateEncodedKeyComponentSize(Long key)
|
||||
{
|
||||
|
|
|
@ -96,7 +96,7 @@ public class RowBasedColumnSelectorFactory<T> implements ColumnSelectorFactory
|
|||
{
|
||||
if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) {
|
||||
// TIME_COLUMN_NAME is handled specially; override the provided rowSignature.
|
||||
return new ColumnCapabilitiesImpl().setType(ValueType.LONG).setIsComplete(true);
|
||||
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG);
|
||||
} else {
|
||||
final ValueType valueType = rowSignature.getColumnType(columnName).orElse(null);
|
||||
|
||||
|
@ -105,12 +105,13 @@ public class RowBasedColumnSelectorFactory<T> implements ColumnSelectorFactory
|
|||
// causes expression selectors to always treat us as arrays. If we might have multiple values (i.e. if our type
|
||||
// is nonnumeric), set isComplete false to compensate.
|
||||
if (valueType != null) {
|
||||
if (valueType.isNumeric()) {
|
||||
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(valueType);
|
||||
}
|
||||
return new ColumnCapabilitiesImpl()
|
||||
.setType(valueType)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
// Numeric types should be reported as complete, but not STRING or COMPLEX (because we don't have full info)
|
||||
.setIsComplete(valueType.isNumeric());
|
||||
.setDictionaryValuesSorted(false);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -233,7 +233,8 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
|||
private final DimensionDictionary dimLookup;
|
||||
private final MultiValueHandling multiValueHandling;
|
||||
private final boolean hasBitmapIndexes;
|
||||
private boolean hasMultipleValues = false;
|
||||
private volatile boolean hasMultipleValues = false;
|
||||
private volatile boolean isSparse = false;
|
||||
|
||||
@Nullable
|
||||
private SortedDimensionDictionary sortedLookup;
|
||||
|
@ -301,6 +302,12 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
|||
return encodedDimensionValues;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setSparseIndexed()
|
||||
{
|
||||
isSparse = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long estimateEncodedKeyComponentSize(int[] key)
|
||||
{
|
||||
|
@ -623,7 +630,9 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
|||
@Override
|
||||
public boolean nameLookupPossibleInAdvance()
|
||||
{
|
||||
return true;
|
||||
// name lookup is possible in advance if we got a value for every row (setSparseIndexed was not called on this
|
||||
// column) or we've encountered an actual null value and it is present in our dictionary
|
||||
return !isSparse || dimLookup.idForNull != ABSENT_VALUE_ID;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
|
|
@ -221,7 +221,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9
|
|||
final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression();
|
||||
|
||||
String filenameBase = StringUtils.format("%s.forward_dim", dimensionName);
|
||||
if (capabilities.hasMultipleValues()) {
|
||||
if (capabilities.hasMultipleValues().isTrue()) {
|
||||
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
|
||||
encodedValueSerializer = V3CompressedVSizeColumnarMultiIntsSerializer.create(
|
||||
dimensionName,
|
||||
|
@ -533,7 +533,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9
|
|||
public ColumnDescriptor makeColumnDescriptor()
|
||||
{
|
||||
// Now write everything
|
||||
boolean hasMultiValue = capabilities.hasMultipleValues();
|
||||
boolean hasMultiValue = capabilities.hasMultipleValues().isTrue();
|
||||
final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression();
|
||||
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
|
||||
|
||||
|
|
|
@ -118,7 +118,6 @@ public class ColumnBuilder
|
|||
.setDictionaryValuesUnique(dictionaryEncoded)
|
||||
.setHasSpatialIndexes(spatialIndex != null)
|
||||
.setHasMultipleValues(hasMultipleValues)
|
||||
.setIsComplete(true)
|
||||
.setFilterable(filterable),
|
||||
columnSupplier,
|
||||
bitmapIndex,
|
||||
|
|
|
@ -19,33 +19,26 @@
|
|||
|
||||
package org.apache.druid.segment.column;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonValue;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
/**
|
||||
*/
|
||||
public interface ColumnCapabilities
|
||||
{
|
||||
ValueType getType();
|
||||
|
||||
boolean isDictionaryEncoded();
|
||||
Capable areDictionaryValuesSorted();
|
||||
Capable areDictionaryValuesUnique();
|
||||
boolean isRunLengthEncoded();
|
||||
boolean hasBitmapIndexes();
|
||||
boolean hasSpatialIndexes();
|
||||
boolean hasMultipleValues();
|
||||
Capable hasMultipleValues();
|
||||
boolean isFilterable();
|
||||
|
||||
/**
|
||||
* This property indicates that this {@link ColumnCapabilities} is "complete" in that all properties can be expected
|
||||
* to supply valid responses. This is mostly a hack to work around {@link ColumnCapabilities} generators that
|
||||
* fail to set {@link #hasMultipleValues()} even when the associated column really could have multiple values.
|
||||
* Until this situation is sorted out, if this method returns false, callers are encouraged to ignore
|
||||
* {@link #hasMultipleValues()} and treat that property as if it were unknown.
|
||||
*
|
||||
* todo: replace all booleans with {@link Capable} and this method can be dropped
|
||||
*/
|
||||
boolean isComplete();
|
||||
|
||||
|
||||
enum Capable
|
||||
{
|
||||
FALSE,
|
||||
|
@ -57,6 +50,21 @@ public interface ColumnCapabilities
|
|||
return this == TRUE;
|
||||
}
|
||||
|
||||
public boolean isMaybeTrue()
|
||||
{
|
||||
return isTrue() || isUnknown();
|
||||
}
|
||||
|
||||
public boolean isUnknown()
|
||||
{
|
||||
return this == UNKNOWN;
|
||||
}
|
||||
|
||||
public Capable coerceUnknownToBoolean(boolean unknownIsTrue)
|
||||
{
|
||||
return this == UNKNOWN ? Capable.of(unknownIsTrue) : this;
|
||||
}
|
||||
|
||||
public Capable and(Capable other)
|
||||
{
|
||||
if (this == UNKNOWN || other == UNKNOWN) {
|
||||
|
@ -65,9 +73,36 @@ public interface ColumnCapabilities
|
|||
return this == TRUE && other == TRUE ? TRUE : FALSE;
|
||||
}
|
||||
|
||||
public Capable or(Capable other)
|
||||
{
|
||||
if (this == TRUE) {
|
||||
return TRUE;
|
||||
}
|
||||
return other;
|
||||
}
|
||||
|
||||
public static Capable of(boolean bool)
|
||||
{
|
||||
return bool ? TRUE : FALSE;
|
||||
}
|
||||
|
||||
@JsonCreator
|
||||
public static Capable ofNullable(@Nullable Boolean bool)
|
||||
{
|
||||
return bool == null ? Capable.UNKNOWN : of(bool);
|
||||
}
|
||||
|
||||
@JsonValue
|
||||
@Nullable
|
||||
public Boolean toJson()
|
||||
{
|
||||
return this == UNKNOWN ? null : isTrue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return StringUtils.toLowerCase(super.toString());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -31,15 +31,65 @@ import javax.annotation.Nullable;
|
|||
*/
|
||||
public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
||||
{
|
||||
public static ColumnCapabilitiesImpl copyOf(final ColumnCapabilities other)
|
||||
public static ColumnCapabilitiesImpl copyOf(@Nullable final ColumnCapabilities other)
|
||||
{
|
||||
final ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl();
|
||||
capabilities.merge(other);
|
||||
capabilities.setFilterable(other.isFilterable());
|
||||
capabilities.setIsComplete(other.isComplete());
|
||||
if (other != null) {
|
||||
capabilities.type = other.getType();
|
||||
capabilities.dictionaryEncoded = other.isDictionaryEncoded();
|
||||
capabilities.runLengthEncoded = other.isRunLengthEncoded();
|
||||
capabilities.hasInvertedIndexes = other.hasBitmapIndexes();
|
||||
capabilities.hasSpatialIndexes = other.hasSpatialIndexes();
|
||||
capabilities.hasMultipleValues = other.hasMultipleValues();
|
||||
capabilities.dictionaryValuesSorted = other.areDictionaryValuesSorted();
|
||||
capabilities.dictionaryValuesUnique = other.areDictionaryValuesUnique();
|
||||
capabilities.filterable = other.isFilterable();
|
||||
}
|
||||
return capabilities;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used at indexing time to finalize all {@link Capable#UNKNOWN} values to
|
||||
* {@link Capable#FALSE}, in order to present a snapshot of the state of the this column
|
||||
*/
|
||||
@Nullable
|
||||
public static ColumnCapabilitiesImpl snapshot(@Nullable final ColumnCapabilities capabilities)
|
||||
{
|
||||
return snapshot(capabilities, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Used at indexing time to finalize all {@link Capable#UNKNOWN} values to
|
||||
* {@link Capable#FALSE} or {@link Capable#TRUE}, in order to present a snapshot of the state of the this column
|
||||
*/
|
||||
@Nullable
|
||||
public static ColumnCapabilitiesImpl snapshot(@Nullable final ColumnCapabilities capabilities, boolean unknownIsTrue)
|
||||
{
|
||||
if (capabilities == null) {
|
||||
return null;
|
||||
}
|
||||
ColumnCapabilitiesImpl copy = copyOf(capabilities);
|
||||
copy.hasMultipleValues = copy.hasMultipleValues.coerceUnknownToBoolean(unknownIsTrue);
|
||||
copy.dictionaryValuesSorted = copy.dictionaryValuesSorted.coerceUnknownToBoolean(unknownIsTrue);
|
||||
copy.dictionaryValuesUnique = copy.dictionaryValuesUnique.coerceUnknownToBoolean(unknownIsTrue);
|
||||
return copy;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a no frills, simple column with {@link ValueType} set and everything else false
|
||||
*/
|
||||
public static ColumnCapabilitiesImpl createSimpleNumericColumnCapabilities(ValueType valueType)
|
||||
{
|
||||
return new ColumnCapabilitiesImpl().setType(valueType)
|
||||
.setHasMultipleValues(false)
|
||||
.setHasBitmapIndexes(false)
|
||||
.setDictionaryEncoded(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setHasSpatialIndexes(false);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private ValueType type = null;
|
||||
|
||||
|
@ -47,7 +97,7 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
private boolean runLengthEncoded = false;
|
||||
private boolean hasInvertedIndexes = false;
|
||||
private boolean hasSpatialIndexes = false;
|
||||
private boolean hasMultipleValues = false;
|
||||
private Capable hasMultipleValues = Capable.UNKNOWN;
|
||||
|
||||
// These capabilities are computed at query time and not persisted in the segment files.
|
||||
@JsonIgnore
|
||||
|
@ -56,8 +106,6 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
private Capable dictionaryValuesUnique = Capable.UNKNOWN;
|
||||
@JsonIgnore
|
||||
private boolean filterable;
|
||||
@JsonIgnore
|
||||
private boolean complete = false;
|
||||
|
||||
@Override
|
||||
@JsonProperty
|
||||
|
@ -144,14 +192,14 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
|
||||
@Override
|
||||
@JsonProperty("hasMultipleValues")
|
||||
public boolean hasMultipleValues()
|
||||
public Capable hasMultipleValues()
|
||||
{
|
||||
return hasMultipleValues;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setHasMultipleValues(boolean hasMultipleValues)
|
||||
{
|
||||
this.hasMultipleValues = hasMultipleValues;
|
||||
this.hasMultipleValues = Capable.of(hasMultipleValues);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -171,22 +219,10 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isComplete()
|
||||
{
|
||||
return complete;
|
||||
}
|
||||
|
||||
public ColumnCapabilitiesImpl setIsComplete(boolean complete)
|
||||
{
|
||||
this.complete = complete;
|
||||
return this;
|
||||
}
|
||||
|
||||
public void merge(ColumnCapabilities other)
|
||||
public ColumnCapabilities merge(@Nullable ColumnCapabilities other)
|
||||
{
|
||||
if (other == null) {
|
||||
return;
|
||||
return this;
|
||||
}
|
||||
|
||||
if (type == null) {
|
||||
|
@ -201,10 +237,11 @@ public class ColumnCapabilitiesImpl implements ColumnCapabilities
|
|||
this.runLengthEncoded |= other.isRunLengthEncoded();
|
||||
this.hasInvertedIndexes |= other.hasBitmapIndexes();
|
||||
this.hasSpatialIndexes |= other.hasSpatialIndexes();
|
||||
this.hasMultipleValues |= other.hasMultipleValues();
|
||||
this.complete &= other.isComplete(); // these should always be the same?
|
||||
this.filterable &= other.isFilterable();
|
||||
this.hasMultipleValues = this.hasMultipleValues.or(other.hasMultipleValues());
|
||||
this.dictionaryValuesSorted = this.dictionaryValuesSorted.and(other.areDictionaryValuesSorted());
|
||||
this.dictionaryValuesUnique = this.dictionaryValuesUnique.and(other.areDictionaryValuesUnique());
|
||||
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -115,7 +115,7 @@ public class ExpressionFilter implements Filter
|
|||
// multiple values. The lack of multiple values is important because expression filters treat multi-value
|
||||
// arrays as nulls, which doesn't permit index based filtering.
|
||||
final String column = Iterables.getOnlyElement(requiredBindings.get());
|
||||
return selector.getBitmapIndex(column) != null && !selector.hasMultipleValues(column);
|
||||
return selector.getBitmapIndex(column) != null && !selector.hasMultipleValues(column).isMaybeTrue();
|
||||
} else {
|
||||
// Multi-column expression.
|
||||
return false;
|
||||
|
|
|
@ -414,7 +414,7 @@ public class Filters
|
|||
if (filter.supportsBitmapIndex(indexSelector)) {
|
||||
final ColumnHolder columnHolder = columnSelector.getColumnHolder(dimension);
|
||||
if (columnHolder != null) {
|
||||
return !columnHolder.getCapabilities().hasMultipleValues();
|
||||
return !columnHolder.getCapabilities().hasMultipleValues().isMaybeTrue();
|
||||
}
|
||||
}
|
||||
return false;
|
||||
|
|
|
@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.google.common.primitives.Longs;
|
||||
import com.google.errorprone.annotations.concurrent.GuardedBy;
|
||||
|
@ -89,6 +90,7 @@ import java.util.Iterator;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentLinkedDeque;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
@ -327,9 +329,10 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
|
|||
}
|
||||
|
||||
//__time capabilities
|
||||
ColumnCapabilitiesImpl timeCapabilities = new ColumnCapabilitiesImpl().setIsComplete(true);
|
||||
timeCapabilities.setType(ValueType.LONG);
|
||||
columnCapabilities.put(ColumnHolder.TIME_COLUMN_NAME, timeCapabilities);
|
||||
columnCapabilities.put(
|
||||
ColumnHolder.TIME_COLUMN_NAME,
|
||||
ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG)
|
||||
);
|
||||
|
||||
// This should really be more generic
|
||||
List<SpatialDimensionSchema> spatialDimensions = dimensionsSpec.getSpatialDimensions();
|
||||
|
@ -640,12 +643,15 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
|
|||
}
|
||||
|
||||
final List<String> rowDimensions = row.getDimensions();
|
||||
|
||||
Object[] dims;
|
||||
List<Object> overflow = null;
|
||||
long dimsKeySize = 0;
|
||||
List<String> parseExceptionMessages = new ArrayList<>();
|
||||
synchronized (dimensionDescs) {
|
||||
// all known dimensions are assumed missing until we encounter in the rowDimensions
|
||||
Set<String> absentDimensions = Sets.newHashSet(dimensionDescs.keySet());
|
||||
|
||||
// first, process dimension values present in the row
|
||||
dims = new Object[dimensionDescs.size()];
|
||||
for (String dimension : rowDimensions) {
|
||||
if (Strings.isNullOrEmpty(dimension)) {
|
||||
|
@ -656,18 +662,13 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
|
|||
DimensionDesc desc = dimensionDescs.get(dimension);
|
||||
if (desc != null) {
|
||||
capabilities = desc.getCapabilities();
|
||||
absentDimensions.remove(dimension);
|
||||
} else {
|
||||
wasNewDim = true;
|
||||
capabilities = columnCapabilities.get(dimension);
|
||||
if (capabilities == null) {
|
||||
capabilities = new ColumnCapabilitiesImpl();
|
||||
// For schemaless type discovery, assume everything is a String for now, can change later.
|
||||
capabilities.setType(ValueType.STRING);
|
||||
capabilities.setDictionaryEncoded(true);
|
||||
capabilities.setHasBitmapIndexes(true);
|
||||
capabilities.setDictionaryValuesSorted(false);
|
||||
capabilities.setDictionaryValuesUnique(true);
|
||||
capabilities.setIsComplete(true);
|
||||
capabilities = makeCapabilitiesFromValueType(ValueType.STRING);
|
||||
columnCapabilities.put(dimension, capabilities);
|
||||
}
|
||||
DimensionHandler handler = DimensionHandlerUtils.getHandlerFromCapabilities(dimension, capabilities, null);
|
||||
|
@ -677,23 +678,24 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
|
|||
DimensionIndexer indexer = desc.getIndexer();
|
||||
Object dimsKey = null;
|
||||
try {
|
||||
dimsKey = indexer.processRowValsToUnsortedEncodedKeyComponent(
|
||||
row.getRaw(dimension),
|
||||
true
|
||||
);
|
||||
dimsKey = indexer.processRowValsToUnsortedEncodedKeyComponent(row.getRaw(dimension), true);
|
||||
}
|
||||
catch (ParseException pe) {
|
||||
parseExceptionMessages.add(pe.getMessage());
|
||||
}
|
||||
dimsKeySize += indexer.estimateEncodedKeyComponentSize(dimsKey);
|
||||
// Set column capabilities as data is coming in
|
||||
if (!capabilities.hasMultipleValues() &&
|
||||
if (!capabilities.hasMultipleValues().isTrue() &&
|
||||
dimsKey != null &&
|
||||
handler.getLengthOfEncodedKeyComponent(dimsKey) > 1) {
|
||||
capabilities.setHasMultipleValues(true);
|
||||
}
|
||||
|
||||
if (wasNewDim) {
|
||||
// unless this is the first row we are processing, all newly discovered columns will be sparse
|
||||
if (maxIngestedEventTime != null) {
|
||||
indexer.setSparseIndexed();
|
||||
}
|
||||
if (overflow == null) {
|
||||
overflow = new ArrayList<>();
|
||||
}
|
||||
|
@ -713,6 +715,11 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
|
|||
dims[desc.getIndex()] = dimsKey;
|
||||
}
|
||||
}
|
||||
|
||||
// process any dimensions with missing values in the row
|
||||
for (String missing : absentDimensions) {
|
||||
dimensionDescs.get(missing).getIndexer().setSparseIndexed();
|
||||
}
|
||||
}
|
||||
|
||||
if (overflow != null) {
|
||||
|
@ -923,16 +930,16 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
|
|||
|
||||
private ColumnCapabilitiesImpl makeCapabilitiesFromValueType(ValueType type)
|
||||
{
|
||||
ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl();
|
||||
capabilities.setDictionaryEncoded(type == ValueType.STRING);
|
||||
capabilities.setHasBitmapIndexes(type == ValueType.STRING);
|
||||
if (type == ValueType.STRING) {
|
||||
capabilities.setDictionaryValuesUnique(true);
|
||||
capabilities.setDictionaryValuesSorted(false);
|
||||
// we start out as not having multiple values, but this might change as we encounter them
|
||||
return new ColumnCapabilitiesImpl().setType(type)
|
||||
.setHasBitmapIndexes(true)
|
||||
.setDictionaryEncoded(true)
|
||||
.setDictionaryValuesUnique(true)
|
||||
.setDictionaryValuesSorted(false);
|
||||
} else {
|
||||
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(type);
|
||||
}
|
||||
capabilities.setType(type);
|
||||
capabilities.setIsComplete(true);
|
||||
return capabilities;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -988,6 +995,7 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
|
|||
return new IncrementalIndexStorageAdapter(this);
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public ColumnCapabilities getCapabilities(String column)
|
||||
{
|
||||
return columnCapabilities.get(column);
|
||||
|
@ -1124,18 +1132,18 @@ public abstract class IncrementalIndex<AggregatorType> extends AbstractIndex imp
|
|||
this.name = factory.getName();
|
||||
|
||||
String typeInfo = factory.getTypeName();
|
||||
this.capabilities = new ColumnCapabilitiesImpl().setIsComplete(true);
|
||||
if ("float".equalsIgnoreCase(typeInfo)) {
|
||||
capabilities.setType(ValueType.FLOAT);
|
||||
capabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT);
|
||||
this.type = typeInfo;
|
||||
} else if ("long".equalsIgnoreCase(typeInfo)) {
|
||||
capabilities.setType(ValueType.LONG);
|
||||
capabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG);
|
||||
this.type = typeInfo;
|
||||
} else if ("double".equalsIgnoreCase(typeInfo)) {
|
||||
capabilities.setType(ValueType.DOUBLE);
|
||||
capabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE);
|
||||
this.type = typeInfo;
|
||||
} else {
|
||||
capabilities.setType(ValueType.COMPLEX);
|
||||
// in an ideal world complex type reports its actual column capabilities...
|
||||
capabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.COMPLEX);
|
||||
this.type = ComplexMetrics.getSerdeForType(typeInfo).getTypeName();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.druid.segment.VirtualColumns;
|
|||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
import org.apache.druid.segment.data.ListIndexed;
|
||||
import org.apache.druid.segment.filter.BooleanValueMatcher;
|
||||
|
@ -150,16 +149,23 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
// at index-persisting time to determine if we need a multi-value column or not. However, that means we
|
||||
// need to tweak the capabilities here in the StorageAdapter (a query-time construct), so at query time
|
||||
// they appear multi-valued.
|
||||
|
||||
final ColumnCapabilities capabilitiesFromIndex = index.getCapabilities(column);
|
||||
final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(column);
|
||||
if (dimensionDesc != null && dimensionDesc.getCapabilities().getType() == ValueType.STRING) {
|
||||
final ColumnCapabilitiesImpl retVal = ColumnCapabilitiesImpl.copyOf(capabilitiesFromIndex);
|
||||
retVal.setHasMultipleValues(true);
|
||||
return retVal;
|
||||
} else {
|
||||
return capabilitiesFromIndex;
|
||||
//
|
||||
// Note that this could be improved if we snapshot the capabilities at cursor creation time and feed those through
|
||||
// to the StringDimensionIndexer so the selector built on top of it can produce values from the snapshot state of
|
||||
// multi-valuedness at cursor creation time, instead of the latest state, and getSnapshotColumnCapabilities could
|
||||
// be removed.
|
||||
return ColumnCapabilitiesImpl.snapshot(index.getCapabilities(column), true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sad workaround for {@link org.apache.druid.query.metadata.SegmentAnalyzer} to deal with the fact that the
|
||||
* response from {@link #getColumnCapabilities} is not accurate for string columns, in that it reports all string
|
||||
* string columns as having multiple values. This method returns the actual capabilities of the underlying
|
||||
* {@link IncrementalIndex}at the time this method is called.
|
||||
*/
|
||||
public ColumnCapabilities getSnapshotColumnCapabilities(String column)
|
||||
{
|
||||
return ColumnCapabilitiesImpl.snapshot(index.getCapabilities(column));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -58,8 +58,9 @@ public class IndexedTableColumnSelectorFactory implements ColumnSelectorFactory
|
|||
|
||||
capabilities.setDictionaryValuesSorted(false);
|
||||
capabilities.setDictionaryValuesUnique(false);
|
||||
capabilities.setHasMultipleValues(false);
|
||||
|
||||
return capabilities.setIsComplete(true);
|
||||
return capabilities;
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -85,7 +85,7 @@ public class QueryableIndexVectorColumnSelectorFactory implements VectorColumnSe
|
|||
if (holder == null
|
||||
|| !holder.getCapabilities().isDictionaryEncoded()
|
||||
|| holder.getCapabilities().getType() != ValueType.STRING
|
||||
|| !holder.getCapabilities().hasMultipleValues()) {
|
||||
|| !holder.getCapabilities().hasMultipleValues().isMaybeTrue()) {
|
||||
throw new ISE(
|
||||
"Column[%s] is not a multi-value string column, do not ask for a multi-value selector",
|
||||
spec.getDimension()
|
||||
|
@ -125,7 +125,7 @@ public class QueryableIndexVectorColumnSelectorFactory implements VectorColumnSe
|
|||
return NilVectorSelector.create(offset);
|
||||
}
|
||||
|
||||
if (holder.getCapabilities().hasMultipleValues()) {
|
||||
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());
|
||||
}
|
||||
|
|
|
@ -154,8 +154,7 @@ public class ExpressionSelectors
|
|||
} else if (capabilities != null
|
||||
&& capabilities.getType() == ValueType.STRING
|
||||
&& capabilities.isDictionaryEncoded()
|
||||
&& capabilities.isComplete()
|
||||
&& !capabilities.hasMultipleValues()
|
||||
&& !capabilities.hasMultipleValues().isMaybeTrue()
|
||||
&& exprDetails.getArrayBindings().isEmpty()) {
|
||||
// Optimization for expressions that hit one scalar string column and nothing else.
|
||||
return new SingleStringInputCachingExpressionColumnValueSelector(
|
||||
|
@ -227,7 +226,7 @@ public class ExpressionSelectors
|
|||
if (capabilities != null
|
||||
&& capabilities.getType() == ValueType.STRING
|
||||
&& capabilities.isDictionaryEncoded()
|
||||
&& capabilities.isComplete()
|
||||
&& !capabilities.hasMultipleValues().isUnknown()
|
||||
&& !exprDetails.hasInputArrays()
|
||||
&& !exprDetails.isOutputArray()
|
||||
) {
|
||||
|
@ -356,7 +355,7 @@ public class ExpressionSelectors
|
|||
final ColumnCapabilities columnCapabilities = columnSelectorFactory
|
||||
.getColumnCapabilities(columnName);
|
||||
final ValueType nativeType = columnCapabilities != null ? columnCapabilities.getType() : null;
|
||||
final boolean multiVal = columnCapabilities != null && columnCapabilities.hasMultipleValues();
|
||||
final boolean multiVal = columnCapabilities != null && columnCapabilities.hasMultipleValues().isTrue();
|
||||
final Supplier<Object> supplier;
|
||||
|
||||
if (nativeType == ValueType.FLOAT) {
|
||||
|
@ -597,11 +596,11 @@ public class ExpressionSelectors
|
|||
for (String column : columns) {
|
||||
final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(column);
|
||||
if (capabilities != null) {
|
||||
if (capabilities.hasMultipleValues()) {
|
||||
if (capabilities.hasMultipleValues().isTrue()) {
|
||||
actualArrays.add(column);
|
||||
} else if (
|
||||
!capabilities.isComplete() &&
|
||||
capabilities.getType().equals(ValueType.STRING) &&
|
||||
capabilities.hasMultipleValues().isMaybeTrue() &&
|
||||
!exprDetails.getArrayBindings().contains(column)
|
||||
) {
|
||||
unknownIfArrays.add(column);
|
||||
|
|
|
@ -131,10 +131,10 @@ public class ExpressionVirtualColumn implements VirtualColumn
|
|||
@Override
|
||||
public ColumnCapabilities capabilities(String columnName)
|
||||
{
|
||||
// Note: Ideally we would only "setHasMultipleValues(true)" if the expression in question could potentially return
|
||||
// multiple values. However, we don't currently have a good way of determining this, so to be safe we always
|
||||
// set the flag.
|
||||
return new ColumnCapabilitiesImpl().setType(outputType).setHasMultipleValues(true);
|
||||
// Note: Ideally we would fill out additional information instead of leaving capabilities as 'unknown', e.g. examine
|
||||
// if the expression in question could potentially return multiple values and anything else. However, we don't
|
||||
// currently have a good way of determining this, so fill this out more once we do
|
||||
return new ColumnCapabilitiesImpl().setType(outputType);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -47,8 +47,7 @@ public class GroupByQueryEngineV2Test
|
|||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(true)
|
||||
.setDictionaryValuesSorted(true)
|
||||
.setDictionaryValuesUnique(true)
|
||||
.setIsComplete(true);
|
||||
.setDictionaryValuesUnique(true);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
|
@ -63,8 +62,7 @@ public class GroupByQueryEngineV2Test
|
|||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setDictionaryValuesUnique(true)
|
||||
.setIsComplete(true);
|
||||
.setDictionaryValuesUnique(true);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
|
@ -79,8 +77,7 @@ public class GroupByQueryEngineV2Test
|
|||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setIsComplete(true);
|
||||
.setDictionaryValuesUnique(false);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
|
@ -95,8 +92,7 @@ public class GroupByQueryEngineV2Test
|
|||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(true)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setIsComplete(true);
|
||||
.setDictionaryValuesUnique(false);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
|
@ -111,8 +107,7 @@ public class GroupByQueryEngineV2Test
|
|||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setIsComplete(true);
|
||||
.setDictionaryValuesUnique(false);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).anyTimes();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
|
@ -131,8 +126,7 @@ public class GroupByQueryEngineV2Test
|
|||
.setHasMultipleValues(false)
|
||||
.setDictionaryEncoded(false)
|
||||
.setDictionaryValuesSorted(false)
|
||||
.setDictionaryValuesUnique(false)
|
||||
.setIsComplete(true);
|
||||
.setDictionaryValuesUnique(false);
|
||||
EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once();
|
||||
EasyMock.replay(factory);
|
||||
Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM));
|
||||
|
|
|
@ -137,9 +137,8 @@ public class LookupSegmentTest
|
|||
// Note: the "k" column does not actually have multiple values, but the RowBasedStorageAdapter doesn't allow
|
||||
// reporting complete single-valued capabilities. It would be good to change this in the future, so query engines
|
||||
// running on top of lookups can take advantage of singly-valued optimizations.
|
||||
Assert.assertFalse(capabilities.hasMultipleValues());
|
||||
Assert.assertTrue(capabilities.hasMultipleValues().isUnknown());
|
||||
Assert.assertFalse(capabilities.isDictionaryEncoded());
|
||||
Assert.assertFalse(capabilities.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -151,9 +150,8 @@ public class LookupSegmentTest
|
|||
// reporting complete single-valued capabilities. It would be good to change this in the future, so query engines
|
||||
// running on top of lookups can take advantage of singly-valued optimizations.
|
||||
Assert.assertEquals(ValueType.STRING, capabilities.getType());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues());
|
||||
Assert.assertTrue(capabilities.hasMultipleValues().isUnknown());
|
||||
Assert.assertFalse(capabilities.isDictionaryEncoded());
|
||||
Assert.assertFalse(capabilities.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -227,7 +227,7 @@ public class SegmentMetadataQueryTest
|
|||
"placement",
|
||||
new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap1,
|
||||
false,
|
||||
preferedSize1,
|
||||
1,
|
||||
"preferred",
|
||||
|
@ -268,7 +268,7 @@ public class SegmentMetadataQueryTest
|
|||
"placement",
|
||||
new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap2,
|
||||
false,
|
||||
placementSize2,
|
||||
1,
|
||||
null,
|
||||
|
@ -304,7 +304,7 @@ public class SegmentMetadataQueryTest
|
|||
"placement",
|
||||
new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap1 || !mmap2,
|
||||
false,
|
||||
0,
|
||||
0,
|
||||
null,
|
||||
|
@ -372,7 +372,7 @@ public class SegmentMetadataQueryTest
|
|||
"placement",
|
||||
new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap1 || !mmap2,
|
||||
false,
|
||||
0,
|
||||
1,
|
||||
null,
|
||||
|
@ -440,7 +440,7 @@ public class SegmentMetadataQueryTest
|
|||
"placement",
|
||||
new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap1 || !mmap2,
|
||||
false,
|
||||
0,
|
||||
1,
|
||||
null,
|
||||
|
@ -509,7 +509,7 @@ public class SegmentMetadataQueryTest
|
|||
}
|
||||
ColumnAnalysis analysis = new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap1 || !mmap2,
|
||||
false,
|
||||
size1 + size2,
|
||||
1,
|
||||
"preferred",
|
||||
|
@ -530,7 +530,7 @@ public class SegmentMetadataQueryTest
|
|||
}
|
||||
ColumnAnalysis analysis = new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap1 || !mmap2,
|
||||
false,
|
||||
size1 + size2,
|
||||
3,
|
||||
"spot",
|
||||
|
@ -551,7 +551,7 @@ public class SegmentMetadataQueryTest
|
|||
}
|
||||
ColumnAnalysis analysis = new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap1 || !mmap2,
|
||||
false,
|
||||
size1 + size2,
|
||||
9,
|
||||
"automotive",
|
||||
|
@ -637,7 +637,7 @@ public class SegmentMetadataQueryTest
|
|||
"placement",
|
||||
new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap1 || !mmap2,
|
||||
false,
|
||||
0,
|
||||
0,
|
||||
null,
|
||||
|
@ -699,7 +699,7 @@ public class SegmentMetadataQueryTest
|
|||
"placement",
|
||||
new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap1 || !mmap2,
|
||||
false,
|
||||
0,
|
||||
0,
|
||||
null,
|
||||
|
@ -757,7 +757,7 @@ public class SegmentMetadataQueryTest
|
|||
"placement",
|
||||
new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap1 || !mmap2,
|
||||
false,
|
||||
0,
|
||||
0,
|
||||
null,
|
||||
|
@ -815,7 +815,7 @@ public class SegmentMetadataQueryTest
|
|||
"placement",
|
||||
new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap1 || !mmap2,
|
||||
false,
|
||||
0,
|
||||
0,
|
||||
null,
|
||||
|
|
|
@ -102,7 +102,7 @@ public class SegmentMetadataUnionQueryTest extends InitializedNullHandlingTest
|
|||
"placement",
|
||||
new ColumnAnalysis(
|
||||
ValueType.STRING.toString(),
|
||||
!mmap,
|
||||
false,
|
||||
mmap ? 43524 : 43056,
|
||||
1,
|
||||
"preferred",
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue;
|
|||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
|
||||
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
@ -66,7 +67,7 @@ import java.util.concurrent.ThreadLocalRandom;
|
|||
/**
|
||||
*/
|
||||
@RunWith(Parameterized.class)
|
||||
public class IndexMergerV9WithSpatialIndexTest
|
||||
public class IndexMergerV9WithSpatialIndexTest extends InitializedNullHandlingTest
|
||||
{
|
||||
|
||||
public static final int NUM_POINTS = 5000;
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.apache.druid.query.aggregation.FloatSumAggregatorFactory;
|
|||
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
|
||||
import org.apache.druid.segment.column.ColumnHolder;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.incremental.IncrementalIndex;
|
||||
|
@ -152,9 +153,12 @@ public class QueryableIndexColumnCapabilitiesTest extends InitializedNullHandlin
|
|||
Assert.assertTrue(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
// multi-value is unknown unless explicitly set to 'true'
|
||||
Assert.assertTrue(caps.hasMultipleValues().isUnknown());
|
||||
// at index merge or query time we 'complete' the capabilities to take a snapshot of the current state,
|
||||
// coercing any 'UNKNOWN' values to false
|
||||
Assert.assertFalse(ColumnCapabilitiesImpl.snapshot(caps).hasMultipleValues().isMaybeTrue());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
|
||||
caps = MMAP_INDEX.getColumnHolder("d1").getCapabilities();
|
||||
Assert.assertEquals(ValueType.STRING, caps.getType());
|
||||
|
@ -162,9 +166,8 @@ public class QueryableIndexColumnCapabilitiesTest extends InitializedNullHandlin
|
|||
Assert.assertTrue(caps.isDictionaryEncoded());
|
||||
Assert.assertTrue(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -176,9 +179,8 @@ public class QueryableIndexColumnCapabilitiesTest extends InitializedNullHandlin
|
|||
Assert.assertTrue(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertTrue(caps.hasMultipleValues());
|
||||
Assert.assertTrue(caps.hasMultipleValues().isTrue());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
|
||||
caps = MMAP_INDEX.getColumnHolder("d2").getCapabilities();
|
||||
Assert.assertEquals(ValueType.STRING, caps.getType());
|
||||
|
@ -186,9 +188,8 @@ public class QueryableIndexColumnCapabilitiesTest extends InitializedNullHandlin
|
|||
Assert.assertTrue(caps.isDictionaryEncoded());
|
||||
Assert.assertTrue(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertTrue(caps.hasMultipleValues());
|
||||
Assert.assertTrue(caps.hasMultipleValues().isTrue());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -206,8 +207,7 @@ public class QueryableIndexColumnCapabilitiesTest extends InitializedNullHandlin
|
|||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -54,9 +54,8 @@ public class RowBasedColumnSelectorFactoryTest
|
|||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -69,9 +68,8 @@ public class RowBasedColumnSelectorFactoryTest
|
|||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertTrue(caps.hasMultipleValues().isUnknown());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertFalse(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -84,9 +82,8 @@ public class RowBasedColumnSelectorFactoryTest
|
|||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -99,9 +96,8 @@ public class RowBasedColumnSelectorFactoryTest
|
|||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -114,9 +110,8 @@ public class RowBasedColumnSelectorFactoryTest
|
|||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertTrue(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -129,9 +124,8 @@ public class RowBasedColumnSelectorFactoryTest
|
|||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertFalse(caps.hasMultipleValues());
|
||||
Assert.assertTrue(caps.hasMultipleValues().isUnknown());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertFalse(caps.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -332,8 +332,7 @@ public class RowBasedStorageAdapterTest
|
|||
|
||||
final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.FLOAT.name());
|
||||
Assert.assertEquals(ValueType.FLOAT, capabilities.getType());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues());
|
||||
Assert.assertTrue(capabilities.isComplete());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues().isMaybeTrue());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -343,8 +342,7 @@ public class RowBasedStorageAdapterTest
|
|||
|
||||
final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.DOUBLE.name());
|
||||
Assert.assertEquals(ValueType.DOUBLE, capabilities.getType());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues());
|
||||
Assert.assertTrue(capabilities.isComplete());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues().isMaybeTrue());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -354,8 +352,7 @@ public class RowBasedStorageAdapterTest
|
|||
|
||||
final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.LONG.name());
|
||||
Assert.assertEquals(ValueType.LONG, capabilities.getType());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues());
|
||||
Assert.assertTrue(capabilities.isComplete());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues().isMaybeTrue());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -369,8 +366,7 @@ public class RowBasedStorageAdapterTest
|
|||
// Note: unlike numeric types, STRING-typed columns might have multiple values, so they report as incomplete. It
|
||||
// would be good in the future to support some way of changing this, when it is known ahead of time that
|
||||
// multi-valuedness is definitely happening or is definitely impossible.
|
||||
Assert.assertFalse(capabilities.hasMultipleValues());
|
||||
Assert.assertFalse(capabilities.isComplete());
|
||||
Assert.assertTrue(capabilities.hasMultipleValues().isUnknown());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -382,8 +378,7 @@ public class RowBasedStorageAdapterTest
|
|||
|
||||
// Note: unlike numeric types, COMPLEX-typed columns report that they are incomplete.
|
||||
Assert.assertEquals(ValueType.COMPLEX, capabilities.getType());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues());
|
||||
Assert.assertFalse(capabilities.isComplete());
|
||||
Assert.assertTrue(capabilities.hasMultipleValues().isUnknown());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -47,7 +47,7 @@ public class ColumnCapabilitiesImplTest
|
|||
Assert.assertTrue(cc.isDictionaryEncoded());
|
||||
Assert.assertFalse(cc.isRunLengthEncoded());
|
||||
Assert.assertTrue(cc.hasSpatialIndexes());
|
||||
Assert.assertTrue(cc.hasMultipleValues());
|
||||
Assert.assertTrue(cc.hasMultipleValues().isTrue());
|
||||
Assert.assertTrue(cc.hasBitmapIndexes());
|
||||
Assert.assertFalse(cc.isFilterable());
|
||||
}
|
||||
|
@ -72,7 +72,7 @@ public class ColumnCapabilitiesImplTest
|
|||
Assert.assertTrue(cc.isDictionaryEncoded());
|
||||
Assert.assertTrue(cc.isRunLengthEncoded());
|
||||
Assert.assertTrue(cc.hasSpatialIndexes());
|
||||
Assert.assertTrue(cc.hasMultipleValues());
|
||||
Assert.assertTrue(cc.hasMultipleValues().isTrue());
|
||||
Assert.assertTrue(cc.hasBitmapIndexes());
|
||||
Assert.assertFalse(cc.isFilterable());
|
||||
}
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.druid.query.filter.Filter;
|
|||
import org.apache.druid.query.filter.SelectorDimFilter;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import org.apache.druid.segment.column.BitmapIndex;
|
||||
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||
import org.apache.druid.segment.data.CloseableIndexed;
|
||||
import org.apache.druid.segment.data.ConciseBitmapSerdeFactory;
|
||||
|
@ -146,9 +147,9 @@ public class ExtractionDimFilterTest
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues(final String dimension)
|
||||
public ColumnCapabilities.Capable hasMultipleValues(final String dimension)
|
||||
{
|
||||
return true;
|
||||
return ColumnCapabilities.Capable.TRUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -137,9 +137,8 @@ public class IndexedTableJoinableTest
|
|||
Assert.assertEquals(ValueType.STRING, capabilities.getType());
|
||||
Assert.assertTrue(capabilities.isDictionaryEncoded());
|
||||
Assert.assertFalse(capabilities.hasBitmapIndexes());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues().isMaybeTrue());
|
||||
Assert.assertFalse(capabilities.hasSpatialIndexes());
|
||||
Assert.assertTrue(capabilities.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -149,9 +148,8 @@ public class IndexedTableJoinableTest
|
|||
Assert.assertEquals(ValueType.LONG, capabilities.getType());
|
||||
Assert.assertFalse(capabilities.isDictionaryEncoded());
|
||||
Assert.assertFalse(capabilities.hasBitmapIndexes());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues());
|
||||
Assert.assertFalse(capabilities.hasMultipleValues().isMaybeTrue());
|
||||
Assert.assertFalse(capabilities.hasSpatialIndexes());
|
||||
Assert.assertTrue(capabilities.isComplete());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -357,8 +357,7 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
{
|
||||
return new ColumnCapabilitiesImpl().setType(ValueType.STRING)
|
||||
.setHasMultipleValues(true)
|
||||
.setDictionaryEncoded(true)
|
||||
.setIsComplete(true);
|
||||
.setDictionaryEncoded(true);
|
||||
}
|
||||
};
|
||||
final BaseObjectColumnValueSelector selectorImplicit =
|
||||
|
@ -814,9 +813,9 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertTrue(caps.hasMultipleValues());
|
||||
Assert.assertTrue(caps.hasMultipleValues().isUnknown());
|
||||
Assert.assertTrue(caps.hasMultipleValues().isMaybeTrue());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertFalse(caps.isComplete());
|
||||
|
||||
caps = Z_CONCAT_X.capabilities("expr");
|
||||
Assert.assertEquals(ValueType.STRING, caps.getType());
|
||||
|
@ -824,8 +823,8 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest
|
|||
Assert.assertFalse(caps.isDictionaryEncoded());
|
||||
Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue());
|
||||
Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue());
|
||||
Assert.assertTrue(caps.hasMultipleValues());
|
||||
Assert.assertTrue(caps.hasMultipleValues().isUnknown());
|
||||
Assert.assertTrue(caps.hasMultipleValues().isMaybeTrue());
|
||||
Assert.assertFalse(caps.hasSpatialIndexes());
|
||||
Assert.assertFalse(caps.isComplete());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -417,7 +417,7 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest
|
|||
@Override
|
||||
public ColumnCapabilities capabilities(String columnName)
|
||||
{
|
||||
return new ColumnCapabilitiesImpl().setType(ValueType.LONG);
|
||||
return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
Loading…
Reference in New Issue