This is an automated email from the ASF dual-hosted git repository.

cwylie pushed a commit to branch 31.0.0
in repository https://gitbox.apache.org/repos/asf/druid.git


The following commit(s) were added to refs/heads/31.0.0 by this push:
     new 51bea569483 add VirtualColumns.findEquivalent and 
VirtualColumn.EquivalenceKey (#17084) (#17166)
51bea569483 is described below

commit 51bea569483e5a306ddf332d9a9cece22bf213b4
Author: Clint Wylie <cwy...@apache.org>
AuthorDate: Fri Sep 27 17:54:06 2024 -0700

    add VirtualColumns.findEquivalent and VirtualColumn.EquivalenceKey (#17084) 
(#17166)
---
 .../org/apache/druid/segment/VirtualColumn.java    |  19 ++
 .../org/apache/druid/segment/VirtualColumns.java   |  24 +-
 .../segment/virtual/ExpressionVirtualColumn.java   |  91 +++++--
 .../segment/virtual/NestedFieldVirtualColumn.java  | 261 ++++++++++++---------
 .../apache/druid/segment/VirtualColumnsTest.java   |  76 ++++--
 .../virtual/NestedFieldVirtualColumnTest.java      |  14 +-
 6 files changed, 336 insertions(+), 149 deletions(-)

diff --git 
a/processing/src/main/java/org/apache/druid/segment/VirtualColumn.java 
b/processing/src/main/java/org/apache/druid/segment/VirtualColumn.java
index ca9408d14e4..b7ca00d9e5d 100644
--- a/processing/src/main/java/org/apache/druid/segment/VirtualColumn.java
+++ b/processing/src/main/java/org/apache/druid/segment/VirtualColumn.java
@@ -21,6 +21,7 @@ package org.apache.druid.segment;
 
 import com.fasterxml.jackson.annotation.JsonSubTypes;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode;
 import org.apache.druid.java.util.common.Cacheable;
 import org.apache.druid.query.dimension.DimensionSpec;
 import org.apache.druid.query.filter.ColumnIndexSelector;
@@ -336,4 +337,22 @@ public interface VirtualColumn extends Cacheable
   {
     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
+  {
+  }
 }
diff --git 
a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java 
b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java
index bc2609b4a00..676067a959f 100644
--- a/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java
+++ b/processing/src/main/java/org/apache/druid/segment/VirtualColumns.java
@@ -26,6 +26,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import org.apache.druid.java.util.common.Cacheable;
 import org.apache.druid.java.util.common.IAE;
@@ -131,10 +132,13 @@ public class VirtualColumns implements Cacheable
   // For equals, hashCode, toString, and serialization:
   private final List<VirtualColumn> virtualColumns;
   private final List<String> virtualColumnNames;
+  // For equivalence
+  private final Map<VirtualColumn.EquivalenceKey, VirtualColumn> equivalence;
 
   // For getVirtualColumn:
   private final Map<String, VirtualColumn> withDotSupport;
   private final Map<String, VirtualColumn> withoutDotSupport;
+  private final boolean hasNoDotColumns;
 
   private VirtualColumns(
       List<VirtualColumn> virtualColumns,
@@ -146,10 +150,15 @@ public class VirtualColumns implements Cacheable
     this.withDotSupport = withDotSupport;
     this.withoutDotSupport = withoutDotSupport;
     this.virtualColumnNames = new ArrayList<>(virtualColumns.size());
-
+    this.hasNoDotColumns = withDotSupport.isEmpty();
+    this.equivalence = Maps.newHashMapWithExpectedSize(virtualColumns.size());
     for (VirtualColumn virtualColumn : virtualColumns) {
       detectCycles(virtualColumn, null);
       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) {
       return vc;
     }
+    if (hasNoDotColumns) {
+      return null;
+    }
     final String baseColumnName = splitColumnName(columnName).lhs;
     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
    * {@link ColumnSelector} to allow reading things from segments. If the 
column does not have indexes this method
diff --git 
a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java
 
b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java
index 42a723907b9..e6f4d57e1d5 100644
--- 
a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java
+++ 
b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java
@@ -61,9 +61,7 @@ public class ExpressionVirtualColumn implements VirtualColumn
   private static final Logger log = new Logger(ExpressionVirtualColumn.class);
 
   private final String name;
-  private final String expression;
-  @Nullable
-  private final ColumnType outputType;
+  private final Expression expression;
   private final Supplier<Expr> parsedExpression;
   private final Supplier<byte[]> cacheKey;
 
@@ -126,8 +124,7 @@ public class ExpressionVirtualColumn implements 
VirtualColumn
   )
   {
     this.name = Preconditions.checkNotNull(name, "name");
-    this.expression = Preconditions.checkNotNull(expression, "expression");
-    this.outputType = outputType;
+    this.expression = new Expression(Preconditions.checkNotNull(expression, 
"expression"), outputType);
     this.parsedExpression = parsedExpression;
     this.cacheKey = makeCacheKeySupplier();
   }
@@ -142,14 +139,14 @@ public class ExpressionVirtualColumn implements 
VirtualColumn
   @JsonProperty
   public String getExpression()
   {
-    return expression;
+    return expression.expressionString;
   }
 
   @Nullable
   @JsonProperty
   public ColumnType getOutputType()
   {
-    return outputType;
+    return expression.outputType;
   }
 
   @JsonIgnore
@@ -273,7 +270,7 @@ public class ExpressionVirtualColumn implements 
VirtualColumn
       ColumnIndexSelector columnIndexSelector
   )
   {
-    return 
getParsedExpression().get().asColumnIndexSupplier(columnIndexSelector, 
outputType);
+    return 
getParsedExpression().get().asColumnIndexSupplier(columnIndexSelector, 
expression.outputType);
   }
 
   @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
     // inputs or because of unimplemented methods on expression 
implementations themselves, or, because a
     // ColumnInspector is not available
-
+    final ColumnType outputType = expression.outputType;
     if (ExpressionProcessing.processArraysAsMultiValueStrings() && outputType 
!= null && outputType.isArray()) {
       return new 
ColumnCapabilitiesImpl().setType(ColumnType.STRING).setHasMultipleValues(true);
     }
@@ -299,6 +296,8 @@ public class ExpressionVirtualColumn implements 
VirtualColumn
       return 
inspector.getColumnCapabilities(parsedExpression.get().getBindingIfIdentifier());
     }
 
+    final ColumnType outputType = expression.outputType;
+
     final ExpressionPlan plan = ExpressionPlanner.plan(inspector, 
parsedExpression.get());
     final ColumnCapabilities inferred = 
plan.inferColumnCapabilities(outputType);
     // 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(
               "Projected output type %s of expression %s does not match 
provided type %s",
               inferred.asTypeString(),
-              expression,
+              expression.expressionString,
               outputType
           );
         } else {
           log.debug(
               "Projected output type %s of expression %s does not match 
provided type %s",
               inferred.asTypeString(),
-              expression,
+              expression.expressionString,
               outputType
           );
         }
