add VirtualColumns.findEquivalent and VirtualColumn.EquivalenceKey (#17084)

This commit is contained in:
Clint Wylie 2024-09-17 13:17:44 -07:00 committed by GitHub
parent 490211f2b1
commit a93546d493
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
6 changed files with 336 additions and 149 deletions

View File

@ -21,6 +21,7 @@ package org.apache.druid.segment;
import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeInfo;
import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode;
import org.apache.druid.java.util.common.Cacheable; import org.apache.druid.java.util.common.Cacheable;
import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.ColumnIndexSelector; import org.apache.druid.query.filter.ColumnIndexSelector;
@ -336,4 +337,22 @@ public interface VirtualColumn extends Cacheable
{ {
return NoIndexesColumnIndexSupplier.getInstance(); return NoIndexesColumnIndexSupplier.getInstance();
} }
/**
* Returns a key used for "equivalence" comparisons, for checking if some virtual column is equivalent to some other
* virtual column, regardless of the output name. If this method returns null, it does not participate in equivalence
* comparisons.
*
* @see VirtualColumns#findEquivalent(VirtualColumn)
*/
@Nullable
default EquivalenceKey getEquivalanceKey()
{
return null;
}
@SubclassesMustOverrideEqualsAndHashCode
interface EquivalenceKey
{
}
} }

View File

@ -26,6 +26,7 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Strings; import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import org.apache.druid.java.util.common.Cacheable; import org.apache.druid.java.util.common.Cacheable;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
@ -131,10 +132,13 @@ public class VirtualColumns implements Cacheable
// For equals, hashCode, toString, and serialization: // For equals, hashCode, toString, and serialization:
private final List<VirtualColumn> virtualColumns; private final List<VirtualColumn> virtualColumns;
private final List<String> virtualColumnNames; private final List<String> virtualColumnNames;
// For equivalence
private final Map<VirtualColumn.EquivalenceKey, VirtualColumn> equivalence;
// For getVirtualColumn: // For getVirtualColumn:
private final Map<String, VirtualColumn> withDotSupport; private final Map<String, VirtualColumn> withDotSupport;
private final Map<String, VirtualColumn> withoutDotSupport; private final Map<String, VirtualColumn> withoutDotSupport;
private final boolean hasNoDotColumns;
private VirtualColumns( private VirtualColumns(
List<VirtualColumn> virtualColumns, List<VirtualColumn> virtualColumns,
@ -146,10 +150,15 @@ public class VirtualColumns implements Cacheable
this.withDotSupport = withDotSupport; this.withDotSupport = withDotSupport;
this.withoutDotSupport = withoutDotSupport; this.withoutDotSupport = withoutDotSupport;
this.virtualColumnNames = new ArrayList<>(virtualColumns.size()); this.virtualColumnNames = new ArrayList<>(virtualColumns.size());
this.hasNoDotColumns = withDotSupport.isEmpty();
this.equivalence = Maps.newHashMapWithExpectedSize(virtualColumns.size());
for (VirtualColumn virtualColumn : virtualColumns) { for (VirtualColumn virtualColumn : virtualColumns) {
detectCycles(virtualColumn, null); detectCycles(virtualColumn, null);
virtualColumnNames.add(virtualColumn.getOutputName()); virtualColumnNames.add(virtualColumn.getOutputName());
VirtualColumn.EquivalenceKey key = virtualColumn.getEquivalanceKey();
if (key != null) {
equivalence.put(key, virtualColumn);
}
} }
} }
@ -172,10 +181,23 @@ public class VirtualColumns implements Cacheable
if (vc != null) { if (vc != null) {
return vc; return vc;
} }
if (hasNoDotColumns) {
return null;
}
final String baseColumnName = splitColumnName(columnName).lhs; final String baseColumnName = splitColumnName(columnName).lhs;
return withDotSupport.get(baseColumnName); return withDotSupport.get(baseColumnName);
} }
/**
* Check if a virtual column is already defined which is the same as some other virtual column, ignoring output name,
* returning that virtual column if it exists, or null if there is no equivalent virtual column.
*/
@Nullable
public VirtualColumn findEquivalent(VirtualColumn virtualColumn)
{
return equivalence.get(virtualColumn.getEquivalanceKey());
}
/** /**
* Get the {@link ColumnIndexSupplier} of the specified virtual column, with the assistance of a * Get the {@link ColumnIndexSupplier} of the specified virtual column, with the assistance of a
* {@link ColumnSelector} to allow reading things from segments. If the column does not have indexes this method * {@link ColumnSelector} to allow reading things from segments. If the column does not have indexes this method

View File

@ -61,9 +61,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
private static final Logger log = new Logger(ExpressionVirtualColumn.class); private static final Logger log = new Logger(ExpressionVirtualColumn.class);
private final String name; private final String name;
private final String expression; private final Expression expression;
@Nullable
private final ColumnType outputType;
private final Supplier<Expr> parsedExpression; private final Supplier<Expr> parsedExpression;
private final Supplier<byte[]> cacheKey; private final Supplier<byte[]> cacheKey;
@ -126,8 +124,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
) )
{ {
this.name = Preconditions.checkNotNull(name, "name"); this.name = Preconditions.checkNotNull(name, "name");
this.expression = Preconditions.checkNotNull(expression, "expression"); this.expression = new Expression(Preconditions.checkNotNull(expression, "expression"), outputType);
this.outputType = outputType;
this.parsedExpression = parsedExpression; this.parsedExpression = parsedExpression;
this.cacheKey = makeCacheKeySupplier(); this.cacheKey = makeCacheKeySupplier();
} }
@ -142,14 +139,14 @@ public class ExpressionVirtualColumn implements VirtualColumn
@JsonProperty @JsonProperty
public String getExpression() public String getExpression()
{ {
return expression; return expression.expressionString;
} }
@Nullable @Nullable
@JsonProperty @JsonProperty
public ColumnType getOutputType() public ColumnType getOutputType()
{ {
return outputType; return expression.outputType;
} }
@JsonIgnore @JsonIgnore
@ -273,7 +270,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
ColumnIndexSelector columnIndexSelector ColumnIndexSelector columnIndexSelector
) )
{ {
return getParsedExpression().get().asColumnIndexSupplier(columnIndexSelector, outputType); return getParsedExpression().get().asColumnIndexSupplier(columnIndexSelector, expression.outputType);
} }
@Override @Override
@ -283,7 +280,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
// are unable to compute the output type of the expression, either due to incomplete type information of the // are unable to compute the output type of the expression, either due to incomplete type information of the
// inputs or because of unimplemented methods on expression implementations themselves, or, because a // inputs or because of unimplemented methods on expression implementations themselves, or, because a
// ColumnInspector is not available // ColumnInspector is not available
final ColumnType outputType = expression.outputType;
if (ExpressionProcessing.processArraysAsMultiValueStrings() && outputType != null && outputType.isArray()) { if (ExpressionProcessing.processArraysAsMultiValueStrings() && outputType != null && outputType.isArray()) {
return new ColumnCapabilitiesImpl().setType(ColumnType.STRING).setHasMultipleValues(true); return new ColumnCapabilitiesImpl().setType(ColumnType.STRING).setHasMultipleValues(true);
} }
@ -299,6 +296,8 @@ public class ExpressionVirtualColumn implements VirtualColumn
return inspector.getColumnCapabilities(parsedExpression.get().getBindingIfIdentifier()); return inspector.getColumnCapabilities(parsedExpression.get().getBindingIfIdentifier());
} }
final ColumnType outputType = expression.outputType;
final ExpressionPlan plan = ExpressionPlanner.plan(inspector, parsedExpression.get()); final ExpressionPlan plan = ExpressionPlanner.plan(inspector, parsedExpression.get());
final ColumnCapabilities inferred = plan.inferColumnCapabilities(outputType); final ColumnCapabilities inferred = plan.inferColumnCapabilities(outputType);
// if we can infer the column capabilities from the expression plan, then use that // if we can infer the column capabilities from the expression plan, then use that
@ -311,14 +310,14 @@ public class ExpressionVirtualColumn implements VirtualColumn
log.warn( log.warn(
"Projected output type %s of expression %s does not match provided type %s", "Projected output type %s of expression %s does not match provided type %s",
inferred.asTypeString(), inferred.asTypeString(),
expression, expression.expressionString,
outputType outputType
); );
} else { } else {
log.debug( log.debug(
"Projected output type %s of expression %s does not match provided type %s", "Projected output type %s of expression %s does not match provided type %s",
inferred.asTypeString(), inferred.asTypeString(),
expression, expression.expressionString,
outputType outputType
); );
} }
@ -348,6 +347,13 @@ public class ExpressionVirtualColumn implements VirtualColumn
return cacheKey.get(); return cacheKey.get();
} }
@Nullable
@Override
public EquivalenceKey getEquivalanceKey()
{
return expression;
}
@Override @Override
public boolean equals(final Object o) public boolean equals(final Object o)
{ {
@ -359,14 +365,13 @@ public class ExpressionVirtualColumn implements VirtualColumn
} }
final ExpressionVirtualColumn that = (ExpressionVirtualColumn) o; final ExpressionVirtualColumn that = (ExpressionVirtualColumn) o;
return Objects.equals(name, that.name) && return Objects.equals(name, that.name) &&
Objects.equals(expression, that.expression) && Objects.equals(expression, that.expression);
Objects.equals(outputType, that.outputType);
} }
@Override @Override
public int hashCode() public int hashCode()
{ {
return Objects.hash(name, expression, outputType); return Objects.hash(name, expression);
} }
@Override @Override
@ -374,8 +379,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
{ {
return "ExpressionVirtualColumn{" + return "ExpressionVirtualColumn{" +
"name='" + name + '\'' + "name='" + name + '\'' +
", expression='" + expression + '\'' + ", expression=" + expression +
", outputType=" + outputType +
'}'; '}';
} }
@ -389,10 +393,10 @@ public class ExpressionVirtualColumn implements VirtualColumn
final ColumnCapabilities baseCapabilities = final ColumnCapabilities baseCapabilities =
inspector.getColumnCapabilities(parsedExpression.get().getBindingIfIdentifier()); inspector.getColumnCapabilities(parsedExpression.get().getBindingIfIdentifier());
if (outputType == null) { if (expression.outputType == null) {
// No desired output type. Anything from the source is fine. // No desired output type. Anything from the source is fine.
return true; return true;
} else if (baseCapabilities != null && outputType.equals(baseCapabilities.toColumnType())) { } else if (baseCapabilities != null && expression.outputType.equals(baseCapabilities.toColumnType())) {
// Desired output type matches the type from the source. // Desired output type matches the type from the source.
return true; return true;
} }
@ -408,10 +412,57 @@ public class ExpressionVirtualColumn implements VirtualColumn
.appendString(name) .appendString(name)
.appendCacheable(parsedExpression.get()); .appendCacheable(parsedExpression.get());
if (outputType != null) { if (expression.outputType != null) {
builder.appendString(outputType.toString()); builder.appendString(expression.outputType.toString());
} }
return builder.build(); return builder.build();
}); });
} }
/**
* {@link VirtualColumn.EquivalenceKey} for expressions. Note that this does not check true equivalence of
* expressions, for example it will not currently consider something like 'a + b' equivalent to 'b + a'. This is ok
* for current uses of this functionality, but in theory we could push down equivalence to the parsed expression
* instead of checking for an identical string expression, it would just be a lot more expensive.
*/
private static final class Expression implements EquivalenceKey
{
private final String expressionString;
@Nullable
private final ColumnType outputType;
private Expression(String expression, @Nullable ColumnType outputType)
{
this.expressionString = expression;
this.outputType = outputType;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Expression that = (Expression) o;
return Objects.equals(expressionString, that.expressionString) && Objects.equals(outputType, that.outputType);
}
@Override
public int hashCode()
{
return Objects.hash(expressionString, outputType);
}
@Override
public String toString()
{
return "Expression{" +
"expression='" + expressionString + '\'' +
", outputType=" + outputType +
'}';
}
}
} }

