mirror of https://github.com/apache/druid.git
add VirtualColumns.findEquivalent and VirtualColumn.EquivalenceKey (#17084)
This commit is contained in:
parent
490211f2b1
commit
a93546d493
|
@ -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
|
||||||
|
{
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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 +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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}.
|
||||||
|
|
|
@ -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
|
||||||
{
|
{
|
||||||
|
|
|
@ -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();
|
||||||
|
|
Loading…
Reference in New Issue