@@ -348,6 +347,13 @@ public class ExpressionVirtualColumn implements 
VirtualColumn
     return cacheKey.get();
   }
 
+  @Nullable
+  @Override
+  public EquivalenceKey getEquivalanceKey()
+  {
+    return expression;
+  }
+
   @Override
   public boolean equals(final Object o)
   {
@@ -359,14 +365,13 @@ public class ExpressionVirtualColumn implements 
VirtualColumn
     }
     final ExpressionVirtualColumn that = (ExpressionVirtualColumn) o;
     return Objects.equals(name, that.name) &&
-           Objects.equals(expression, that.expression) &&
-           Objects.equals(outputType, that.outputType);
+           Objects.equals(expression, that.expression);
   }
 
   @Override
   public int hashCode()
   {
-    return Objects.hash(name, expression, outputType);
+    return Objects.hash(name, expression);
   }
 
   @Override
@@ -374,8 +379,7 @@ public class ExpressionVirtualColumn implements 
VirtualColumn
   {
     return "ExpressionVirtualColumn{" +
            "name='" + name + '\'' +
-           ", expression='" + expression + '\'' +
-           ", outputType=" + outputType +
+           ", expression=" + expression +
            '}';
   }
 
@@ -389,10 +393,10 @@ public class ExpressionVirtualColumn implements 
VirtualColumn
       final ColumnCapabilities baseCapabilities =
           
inspector.getColumnCapabilities(parsedExpression.get().getBindingIfIdentifier());
 