View File

@ -93,27 +93,23 @@ import java.util.Set;
* nested fields ({@link NestedFieldDictionaryEncodedColumn}) including using * nested fields ({@link NestedFieldDictionaryEncodedColumn}) including using
* their indexes. * their indexes.
* <p> * <p>
* This virtual column is used for the SQL operators JSON_VALUE (if {@link #processFromRaw} is set to false) or * This virtual column is used for the SQL operators JSON_VALUE (if {@link #isProcessFromRaw()} is set to false) or
* JSON_QUERY (if it is true), and accepts 'JSONPath' or 'jq' syntax string representations of paths, or a parsed * JSON_QUERY (if it is true), and accepts 'JSONPath' or 'jq' syntax string representations of paths, or a parsed
* list of {@link NestedPathPart} in order to determine what should be selected from the column. * list of {@link NestedPathPart} in order to determine what should be selected from the column.
* <p> * <p>
* Type information for nested fields is completely absent in the SQL planner, so it guesses the best it can to set * Type information for nested fields is completely absent in the SQL planner, so it guesses the best it can to set
* {@link #expectedType} from the context of how something is being used, e.g. an aggregators default type or an * {@link #getExpectedType()} from the context of how something is being used, e.g. an aggregators default type or an
* explicit cast, or, if using the 'RETURNING' syntax which explicitly specifies type. This might not be the same as * explicit cast, or, if using the 'RETURNING' syntax which explicitly specifies type. This might not be the same as
* if it had actual type information, but, we try to stick with whatever we chose there to do the best we can for now. * if it had actual type information, but, we try to stick with whatever we chose there to do the best we can for now.
* <p> * <p>
* Since {@link #capabilities(ColumnInspector, String)} is determined by the {@link #expectedType}, the results will * Since {@link #capabilities(ColumnInspector, String)} is determined by the {@link #getExpectedType()}, the results
* be best effor cast to the expected type if the column is not natively the expected type so that this column can * will be best effor cast to the expected type if the column is not natively the expected type so that this column can
* fulfill the contract of the type of selector that is likely to be created to read this column. * fulfill the contract of the type of selector that is likely to be created to read this column.
*/ */
public class NestedFieldVirtualColumn implements VirtualColumn public class NestedFieldVirtualColumn implements VirtualColumn
{ {
private final String columnName;
private final String outputName; private final String outputName;
@Nullable private final NestedFieldSpec fieldSpec;
private final ColumnType expectedType;
private final List<NestedPathPart> parts;
private final boolean processFromRaw;
private final boolean hasNegativeArrayIndex; private final boolean hasNegativeArrayIndex;
@ -128,22 +124,21 @@ public class NestedFieldVirtualColumn implements VirtualColumn
@JsonProperty("useJqSyntax") @Nullable Boolean useJqSyntax @JsonProperty("useJqSyntax") @Nullable Boolean useJqSyntax
) )
{ {
this.columnName = columnName;
this.outputName = outputName; this.outputName = outputName;
if (path != null) { if (path != null) {
Preconditions.checkArgument(parts == null, "Cannot define both 'path' and 'pathParts'"); Preconditions.checkArgument(parts == null, "Cannot define both 'path' and 'pathParts'");
} else if (parts == null) { } else if (parts == null) {
throw new IllegalArgumentException("Must define exactly one of 'path' or 'pathParts'"); throw new IllegalArgumentException("Must define exactly one of 'path' or 'pathParts'");
} }
final List<NestedPathPart> pathParts;
if (parts != null) { if (parts != null) {
this.parts = parts; pathParts = parts;
} else { } else {
boolean isInputJq = useJqSyntax != null && useJqSyntax; boolean isInputJq = useJqSyntax != null && useJqSyntax;
this.parts = isInputJq ? NestedPathFinder.parseJqPath(path) : NestedPathFinder.parseJsonPath(path); pathParts = isInputJq ? NestedPathFinder.parseJqPath(path) : NestedPathFinder.parseJsonPath(path);
} }
boolean hasNegative = false; boolean hasNegative = false;
for (NestedPathPart part : this.parts) { for (NestedPathPart part : pathParts) {
if (part instanceof NestedPathArrayElement) { if (part instanceof NestedPathArrayElement) {
NestedPathArrayElement elementPart = (NestedPathArrayElement) part; NestedPathArrayElement elementPart = (NestedPathArrayElement) part;
if (elementPart.getIndex() < 0) { if (elementPart.getIndex() < 0) {
@ -153,8 +148,12 @@ public class NestedFieldVirtualColumn implements VirtualColumn
} }
} }
this.hasNegativeArrayIndex = hasNegative; this.hasNegativeArrayIndex = hasNegative;
this.expectedType = expectedType; this.fieldSpec = new NestedFieldSpec(
this.processFromRaw = processFromRaw == null ? false : processFromRaw; columnName,
expectedType,
pathParts,
processFromRaw != null && processFromRaw
);
} }
@VisibleForTesting @VisibleForTesting
@ -181,12 +180,12 @@ public class NestedFieldVirtualColumn implements VirtualColumn
@Override @Override
public byte[] getCacheKey() public byte[] getCacheKey()
{ {
final String partsString = NestedPathFinder.toNormalizedJsonPath(parts); final String partsString = NestedPathFinder.toNormalizedJsonPath(fieldSpec.parts);
return new CacheKeyBuilder(VirtualColumnCacheHelper.CACHE_TYPE_ID_USER_DEFINED).appendString("nested-field") return new CacheKeyBuilder(VirtualColumnCacheHelper.CACHE_TYPE_ID_USER_DEFINED).appendString("nested-field")
.appendString(outputName) .appendString(outputName)
.appendString(columnName) .appendString(fieldSpec.columnName)
.appendString(partsString) .appendString(partsString)
.appendBoolean(processFromRaw) .appendBoolean(fieldSpec.processFromRaw)
.build(); .build();
} }
@ -200,25 +199,25 @@ public class NestedFieldVirtualColumn implements VirtualColumn
@JsonProperty @JsonProperty
public String getColumnName() public String getColumnName()
{ {
return columnName; return fieldSpec.columnName;
} }
@JsonProperty("pathParts") @JsonProperty("pathParts")
public List<NestedPathPart> getPathParts() public List<NestedPathPart> getPathParts()
{ {
return parts; return fieldSpec.parts;
} }
@JsonProperty @JsonProperty
public ColumnType getExpectedType() public ColumnType getExpectedType()
{ {
return expectedType; return fieldSpec.expectedType;
} }
@JsonProperty @JsonProperty
public boolean isProcessFromRaw() public boolean isProcessFromRaw()
{ {
return processFromRaw; return fieldSpec.processFromRaw;
} }
@Override @Override
@ -241,13 +240,13 @@ public class NestedFieldVirtualColumn implements VirtualColumn
) )
{ {
// this column value selector is used for realtime queries, so we always process StructuredData // this column value selector is used for realtime queries, so we always process StructuredData
final ColumnValueSelector<?> baseSelector = factory.makeColumnValueSelector(this.columnName); final ColumnValueSelector<?> baseSelector = factory.makeColumnValueSelector(fieldSpec.columnName);
// processFromRaw is true that means JSON_QUERY, which can return partial results, otherwise this virtual column // processFromRaw is true that means JSON_QUERY, which can return partial results, otherwise this virtual column
// is JSON_VALUE which only returns literals, so use the literal value selector instead // is JSON_VALUE which only returns literals, so use the literal value selector instead
return processFromRaw return fieldSpec.processFromRaw
? new RawFieldColumnSelector(baseSelector, parts) ? new RawFieldColumnSelector(baseSelector, fieldSpec.parts)
: new RawFieldLiteralColumnValueSelector(baseSelector, parts); : new RawFieldLiteralColumnValueSelector(baseSelector, fieldSpec.parts);
} }
@Nullable @Nullable
@ -258,7 +257,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
ReadableOffset offset ReadableOffset offset
) )
{ {
ColumnHolder holder = columnSelector.getColumnHolder(columnName); ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName);
if (holder == null) { if (holder == null) {
// column doesn't exist // column doesn't exist
return dimensionSpec.decorate(DimensionSelector.constant(null, dimensionSpec.getExtractionFn())); return dimensionSpec.decorate(DimensionSelector.constant(null, dimensionSpec.getExtractionFn()));
@ -283,11 +282,11 @@ public class NestedFieldVirtualColumn implements VirtualColumn
BaseColumn theColumn = holder.getColumn(); BaseColumn theColumn = holder.getColumn();
if (theColumn instanceof NestedDataComplexColumn) { if (theColumn instanceof NestedDataComplexColumn) {
final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn; final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn;
return column.makeDimensionSelector(parts, offset, extractionFn); return column.makeDimensionSelector(fieldSpec.parts, offset, extractionFn);
} }
// not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts // not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts
if (parts.isEmpty()) { if (fieldSpec.parts.isEmpty()) {
// dictionary encoded columns do not typically implement the value selector methods (getLong, getDouble, getFloat) // dictionary encoded columns do not typically implement the value selector methods (getLong, getDouble, getFloat)
// nothing *should* be using a dimension selector to call the numeric getters, but just in case... wrap their // nothing *should* be using a dimension selector to call the numeric getters, but just in case... wrap their
// selector in a "best effort" casting selector to implement them // selector in a "best effort" casting selector to implement them
@ -303,10 +302,10 @@ public class NestedFieldVirtualColumn implements VirtualColumn
); );
} }
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { if (fieldSpec.parts.size() == 1 && fieldSpec.parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) {
final VariantColumn<?> arrayColumn = (VariantColumn<?>) theColumn; final VariantColumn<?> arrayColumn = (VariantColumn<?>) theColumn;
ColumnValueSelector<?> arraySelector = arrayColumn.makeColumnValueSelector(offset); ColumnValueSelector<?> arraySelector = arrayColumn.makeColumnValueSelector(offset);
final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); final int elementNumber = ((NestedPathArrayElement) fieldSpec.parts.get(0)).getIndex();
if (elementNumber < 0) { if (elementNumber < 0) {
throw new IAE("Cannot make array element selector, negative array index not supported"); throw new IAE("Cannot make array element selector, negative array index not supported");
} }
@ -351,13 +350,13 @@ public class NestedFieldVirtualColumn implements VirtualColumn
ReadableOffset offset ReadableOffset offset
) )
{ {
ColumnHolder holder = columnSelector.getColumnHolder(this.columnName); ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName);
if (holder == null) { if (holder == null) {
return NilColumnValueSelector.instance(); return NilColumnValueSelector.instance();
} }
BaseColumn theColumn = holder.getColumn(); BaseColumn theColumn = holder.getColumn();
if (processFromRaw || hasNegativeArrayIndex) { if (fieldSpec.processFromRaw || hasNegativeArrayIndex) {
// if the path has negative array elements, or has set the flag to process 'raw' values explicitly (JSON_QUERY), // if the path has negative array elements, or has set the flag to process 'raw' values explicitly (JSON_QUERY),
// then we use the 'raw' processing of the RawFieldColumnSelector/RawFieldLiteralColumnValueSelector created // then we use the 'raw' processing of the RawFieldColumnSelector/RawFieldLiteralColumnValueSelector created
// with the column selector factory instead of using the optimized nested field column // with the column selector factory instead of using the optimized nested field column
@ -367,11 +366,11 @@ public class NestedFieldVirtualColumn implements VirtualColumn
// "JSON_VALUE", which only returns literals, on a NestedDataComplexColumn, so we can use the fields value selector // "JSON_VALUE", which only returns literals, on a NestedDataComplexColumn, so we can use the fields value selector
if (theColumn instanceof NestedDataComplexColumn) { if (theColumn instanceof NestedDataComplexColumn) {
final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn; final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn;
return column.makeColumnValueSelector(parts, offset); return column.makeColumnValueSelector(fieldSpec.parts, offset);
} }
// not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts // not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts
if (parts.isEmpty()) { if (fieldSpec.parts.isEmpty()) {
// dictionary encoded columns do not typically implement the value selector methods (getLong, getDouble, getFloat) // dictionary encoded columns do not typically implement the value selector methods (getLong, getDouble, getFloat)
// so we want to wrap their selector in a "best effort" casting selector to implement them // so we want to wrap their selector in a "best effort" casting selector to implement them
if (theColumn instanceof DictionaryEncodedColumn && !(theColumn instanceof VariantColumn)) { if (theColumn instanceof DictionaryEncodedColumn && !(theColumn instanceof VariantColumn)) {
@ -383,10 +382,10 @@ public class NestedFieldVirtualColumn implements VirtualColumn
return theColumn.makeColumnValueSelector(offset); return theColumn.makeColumnValueSelector(offset);
} }
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { if (fieldSpec.parts.size() == 1 && fieldSpec.parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) {
final VariantColumn<?> arrayColumn = (VariantColumn<?>) theColumn; final VariantColumn<?> arrayColumn = (VariantColumn<?>) theColumn;
ColumnValueSelector<?> arraySelector = arrayColumn.makeColumnValueSelector(offset); ColumnValueSelector<?> arraySelector = arrayColumn.makeColumnValueSelector(offset);
final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); final int elementNumber = ((NestedPathArrayElement) fieldSpec.parts.get(0)).getIndex();
if (elementNumber < 0) { if (elementNumber < 0) {
throw new IAE("Cannot make array element selector, negative array index not supported"); throw new IAE("Cannot make array element selector, negative array index not supported");
} }
@ -466,7 +465,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
ReadableVectorOffset offset ReadableVectorOffset offset
) )
{ {
ColumnHolder holder = columnSelector.getColumnHolder(columnName); ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName);
if (holder == null) { if (holder == null) {
return dimensionSpec.decorate(NilVectorSelector.create(offset)); return dimensionSpec.decorate(NilVectorSelector.create(offset));
} }
@ -482,11 +481,11 @@ public class NestedFieldVirtualColumn implements VirtualColumn
BaseColumn theColumn = holder.getColumn(); BaseColumn theColumn = holder.getColumn();
if (theColumn instanceof NestedDataComplexColumn) { if (theColumn instanceof NestedDataComplexColumn) {
final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn; final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn;
return column.makeSingleValueDimensionVectorSelector(parts, offset); return column.makeSingleValueDimensionVectorSelector(fieldSpec.parts, offset);
} }
// not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts // not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts
if (parts.isEmpty()) { if (fieldSpec.parts.isEmpty()) {
// we will not end up here unless underlying column capabilities lied about something being dictionary encoded... // we will not end up here unless underlying column capabilities lied about something being dictionary encoded...
// so no need for magic casting like nonvectorized engine // so no need for magic casting like nonvectorized engine
return ((DictionaryEncodedColumn) theColumn).makeSingleValueDimensionVectorSelector(offset); return ((DictionaryEncodedColumn) theColumn).makeSingleValueDimensionVectorSelector(offset);
@ -505,7 +504,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
ReadableVectorOffset offset ReadableVectorOffset offset
) )
{ {
ColumnHolder holder = columnSelector.getColumnHolder(this.columnName); ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName);
if (holder == null) { if (holder == null) {
return NilVectorSelector.create(offset); return NilVectorSelector.create(offset);
} }
@ -514,80 +513,80 @@ public class NestedFieldVirtualColumn implements VirtualColumn
if (column instanceof NestedDataComplexColumn) { if (column instanceof NestedDataComplexColumn) {
final NestedDataComplexColumn complexColumn = (NestedDataComplexColumn) column; final NestedDataComplexColumn complexColumn = (NestedDataComplexColumn) column;
if (processFromRaw) { if (fieldSpec.processFromRaw) {
// processFromRaw is true, that means JSON_QUERY, which can return partial results, otherwise this virtual column // processFromRaw is true, that means JSON_QUERY, which can return partial results, otherwise this virtual column
// is JSON_VALUE which only returns literals, so we can use the nested columns value selector // is JSON_VALUE which only returns literals, so we can use the nested columns value selector
return new RawFieldVectorObjectSelector(complexColumn.makeVectorObjectSelector(offset), parts); return new RawFieldVectorObjectSelector(complexColumn.makeVectorObjectSelector(offset), fieldSpec.parts);
} }
Set<ColumnType> types = complexColumn.getColumnTypes(parts); Set<ColumnType> types = complexColumn.getColumnTypes(fieldSpec.parts);
ColumnType leastRestrictiveType = null; ColumnType leastRestrictiveType = null;
if (types != null) { if (types != null) {
for (ColumnType type : types) { for (ColumnType type : types) {
leastRestrictiveType = ColumnType.leastRestrictiveType(leastRestrictiveType, type); leastRestrictiveType = ColumnType.leastRestrictiveType(leastRestrictiveType, type);
} }
} }
if (leastRestrictiveType != null && leastRestrictiveType.isNumeric() && !Types.isNumeric(expectedType)) { if (leastRestrictiveType != null && leastRestrictiveType.isNumeric() && !Types.isNumeric(fieldSpec.expectedType)) {
return ExpressionVectorSelectors.castValueSelectorToObject( return ExpressionVectorSelectors.castValueSelectorToObject(
offset, offset,
columnName, columnName,
complexColumn.makeVectorValueSelector(parts, offset), complexColumn.makeVectorValueSelector(fieldSpec.parts, offset),
leastRestrictiveType, leastRestrictiveType,
expectedType == null ? ColumnType.STRING : expectedType fieldSpec.expectedType == null ? ColumnType.STRING : fieldSpec.expectedType
); );
} }
final VectorObjectSelector objectSelector = complexColumn.makeVectorObjectSelector(parts, offset); final VectorObjectSelector objectSelector = complexColumn.makeVectorObjectSelector(fieldSpec.parts, offset);
if (leastRestrictiveType != null && if (leastRestrictiveType != null &&
leastRestrictiveType.isArray() && leastRestrictiveType.isArray() &&
expectedType != null && fieldSpec.expectedType != null &&
!expectedType.isArray() !fieldSpec.expectedType.isArray()
) { ) {
final ExpressionType elementType = ExpressionType.fromColumnTypeStrict(leastRestrictiveType.getElementType()); final ExpressionType elementType = ExpressionType.fromColumnTypeStrict(leastRestrictiveType.getElementType());
final ExpressionType castTo = ExpressionType.fromColumnTypeStrict(expectedType); final ExpressionType castTo = ExpressionType.fromColumnTypeStrict(fieldSpec.expectedType);
return makeVectorArrayToScalarObjectSelector(offset, objectSelector, elementType, castTo); return makeVectorArrayToScalarObjectSelector(offset, objectSelector, elementType, castTo);
} }
return objectSelector; return objectSelector;
} }
// not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts // not a nested column, but we can still do stuff if the path is the 'root', indicated by an empty path parts
if (parts.isEmpty()) { if (fieldSpec.parts.isEmpty()) {
ColumnCapabilities capabilities = holder.getCapabilities(); ColumnCapabilities capabilities = holder.getCapabilities();
// expectedType shouldn't possibly be null if we are being asked for an object selector and the underlying column // expectedType shouldn't possibly be null if we are being asked for an object selector and the underlying column
// is numeric, else we would have been asked for a value selector // is numeric, else we would have been asked for a value selector
Preconditions.checkArgument( Preconditions.checkArgument(
expectedType != null, fieldSpec.expectedType != null,
"Asked for a VectorObjectSelector on a numeric column, 'expectedType' must not be null" "Asked for a VectorObjectSelector on a numeric column, 'expectedType' must not be null"
); );
if (capabilities.isNumeric()) { if (capabilities.isNumeric()) {
return ExpressionVectorSelectors.castValueSelectorToObject( return ExpressionVectorSelectors.castValueSelectorToObject(
offset, offset,
this.columnName, fieldSpec.columnName,
column.makeVectorValueSelector(offset), column.makeVectorValueSelector(offset),
capabilities.toColumnType(), capabilities.toColumnType(),
expectedType fieldSpec.expectedType
); );
} }
// if the underlying column is array typed, the vector object selector it spits out will homogenize stuff to // if the underlying column is array typed, the vector object selector it spits out will homogenize stuff to
// make all of the objects a consistent type, which is typically a good thing, but if we are doing mixed type // make all of the objects a consistent type, which is typically a good thing, but if we are doing mixed type
// stuff and expect the output type to be scalar typed, then we should coerce things to only extract the scalars // stuff and expect the output type to be scalar typed, then we should coerce things to only extract the scalars
if (capabilities.isArray() && !expectedType.isArray()) { if (capabilities.isArray() && !fieldSpec.expectedType.isArray()) {
final VectorObjectSelector delegate = column.makeVectorObjectSelector(offset); final VectorObjectSelector delegate = column.makeVectorObjectSelector(offset);
final ExpressionType elementType = ExpressionType.fromColumnTypeStrict(capabilities.getElementType()); final ExpressionType elementType = ExpressionType.fromColumnTypeStrict(capabilities.getElementType());
final ExpressionType castTo = ExpressionType.fromColumnTypeStrict(expectedType); final ExpressionType castTo = ExpressionType.fromColumnTypeStrict(fieldSpec.expectedType);
return makeVectorArrayToScalarObjectSelector(offset, delegate, elementType, castTo); return makeVectorArrayToScalarObjectSelector(offset, delegate, elementType, castTo);
} }
return column.makeVectorObjectSelector(offset); return column.makeVectorObjectSelector(offset);
} }
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && column instanceof VariantColumn) { if (fieldSpec.parts.size() == 1 && fieldSpec.parts.get(0) instanceof NestedPathArrayElement && column instanceof VariantColumn) {
final VariantColumn<?> arrayColumn = (VariantColumn<?>) column; final VariantColumn<?> arrayColumn = (VariantColumn<?>) column;
final ExpressionType elementType = ExpressionType.fromColumnTypeStrict( final ExpressionType elementType = ExpressionType.fromColumnTypeStrict(
arrayColumn.getLogicalType().isArray() ? arrayColumn.getLogicalType().getElementType() : arrayColumn.getLogicalType() arrayColumn.getLogicalType().isArray() ? arrayColumn.getLogicalType().getElementType() : arrayColumn.getLogicalType()
); );
final ExpressionType castTo = expectedType == null final ExpressionType castTo = fieldSpec.expectedType == null
? ExpressionType.STRING ? ExpressionType.STRING
: ExpressionType.fromColumnTypeStrict(expectedType); : ExpressionType.fromColumnTypeStrict(fieldSpec.expectedType);
VectorObjectSelector arraySelector = arrayColumn.makeVectorObjectSelector(offset); VectorObjectSelector arraySelector = arrayColumn.makeVectorObjectSelector(offset);
final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); final int elementNumber = ((NestedPathArrayElement) fieldSpec.parts.get(0)).getIndex();
if (elementNumber < 0) { if (elementNumber < 0) {
throw new IAE("Cannot make array element selector, negative array index not supported"); throw new IAE("Cannot make array element selector, negative array index not supported");
} }
@ -646,17 +645,17 @@ public class NestedFieldVirtualColumn implements VirtualColumn
ReadableVectorOffset offset ReadableVectorOffset offset
) )
{ {
ColumnHolder holder = columnSelector.getColumnHolder(this.columnName); ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName);
if (holder == null) { if (holder == null) {
return NilVectorSelector.create(offset); return NilVectorSelector.create(offset);
} }
BaseColumn theColumn = holder.getColumn(); BaseColumn theColumn = holder.getColumn();
if (!(theColumn instanceof NestedDataComplexColumn)) { if (!(theColumn instanceof NestedDataComplexColumn)) {
if (parts.isEmpty()) { if (fieldSpec.parts.isEmpty()) {
if (theColumn instanceof DictionaryEncodedColumn) { if (theColumn instanceof DictionaryEncodedColumn) {
final VectorObjectSelector delegate = theColumn.makeVectorObjectSelector(offset); final VectorObjectSelector delegate = theColumn.makeVectorObjectSelector(offset);
if (expectedType != null && expectedType.is(ValueType.LONG)) { if (fieldSpec.expectedType != null && fieldSpec.expectedType.is(ValueType.LONG)) {
return new BaseLongVectorValueSelector(offset) return new BaseLongVectorValueSelector(offset)
{ {
private int currentOffsetId = ReadableVectorInspector.NULL_ID; private int currentOffsetId = ReadableVectorInspector.NULL_ID;
@ -701,7 +700,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
} }
} }
}; };
} else if (expectedType != null && expectedType.is(ValueType.FLOAT)) { } else if (fieldSpec.expectedType != null && fieldSpec.expectedType.is(ValueType.FLOAT)) {
return new BaseFloatVectorValueSelector(offset) return new BaseFloatVectorValueSelector(offset)
{ {
private int currentOffsetId = ReadableVectorInspector.NULL_ID; private int currentOffsetId = ReadableVectorInspector.NULL_ID;
@ -794,15 +793,15 @@ public class NestedFieldVirtualColumn implements VirtualColumn
} }
return theColumn.makeVectorValueSelector(offset); return theColumn.makeVectorValueSelector(offset);
} }
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { if (fieldSpec.parts.size() == 1 && fieldSpec.parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) {
final VariantColumn<?> arrayColumn = (VariantColumn<?>) theColumn; final VariantColumn<?> arrayColumn = (VariantColumn<?>) theColumn;
VectorObjectSelector arraySelector = arrayColumn.makeVectorObjectSelector(offset); VectorObjectSelector arraySelector = arrayColumn.makeVectorObjectSelector(offset);
final int elementNumber = ((NestedPathArrayElement) parts.get(0)).getIndex(); final int elementNumber = ((NestedPathArrayElement) fieldSpec.parts.get(0)).getIndex();
if (elementNumber < 0) { if (elementNumber < 0) {
throw new IAE("Cannot make array element selector, negative array index not supported"); throw new IAE("Cannot make array element selector, negative array index not supported");
} }
if (expectedType != null && expectedType.is(ValueType.LONG)) { if (fieldSpec.expectedType != null && fieldSpec.expectedType.is(ValueType.LONG)) {
return new BaseLongVectorValueSelector(offset) return new BaseLongVectorValueSelector(offset)
{ {
private final long[] longs = new long[offset.getMaxVectorSize()]; private final long[] longs = new long[offset.getMaxVectorSize()];
@ -871,7 +870,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
return nulls; return nulls;
} }
}; };
} else if (expectedType != null && expectedType.is(ValueType.FLOAT)) { } else if (fieldSpec.expectedType != null && fieldSpec.expectedType.is(ValueType.FLOAT)) {
return new BaseFloatVectorValueSelector(offset) return new BaseFloatVectorValueSelector(offset)
{ {
private final float[] floats = new float[offset.getMaxVectorSize()]; private final float[] floats = new float[offset.getMaxVectorSize()];
@ -1015,12 +1014,12 @@ public class NestedFieldVirtualColumn implements VirtualColumn
final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn; final NestedDataComplexColumn column = (NestedDataComplexColumn) theColumn;
// if column is numeric, it has a vector value selector, so we can directly make a vector value selector // if column is numeric, it has a vector value selector, so we can directly make a vector value selector
// if we are missing an expectedType, then we've got nothing else to work with so try it anyway // if we are missing an expectedType, then we've got nothing else to work with so try it anyway
if (column.isNumeric(parts) || expectedType == null) { if (column.isNumeric(fieldSpec.parts) || fieldSpec.expectedType == null) {
return column.makeVectorValueSelector(parts, offset); return column.makeVectorValueSelector(fieldSpec.parts, offset);
} }
final VectorObjectSelector objectSelector = column.makeVectorObjectSelector(parts, offset); final VectorObjectSelector objectSelector = column.makeVectorObjectSelector(fieldSpec.parts, offset);
if (expectedType.is(ValueType.LONG)) { if (fieldSpec.expectedType.is(ValueType.LONG)) {
return new BaseLongVectorValueSelector(offset) return new BaseLongVectorValueSelector(offset)
{ {
private final long[] longVector = new long[offset.getMaxVectorSize()]; private final long[] longVector = new long[offset.getMaxVectorSize()];
@ -1162,47 +1161,47 @@ public class NestedFieldVirtualColumn implements VirtualColumn
ColumnIndexSelector indexSelector ColumnIndexSelector indexSelector
) )
{ {
ColumnHolder holder = indexSelector.getColumnHolder(this.columnName); ColumnHolder holder = indexSelector.getColumnHolder(fieldSpec.columnName);
if (holder == null) { if (holder == null) {
return null; return null;
} }
BaseColumn theColumn = holder.getColumn(); BaseColumn theColumn = holder.getColumn();
if (theColumn instanceof CompressedNestedDataComplexColumn) { if (theColumn instanceof CompressedNestedDataComplexColumn) {
final CompressedNestedDataComplexColumn<?> nestedColumn = (CompressedNestedDataComplexColumn<?>) theColumn; final CompressedNestedDataComplexColumn<?> nestedColumn = (CompressedNestedDataComplexColumn<?>) theColumn;
final ColumnIndexSupplier nestedColumnPathIndexSupplier = nestedColumn.getColumnIndexSupplier(parts); final ColumnIndexSupplier nestedColumnPathIndexSupplier = nestedColumn.getColumnIndexSupplier(fieldSpec.parts);
if (nestedColumnPathIndexSupplier == null && processFromRaw) { if (nestedColumnPathIndexSupplier == null && fieldSpec.processFromRaw) {
// if processing from raw, a non-exstent path from parts doesn't mean the path doesn't really exist // if processing from raw, a non-exstent path from parts doesn't mean the path doesn't really exist
// so fall back to no indexes // so fall back to no indexes
return NoIndexesColumnIndexSupplier.getInstance(); return NoIndexesColumnIndexSupplier.getInstance();
} }
if (expectedType != null) { if (fieldSpec.expectedType != null) {
final Set<ColumnType> types = nestedColumn.getColumnTypes(parts); final Set<ColumnType> types = nestedColumn.getColumnTypes(fieldSpec.parts);
// if the expected output type is numeric but not all of the input types are numeric, we might have additional // if the expected output type is numeric but not all of the input types are numeric, we might have additional
// null values than what the null value bitmap is tracking, fall back to not using indexes // null values than what the null value bitmap is tracking, fall back to not using indexes
if (expectedType.isNumeric() && (types == null || types.stream().anyMatch(t -> !t.isNumeric()))) { if (fieldSpec.expectedType.isNumeric() && (types == null || types.stream().anyMatch(t -> !t.isNumeric()))) {
return NoIndexesColumnIndexSupplier.getInstance(); return NoIndexesColumnIndexSupplier.getInstance();
} }
} }
return nestedColumnPathIndexSupplier; return nestedColumnPathIndexSupplier;
} }
if (parts.isEmpty()) { if (fieldSpec.parts.isEmpty()) {
final ColumnIndexSupplier baseIndexSupplier = holder.getIndexSupplier(); final ColumnIndexSupplier baseIndexSupplier = holder.getIndexSupplier();
if (expectedType != null) { if (fieldSpec.expectedType != null) {
if (theColumn instanceof NumericColumn) { if (theColumn instanceof NumericColumn) {
return baseIndexSupplier; return baseIndexSupplier;
} }
if (theColumn instanceof NestedCommonFormatColumn) { if (theColumn instanceof NestedCommonFormatColumn) {
final NestedCommonFormatColumn commonFormat = (NestedCommonFormatColumn) theColumn; final NestedCommonFormatColumn commonFormat = (NestedCommonFormatColumn) theColumn;
if (expectedType.isNumeric() && !commonFormat.getLogicalType().isNumeric()) { if (fieldSpec.expectedType.isNumeric() && !commonFormat.getLogicalType().isNumeric()) {
return NoIndexesColumnIndexSupplier.getInstance(); return NoIndexesColumnIndexSupplier.getInstance();
} }
} else { } else {
return expectedType.isNumeric() ? NoIndexesColumnIndexSupplier.getInstance() : baseIndexSupplier; return fieldSpec.expectedType.isNumeric() ? NoIndexesColumnIndexSupplier.getInstance() : baseIndexSupplier;
} }
} }
return baseIndexSupplier; return baseIndexSupplier;
} }
if (parts.size() == 1 && parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) { if (fieldSpec.parts.size() == 1 && fieldSpec.parts.get(0) instanceof NestedPathArrayElement && theColumn instanceof VariantColumn) {
// cannot use the array column index supplier directly, in the future array columns should expose a function // cannot use the array column index supplier directly, in the future array columns should expose a function
// with a signature like 'getArrayElementIndexSupplier(int index)' to allow getting indexes for specific elements // with a signature like 'getArrayElementIndexSupplier(int index)' to allow getting indexes for specific elements
// if we want to support this stuff. Right now VariantArrayColumn doesn't actually retain enough information about // if we want to support this stuff. Right now VariantArrayColumn doesn't actually retain enough information about
@ -1215,7 +1214,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
@Override @Override
public ColumnCapabilities capabilities(String columnName) public ColumnCapabilities capabilities(String columnName)
{ {
if (processFromRaw) { if (fieldSpec.processFromRaw) {
// JSON_QUERY always returns a StructuredData // JSON_QUERY always returns a StructuredData
return ColumnCapabilitiesImpl.createDefault() return ColumnCapabilitiesImpl.createDefault()
.setType(ColumnType.NESTED_DATA) .setType(ColumnType.NESTED_DATA)
@ -1225,7 +1224,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
// this should only be used for 'realtime' queries, so don't indicate that we are dictionary encoded or have indexes // this should only be used for 'realtime' queries, so don't indicate that we are dictionary encoded or have indexes
// from here // from here
return ColumnCapabilitiesImpl.createDefault() return ColumnCapabilitiesImpl.createDefault()
.setType(expectedType != null ? expectedType : ColumnType.STRING) .setType(fieldSpec.expectedType != null ? fieldSpec.expectedType : ColumnType.STRING)
.setHasNulls(true); .setHasNulls(true);
} }
@ -1233,8 +1232,8 @@ public class NestedFieldVirtualColumn implements VirtualColumn
@Override @Override
public ColumnCapabilities capabilities(ColumnInspector inspector, String columnName) public ColumnCapabilities capabilities(ColumnInspector inspector, String columnName)
{ {
if (processFromRaw) { if (fieldSpec.processFromRaw) {
if (expectedType != null && expectedType.isArray() && ColumnType.NESTED_DATA.equals(expectedType.getElementType())) { if (fieldSpec.expectedType != null && fieldSpec.expectedType.isArray() && ColumnType.NESTED_DATA.equals(fieldSpec.expectedType.getElementType())) {
// arrays of objects! // arrays of objects!
return ColumnCapabilitiesImpl.createDefault() return ColumnCapabilitiesImpl.createDefault()
.setType(ColumnType.ofArray(ColumnType.NESTED_DATA)) .setType(ColumnType.ofArray(ColumnType.NESTED_DATA))
@ -1249,16 +1248,16 @@ public class NestedFieldVirtualColumn implements VirtualColumn
} }
// ColumnInspector isn't really enough... we need the ability to read the complex column itself to examine // ColumnInspector isn't really enough... we need the ability to read the complex column itself to examine
// the nested fields type information to really be accurate here, so we rely on the expectedType to guide us // the nested fields type information to really be accurate here, so we rely on the expectedType to guide us
final ColumnCapabilities capabilities = inspector.getColumnCapabilities(this.columnName); final ColumnCapabilities capabilities = inspector.getColumnCapabilities(fieldSpec.columnName);
if (capabilities != null) { if (capabilities != null) {
// if the underlying column is a nested column (and persisted to disk, re: the dictionary encoded check) // if the underlying column is a nested column (and persisted to disk, re: the dictionary encoded check)
if (capabilities.is(ValueType.COMPLEX) && if (capabilities.is(ValueType.COMPLEX) &&
capabilities.getComplexTypeName().equals(NestedDataComplexTypeSerde.TYPE_NAME) && capabilities.getComplexTypeName().equals(NestedDataComplexTypeSerde.TYPE_NAME) &&
capabilities.isDictionaryEncoded().isTrue()) { capabilities.isDictionaryEncoded().isTrue()) {
final boolean useDictionary = parts.isEmpty() || !(parts.get(parts.size() - 1) instanceof NestedPathArrayElement); final boolean useDictionary = fieldSpec.parts.isEmpty() || !(fieldSpec.parts.get(fieldSpec.parts.size() - 1) instanceof NestedPathArrayElement);
return ColumnCapabilitiesImpl.createDefault() return ColumnCapabilitiesImpl.createDefault()
.setType(expectedType != null ? expectedType : ColumnType.STRING) .setType(fieldSpec.expectedType != null ? fieldSpec.expectedType : ColumnType.STRING)
.setDictionaryEncoded(useDictionary) .setDictionaryEncoded(useDictionary)
.setDictionaryValuesSorted(useDictionary) .setDictionaryValuesSorted(useDictionary)
.setDictionaryValuesUnique(useDictionary) .setDictionaryValuesUnique(useDictionary)
@ -1266,12 +1265,12 @@ public class NestedFieldVirtualColumn implements VirtualColumn
.setHasNulls(true); .setHasNulls(true);
} }
// column is not nested, use underlying column capabilities, adjusted for expectedType as necessary // column is not nested, use underlying column capabilities, adjusted for expectedType as necessary
if (parts.isEmpty()) { if (fieldSpec.parts.isEmpty()) {
ColumnCapabilitiesImpl copy = ColumnCapabilitiesImpl.copyOf(capabilities); ColumnCapabilitiesImpl copy = ColumnCapabilitiesImpl.copyOf(capabilities);
if (expectedType != null) { if (fieldSpec.expectedType != null) {
copy.setType(expectedType); copy.setType(fieldSpec.expectedType);
copy.setHasNulls( copy.setHasNulls(
copy.hasNulls().or(ColumnCapabilities.Capable.of(expectedType.getType() != capabilities.getType())) copy.hasNulls().or(ColumnCapabilities.Capable.of(fieldSpec.expectedType.getType() != capabilities.getType()))
); );
} }
return copy; return copy;
@ -1287,7 +1286,7 @@ public class NestedFieldVirtualColumn implements VirtualColumn
@Override @Override
public List<String> requiredColumns() public List<String> requiredColumns()
{ {
return Collections.singletonList(columnName); return Collections.singletonList(fieldSpec.columnName);
} }
@Override @Override
@ -1296,6 +1295,13 @@ public class NestedFieldVirtualColumn implements VirtualColumn
return false; return false;
} }
@Nullable
@Override
public EquivalenceKey getEquivalanceKey()
{
return fieldSpec;
}
@Override @Override
public boolean equals(Object o) public boolean equals(Object o)
{ {
@ -1306,28 +1312,25 @@ public class NestedFieldVirtualColumn implements VirtualColumn
return false; return false;
} }
NestedFieldVirtualColumn that = (NestedFieldVirtualColumn) o; NestedFieldVirtualColumn that = (NestedFieldVirtualColumn) o;
return columnName.equals(that.columnName) && return outputName.equals(that.outputName) &&
outputName.equals(that.outputName) && fieldSpec.equals(that.fieldSpec);
parts.equals(that.parts) &&
Objects.equals(expectedType, that.expectedType) &&
processFromRaw == that.processFromRaw;
} }
@Override @Override
public int hashCode() public int hashCode()
{ {
return Objects.hash(columnName, parts, outputName, expectedType, processFromRaw); return Objects.hash(outputName, fieldSpec);
} }
@Override @Override
public String toString() public String toString()
{ {
return "NestedFieldVirtualColumn{" + return "NestedFieldVirtualColumn{" +
"columnName='" + columnName + '\'' + "columnName='" + fieldSpec.columnName + '\'' +
", outputName='" + outputName + '\'' + ", outputName='" + outputName + '\'' +
", typeHint='" + expectedType + '\'' + ", typeHint='" + fieldSpec.expectedType + '\'' +
", pathParts='" + parts + '\'' + ", pathParts='" + fieldSpec.parts + '\'' +
", allowFallback=" + processFromRaw + ", allowFallback=" + fieldSpec.processFromRaw +
'}'; '}';
} }
@ -1386,6 +1389,50 @@ public class NestedFieldVirtualColumn implements VirtualColumn
}; };
} }
private static class NestedFieldSpec implements EquivalenceKey
{
private final String columnName;
@Nullable
private final ColumnType expectedType;
private final List<NestedPathPart> parts;
private final boolean processFromRaw;
private NestedFieldSpec(
String columnName,
@Nullable ColumnType expectedType,
List<NestedPathPart> parts,
boolean processFromRaw
)
{
this.columnName = columnName;
this.expectedType = expectedType;
this.parts = parts;
this.processFromRaw = processFromRaw;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
NestedFieldSpec that = (NestedFieldSpec) o;
return processFromRaw == that.processFromRaw
&& Objects.equals(columnName, that.columnName)
&& Objects.equals(expectedType, that.expectedType)
&& Objects.equals(parts, that.parts);
}
@Override
public int hashCode()
{
return Objects.hash(columnName, expectedType, parts, processFromRaw);
}
}
/** /**
* Process the "raw" data to extract non-complex values. Like {@link RawFieldColumnSelector} but does not return * Process the "raw" data to extract non-complex values. Like {@link RawFieldColumnSelector} but does not return
* complex nested objects and does not wrap the results in {@link StructuredData}. * complex nested objects and does not wrap the results in {@link StructuredData}.

View File

@ -43,7 +43,6 @@ import org.apache.druid.testing.InitializedNullHandlingTest;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.mockito.Mock; import org.mockito.Mock;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.mockito.junit.MockitoJUnit; import org.mockito.junit.MockitoJUnit;
@ -62,9 +61,6 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest
{ {
private static final String REAL_COLUMN_NAME = "real_column"; private static final String REAL_COLUMN_NAME = "real_column";
@Rule
public ExpectedException expectedException = ExpectedException.none();
@Rule @Rule
public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
@ -217,10 +213,11 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest
{ {
final VirtualColumns virtualColumns = makeVirtualColumns(); final VirtualColumns virtualColumns = makeVirtualColumns();
expectedException.expect(IllegalArgumentException.class); Throwable t = Assert.assertThrows(
expectedException.expectMessage("No such virtual column[bar]"); IllegalArgumentException.class,
() -> virtualColumns.makeColumnValueSelector("bar", baseColumnSelectorFactory)
virtualColumns.makeColumnValueSelector("bar", baseColumnSelectorFactory); );
Assert.assertEquals("No such virtual column[bar]", t.getMessage());
} }
@Test @Test
@ -321,10 +318,11 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest
TestExprMacroTable.INSTANCE TestExprMacroTable.INSTANCE
); );
expectedException.expect(IllegalArgumentException.class); Throwable t = Assert.assertThrows(
expectedException.expectMessage("virtualColumn name[__time] not allowed"); IllegalArgumentException.class,
() -> VirtualColumns.create(ImmutableList.of(expr))
VirtualColumns.create(ImmutableList.of(expr)); );
Assert.assertEquals("virtualColumn name[__time] not allowed", t.getMessage());
} }
@Test @Test
@ -344,10 +342,11 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest
TestExprMacroTable.INSTANCE TestExprMacroTable.INSTANCE
); );
expectedException.expect(IllegalArgumentException.class); Throwable t = Assert.assertThrows(
expectedException.expectMessage("Duplicate virtualColumn name[expr]"); IllegalArgumentException.class,
() -> VirtualColumns.create(ImmutableList.of(expr, expr2))
VirtualColumns.create(ImmutableList.of(expr, expr2)); );
Assert.assertEquals("Duplicate virtualColumn name[expr]", t.getMessage());
} }
@Test @Test
@ -367,10 +366,11 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest
TestExprMacroTable.INSTANCE TestExprMacroTable.INSTANCE
); );
expectedException.expect(IllegalArgumentException.class); Throwable t = Assert.assertThrows(
expectedException.expectMessage("Self-referential column[expr]"); IllegalArgumentException.class,
() -> VirtualColumns.create(ImmutableList.of(expr, expr2))
VirtualColumns.create(ImmutableList.of(expr, expr2)); );
Assert.assertEquals("Self-referential column[expr]", t.getMessage());
} }
@Test @Test
@ -417,6 +417,42 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest
Assert.assertNotEquals(VirtualColumns.EMPTY.hashCode(), virtualColumns.hashCode()); Assert.assertNotEquals(VirtualColumns.EMPTY.hashCode(), virtualColumns.hashCode());
} }
@Test
public void testEquivalence()
{
final VirtualColumn v0 = new ExpressionVirtualColumn(
"expr",
"x + y",
ColumnType.FLOAT,
TestExprMacroTable.INSTANCE
);
final VirtualColumns virtualColumns = VirtualColumns.create(ImmutableList.of(v0));
final VirtualColumn v1 = new ExpressionVirtualColumn(
"differentNameExpr",
"x + y",
ColumnType.FLOAT,
TestExprMacroTable.INSTANCE
);
final VirtualColumn v2 = new ExpressionVirtualColumn(
"differentNameTypeExpr",
"x + y",
ColumnType.DOUBLE,
TestExprMacroTable.INSTANCE
);
final VirtualColumn v3 = new ExpressionVirtualColumn(
"expr",
"x + y",
ColumnType.DOUBLE,
TestExprMacroTable.INSTANCE
);
Assert.assertEquals(v0, virtualColumns.findEquivalent(v0));
Assert.assertEquals(v0, virtualColumns.findEquivalent(v1));
Assert.assertNull(virtualColumns.findEquivalent(v2));
Assert.assertNull(virtualColumns.findEquivalent(v3));
}
@Test @Test
public void testSerde() throws Exception public void testSerde() throws Exception
{ {

View File

@ -86,11 +86,23 @@ public class NestedFieldVirtualColumnTest
); );
} }
@Test
public void testEquivalence()
{
NestedFieldVirtualColumn v1 = new NestedFieldVirtualColumn("nested", "$.x.y.z", "v0", ColumnType.LONG);
NestedFieldVirtualColumn v2 = new NestedFieldVirtualColumn("nested", "$.x.y.z", "v1", ColumnType.LONG);
NestedFieldVirtualColumn v3 = new NestedFieldVirtualColumn("nested", "$.x.y.z[0]", "v0", ColumnType.LONG);
Assert.assertNotEquals(v1, v2);
Assert.assertEquals(v1.getEquivalanceKey(), v2.getEquivalanceKey());
Assert.assertNotEquals(v1, v3);
Assert.assertNotEquals(v1.getEquivalanceKey(), v3.getEquivalanceKey());
}
@Test @Test
public void testEqualsAndHashcode() public void testEqualsAndHashcode()
{ {
EqualsVerifier.forClass(NestedFieldVirtualColumn.class) EqualsVerifier.forClass(NestedFieldVirtualColumn.class)
.withNonnullFields("columnName", "outputName") .withNonnullFields("fieldSpec", "outputName")
.withIgnoredFields("hasNegativeArrayIndex") .withIgnoredFields("hasNegativeArrayIndex")
.usingGetClass() .usingGetClass()
.verify(); .verify();