-      if (outputType == null) {
+      if (expression.outputType == null) {
         // No desired output type. Anything from the source is fine.
         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.
         return true;
       }
@@ -408,10 +412,57 @@ public class ExpressionVirtualColumn implements 
VirtualColumn
           .appendString(name)
           .appendCacheable(parsedExpression.get());
 
-      if (outputType != null) {
-        builder.appendString(outputType.toString());
+      if (expression.outputType != null) {
+        builder.appendString(expression.outputType.toString());
       }
       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 +
+             '}';
+    }
+  }
 }
diff --git 
a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java
 
b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java
index aa7dd8b88b7..5ec5958f586 100644
--- 
a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java
+++ 
b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java
@@ -93,27 +93,23 @@ import java.util.Set;
  * nested fields ({@link NestedFieldDictionaryEncodedColumn}) including using
  * their indexes.
  * <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
  * list of {@link NestedPathPart} in order to determine what should be 
selected from the column.
  * <p>
  * 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
  * 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>
- * Since {@link #capabilities(ColumnInspector, String)} is determined by the 
{@link #expectedType}, the results will
- * be best effor cast to the expected type if the column is not natively the 
expected type so that this column can
+ * Since {@link #capabilities(ColumnInspector, String)} is determined by the 
{@link #getExpectedType()}, the results
+ * 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.
  */
 public class NestedFieldVirtualColumn implements VirtualColumn
 {
-  private final String columnName;
   private final String outputName;
-  @Nullable
-  private final ColumnType expectedType;
-  private final List<NestedPathPart> parts;
-  private final boolean processFromRaw;
+  private final NestedFieldSpec fieldSpec;
 
   private final boolean hasNegativeArrayIndex;
 
@@ -128,22 +124,21 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
       @JsonProperty("useJqSyntax") @Nullable Boolean useJqSyntax
   )
   {
-    this.columnName = columnName;
     this.outputName = outputName;
     if (path != null) {
       Preconditions.checkArgument(parts == null, "Cannot define both 'path' 
and 'pathParts'");
     } else if (parts == null) {
       throw new IllegalArgumentException("Must define exactly one of 'path' or 
'pathParts'");
     }
-
+    final List<NestedPathPart> pathParts;
     if (parts != null) {
-      this.parts = parts;
+      pathParts = parts;
     } else {
       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;
-    for (NestedPathPart part : this.parts) {
+    for (NestedPathPart part : pathParts) {
       if (part instanceof NestedPathArrayElement) {
         NestedPathArrayElement elementPart = (NestedPathArrayElement) part;
         if (elementPart.getIndex() < 0) {
@@ -153,8 +148,12 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
       }
     }
     this.hasNegativeArrayIndex = hasNegative;
-    this.expectedType = expectedType;
-    this.processFromRaw = processFromRaw == null ? false : processFromRaw;
+    this.fieldSpec = new NestedFieldSpec(
+        columnName,
+        expectedType,
+        pathParts,
+        processFromRaw != null && processFromRaw
+    );
   }
 
   @VisibleForTesting
@@ -181,12 +180,12 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
   @Override
   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")
                                                                                
    .appendString(outputName)
-                                                                               
    .appendString(columnName)
+                                                                               
    .appendString(fieldSpec.columnName)
                                                                                
    .appendString(partsString)
-                                                                               
    .appendBoolean(processFromRaw)
+                                                                               
    .appendBoolean(fieldSpec.processFromRaw)
                                                                                
    .build();
   }
 
@@ -200,25 +199,25 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
   @JsonProperty
   public String getColumnName()
   {
-    return columnName;
+    return fieldSpec.columnName;
   }
 
   @JsonProperty("pathParts")
   public List<NestedPathPart> getPathParts()
   {
-    return parts;
+    return fieldSpec.parts;
   }
 
   @JsonProperty
   public ColumnType getExpectedType()
   {
-    return expectedType;
+    return fieldSpec.expectedType;
   }
 
   @JsonProperty
   public boolean isProcessFromRaw()
   {
-    return processFromRaw;
+    return fieldSpec.processFromRaw;
   }
 
   @Override
@@ -241,13 +240,13 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
   )
   {
     // 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
     // is JSON_VALUE which only returns literals, so use the literal value 
selector instead
-    return processFromRaw
-           ? new RawFieldColumnSelector(baseSelector, parts)
-           : new RawFieldLiteralColumnValueSelector(baseSelector, parts);
+    return fieldSpec.processFromRaw
+           ? new RawFieldColumnSelector(baseSelector, fieldSpec.parts)
+           : new RawFieldLiteralColumnValueSelector(baseSelector, 
fieldSpec.parts);
   }
 
   @Nullable
@@ -258,7 +257,7 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
       ReadableOffset offset
   )
   {
-    ColumnHolder holder = columnSelector.getColumnHolder(columnName);
+    ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName);
     if (holder == null) {
       // column doesn't exist
       return dimensionSpec.decorate(DimensionSelector.constant(null, 
dimensionSpec.getExtractionFn()));
@@ -283,11 +282,11 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
     BaseColumn theColumn = holder.getColumn();
     if (theColumn instanceof NestedDataComplexColumn) {
       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
-    if (parts.isEmpty()) {
+    if (fieldSpec.parts.isEmpty()) {
       // 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
       // 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;
       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) {
         throw new IAE("Cannot make array element selector, negative array 
index not supported");
       }
@@ -351,13 +350,13 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
       ReadableOffset offset
   )
   {
-    ColumnHolder holder = columnSelector.getColumnHolder(this.columnName);
+    ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName);
     if (holder == null) {
       return NilColumnValueSelector.instance();
     }
     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),
       // then we use the 'raw' processing of the 
RawFieldColumnSelector/RawFieldLiteralColumnValueSelector created
       // 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
     if (theColumn instanceof NestedDataComplexColumn) {
       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
-    if (parts.isEmpty()) {
+    if (fieldSpec.parts.isEmpty()) {
       // 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
       if (theColumn instanceof DictionaryEncodedColumn && !(theColumn 
instanceof VariantColumn)) {
@@ -383,10 +382,10 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
       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;
       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) {
         throw new IAE("Cannot make array element selector, negative array 
index not supported");
       }
@@ -466,7 +465,7 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
       ReadableVectorOffset offset
   )
   {
-    ColumnHolder holder = columnSelector.getColumnHolder(columnName);
+    ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName);
     if (holder == null) {
       return dimensionSpec.decorate(NilVectorSelector.create(offset));
     }
@@ -482,11 +481,11 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
     BaseColumn theColumn = holder.getColumn();
     if (theColumn instanceof NestedDataComplexColumn) {
       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
-    if (parts.isEmpty()) {
+    if (fieldSpec.parts.isEmpty()) {
       // 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
       return ((DictionaryEncodedColumn) 
theColumn).makeSingleValueDimensionVectorSelector(offset);
@@ -505,7 +504,7 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
       ReadableVectorOffset offset
   )
   {
-    ColumnHolder holder = columnSelector.getColumnHolder(this.columnName);
+    ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName);
     if (holder == null) {
       return NilVectorSelector.create(offset);
     }
@@ -514,80 +513,80 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
 
     if (column instanceof NestedDataComplexColumn) {
       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
         // 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;
       if (types != null) {
         for (ColumnType type : types) {
           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(
             offset,
             columnName,
-            complexColumn.makeVectorValueSelector(parts, offset),
+            complexColumn.makeVectorValueSelector(fieldSpec.parts, offset),
             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 &&
           leastRestrictiveType.isArray() &&
-          expectedType != null &&
-          !expectedType.isArray()
+          fieldSpec.expectedType != null &&
+          !fieldSpec.expectedType.isArray()
       ) {
         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 objectSelector;
     }
     // 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();
       // 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
       Preconditions.checkArgument(
-          expectedType != null,
+          fieldSpec.expectedType != null,
           "Asked for a VectorObjectSelector on a numeric column, 
'expectedType' must not be null"
       );
       if (capabilities.isNumeric()) {
         return ExpressionVectorSelectors.castValueSelectorToObject(
             offset,
-            this.columnName,
+            fieldSpec.columnName,
             column.makeVectorValueSelector(offset),
             capabilities.toColumnType(),
-            expectedType
+            fieldSpec.expectedType
         );
       }
       // 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
       // 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 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 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 ExpressionType elementType = ExpressionType.fromColumnTypeStrict(
           arrayColumn.getLogicalType().isArray() ? 
arrayColumn.getLogicalType().getElementType() : arrayColumn.getLogicalType()
       );
-      final ExpressionType castTo = expectedType == null
+      final ExpressionType castTo = fieldSpec.expectedType == null
                                     ? ExpressionType.STRING
-                                    : 
ExpressionType.fromColumnTypeStrict(expectedType);
+                                    : 
ExpressionType.fromColumnTypeStrict(fieldSpec.expectedType);
       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) {
         throw new IAE("Cannot make array element selector, negative array 
index not supported");
       }
@@ -646,17 +645,17 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
       ReadableVectorOffset offset
   )
   {
-    ColumnHolder holder = columnSelector.getColumnHolder(this.columnName);
+    ColumnHolder holder = columnSelector.getColumnHolder(fieldSpec.columnName);
     if (holder == null) {
       return NilVectorSelector.create(offset);
     }
     BaseColumn theColumn = holder.getColumn();
     if (!(theColumn instanceof NestedDataComplexColumn)) {
 
-      if (parts.isEmpty()) {
+      if (fieldSpec.parts.isEmpty()) {
         if (theColumn instanceof DictionaryEncodedColumn) {
           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)
             {
               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)
             {
               private int currentOffsetId = ReadableVectorInspector.NULL_ID;
@@ -794,15 +793,15 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
         }
         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;
         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) {
           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)
           {
             private final long[] longs = new long[offset.getMaxVectorSize()];
@@ -871,7 +870,7 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
               return nulls;
             }
           };
-        } else if (expectedType != null && expectedType.is(ValueType.FLOAT)) {
+        } else if (fieldSpec.expectedType != null && 
fieldSpec.expectedType.is(ValueType.FLOAT)) {
           return new BaseFloatVectorValueSelector(offset)
           {
             private final float[] floats = new 
float[offset.getMaxVectorSize()];
@@ -1015,12 +1014,12 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
     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 we are missing an expectedType, then we've got nothing else to work 
with so try it anyway
-    if (column.isNumeric(parts) || expectedType == null) {
-      return column.makeVectorValueSelector(parts, offset);
+    if (column.isNumeric(fieldSpec.parts) || fieldSpec.expectedType == null) {
+      return column.makeVectorValueSelector(fieldSpec.parts, offset);
     }
 
-    final VectorObjectSelector objectSelector = 
column.makeVectorObjectSelector(parts, offset);
-    if (expectedType.is(ValueType.LONG)) {
+    final VectorObjectSelector objectSelector = 
column.makeVectorObjectSelector(fieldSpec.parts, offset);
+    if (fieldSpec.expectedType.is(ValueType.LONG)) {
       return new BaseLongVectorValueSelector(offset)
       {
         private final long[] longVector = new long[offset.getMaxVectorSize()];
@@ -1162,47 +1161,47 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
       ColumnIndexSelector indexSelector
   )
   {
-    ColumnHolder holder = indexSelector.getColumnHolder(this.columnName);
+    ColumnHolder holder = indexSelector.getColumnHolder(fieldSpec.columnName);
     if (holder == null) {
       return null;
     }
     BaseColumn theColumn = holder.getColumn();
     if (theColumn instanceof CompressedNestedDataComplexColumn) {
       final CompressedNestedDataComplexColumn<?> nestedColumn = 
(CompressedNestedDataComplexColumn<?>) theColumn;
-      final ColumnIndexSupplier nestedColumnPathIndexSupplier = 
nestedColumn.getColumnIndexSupplier(parts);
-      if (nestedColumnPathIndexSupplier == null && processFromRaw) {
+      final ColumnIndexSupplier nestedColumnPathIndexSupplier = 
nestedColumn.getColumnIndexSupplier(fieldSpec.parts);
+      if (nestedColumnPathIndexSupplier == null && fieldSpec.processFromRaw) {
         // 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
         return NoIndexesColumnIndexSupplier.getInstance();
       }
-      if (expectedType != null) {
-        final Set<ColumnType> types = nestedColumn.getColumnTypes(parts);
+      if (fieldSpec.expectedType != null) {
+        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
         // 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 nestedColumnPathIndexSupplier;
     }
-    if (parts.isEmpty()) {
+    if (fieldSpec.parts.isEmpty()) {
       final ColumnIndexSupplier baseIndexSupplier = holder.getIndexSupplier();
-      if (expectedType != null) {
+      if (fieldSpec.expectedType != null) {
         if (theColumn instanceof NumericColumn) {
           return baseIndexSupplier;
         }
         if (theColumn instanceof NestedCommonFormatColumn) {
           final NestedCommonFormatColumn commonFormat = 
(NestedCommonFormatColumn) theColumn;
-          if (expectedType.isNumeric() && 
!commonFormat.getLogicalType().isNumeric()) {
+          if (fieldSpec.expectedType.isNumeric() && 
!commonFormat.getLogicalType().isNumeric()) {
             return NoIndexesColumnIndexSupplier.getInstance();
           }
         } else {
-          return expectedType.isNumeric() ? 
NoIndexesColumnIndexSupplier.getInstance() : baseIndexSupplier;
+          return fieldSpec.expectedType.isNumeric() ? 
NoIndexesColumnIndexSupplier.getInstance() : 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
       // 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
@@ -1215,7 +1214,7 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
   @Override
   public ColumnCapabilities capabilities(String columnName)
   {
-    if (processFromRaw) {
+    if (fieldSpec.processFromRaw) {
       // JSON_QUERY always returns a StructuredData
       return ColumnCapabilitiesImpl.createDefault()
                                    .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
     // from here
     return ColumnCapabilitiesImpl.createDefault()
-                                 .setType(expectedType != null ? expectedType 
: ColumnType.STRING)
+                                 .setType(fieldSpec.expectedType != null ? 
fieldSpec.expectedType : ColumnType.STRING)
                                  .setHasNulls(true);
   }
 
@@ -1233,8 +1232,8 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
   @Override
   public ColumnCapabilities capabilities(ColumnInspector inspector, String 
columnName)
   {
-    if (processFromRaw) {
-      if (expectedType != null && expectedType.isArray() && 
ColumnType.NESTED_DATA.equals(expectedType.getElementType())) {
+    if (fieldSpec.processFromRaw) {
+      if (fieldSpec.expectedType != null && fieldSpec.expectedType.isArray() 
&& ColumnType.NESTED_DATA.equals(fieldSpec.expectedType.getElementType())) {
         // arrays of objects!
         return ColumnCapabilitiesImpl.createDefault()
                                      
.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
     // 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 the underlying column is a nested column (and persisted to disk, 
re: the dictionary encoded check)
       if (capabilities.is(ValueType.COMPLEX) &&
           
capabilities.getComplexTypeName().equals(NestedDataComplexTypeSerde.TYPE_NAME) 
&&
           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()
-                                     .setType(expectedType != null ? 
expectedType : ColumnType.STRING)
+                                     .setType(fieldSpec.expectedType != null ? 
fieldSpec.expectedType : ColumnType.STRING)
                                      .setDictionaryEncoded(useDictionary)
                                      .setDictionaryValuesSorted(useDictionary)
                                      .setDictionaryValuesUnique(useDictionary)
@@ -1266,12 +1265,12 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
                                      .setHasNulls(true);
       }
       // 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);
-        if (expectedType != null) {
-          copy.setType(expectedType);
+        if (fieldSpec.expectedType != null) {
+          copy.setType(fieldSpec.expectedType);
           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;
@@ -1287,7 +1286,7 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
   @Override
   public List<String> requiredColumns()
   {
-    return Collections.singletonList(columnName);
+    return Collections.singletonList(fieldSpec.columnName);
   }
 
   @Override
@@ -1296,6 +1295,13 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
     return false;
   }
 
+  @Nullable
+  @Override
+  public EquivalenceKey getEquivalanceKey()
+  {
+    return fieldSpec;
+  }
+
   @Override
   public boolean equals(Object o)
   {
@@ -1306,28 +1312,25 @@ public class NestedFieldVirtualColumn implements 
VirtualColumn
       return false;
     }
     NestedFieldVirtualColumn that = (NestedFieldVirtualColumn) o;
-    return columnName.equals(that.columnName) &&
-           outputName.equals(that.outputName) &&
-           parts.equals(that.parts) &&
-           Objects.equals(expectedType, that.expectedType) &&
-           processFromRaw == that.processFromRaw;
+    return outputName.equals(that.outputName) &&
+           fieldSpec.equals(that.fieldSpec);
   }
 
   @Override
   public int hashCode()
   {
-    return Objects.hash(columnName, parts, outputName, expectedType, 
processFromRaw);
+    return Objects.hash(outputName, fieldSpec);
   }
 
   @Override
   public String toString()
   {
     return "NestedFieldVirtualColumn{" +
-           "columnName='" + columnName + '\'' +
+           "columnName='" + fieldSpec.columnName + '\'' +
            ", outputName='" + outputName + '\'' +
-           ", typeHint='" + expectedType + '\'' +
-           ", pathParts='" + parts + '\'' +
-           ", allowFallback=" + processFromRaw +
+           ", typeHint='" + fieldSpec.expectedType + '\'' +
+           ", pathParts='" + fieldSpec.parts + '\'' +
+           ", 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
    * complex nested objects and does not wrap the results in {@link 
StructuredData}.
diff --git 
a/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java 
b/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java
index 71d9adbd017..9f0e8c5ea31 100644
--- a/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java
+++ b/processing/src/test/java/org/apache/druid/segment/VirtualColumnsTest.java
@@ -43,7 +43,6 @@ import org.apache.druid.testing.InitializedNullHandlingTest;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.ExpectedException;
 import org.mockito.Mock;
 import org.mockito.Mockito;
 import org.mockito.junit.MockitoJUnit;
@@ -62,9 +61,6 @@ public class VirtualColumnsTest extends 
InitializedNullHandlingTest
 {
   private static final String REAL_COLUMN_NAME = "real_column";
 
-  @Rule
-  public ExpectedException expectedException = ExpectedException.none();
-
   @Rule
   public MockitoRule mockitoRule = 
MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS);
 
@@ -217,10 +213,11 @@ public class VirtualColumnsTest extends 
InitializedNullHandlingTest
   {
     final VirtualColumns virtualColumns = makeVirtualColumns();
 
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage("No such virtual column[bar]");
-
-    virtualColumns.makeColumnValueSelector("bar", baseColumnSelectorFactory);
+    Throwable t = Assert.assertThrows(
+        IllegalArgumentException.class,
+        () -> virtualColumns.makeColumnValueSelector("bar", 
baseColumnSelectorFactory)
+    );
+    Assert.assertEquals("No such virtual column[bar]", t.getMessage());
   }
 
   @Test
@@ -321,10 +318,11 @@ public class VirtualColumnsTest extends 
InitializedNullHandlingTest
         TestExprMacroTable.INSTANCE
     );
 
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage("virtualColumn name[__time] not allowed");
-
-    VirtualColumns.create(ImmutableList.of(expr));
+    Throwable t = Assert.assertThrows(
+        IllegalArgumentException.class,
+        () -> VirtualColumns.create(ImmutableList.of(expr))
+    );
+    Assert.assertEquals("virtualColumn name[__time] not allowed", 
t.getMessage());
   }
 
   @Test
@@ -344,10 +342,11 @@ public class VirtualColumnsTest extends 
InitializedNullHandlingTest
         TestExprMacroTable.INSTANCE
     );
 
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage("Duplicate virtualColumn name[expr]");
-
-    VirtualColumns.create(ImmutableList.of(expr, expr2));
+    Throwable t = Assert.assertThrows(
+        IllegalArgumentException.class,
+        () -> VirtualColumns.create(ImmutableList.of(expr, expr2))
+    );
+    Assert.assertEquals("Duplicate virtualColumn name[expr]", t.getMessage());
   }
 
   @Test
@@ -367,10 +366,11 @@ public class VirtualColumnsTest extends 
InitializedNullHandlingTest
         TestExprMacroTable.INSTANCE
     );
 
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage("Self-referential column[expr]");
-
-    VirtualColumns.create(ImmutableList.of(expr, expr2));
+    Throwable t = Assert.assertThrows(
+        IllegalArgumentException.class,
+        () -> VirtualColumns.create(ImmutableList.of(expr, expr2))
+    );
+    Assert.assertEquals("Self-referential column[expr]", t.getMessage());
   }
 
   @Test
@@ -417,6 +417,42 @@ public class VirtualColumnsTest extends 
InitializedNullHandlingTest
     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
   public void testSerde() throws Exception
   {
diff --git 
a/processing/src/test/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumnTest.java
 
b/processing/src/test/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumnTest.java
index 581c8674da9..62f26588009 100644
--- 
a/processing/src/test/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumnTest.java
+++ 
b/processing/src/test/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumnTest.java
@@ -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
   public void testEqualsAndHashcode()
   {
     EqualsVerifier.forClass(NestedFieldVirtualColumn.class)
-                  .withNonnullFields("columnName", "outputName")
+                  .withNonnullFields("fieldSpec", "outputName")
                   .withIgnoredFields("hasNegativeArrayIndex")
                   .usingGetClass()
                   .verify();


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@druid.apache.org
For additional commands, e-mail: commits-h...@druid.apache.org

Reply via email to