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

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


The following commit(s) were added to refs/heads/master by this push:
     new bf1b2fe04bf Additional expr type alignment. (#18503)
bf1b2fe04bf is described below

commit bf1b2fe04bf7d0ff8401e23f65988bbfed00807b
Author: Gian Merlino <[email protected]>
AuthorDate: Thu Sep 18 16:20:09 2025 -0700

    Additional expr type alignment. (#18503)
    
    * Additional expr type alignment.
    
    PR #16366 originally added fallback vectorization, a mechanism for
    making all expressions vectorizable. Later, #17098 fixed some issues
    that arose and #17248 disabled fallback vectorization in the out-of-box
    configuration.
    
    This patch fixes various remaining issues with inconsistent type handling
    between the vectorized and nonvectorized expr implementations. It does not
    yet re-enable fallback vectorization out of the box, due to remaining
    inconsistencies with conditional exprs like "case_searched", "case_simple",
    and "if".
    
    1) Aligns the behavior of missing columns and literal nulls so they are
       always treated as null longs. This was already the case for vectorized
       identifiers, but non-vectorized identifiers and literal nulls were still
       represented as strings.
    
    2) Replaces all occurrences of "ExprEval.of(null)" with either an explicit
       type, or a call to "ExprEval.ofMissing()". ofMissing is a new function
       for situations where an eval represents a null value of unknown type.
       It is equivalent to "ExprEval.ofLong(null)", but is a separate function
       for clarity at the call site.
    
    3) Update "cast" to return the target type even for null values.
    
    4) Update "greatest", "least", and "array" so they eval to types that
       match what is reported by "getOutputType".
    
    5) Update "scalb" to coerce input strings as numbers, to better allow
       for type evolution and missing columns.
    
    6) Update "reverse" to coerce inputs to strings, to better allow for
       type evolution and missing columns.
    
    * Restore fallback in testArrayFns.
    
    * Fix issues.
---
 .../datasketches/theta/sql/ThetaPostAggMacros.java |   2 +-
 .../druid/query/expressions/SleepExprMacro.java    |   2 +-
 .../apache/druid/testing/tools/SleepExprMacro.java |   2 +-
 .../org/apache/druid/math/expr/ApplyFunction.java  |   8 +-
 .../druid/math/expr/BinaryEvalOpExprBase.java      |   8 +-
 .../druid/math/expr/BinaryMathOperatorExpr.java    |   2 +-
 .../apache/druid/math/expr/BuiltInExprMacros.java  |   2 +-
 .../org/apache/druid/math/expr/ConstantExpr.java   |   2 +-
 .../java/org/apache/druid/math/expr/ExprEval.java  |  65 +++++---
 .../apache/druid/math/expr/ExprListenerImpl.java   |   2 +-
 .../druid/math/expr/ExpressionProcessing.java      |   6 +-
 .../math/expr/ExpressionProcessingConfig.java      |  13 --
 .../java/org/apache/druid/math/expr/Function.java  | 182 +++++++++------------
 .../apache/druid/math/expr/UnaryOperatorExpr.java  |   4 +-
 .../math/expr/vector/FallbackVectorProcessor.java  |   8 +-
 .../expression/IPv4AddressStringifyExprMacro.java  |  10 +-
 .../druid/query/expression/LookupExprMacro.java    |   2 +-
 .../query/expression/NestedDataExpressions.java    |   4 +-
 .../query/expression/RegexpExtractExprMacro.java   |   4 +-
 .../query/expression/RegexpReplaceExprMacro.java   |  10 +-
 .../query/expression/TimestampCeilExprMacro.java   |   2 +-
 .../expression/TimestampExtractExprMacro.java      |   4 +-
 .../query/expression/TimestampFloorExprMacro.java  |   2 +-
 .../query/expression/TimestampFormatExprMacro.java |   4 +-
 .../query/expression/TimestampParseExprMacro.java  |   4 +-
 .../query/expression/TimestampShiftExprMacro.java  |   4 +-
 .../druid/query/expression/TrimExprMacro.java      |   4 +-
 .../apache/druid/math/expr/ConstantExprTest.java   |  11 +-
 .../java/org/apache/druid/math/expr/EvalTest.java  |  12 +-
 .../org/apache/druid/math/expr/ExprEvalTest.java   |   6 +-
 .../org/apache/druid/math/expr/FunctionTest.java   |   4 +-
 .../org/apache/druid/math/expr/OutputTypeTest.java |   4 +-
 .../math/expr/VectorExprResultConsistencyTest.java |  38 ++++-
 .../expression/IPv4AddressMatchExprMacroTest.java  |  38 ++---
 .../expression/IPv4AddressParseExprMacroTest.java  |  12 +-
 .../IPv4AddressStringifyExprMacroTest.java         |  10 +-
 .../expression/IPv6AddressMatchExprMacroTest.java  |  14 +-
 .../query/expression/LookupExprMacroTest.java      |   2 +-
 .../expression/TimestampExtractExprMacroTest.java  |  26 +--
 .../query/expression/TimestampShiftMacroTest.java  |  30 ++--
 .../druid/segment/filter/EqualityFilterTests.java  |   2 +-
 .../druid/segment/filter/ExpressionFilterTest.java |   2 +-
 .../druid/sql/calcite/CalciteJoinQueryTest.java    |  11 +-
 .../calcite/CalciteLookupFunctionQueryTest.java    |  18 +-
 .../apache/druid/sql/calcite/CalciteQueryTest.java |  10 +-
 .../sql/calcite/expression/ExpressionsTest.java    |  21 ---
 46 files changed, 315 insertions(+), 318 deletions(-)

diff --git 
a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaPostAggMacros.java
 
b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaPostAggMacros.java
index 3f21ebca105..4b7ce852fd1 100644
--- 
a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaPostAggMacros.java
+++ 
b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaPostAggMacros.java
@@ -85,7 +85,7 @@ public class ThetaPostAggMacros
       ExprEval eval = estimateExpr.eval(bindings);
       final Object valObj = eval.value();
       if (valObj == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofDouble(null);
       }
       if (valObj instanceof SketchHolder) {
         SketchHolder thetaSketchHolder = (SketchHolder) valObj;
diff --git 
a/extensions-core/testing-tools/src/main/java/org/apache/druid/query/expressions/SleepExprMacro.java
 
b/extensions-core/testing-tools/src/main/java/org/apache/druid/query/expressions/SleepExprMacro.java
index a947f9ef4b7..413625b3988 100644
--- 
a/extensions-core/testing-tools/src/main/java/org/apache/druid/query/expressions/SleepExprMacro.java
+++ 
b/extensions-core/testing-tools/src/main/java/org/apache/druid/query/expressions/SleepExprMacro.java
@@ -70,7 +70,7 @@ public class SleepExprMacro implements ExprMacro
               Thread.sleep((long) (seconds * 1000));
             }
           }
-          return ExprEval.of(null);
+          return ExprEval.ofMissing();
         }
         catch (InterruptedException e) {
           Thread.currentThread().interrupt();
diff --git 
a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/SleepExprMacro.java
 
b/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/SleepExprMacro.java
index ea8d29cc1dc..85d00f2d65e 100644
--- 
a/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/SleepExprMacro.java
+++ 
b/integration-tests-ex/tools/src/main/java/org/apache/druid/testing/tools/SleepExprMacro.java
@@ -70,7 +70,7 @@ public class SleepExprMacro implements ExprMacro
               Thread.sleep((long) (seconds * 1000));
             }
           }
-          return ExprEval.of(null);
+          return ExprEval.ofMissing();
         }
         catch (InterruptedException e) {
           Thread.currentThread().interrupt();
diff --git 
a/processing/src/main/java/org/apache/druid/math/expr/ApplyFunction.java 
b/processing/src/main/java/org/apache/druid/math/expr/ApplyFunction.java
index bd39110c76a..3f00f943bf7 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/ApplyFunction.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/ApplyFunction.java
@@ -187,7 +187,7 @@ public interface ApplyFunction extends NamedFunction
 
       Object[] array = arrayEval.asArray();
       if (array == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofMissing();
       }
       if (array.length == 0) {
         return arrayEval;
@@ -250,7 +250,7 @@ public interface ApplyFunction extends NamedFunction
         arrayInputs.add(Arrays.asList(array));
       }
       if (hadNull) {
-        return ExprEval.of(null);
+        return ExprEval.ofMissing();
       }
       if (hadEmpty) {
         return ExprEval.ofStringArray(new String[0]);
@@ -338,7 +338,7 @@ public interface ApplyFunction extends NamedFunction
 
       Object[] array = arrayEval.asArray();
       if (array == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofMissing();
       }
       Object accumulator = accEval.value();
 
@@ -405,7 +405,7 @@ public interface ApplyFunction extends NamedFunction
         arrayInputs.add(Arrays.asList(array));
       }
       if (hadNull) {
-        return ExprEval.of(null);
+        return ExprEval.ofMissing();
       }
       if (hadEmpty) {
         return ExprEval.ofStringArray(new Object[0]);
diff --git 
a/processing/src/main/java/org/apache/druid/math/expr/BinaryEvalOpExprBase.java 
b/processing/src/main/java/org/apache/druid/math/expr/BinaryEvalOpExprBase.java
index 1ec567f97b0..588a425aae5 100644
--- 
a/processing/src/main/java/org/apache/druid/math/expr/BinaryEvalOpExprBase.java
+++ 
b/processing/src/main/java/org/apache/druid/math/expr/BinaryEvalOpExprBase.java
@@ -132,7 +132,7 @@ abstract class BinaryEvalOpExprBase extends BinaryOpExprBase
     // Result of any Binary expressions is null if any of the argument is null.
     // e.g "select null * 2 as c;" or "select null + 1 as c;" will return null 
as per Standard SQL spec.
     if (leftVal.value() == null || rightVal.value() == null) {
-      return ExprEval.of(null);
+      return ExprEval.ofMissing();
     }
 
     ExpressionType type = ExpressionTypeConversion.autoDetect(leftVal, 
rightVal);
@@ -144,7 +144,7 @@ abstract class BinaryEvalOpExprBase extends BinaryOpExprBase
       case DOUBLE:
       default:
         if (leftVal.isNumericNull() || rightVal.isNumericNull()) {
-          return ExprEval.of(null);
+          return ExprEval.ofMissing();
         }
         return ExprEval.of(evalDouble(leftVal.asDouble(), 
rightVal.asDouble()));
     }
@@ -183,7 +183,7 @@ abstract class BinaryBooleanOpExprBase extends 
BinaryOpExprBase
     // Result of any Binary expressions is null if any of the argument is null.
     // e.g "select null * 2 as c;" or "select null + 1 as c;" will return null 
as per Standard SQL spec.
     if (leftVal.value() == null || rightVal.value() == null) {
-      return ExprEval.of(null);
+      return ExprEval.ofMissing();
     }
 
     ExpressionType type = ExpressionTypeConversion.autoDetect(leftVal, 
rightVal);
@@ -201,7 +201,7 @@ abstract class BinaryBooleanOpExprBase extends 
BinaryOpExprBase
       case DOUBLE:
       default:
         if (leftVal.isNumericNull() || rightVal.isNumericNull()) {
-          return ExprEval.of(null);
+          return ExprEval.ofMissing();
         }
         result = evalDouble(leftVal.asDouble(), rightVal.asDouble());
         break;
diff --git 
a/processing/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java
 
b/processing/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java
index fa31c10c61e..bffc0435335 100644
--- 
a/processing/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java
+++ 
b/processing/src/main/java/org/apache/druid/math/expr/BinaryMathOperatorExpr.java
@@ -53,7 +53,7 @@ final class BinPlusExpr extends BinaryEvalOpExprBase
   @Override
   protected ExprEval evalString(@Nullable String left, @Nullable String right)
   {
-    return ExprEval.of(left + right);
+    return ExprEval.ofString(left + right);
   }
 
   @Override
diff --git 
a/processing/src/main/java/org/apache/druid/math/expr/BuiltInExprMacros.java 
b/processing/src/main/java/org/apache/druid/math/expr/BuiltInExprMacros.java
index 0f5a08d2eb7..189ae918c3d 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/BuiltInExprMacros.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/BuiltInExprMacros.java
@@ -181,7 +181,7 @@ public class BuiltInExprMacros
       {
         ExprEval<?> toDecode = arg.eval(bindings);
         if (toDecode.value() == null) {
-          return ExprEval.of(null);
+          return ExprEval.ofString(null);
         }
         return new 
StringExpr(StringUtils.fromUtf8(StringUtils.decodeBase64String(toDecode.asString()))).eval(bindings);
       }
diff --git 
a/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java 
b/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java
index 6d44542171a..c3f5fc0c012 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/ConstantExpr.java
@@ -412,7 +412,7 @@ class StringExpr extends ConstantExpr<String>
   @Override
   protected ExprEval realEval()
   {
-    return ExprEval.of(value);
+    return ExprEval.ofString(value);
   }
 
   @Override
diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java 
b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java
index 0afa96c0f94..d69235bd11c 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/ExprEval.java
@@ -64,7 +64,7 @@ public abstract class ExprEval<T>
    *                                 {@link ByteBuffer}. Certain types are 
deserialized more efficiently if allowed
    *                                 to retain references to the provided 
buffer.
    */
-  public static ExprEval deserialize(
+  public static ExprEval<?> deserialize(
       final ByteBuffer buffer,
       final int offset,
       final int maxSize,
@@ -281,17 +281,40 @@ public abstract class ExprEval<T>
     return Object.class;
   }
 
-  public static ExprEval of(long longValue)
+  /**
+   * Eval that represents a null value of undefined type. Commonly used to 
represent columns that do not exist.
+   *
+   * Behaviorally equivalent to {@code ofLong(null)}. Generally, this function 
is preferred if the type is unknown, and
+   * {@code ofLong(null)} is preferred if the type is known to be long.
+   */
+  public static ExprEval<?> ofMissing()
+  {
+    return LongExprEval.OF_NULL;
+  }
+
+  public static ExprEval<?> of(long longValue)
   {
     return new LongExprEval(longValue);
   }
 
-  public static ExprEval of(double doubleValue)
+  public static ExprEval<?> of(double doubleValue)
   {
     return new DoubleExprEval(doubleValue);
   }
 
-  public static ExprEval of(@Nullable String stringValue)
+  /**
+   * Equivalent to {@link #ofString(String)}. Deprecated because the pattern 
{@code ExprEval.of(null)} for
+   * an "unknown type" null is not recommended-- instead it should be {@link 
ExprEval#ofMissing()}
+   *
+   * @deprecated use {@link #ofString(String)} instead, which is clearer as to 
type
+   */
+  @Deprecated
+  public static ExprEval<?> of(@Nullable String stringValue)
+  {
+    return ofString(stringValue);
+  }
+
+  public static ExprEval<?> ofString(@Nullable String stringValue)
   {
     if (stringValue == null) {
       return StringExprEval.OF_NULL;
@@ -299,7 +322,7 @@ public abstract class ExprEval<T>
     return new StringExprEval(stringValue);
   }
 
-  public static ExprEval ofLong(@Nullable Number longValue)
+  public static ExprEval<?> ofLong(@Nullable Number longValue)
   {
     if (longValue == null) {
       return LongExprEval.OF_NULL;
@@ -307,7 +330,7 @@ public abstract class ExprEval<T>
     return new LongExprEval(longValue);
   }
 
-  public static ExprEval ofDouble(@Nullable Number doubleValue)
+  public static ExprEval<?> ofDouble(@Nullable Number doubleValue)
   {
     if (doubleValue == null) {
       return DoubleExprEval.OF_NULL;
@@ -315,7 +338,7 @@ public abstract class ExprEval<T>
     return new DoubleExprEval(doubleValue);
   }
 
-  public static ExprEval ofLongArray(@Nullable Object[] longValue)
+  public static ExprEval<?> ofLongArray(@Nullable Object[] longValue)
   {
     if (longValue == null) {
       return ArrayExprEval.OF_NULL_LONG;
@@ -323,7 +346,7 @@ public abstract class ExprEval<T>
     return new ArrayExprEval(ExpressionType.LONG_ARRAY, longValue);
   }
 
-  public static ExprEval ofDoubleArray(@Nullable Object[] doubleValue)
+  public static ExprEval<?> ofDoubleArray(@Nullable Object[] doubleValue)
   {
     if (doubleValue == null) {
       return ArrayExprEval.OF_NULL_DOUBLE;
@@ -331,7 +354,7 @@ public abstract class ExprEval<T>
     return new ArrayExprEval(ExpressionType.DOUBLE_ARRAY, doubleValue);
   }
 
-  public static ExprEval ofStringArray(@Nullable Object[] stringValue)
+  public static ExprEval<?> ofStringArray(@Nullable Object[] stringValue)
   {
     if (stringValue == null) {
       return ArrayExprEval.OF_NULL_STRING;
@@ -340,7 +363,7 @@ public abstract class ExprEval<T>
   }
 
 
-  public static ExprEval ofArray(ExpressionType outputType, @Nullable Object[] 
value)
+  public static ExprEval<?> ofArray(ExpressionType outputType, @Nullable 
Object[] value)
   {
     Preconditions.checkArgument(outputType.isArray(), "Output type %s is not 
an array", outputType);
     return new ArrayExprEval(outputType, value);
@@ -349,12 +372,12 @@ public abstract class ExprEval<T>
   /**
    * Convert a boolean into a long expression type
    */
-  public static ExprEval ofLongBoolean(boolean value)
+  public static ExprEval<?> ofLongBoolean(boolean value)
   {
     return value ? LongExprEval.TRUE : LongExprEval.FALSE;
   }
 
-  public static ExprEval ofComplex(ExpressionType outputType, @Nullable Object 
value)
+  public static ExprEval<?> ofComplex(ExpressionType outputType, @Nullable 
Object value)
   {
     if (ExpressionType.NESTED_DATA.equals(outputType)) {
       return new NestedDataExprEval(value);
@@ -362,7 +385,7 @@ public abstract class ExprEval<T>
     return new ComplexExprEval(outputType, value);
   }
 
-  public static ExprEval bestEffortArray(@Nullable List<?> theList)
+  public static ExprEval<?> bestEffortArray(@Nullable List<?> theList)
   {
     // do not convert empty lists to arrays with a single null element here, 
because that should have been done
     // by the selectors preparing their ObjectBindings if necessary. If we get 
to this point it was legitimately
@@ -377,13 +400,13 @@ public abstract class ExprEval<T>
   /**
    * Examine java type to find most appropriate expression type
    */
-  public static ExprEval bestEffortOf(@Nullable Object val)
+  public static ExprEval<?> bestEffortOf(@Nullable Object val)
   {
     if (val == null) {
-      return StringExprEval.OF_NULL;
+      return LongExprEval.OF_NULL;
     }
     if (val instanceof ExprEval) {
-      return (ExprEval) val;
+      return (ExprEval<?>) val;
     }
     if (val instanceof String) {
       return new StringExprEval((String) val);
@@ -496,7 +519,7 @@ public abstract class ExprEval<T>
    * @param type  type, or null to be equivalent to {@link 
#bestEffortOf(Object)}
    * @param value object to be coerced to the type
    */
-  public static ExprEval ofType(@Nullable ExpressionType type, @Nullable 
Object value)
+  public static ExprEval<?> ofType(@Nullable ExpressionType type, @Nullable 
Object value)
   {
     if (type == null) {
       return bestEffortOf(value);
@@ -521,7 +544,7 @@ public abstract class ExprEval<T>
         if (value instanceof byte[]) {
           return new StringExprEval(StringUtils.encodeBase64String((byte[]) 
value));
         }
-        return of(Evals.asString(value));
+        return ofString(Evals.asString(value));
       case LONG:
         if (value instanceof Number) {
           return ofLong((Number) value);
@@ -844,7 +867,7 @@ public abstract class ExprEval<T>
             return ExprEval.of(asLong());
           }
         case STRING:
-          return ExprEval.of(asString());
+          return ExprEval.ofString(asString());
         case ARRAY:
           switch (castTo.getElementType().getType()) {
             case DOUBLE:
@@ -918,7 +941,7 @@ public abstract class ExprEval<T>
         case LONG:
           return this;
         case STRING:
-          return ExprEval.of(asString());
+          return ExprEval.ofString(asString());
         case ARRAY:
           if (value == null) {
             return new ArrayExprEval(castTo, null);
@@ -1283,7 +1306,7 @@ public abstract class ExprEval<T>
       switch (castTo.getType()) {
         case STRING:
           if (value.length == 1) {
-            return ExprEval.of(asString());
+            return ExprEval.ofString(asString());
           }
           return ExprEval.ofType(castTo, null);
         case LONG:
diff --git 
a/processing/src/main/java/org/apache/druid/math/expr/ExprListenerImpl.java 
b/processing/src/main/java/org/apache/druid/math/expr/ExprListenerImpl.java
index b2aaeadf214..8d2e1a690f8 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/ExprListenerImpl.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/ExprListenerImpl.java
@@ -385,7 +385,7 @@ public class ExprListenerImpl extends ExprBaseListener
   @Override
   public void exitNull(ExprParser.NullContext ctx)
   {
-    nodes.put(ctx, new StringExpr(null));
+    nodes.put(ctx, new NullLongExpr());
   }
 
   @Override
diff --git 
a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java 
b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java
index 2e84b33d7bf..e0582c0774e 100644
--- 
a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java
+++ 
b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessing.java
@@ -45,19 +45,19 @@ public class ExpressionProcessing
   @VisibleForTesting
   public static void initializeForTests()
   {
-    INSTANCE = new ExpressionProcessingConfig(null, null, null, null);
+    INSTANCE = new ExpressionProcessingConfig(null, null, null);
   }
 
   @VisibleForTesting
   public static void initializeForHomogenizeNullMultiValueStrings()
   {
-    INSTANCE = new ExpressionProcessingConfig(null, null, true, null);
+    INSTANCE = new ExpressionProcessingConfig(null, true, null);
   }
 
   @VisibleForTesting
   public static void initializeForFallback()
   {
-    INSTANCE = new ExpressionProcessingConfig(null, null, null, true);
+    INSTANCE = new ExpressionProcessingConfig(null, null, true);
   }
 
   /**
diff --git 
a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java
 
b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java
index 712fac379c8..b437bb7e3d2 100644
--- 
a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java
+++ 
b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java
@@ -21,14 +21,11 @@ package org.apache.druid.math.expr;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.druid.java.util.common.logger.Logger;
 
 import javax.annotation.Nullable;
 
 public class ExpressionProcessingConfig
 {
-  private static final Logger LOG = new 
Logger(ExpressionProcessingConfig.class);
-
   public static final String NULL_HANDLING_LEGACY_LOGICAL_OPS_STRING = 
"druid.expressions.useStrictBooleans";
   // Coerce arrays to multi value strings
   public static final String 
PROCESS_ARRAYS_AS_MULTIVALUE_STRINGS_CONFIG_STRING =
@@ -47,23 +44,13 @@ public class ExpressionProcessingConfig
   @JsonProperty("allowVectorizeFallback")
   private final boolean allowVectorizeFallback;
 
-  @Deprecated
-  @JsonProperty("useStrictBooleans")
-  private final boolean useStrictBooleans;
-
   @JsonCreator
   public ExpressionProcessingConfig(
-      @Deprecated @JsonProperty("useStrictBooleans") @Nullable Boolean 
useStrictBooleans,
       @JsonProperty("processArraysAsMultiValueStrings") @Nullable Boolean 
processArraysAsMultiValueStrings,
       @JsonProperty("homogenizeNullMultiValueStringArrays") @Nullable Boolean 
homogenizeNullMultiValueStringArrays,
       @JsonProperty("allowVectorizeFallback") @Nullable Boolean 
allowVectorizeFallback
   )
   {
-    this.useStrictBooleans = getWithPropertyFallback(
-        useStrictBooleans,
-        NULL_HANDLING_LEGACY_LOGICAL_OPS_STRING,
-        "true"
-    );
     this.processArraysAsMultiValueStrings = getWithPropertyFallbackFalse(
         processArraysAsMultiValueStrings,
         PROCESS_ARRAYS_AS_MULTIVALUE_STRINGS_CONFIG_STRING
diff --git a/processing/src/main/java/org/apache/druid/math/expr/Function.java 
b/processing/src/main/java/org/apache/druid/math/expr/Function.java
index 16a46cbda9b..373fdcc4f42 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/Function.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/Function.java
@@ -257,14 +257,14 @@ public interface Function extends NamedFunction
     protected final ExprEval eval(ExprEval param)
     {
       if (param.isNumericNull()) {
-        return ExprEval.of(null);
-      }
-      if (param.type().is(ExprType.LONG)) {
+        return ExprEval.ofMissing();
+      } else if (param.type().is(ExprType.LONG)) {
         return eval(param.asLong());
       } else if (param.type().is(ExprType.DOUBLE)) {
         return eval(param.asDouble());
+      } else {
+        return ExprEval.ofMissing();
       }
-      return ExprEval.of(null);
     }
 
     protected ExprEval eval(long param)
@@ -322,13 +322,13 @@ public interface Function extends NamedFunction
     {
       // match the logic of BinaryEvalOpExprBase.eval, except there is no 
string handling so both strings is also null
       if (x.value() == null || y.value() == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofMissing();
       }
 
       ExpressionType type = ExpressionTypeConversion.autoDetect(x, y);
       switch (type.getType()) {
         case STRING:
-          return ExprEval.of(null);
+          return ExprEval.ofString(null);
         case LONG:
           return eval(x.asLong(), y.asLong());
         case DOUBLE:
@@ -385,12 +385,12 @@ public interface Function extends NamedFunction
       // this is a copy of the logic of BivariateMathFunction for string 
handling, which itself is a
       // remix of BinaryEvalOpExprBase.eval modified so that string inputs are 
always null outputs
       if (x.value() == null || y.value() == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofMissing();
       }
 
       ExpressionType type = ExpressionTypeConversion.autoDetect(x, y);
       if (type.is(ExprType.STRING)) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
       return eval(x.asLong(), y.asLong());
     }
@@ -422,10 +422,10 @@ public interface Function extends NamedFunction
     {
       final String xString = x.asString();
       if (xString == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
       if (y.isNumericNull()) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
       return eval(xString, y.asLong());
     }
@@ -468,7 +468,7 @@ public interface Function extends NamedFunction
       final ExprEval arrayExpr = getArrayArgument(args).eval(bindings);
       final ExprEval scalarExpr = getScalarArgument(args).eval(bindings);
       if (arrayExpr.asArray() == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofMissing();
       }
       return doApply(arrayExpr, scalarExpr);
     }
@@ -597,7 +597,7 @@ public interface Function extends NamedFunction
       final Object[] array2 = rhsExpr.asArray();
 
       if (array1 == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofMissing();
       }
       if (array2 == null) {
         return lhsExpr;
@@ -655,7 +655,7 @@ public interface Function extends NamedFunction
     public ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings)
     {
       if (args.isEmpty()) {
-        return ExprEval.of(null);
+        return ExprEval.ofLong(null);
       }
 
       // evaluate arguments and collect output type
@@ -666,10 +666,12 @@ public interface Function extends NamedFunction
         ExprEval<?> exprEval = expr.eval(bindings);
         ExpressionType exprType = exprEval.type();
 
+        if (!isValidType(exprType)) {
+          throw validationFailed("does not accept %s types", exprType);
+        }
+        outputType = ExpressionTypeConversion.function(outputType, exprType);
+
         if (exprEval.value() != null) {
-          if (isValidType(exprType)) {
-            outputType = ExpressionTypeConversion.function(outputType, 
exprType);
-          }
           evals.add(exprEval);
         }
       }
@@ -681,19 +683,16 @@ public interface Function extends NamedFunction
         // databases (e.g., MySQL) return null if any expression is null.
         // https://www.postgresql.org/docs/9.5/functions-conditional.html
         // 
https://dev.mysql.com/doc/refman/8.0/en/comparison-operators.html#function_least
-        return ExprEval.of(null);
+        return ExprEval.ofType(outputType, null);
       }
 
       switch (outputType.getType()) {
         case DOUBLE:
-          //noinspection OptionalGetWithoutIsPresent (empty list handled 
earlier)
           return 
ExprEval.of(evals.stream().mapToDouble(ExprEval::asDouble).reduce(doubleReducer).getAsDouble());
         case LONG:
-          //noinspection OptionalGetWithoutIsPresent (empty list handled 
earlier)
           return 
ExprEval.of(evals.stream().mapToLong(ExprEval::asLong).reduce(longReducer).getAsLong());
         default:
-          //noinspection OptionalGetWithoutIsPresent (empty list handled 
earlier)
-          return 
ExprEval.of(evals.stream().map(ExprEval::asString).reduce(stringReducer).get());
+          return 
ExprEval.ofString(evals.stream().map(ExprEval::asString).reduce(stringReducer).get());
       }
     }
 
@@ -705,7 +704,7 @@ public interface Function extends NamedFunction
         case STRING:
           return true;
         default:
-          throw validationFailed("does not accept %s types", exprType);
+          return false;
       }
     }
   }
@@ -1506,7 +1505,7 @@ public interface Function extends NamedFunction
       ExprEval value1 = args.get(0).eval(bindings);
 
       if (value1.isNumericNull()) {
-        return ExprEval.of(null);
+        return ExprEval.ofLong(null);
       }
 
       if (!value1.type().anyOf(ExprType.LONG, ExprType.DOUBLE)) {
@@ -1556,7 +1555,7 @@ public interface Function extends NamedFunction
         BigDecimal decimal = safeGetFromDouble(param.asDouble());
         return ExprEval.of(decimal.setScale(scale, 
RoundingMode.HALF_UP).doubleValue());
       } else {
-        return ExprEval.of(null);
+        return ExprEval.ofLong(null);
       }
     }
 
@@ -1964,16 +1963,10 @@ public interface Function extends NamedFunction
     protected ExprEval eval(ExprEval x, ExprEval y)
     {
       if (x.value() == null || y.value() == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofDouble(null);
       }
 
-      ExpressionType type = ExpressionTypeConversion.autoDetect(x, y);
-      switch (type.getType()) {
-        case STRING:
-          return ExprEval.of(null);
-        default:
-          return ExprEval.of(Math.scalb(x.asDouble(), y.asInt()));
-      }
+      return ExprEval.of(Math.scalb(x.asDouble(), y.asInt()));
     }
 
     @Override
@@ -2000,9 +1993,6 @@ public interface Function extends NamedFunction
     @Override
     protected ExprEval eval(ExprEval x, ExprEval y)
     {
-      if (x.value() == null) {
-        return ExprEval.of(null);
-      }
       ExpressionType castTo;
       try {
         castTo = 
ExpressionType.fromString(StringUtils.toUpperCase(y.asString()));
@@ -2010,7 +2000,11 @@ public interface Function extends NamedFunction
       catch (IllegalArgumentException e) {
         throw validationFailed("Invalid type [%s]", y.asString());
       }
-      return x.castTo(castTo);
+      if (x.value() == null) {
+        return ExprEval.ofType(castTo, null);
+      } else {
+        return x.castTo(castTo);
+      }
     }
 
     @Override
@@ -2191,7 +2185,7 @@ public interface Function extends NamedFunction
         }
       }
 
-      return ExprEval.of(null);
+      return ExprEval.ofMissing();
     }
 
     @Override
@@ -2266,7 +2260,7 @@ public interface Function extends NamedFunction
         }
       }
 
-      return ExprEval.of(null);
+      return ExprEval.ofMissing();
     }
 
     @Override
@@ -2694,15 +2688,15 @@ public interface Function extends NamedFunction
     @Override
     public ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings)
     {
-      if (args.size() == 0) {
-        return ExprEval.of(null);
+      if (args.isEmpty()) {
+        return ExprEval.ofString(null);
       } else {
         // Pass first argument in to the constructor to provide StringBuilder 
a little extra sizing hint.
         String first = args.get(0).eval(bindings).asString();
         if (first == null) {
           // Result of concatenation is null if any of the Values is null.
           // e.g. 'select CONCAT(null, "abc") as c;' will return null as per 
Standard SQL spec.
-          return ExprEval.of(null);
+          return ExprEval.ofString(null);
         }
         final StringBuilder builder = new StringBuilder(first);
         for (int i = 1; i < args.size(); i++) {
@@ -2710,12 +2704,12 @@ public interface Function extends NamedFunction
           if (s == null) {
             // Result of concatenation is null if any of the Values is null.
             // e.g. 'select CONCAT(null, "abc") as c;' will return null as per 
Standard SQL spec.
-            return ExprEval.of(null);
+            return ExprEval.ofString(null);
           } else {
             builder.append(s);
           }
         }
-        return ExprEval.of(builder.toString());
+        return ExprEval.ofString(builder.toString());
       }
     }
 
@@ -2791,7 +2785,7 @@ public interface Function extends NamedFunction
       final String formatString = args.get(0).eval(bindings).asString();
 
       if (formatString == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
 
       final Object[] formatArgs = new Object[args.size() - 1];
@@ -2799,7 +2793,7 @@ public interface Function extends NamedFunction
         formatArgs[i - 1] = args.get(i).eval(bindings).value();
       }
 
-      return ExprEval.of(StringUtils.nonStrictFormat(formatString, 
formatArgs));
+      return ExprEval.ofString(StringUtils.nonStrictFormat(formatString, 
formatArgs));
     }
 
     @Override
@@ -2831,7 +2825,7 @@ public interface Function extends NamedFunction
       final String needle = args.get(1).eval(bindings).asString();
 
       if (haystack == null || needle == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofLong(null);
       }
 
       final int fromIndex;
@@ -2873,7 +2867,7 @@ public interface Function extends NamedFunction
       final String arg = args.get(0).eval(bindings).asString();
 
       if (arg == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
 
       // Behaves like SubstringDimExtractionFn, not SQL SUBSTRING
@@ -2882,14 +2876,14 @@ public interface Function extends NamedFunction
 
       if (index < arg.length()) {
         if (length >= 0) {
-          return ExprEval.of(arg.substring(index, Math.min(index + length, 
arg.length())));
+          return ExprEval.ofString(arg.substring(index, Math.min(index + 
length, arg.length())));
         } else {
-          return ExprEval.of(arg.substring(index));
+          return ExprEval.ofString(arg.substring(index));
         }
       } else {
         // this is a behavior mismatch with SQL SUBSTRING to be consistent 
with SubstringDimExtractionFn
         // In SQL, something like 'select substring("abc", 4,5) as c;' will 
return an empty string
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
     }
 
@@ -2930,7 +2924,7 @@ public interface Function extends NamedFunction
         throw validationFailed("needs a positive integer as the second 
argument");
       }
       int len = x.length();
-      return ExprEval.of(y < len ? x.substring(len - yInt) : x);
+      return ExprEval.ofString(y < len ? x.substring(len - yInt) : x);
     }
   }
 
@@ -2956,7 +2950,7 @@ public interface Function extends NamedFunction
       if (yInt < 0 || yInt != y) {
         throw validationFailed("needs a positive integer as the second 
argument");
       }
-      return ExprEval.of(y < x.length() ? x.substring(0, yInt) : x);
+      return ExprEval.ofString(y < x.length() ? x.substring(0, yInt) : x);
     }
   }
 
@@ -2975,9 +2969,9 @@ public interface Function extends NamedFunction
       final String pattern = args.get(1).eval(bindings).asString();
       final String replacement = args.get(2).eval(bindings).asString();
       if (arg == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
-      return ExprEval.of(StringUtils.replace(arg, pattern, replacement));
+      return ExprEval.ofString(StringUtils.replace(arg, pattern, replacement));
     }
 
     @Override
@@ -3007,9 +3001,9 @@ public interface Function extends NamedFunction
     {
       final String arg = args.get(0).eval(bindings).asString();
       if (arg == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
-      return ExprEval.of(StringUtils.toLowerCase(arg));
+      return ExprEval.ofString(StringUtils.toLowerCase(arg));
     }
 
     @Override
@@ -3039,9 +3033,9 @@ public interface Function extends NamedFunction
     {
       final String arg = args.get(0).eval(bindings).asString();
       if (arg == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
-      return ExprEval.of(StringUtils.toUpperCase(arg));
+      return ExprEval.ofString(StringUtils.toUpperCase(arg));
     }
 
     @Override
@@ -3076,11 +3070,8 @@ public interface Function extends NamedFunction
     @Override
     protected ExprEval eval(ExprEval param)
     {
-      if (!param.type().is(ExprType.STRING)) {
-        throw validationFailed("needs a STRING argument but got %s instead", 
param.type());
-      }
       final String arg = param.asString();
-      return ExprEval.of(arg == null ? null : new 
StringBuilder(arg).reverse().toString());
+      return ExprEval.ofString(arg == null ? null : new 
StringBuilder(arg).reverse().toString());
     }
   }
 
@@ -3106,7 +3097,7 @@ public interface Function extends NamedFunction
       if (yInt != y) {
         throw validationFailed("needs an integer as the second argument");
       }
-      return ExprEval.of(y < 1 ? null : StringUtils.repeat(x, yInt));
+      return ExprEval.ofString(y < 1 ? null : StringUtils.repeat(x, yInt));
     }
   }
 
@@ -3126,9 +3117,9 @@ public interface Function extends NamedFunction
       String pad = args.get(2).eval(bindings).asString();
 
       if (base == null || pad == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       } else {
-        return ExprEval.of(len == 0 ? null : StringUtils.lpad(base, len, pad));
+        return ExprEval.ofString(len == 0 ? null : StringUtils.lpad(base, len, 
pad));
       }
 
     }
@@ -3163,9 +3154,9 @@ public interface Function extends NamedFunction
       String pad = args.get(2).eval(bindings).asString();
 
       if (base == null || pad == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       } else {
-        return ExprEval.of(len == 0 ? null : StringUtils.rpad(base, len, pad));
+        return ExprEval.ofString(len == 0 ? null : StringUtils.rpad(base, len, 
pad));
       }
 
     }
@@ -3196,11 +3187,8 @@ public interface Function extends NamedFunction
     public ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings)
     {
       ExprEval value = args.get(0).eval(bindings);
-      if (!value.type().is(ExprType.STRING)) {
-        throw validationFailed(
-            "first argument should be a STRING but got %s instead",
-            value.type()
-        );
+      if (value.value() == null) {
+        return ExprEval.ofLong(null);
       }
 
       DateTimes.UtcFormatter formatter = DateTimes.ISO_DATE_OPTIONAL_TIME;
@@ -3274,7 +3262,7 @@ public interface Function extends NamedFunction
       DateTimeZone timeZone = 
DateTimes.inferTzFromString(args.get(2).eval(bindings).asString());
 
       if (left == null || right == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofLong(null);
       } else {
         return ExprEval.of(DateTimes.subMonths(right, left, timeZone));
       }
@@ -3491,18 +3479,9 @@ public interface Function extends NamedFunction
 
       ExpressionType arrayElementType = null;
 
-      // Try first to determine the element type, only considering nonnull 
values.
+      // Determine the element type, considering null and nonnull values, for 
consistency with getOutputType.
       for (final ExprEval<?> eval : outEval) {
-        if (eval.value() != null) {
-          arrayElementType = 
ExpressionTypeConversion.leastRestrictiveType(arrayElementType, eval.type());
-        }
-      }
-
-      if (arrayElementType == null) {
-        // Try again to determine the element type, this time considering 
nulls.
-        for (final ExprEval<?> eval : outEval) {
-          arrayElementType = 
ExpressionTypeConversion.leastRestrictiveType(arrayElementType, eval.type());
-        }
+        arrayElementType = 
ExpressionTypeConversion.leastRestrictiveType(arrayElementType, eval.type());
       }
 
       final Object[] out = new Object[length];
@@ -3550,7 +3529,7 @@ public interface Function extends NamedFunction
       final ExprEval expr = args.get(0).eval(bindings);
       final Object[] array = expr.asArray();
       if (array == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofLong(null);
       }
 
       return ExprEval.ofLong(array.length);
@@ -3616,7 +3595,7 @@ public interface Function extends NamedFunction
       final ExprEval expr = args.get(0).eval(bindings);
       final String arrayString = expr.asString();
       if (arrayString == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofStringArray(null);
       }
 
       final String split = args.get(1).eval(bindings).asString();
@@ -3651,9 +3630,9 @@ public interface Function extends NamedFunction
       final String join = scalarExpr.asString();
       final Object[] raw = arrayExpr.asArray();
       if (raw == null || raw.length == 1 && raw[0] == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
-      return ExprEval.of(
+      return ExprEval.ofString(
           
Arrays.stream(raw).map(String::valueOf).collect(Collectors.joining(join != null 
? join : ""))
       );
     }
@@ -3683,7 +3662,7 @@ public interface Function extends NamedFunction
       if (array.length > position && position >= 0) {
         return ExprEval.ofType(arrayExpr.elementType(), array[position]);
       }
-      return ExprEval.of(null);
+      return ExprEval.ofType(arrayExpr.elementType(), null);
     }
   }
 
@@ -3711,7 +3690,7 @@ public interface Function extends NamedFunction
       if (array.length > position && position >= 0) {
         return ExprEval.ofType(arrayExpr.elementType(), array[position]);
       }
-      return ExprEval.of(null);
+      return ExprEval.ofType(arrayExpr.elementType(), null);
     }
   }
 
@@ -3883,7 +3862,7 @@ public interface Function extends NamedFunction
       @Override
       public ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings)
       {
-        return ExprEval.of(null);
+        return ExprEval.ofLong(null);
       }
     }
 
@@ -4649,7 +4628,7 @@ public interface Function extends NamedFunction
       final ExprEval expr = args.get(0).eval(bindings);
       final Object[] array = expr.asArray();
       if (array == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofArray(expr.asArrayType(), null);
       }
 
       final int start = args.get(1).eval(bindings).asInt();
@@ -4660,7 +4639,7 @@ public interface Function extends NamedFunction
 
       if (start < 0 || start > array.length || start > end) {
         // Arrays.copyOfRange will throw exception in these cases
-        return ExprEval.of(null);
+        return ExprEval.ofArray(expr.asArrayType(), null);
       }
 
       return ExprEval.ofArray(expr.asArrayType(), 
Arrays.copyOfRange(expr.asArray(), start, end));
@@ -4676,7 +4655,7 @@ public interface Function extends NamedFunction
     {
       final ExprEval valueParam = args.get(0).eval(bindings);
       if (valueParam.isNumericNull()) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
 
       /**
@@ -4696,22 +4675,19 @@ public interface Function extends NamedFunction
       long precision = 2;
       if (args.size() > 1) {
         ExprEval precisionParam = args.get(1).eval(bindings);
+        if (precisionParam.value() == null) {
+          throw validationFailed("needs a LONG as its second argument but got 
null");
+        }
         if (!precisionParam.type().is(ExprType.LONG)) {
-          throw validationFailed(
-              "needs a LONG as its second argument but got %s instead",
-              precisionParam.type()
-          );
+          throw validationFailed("needs a LONG as its second argument but got 
%s instead", precisionParam.type());
         }
         precision = precisionParam.asLong();
         if (precision < 0 || precision > 3) {
-          throw validationFailed(
-              "given precision[%d] must be in the range of [0,3]",
-              precision
-          );
+          throw validationFailed("given precision[%d] must be in the range of 
[0,3]", precision);
         }
       }
 
-      return ExprEval.of(HumanReadableBytes.format(valueParam.asLong(), 
precision, this.getUnitSystem()));
+      return ExprEval.ofString(HumanReadableBytes.format(valueParam.asLong(), 
precision, this.getUnitSystem()));
     }
 
     @Override
diff --git 
a/processing/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java 
b/processing/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java
index 97aad5add51..60abd51a99d 100644
--- a/processing/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java
+++ b/processing/src/main/java/org/apache/druid/math/expr/UnaryOperatorExpr.java
@@ -135,7 +135,7 @@ class UnaryMinusExpr extends UnaryExpr
     }
     ExprEval ret = expr.eval(bindings);
     if (ret.value() == null) {
-      return ExprEval.of(null);
+      return ExprEval.ofType(ret.type(), null);
     }
     if (ret.type().is(ExprType.LONG)) {
       return ExprEval.of(-ret.asLong());
@@ -178,7 +178,7 @@ class UnaryNotExpr extends UnaryExpr
   {
     ExprEval ret = expr.eval(bindings);
     if (ret.value() == null) {
-      return ExprEval.of(null);
+      return ExprEval.ofLong(null);
     }
     return ExprEval.ofLongBoolean(!ret.asBoolean());
   }
diff --git 
a/processing/src/main/java/org/apache/druid/math/expr/vector/FallbackVectorProcessor.java
 
b/processing/src/main/java/org/apache/druid/math/expr/vector/FallbackVectorProcessor.java
index e8cf412463d..402739c3613 100644
--- 
a/processing/src/main/java/org/apache/druid/math/expr/vector/FallbackVectorProcessor.java
+++ 
b/processing/src/main/java/org/apache/druid/math/expr/vector/FallbackVectorProcessor.java
@@ -45,7 +45,7 @@ public abstract class FallbackVectorProcessor<T> implements 
ExprVectorProcessor<
   final Supplier<ExprEval<?>> fn;
   final List<AdaptedExpr> adaptedArgs;
 
-  private final ExpressionType outputType;
+  protected final ExpressionType outputType;
 
   private FallbackVectorProcessor(
       final Supplier<ExprEval<?>> fn,
@@ -216,10 +216,10 @@ public abstract class FallbackVectorProcessor<T> 
implements ExprVectorProcessor<
           adaptedArg.setRowNumber(i);
         }
 
-        outValues[i] = fn.get().value();
+        outValues[i] = fn.get().castTo(outputType).value();
       }
 
-      return new ExprEvalObjectVector(outValues, getOutputType());
+      return new ExprEvalObjectVector(outValues, outputType);
     }
 
     @Override
@@ -384,7 +384,7 @@ public abstract class FallbackVectorProcessor<T> implements 
ExprVectorProcessor<
         final boolean isNull = results.getNullVector() != null && 
results.getNullVector()[rowNum];
         return ExprEval.ofDouble(isNull ? null : 
results.getDoubleVector()[rowNum]);
       } else {
-        return ExprEval.ofType(type, results.getObjectVector()[rowNum]);
+        return ExprEval.bestEffortOf(results.getObjectVector()[rowNum]);
       }
     }
 
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacro.java
 
b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacro.java
index e56d6a1146f..f6a83700b73 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacro.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacro.java
@@ -80,7 +80,7 @@ public class IPv4AddressStringifyExprMacro implements 
ExprMacroTable.ExprMacro
           case LONG:
             return evalAsLong(eval);
           default:
-            return ExprEval.of(null);
+            return ExprEval.ofString(null);
         }
       }
 
@@ -100,19 +100,19 @@ public class IPv4AddressStringifyExprMacro implements 
ExprMacroTable.ExprMacro
     if (IPv4AddressExprUtils.isValidIPv4Address(eval.asString())) {
       return eval;
     }
-    return ExprEval.of(null);
+    return ExprEval.ofString(null);
   }
 
   private static ExprEval evalAsLong(ExprEval eval)
   {
     if (eval.isNumericNull()) {
-      return ExprEval.of(null);
+      return ExprEval.ofString(null);
     }
 
     IPv4Address address = IPv4AddressExprUtils.parse(eval.asLong());
     if (address == null) {
-      return ExprEval.of(null);
+      return ExprEval.ofString(null);
     }
-    return ExprEval.of(IPv4AddressExprUtils.toString(address));
+    return ExprEval.ofString(IPv4AddressExprUtils.toString(address));
   }
 }
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/LookupExprMacro.java
 
b/processing/src/main/java/org/apache/druid/query/expression/LookupExprMacro.java
index 7a1d1345a89..559623d170c 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/LookupExprMacro.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/LookupExprMacro.java
@@ -92,7 +92,7 @@ public class LookupExprMacro implements 
ExprMacroTable.ExprMacro
       @Override
       public ExprEval eval(final ObjectBinding bindings)
       {
-        return ExprEval.of(extractionFn.apply(arg.eval(bindings).asString()));
+        return 
ExprEval.ofString(extractionFn.apply(arg.eval(bindings).asString()));
       }
 
       @Nullable
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java
 
b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java
index 3939dd8c39e..66b4321ee82 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/NestedDataExpressions.java
@@ -447,7 +447,7 @@ public class NestedDataExpressions
         if (valAtPath.type().isPrimitive() || 
valAtPath.type().isPrimitiveArray()) {
           return valAtPath;
         }
-        return ExprEval.of(null);
+        return ExprEval.ofMissing();
       }
 
       @Nullable
@@ -527,7 +527,7 @@ public class NestedDataExpressions
         if (valAtPath.type().isPrimitive() || 
valAtPath.type().isPrimitiveArray()) {
           return castTo == null ? valAtPath : valAtPath.castTo(castTo);
         }
-        return castTo == null ? ExprEval.of(null) : ExprEval.ofType(castTo, 
null);
+        return castTo == null ? ExprEval.ofMissing() : ExprEval.ofType(castTo, 
null);
       }
 
       @Nullable
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java
 
b/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java
index 9d95ec1e9f1..6a690787735 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java
@@ -80,11 +80,11 @@ public class RegexpExtractExprMacro implements 
ExprMacroTable.ExprMacro
 
         if (s == null) {
           // True nulls do not match anything.
-          return ExprEval.of(null);
+          return ExprEval.ofString(null);
         } else {
           final Matcher matcher = pattern.matcher(s);
           final String retVal = matcher.find() ? matcher.group(index) : null;
-          return ExprEval.of(retVal);
+          return ExprEval.ofString(retVal);
         }
       }
 
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/RegexpReplaceExprMacro.java
 
b/processing/src/main/java/org/apache/druid/query/expression/RegexpReplaceExprMacro.java
index cf7dbdd7de8..835fdcd6c91 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/RegexpReplaceExprMacro.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/RegexpReplaceExprMacro.java
@@ -105,17 +105,17 @@ public class RegexpReplaceExprMacro implements 
ExprMacroTable.ExprMacro
     public ExprEval<?> eval(final ObjectBinding bindings)
     {
       if (pattern == null || replacement == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       }
 
       final String s = arg.eval(bindings).asString();
 
       if (s == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       } else {
         final Matcher matcher = pattern.matcher(s);
         final String retVal = matcher.replaceAll(replacement);
-        return ExprEval.of(retVal);
+        return ExprEval.ofString(retVal);
       }
     }
   }
@@ -139,11 +139,11 @@ public class RegexpReplaceExprMacro implements 
ExprMacroTable.ExprMacro
       final String replacement = args.get(2).eval(bindings).asString();
 
       if (s == null || pattern == null || replacement == null) {
-        return ExprEval.of(null);
+        return ExprEval.ofString(null);
       } else {
         final Matcher matcher = Pattern.compile(pattern).matcher(s);
         final String retVal = matcher.replaceAll(replacement);
-        return ExprEval.of(retVal);
+        return ExprEval.ofString(retVal);
       }
     }
   }
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java
 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java
index 29460529afa..6894276a848 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampCeilExprMacro.java
@@ -76,7 +76,7 @@ public class TimestampCeilExprMacro implements 
ExprMacroTable.ExprMacro
       ExprEval eval = args.get(0).eval(bindings);
       if (eval.isNumericNull()) {
         // Return null if the argument if null.
-        return ExprEval.of(null);
+        return ExprEval.ofLong(null);
       }
       long argTime = eval.asLong();
       long bucketStartTime = granularity.bucketStart(argTime);
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampExtractExprMacro.java
 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampExtractExprMacro.java
index 722f75e51ab..79b8dd5dcd7 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampExtractExprMacro.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampExtractExprMacro.java
@@ -182,7 +182,7 @@ public class TimestampExtractExprMacro implements 
ExprMacroTable.ExprMacro
       final ExprEval<?> eval = args.get(0).eval(bindings);
       if (eval.value() == null) {
         // Return null if the argument if null.
-        return ExprEval.of(null);
+        return ExprEval.ofType(getOutputExpressionType(unit), null);
       }
       final DateTime dateTime = new DateTime(eval.value(), chronology);
       switch (getOutputExpressionType(unit).getType()) {
@@ -246,7 +246,7 @@ public class TimestampExtractExprMacro implements 
ExprMacroTable.ExprMacro
       final ExprEval<?> eval = args.get(0).eval(bindings);
       if (eval.value() == null) {
         // Return null if the argument if null.
-        return ExprEval.of(null);
+        return ExprEval.ofType(getOutputExpressionType(unit), null);
       }
       final Chronology chronology = computeChronology(args, bindings);
       final DateTime dateTime = new DateTime(eval.value(), chronology);
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java
 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java
index 3a17c7cd6df..c0b4660641a 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampFloorExprMacro.java
@@ -115,7 +115,7 @@ public class TimestampFloorExprMacro implements 
ExprMacroTable.ExprMacro
       ExprEval eval = args.get(0).eval(bindings);
       if (eval.isNumericNull()) {
         // Return null if the argument if null.
-        return ExprEval.of(null);
+        return ExprEval.ofLong(null);
       }
       return ExprEval.of(granularity.bucketStart(eval.asLong()));
     }
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampFormatExprMacro.java
 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampFormatExprMacro.java
index 299cb19fcfc..8b0e6e58bec 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampFormatExprMacro.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampFormatExprMacro.java
@@ -82,9 +82,9 @@ public class TimestampFormatExprMacro implements 
ExprMacroTable.ExprMacro
         ExprEval eval = arg.eval(bindings);
         if (eval.isNumericNull()) {
           // Return null if the argument if null.
-          return ExprEval.of(null);
+          return ExprEval.ofString(null);
         }
-        return ExprEval.of(formatter.print(arg.eval(bindings).asLong()));
+        return ExprEval.ofString(formatter.print(arg.eval(bindings).asLong()));
       }
 
       @Nullable
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java
 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java
index dc1fc3b73ff..070254cf69a 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java
@@ -78,7 +78,7 @@ public class TimestampParseExprMacro implements 
ExprMacroTable.ExprMacro
       {
         final String value = arg.eval(bindings).asString();
         if (value == null) {
-          return ExprEval.of(null);
+          return ExprEval.ofLong(null);
         }
 
         try {
@@ -87,7 +87,7 @@ public class TimestampParseExprMacro implements 
ExprMacroTable.ExprMacro
         catch (IllegalArgumentException e) {
           // Catch exceptions potentially thrown by formatter.parseDateTime. 
Our docs say that unparseable timestamps
           // are returned as nulls.
-          return ExprEval.of(null);
+          return ExprEval.ofLong(null);
         }
       }
 
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampShiftExprMacro.java
 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampShiftExprMacro.java
index 3e2535d787f..dbc86b6b77d 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/TimestampShiftExprMacro.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/TimestampShiftExprMacro.java
@@ -101,7 +101,7 @@ public class TimestampShiftExprMacro implements 
ExprMacroTable.ExprMacro
     {
       ExprEval timestamp = args.get(0).eval(bindings);
       if (timestamp.isNumericNull()) {
-        return ExprEval.of(null);
+        return ExprEval.ofLong(null);
       }
       return ExprEval.of(chronology.add(period, timestamp.asLong(), step));
     }
@@ -144,7 +144,7 @@ public class TimestampShiftExprMacro implements 
ExprMacroTable.ExprMacro
     {
       ExprEval timestamp = args.get(0).eval(bindings);
       if (timestamp.isNumericNull()) {
-        return ExprEval.of(null);
+        return ExprEval.ofLong(null);
       }
       final Period period = getPeriod(args, bindings);
       final Chronology chronology = getTimeZone(args, bindings);
diff --git 
a/processing/src/main/java/org/apache/druid/query/expression/TrimExprMacro.java 
b/processing/src/main/java/org/apache/druid/query/expression/TrimExprMacro.java
index 33bb58cb7b0..62997e1be79 100644
--- 
a/processing/src/main/java/org/apache/druid/query/expression/TrimExprMacro.java
+++ 
b/processing/src/main/java/org/apache/druid/query/expression/TrimExprMacro.java
@@ -157,7 +157,7 @@ public abstract class TrimExprMacro implements 
ExprMacroTable.ExprMacro
       if (start == 0 && end == s.length()) {
         return stringEval;
       } else {
-        return ExprEval.of(s.substring(start, end));
+        return ExprEval.ofString(s.substring(start, end));
       }
     }
 
@@ -232,7 +232,7 @@ public abstract class TrimExprMacro implements 
ExprMacroTable.ExprMacro
       if (start == 0 && end == s.length()) {
         return stringEval;
       } else {
-        return ExprEval.of(s.substring(start, end));
+        return ExprEval.ofString(s.substring(start, end));
       }
     }
 
diff --git 
a/processing/src/test/java/org/apache/druid/math/expr/ConstantExprTest.java 
b/processing/src/test/java/org/apache/druid/math/expr/ConstantExprTest.java
index a7812f2d160..c61b1967896 100644
--- a/processing/src/test/java/org/apache/druid/math/expr/ConstantExprTest.java
+++ b/processing/src/test/java/org/apache/druid/math/expr/ConstantExprTest.java
@@ -102,8 +102,8 @@ public class ConstantExprTest extends 
InitializedNullHandlingTest
         new NullDoubleExpr(),
         "null",
         "null",
-        // the expressions 'null' is always parsed as a StringExpr(null)
-        new StringExpr(null)
+        // the expressions 'null' is always parsed as a NullLongExpr()
+        new NullLongExpr()
     );
   }
 
@@ -114,8 +114,8 @@ public class ConstantExprTest extends 
InitializedNullHandlingTest
         new NullLongExpr(),
         "null",
         "null",
-        // the expressions 'null' is always parsed as a StringExpr(null)
-        new StringExpr(null)
+        // the expressions 'null' is always parsed as a NullLongExpr()
+        new NullLongExpr()
     );
   }
 
@@ -148,7 +148,8 @@ public class ConstantExprTest extends 
InitializedNullHandlingTest
         new StringExpr(null),
         null,
         "null",
-        new StringExpr(null)
+        // the expressions 'null' is always parsed as a NullLongExpr()
+        new NullLongExpr()
     );
   }
 
diff --git a/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java 
b/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java
index 047c73ffe2e..e0f64a7c36b 100644
--- a/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java
+++ b/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java
@@ -522,7 +522,7 @@ public class EvalTest extends InitializedNullHandlingTest
     Assert.assertEquals("hello", 
ExprEval.ofComplex(ExpressionType.NESTED_DATA, "hello").asString());
     Assert.assertEquals(ExpressionType.STRING, cast.type());
 
-    cast = ExprEval.of("hello").castTo(ExpressionType.NESTED_DATA);
+    cast = ExprEval.ofString("hello").castTo(ExpressionType.NESTED_DATA);
     Assert.assertEquals("hello", cast.value());
     Assert.assertEquals(ExpressionType.NESTED_DATA, cast.type());
 
@@ -613,7 +613,7 @@ public class EvalTest extends InitializedNullHandlingTest
         new Object[]{1234L},
         cast.asArray()
     );
-    cast = ExprEval.of("hello").castTo(nestedArray);
+    cast = ExprEval.ofString("hello").castTo(nestedArray);
     Assert.assertEquals(nestedArray, cast.type());
     Assert.assertArrayEquals(
         new Object[]{"hello"},
@@ -770,7 +770,7 @@ public class EvalTest extends InitializedNullHandlingTest
 
     t = Assert.assertThrows(
         IllegalArgumentException.class,
-        () -> ExprEval.of("hello").castTo(someComplex)
+        () -> ExprEval.ofString("hello").castTo(someComplex)
     );
     Assert.assertEquals("Invalid type, cannot cast [STRING] to 
[COMPLEX<tester>]", t.getMessage());
 
@@ -820,9 +820,9 @@ public class EvalTest extends InitializedNullHandlingTest
     Assert.assertFalse(ExprEval.ofDouble(1.0).isNumericNull());
     Assert.assertTrue(ExprEval.ofDouble(null).isNumericNull());
 
-    Assert.assertTrue(ExprEval.of(null).isNumericNull());
-    Assert.assertTrue(ExprEval.of("one").isNumericNull());
-    Assert.assertFalse(ExprEval.of("1").isNumericNull());
+    Assert.assertTrue(ExprEval.ofString(null).isNumericNull());
+    Assert.assertTrue(ExprEval.ofString("one").isNumericNull());
+    Assert.assertFalse(ExprEval.ofString("1").isNumericNull());
 
     Assert.assertFalse(ExprEval.ofLongArray(new Long[]{1L}).isNumericNull());
     Assert.assertTrue(ExprEval.ofLongArray(new Long[]{null, 2L, 
3L}).isNumericNull());
diff --git 
a/processing/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java 
b/processing/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java
index 528d4844bf1..04ec0492bb2 100644
--- a/processing/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java
+++ b/processing/src/test/java/org/apache/druid/math/expr/ExprEvalTest.java
@@ -77,7 +77,7 @@ public class ExprEvalTest extends InitializedNullHandlingTest
         10,
         16
     ));
-    assertExpr(0, ExprEval.of("hello world"), 10);
+    assertExpr(0, ExprEval.ofString("hello world"), 10);
   }
 
 
@@ -541,7 +541,7 @@ public class ExprEvalTest extends 
InitializedNullHandlingTest
   @Test
   public void testCastString()
   {
-    ExprEval<?> eval = ExprEval.of("hello");
+    ExprEval<?> eval = ExprEval.ofString("hello");
 
     ExprEval<?> cast = eval.castTo(ExpressionType.DOUBLE);
     Assert.assertNull(cast.value());
@@ -564,7 +564,7 @@ public class ExprEvalTest extends 
InitializedNullHandlingTest
     cast = 
eval.castTo(ExpressionTypeFactory.getInstance().ofArray(ExpressionType.NESTED_DATA));
     Assert.assertArrayEquals(new Object[]{"hello"}, (Object[]) cast.value());
 
-    eval = ExprEval.of("1234.3");
+    eval = ExprEval.ofString("1234.3");
 
     cast = eval.castTo(ExpressionType.DOUBLE);
     Assert.assertEquals(1234.3, cast.value());
diff --git 
a/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java 
b/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java
index 22030ac4928..51068feefd4 100644
--- a/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java
+++ b/processing/src/test/java/org/apache/druid/math/expr/FunctionTest.java
@@ -953,7 +953,7 @@ public class FunctionTest extends 
InitializedNullHandlingTest
   }
 
   @Test
-  public void testSizeForatInvalidArgumentType()
+  public void testSizeFormatInvalidArgumentType()
   {
     // x = "foo"
     Throwable t = Assert.assertThrows(
@@ -981,7 +981,7 @@ public class FunctionTest extends 
InitializedNullHandlingTest
                     .eval(bestEffortBindings)
     );
     Assert.assertEquals(
-        "Function[human_readable_binary_byte_format] needs a LONG as its 
second argument but got STRING instead",
+        "Function[human_readable_binary_byte_format] needs a LONG as its 
second argument but got null",
         t.getMessage()
     );
   }
diff --git 
a/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java 
b/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java
index 088df4c9ef5..267ce565ff9 100644
--- a/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java
+++ b/processing/src/test/java/org/apache/druid/math/expr/OutputTypeTest.java
@@ -396,8 +396,8 @@ public class OutputTypeTest extends 
InitializedNullHandlingTest
   @Test
   public void testEvalAutoConversion()
   {
-    final ExprEval<?> nullStringEval = ExprEval.of(null);
-    final ExprEval<?> stringEval = ExprEval.of("wat");
+    final ExprEval<?> nullStringEval = ExprEval.ofString(null);
+    final ExprEval<?> stringEval = ExprEval.ofString("wat");
     final ExprEval<?> longEval = ExprEval.of(1L);
     final ExprEval<?> doubleEval = ExprEval.of(1.0);
     final ExprEval<?> arrayEval = ExprEval.ofLongArray(new Long[]{1L, 2L, 3L});
diff --git 
a/processing/src/test/java/org/apache/druid/math/expr/VectorExprResultConsistencyTest.java
 
b/processing/src/test/java/org/apache/druid/math/expr/VectorExprResultConsistencyTest.java
index bb199f04ee5..01d7aa551db 100644
--- 
a/processing/src/test/java/org/apache/druid/math/expr/VectorExprResultConsistencyTest.java
+++ 
b/processing/src/test/java/org/apache/druid/math/expr/VectorExprResultConsistencyTest.java
@@ -499,6 +499,30 @@ public class VectorExprResultConsistencyTest extends 
InitializedNullHandlingTest
     }
   }
 
+  @Test
+  public void testReduceFns()
+  {
+    try {
+      ExpressionProcessing.initializeForFallback();
+      testExpression("greatest(s1, s2)", types);
+      testExpression("greatest(l1, l2)", types);
+      testExpression("greatest(l1, nonexistent)", types);
+      testExpression("greatest(d1, d2)", types);
+      testExpression("greatest(l1, d2)", types);
+      testExpression("greatest(s1, l2)", types);
+
+      testExpression("least(s1, s2)", types);
+      testExpression("least(l1, l2)", types);
+      testExpression("least(l1, nonexistent)", types);
+      testExpression("least(d1, d2)", types);
+      testExpression("least(l1, d2)", types);
+      testExpression("least(s1, l2)", types);
+    }
+    finally {
+      ExpressionProcessing.initializeForTests();
+    }
+  }
+
   @Test
   public void testJsonFns()
   {
@@ -657,11 +681,14 @@ public class VectorExprResultConsistencyTest extends 
InitializedNullHandlingTest
       NonnullPair<Expr.ObjectBinding[], Expr.VectorInputBinding> bindings = 
makeRandomizedBindings(VECTOR_SIZE, types);
       ExprEvalVector<?> vectorEval = processor.evalVector(bindings.rhs);
       final Object[] vectorVals = vectorEval.getObjectVector();
+      if (outputType != null) {
+        Assert.assertEquals("vector eval type", outputType, 
vectorEval.getType());
+      }
       for (int i = 0; i < VECTOR_SIZE; i++) {
         ExprEval<?> eval = parsed.eval(bindings.lhs[i]);
         // 'null' expressions can have an output type of null, but still 
evaluate in default mode, so skip type checks
-        if (outputType != null && !eval.isNumericNull()) {
-          Assert.assertEquals(eval.type(), outputType);
+        if (outputType != null && eval.value() != null) {
+          Assert.assertEquals("nonvector eval type", eval.type(), outputType);
         }
         if (outputType != null && outputType.isArray()) {
           Assert.assertArrayEquals(
@@ -762,6 +789,7 @@ public class VectorExprResultConsistencyTest extends 
InitializedNullHandlingTest
   {
     SettableVectorInputBinding vectorBinding = new 
SettableVectorInputBinding(vectorSize);
     SettableObjectBinding[] objectBindings = new 
SettableObjectBinding[vectorSize];
+    Expr.InputBindingInspector inspector = 
InputBindings.inspectorFromTypeMap(types);
 
     for (Map.Entry<String, ExpressionType> entry : types.entrySet()) {
       boolean[] nulls = new boolean[vectorSize];
@@ -773,7 +801,7 @@ public class VectorExprResultConsistencyTest extends 
InitializedNullHandlingTest
             nulls[i] = nullsFn.getAsBoolean();
             longs[i] = nulls[i] ? 0L : longsFn.getAsLong();
             if (objectBindings[i] == null) {
-              objectBindings[i] = new SettableObjectBinding();
+              objectBindings[i] = new 
SettableObjectBinding().withInspector(inspector);
             }
             objectBindings[i].withBinding(entry.getKey(), nulls[i] ? null : 
longs[i]);
           }
@@ -785,7 +813,7 @@ public class VectorExprResultConsistencyTest extends 
InitializedNullHandlingTest
             nulls[i] = nullsFn.getAsBoolean();
             doubles[i] = nulls[i] ? 0.0 : doublesFn.getAsDouble();
             if (objectBindings[i] == null) {
-              objectBindings[i] = new SettableObjectBinding();
+              objectBindings[i] = new 
SettableObjectBinding().withInspector(inspector);
             }
             objectBindings[i].withBinding(entry.getKey(), nulls[i] ? null : 
doubles[i]);
           }
@@ -801,7 +829,7 @@ public class VectorExprResultConsistencyTest extends 
InitializedNullHandlingTest
               strings[i] = nulls[i] ? null : String.valueOf(stringFn.get());
             }
             if (objectBindings[i] == null) {
-              objectBindings[i] = new SettableObjectBinding();
+              objectBindings[i] = new 
SettableObjectBinding().withInspector(inspector);
             }
             objectBindings[i].withBinding(entry.getKey(), nulls[i] ? null : 
strings[i]);
           }
diff --git 
a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacroTest.java
 
b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacroTest.java
index bd3298e59de..50251895385 100644
--- 
a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacroTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacroTest.java
@@ -33,15 +33,15 @@ import java.util.Collections;
 
 public class IPv4AddressMatchExprMacroTest extends MacroTestBase
 {
-  private static final Expr IPV4 = ExprEval.of("192.168.0.1").toExpr();
+  private static final Expr IPV4 = ExprEval.ofString("192.168.0.1").toExpr();
   private static final Expr IPV4_LONG = ExprEval.of(3232235521L).toExpr();
-  private static final Expr IPV4_UINT = ExprEval.of("3232235521").toExpr();
-  private static final Expr IPV4_NETWORK = ExprEval.of("192.168.0.0").toExpr();
-  private static final Expr IPV4_BROADCAST = 
ExprEval.of("192.168.255.255").toExpr();
-  private static final Expr IPV6_COMPATIBLE = 
ExprEval.of("::192.168.0.1").toExpr();
-  private static final Expr IPV6_MAPPED = 
ExprEval.of("::ffff:192.168.0.1").toExpr();
-  private static final Expr SUBNET_192_168 = 
ExprEval.of("192.168.0.0/16").toExpr();
-  private static final Expr SUBNET_10 = ExprEval.of("10.0.0.0/8").toExpr();
+  private static final Expr IPV4_UINT = 
ExprEval.ofString("3232235521").toExpr();
+  private static final Expr IPV4_NETWORK = 
ExprEval.ofString("192.168.0.0").toExpr();
+  private static final Expr IPV4_BROADCAST = 
ExprEval.ofString("192.168.255.255").toExpr();
+  private static final Expr IPV6_COMPATIBLE = 
ExprEval.ofString("::192.168.0.1").toExpr();
+  private static final Expr IPV6_MAPPED = 
ExprEval.ofString("::ffff:192.168.0.1").toExpr();
+  private static final Expr SUBNET_192_168 = 
ExprEval.ofString("192.168.0.0/16").toExpr();
+  private static final Expr SUBNET_10 = 
ExprEval.ofString("10.0.0.0/8").toExpr();
   private static final Expr NOT_LITERAL = Parser.parse("\"notliteral\"", 
ExprMacroTable.nil());
 
   public IPv4AddressMatchExprMacroTest()
@@ -78,14 +78,14 @@ public class IPv4AddressMatchExprMacroTest extends 
MacroTestBase
   {
     expectException(IllegalArgumentException.class, "subnet arg has an invalid 
format");
 
-    Expr invalidSubnet = ExprEval.of("192.168.0.1/invalid").toExpr();
+    Expr invalidSubnet = ExprEval.ofString("192.168.0.1/invalid").toExpr();
     apply(Arrays.asList(IPV4, invalidSubnet));
   }
 
   @Test
   public void testNullStringArg()
   {
-    Expr nullString = ExprEval.of(null).toExpr();
+    Expr nullString = ExprEval.ofString(null).toExpr();
     Assert.assertFalse(eval(nullString, SUBNET_192_168));
   }
 
@@ -142,7 +142,7 @@ public class IPv4AddressMatchExprMacroTest extends 
MacroTestBase
   @Test
   public void testNotIpAddress()
   {
-    Expr notIpAddress = ExprEval.of("druid.apache.org").toExpr();
+    Expr notIpAddress = ExprEval.ofString("druid.apache.org").toExpr();
     Assert.assertFalse(eval(notIpAddress, SUBNET_192_168));
   }
 
@@ -182,26 +182,26 @@ public class IPv4AddressMatchExprMacroTest extends 
MacroTestBase
   @Test
   public void testMatchesPrefix()
   {
-    Assert.assertTrue(eval(ExprEval.of("192.168.1.250").toExpr(), 
ExprEval.of("192.168.1.251/31").toExpr()));
-    Assert.assertFalse(eval(ExprEval.of("192.168.1.240").toExpr(), 
ExprEval.of("192.168.1.251/31").toExpr()));
-    Assert.assertFalse(eval(ExprEval.of("192.168.1.250").toExpr(), 
ExprEval.of("192.168.1.251/32").toExpr()));
-    Assert.assertTrue(eval(ExprEval.of("192.168.1.251").toExpr(), 
ExprEval.of("192.168.1.251/32").toExpr()));
+    Assert.assertTrue(eval(ExprEval.ofString("192.168.1.250").toExpr(), 
ExprEval.ofString("192.168.1.251/31").toExpr()));
+    Assert.assertFalse(eval(ExprEval.ofString("192.168.1.240").toExpr(), 
ExprEval.ofString("192.168.1.251/31").toExpr()));
+    Assert.assertFalse(eval(ExprEval.ofString("192.168.1.250").toExpr(), 
ExprEval.ofString("192.168.1.251/32").toExpr()));
+    Assert.assertTrue(eval(ExprEval.ofString("192.168.1.251").toExpr(), 
ExprEval.ofString("192.168.1.251/32").toExpr()));
 
     Assert.assertTrue(eval(
         
ExprEval.of(IPv4AddressExprUtils.parse("192.168.1.250").longValue()).toExpr(),
-        ExprEval.of("192.168.1.251/31").toExpr()
+        ExprEval.ofString("192.168.1.251/31").toExpr()
     ));
     Assert.assertFalse(eval(
         
ExprEval.of(IPv4AddressExprUtils.parse("192.168.1.240").longValue()).toExpr(),
-        ExprEval.of("192.168.1.251/31").toExpr()
+        ExprEval.ofString("192.168.1.251/31").toExpr()
     ));
     Assert.assertFalse(eval(
         
ExprEval.of(IPv4AddressExprUtils.parse("192.168.1.250").longValue()).toExpr(),
-        ExprEval.of("192.168.1.251/32").toExpr()
+        ExprEval.ofString("192.168.1.251/32").toExpr()
     ));
     Assert.assertTrue(eval(
         
ExprEval.of(IPv4AddressExprUtils.parse("192.168.1.251").longValue()).toExpr(),
-        ExprEval.of("192.168.1.251/32").toExpr()
+        ExprEval.ofString("192.168.1.251/32").toExpr()
     ));
   }
 
diff --git 
a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressParseExprMacroTest.java
 
b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressParseExprMacroTest.java
index d48955d02ce..dbafc12a07a 100644
--- 
a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressParseExprMacroTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressParseExprMacroTest.java
@@ -30,7 +30,7 @@ import java.util.Collections;
 
 public class IPv4AddressParseExprMacroTest extends MacroTestBase
 {
-  private static final Expr VALID = ExprEval.of("192.168.0.1").toExpr();
+  private static final Expr VALID = ExprEval.ofString("192.168.0.1").toExpr();
   private static final long EXPECTED = 3232235521L;
 
   public IPv4AddressParseExprMacroTest()
@@ -57,7 +57,7 @@ public class IPv4AddressParseExprMacroTest extends 
MacroTestBase
   @Test
   public void testnullStringArg()
   {
-    Expr nullString = ExprEval.of(null).toExpr();
+    Expr nullString = ExprEval.ofString(null).toExpr();
     Assert.assertNull(eval(nullString));
   }
 
@@ -78,21 +78,21 @@ public class IPv4AddressParseExprMacroTest extends 
MacroTestBase
   @Test
   public void testInvalidStringArgNotIPAddress()
   {
-    Expr notIpAddress = ExprEval.of("druid.apache.org").toExpr();
+    Expr notIpAddress = ExprEval.ofString("druid.apache.org").toExpr();
     Assert.assertNull(eval(notIpAddress));
   }
 
   @Test
   public void testInvalidStringArgIPv6Compatible()
   {
-    Expr ipv6Compatible = ExprEval.of("::192.168.0.1").toExpr();
+    Expr ipv6Compatible = ExprEval.ofString("::192.168.0.1").toExpr();
     Assert.assertNull(eval(ipv6Compatible));
   }
 
   @Test
   public void testValidStringArgIPv6Mapped()
   {
-    Expr ipv6Mapped = ExprEval.of("::ffff:192.168.0.1").toExpr();
+    Expr ipv6Mapped = ExprEval.ofString("::ffff:192.168.0.1").toExpr();
     Assert.assertNull(eval(ipv6Mapped));
   }
 
@@ -105,7 +105,7 @@ public class IPv4AddressParseExprMacroTest extends 
MacroTestBase
   @Test
   public void testValidStringArgUnsignedInt()
   {
-    Expr unsignedInt = ExprEval.of("3232235521").toExpr();
+    Expr unsignedInt = ExprEval.ofString("3232235521").toExpr();
     Assert.assertNull(eval(unsignedInt));
   }
 
diff --git 
a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacroTest.java
 
b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacroTest.java
index aefc4f24bea..c369bfe7b54 100644
--- 
a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacroTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacroTest.java
@@ -105,28 +105,28 @@ public class IPv4AddressStringifyExprMacroTest extends 
MacroTestBase
   @Test
   public void testNullStringArg()
   {
-    Expr nullString = ExprEval.of(null).toExpr();
+    Expr nullString = ExprEval.ofString(null).toExpr();
     Assert.assertNull(eval(nullString));
   }
 
   @Test
   public void testInvalidStringArgNotIPAddress()
   {
-    Expr notIpAddress = ExprEval.of("druid.apache.org").toExpr();
+    Expr notIpAddress = ExprEval.ofString("druid.apache.org").toExpr();
     Assert.assertNull(eval(notIpAddress));
   }
 
   @Test
   public void testInvalidStringArgIPv6Compatible()
   {
-    Expr ipv6Compatible = ExprEval.of("::192.168.0.1").toExpr();
+    Expr ipv6Compatible = ExprEval.ofString("::192.168.0.1").toExpr();
     Assert.assertNull(eval(ipv6Compatible));
   }
 
   @Test
   public void testValidStringArgIPv6Mapped()
   {
-    Expr ipv6Mapped = ExprEval.of("::ffff:192.168.0.1").toExpr();
+    Expr ipv6Mapped = ExprEval.ofString("::ffff:192.168.0.1").toExpr();
     Assert.assertNull(eval(ipv6Mapped));
   }
 
@@ -139,7 +139,7 @@ public class IPv4AddressStringifyExprMacroTest extends 
MacroTestBase
   @Test
   public void testValidStringArgUnsignedInt()
   {
-    Expr unsignedInt = ExprEval.of("3232235521").toExpr();
+    Expr unsignedInt = ExprEval.ofString("3232235521").toExpr();
     Assert.assertNull(eval(unsignedInt));
   }
 
diff --git 
a/processing/src/test/java/org/apache/druid/query/expression/IPv6AddressMatchExprMacroTest.java
 
b/processing/src/test/java/org/apache/druid/query/expression/IPv6AddressMatchExprMacroTest.java
index d8371298dcf..fd04e634dec 100644
--- 
a/processing/src/test/java/org/apache/druid/query/expression/IPv6AddressMatchExprMacroTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/expression/IPv6AddressMatchExprMacroTest.java
@@ -32,8 +32,8 @@ import java.util.Collections;
  
 public class IPv6AddressMatchExprMacroTest extends MacroTestBase
 {
-  private static final Expr IPV6 = ExprEval.of("201:ef:168::").toExpr();
-  private static final Expr IPV6_CIDR = 
ExprEval.of("201:ef:168::/32").toExpr();
+  private static final Expr IPV6 = ExprEval.ofString("201:ef:168::").toExpr();
+  private static final Expr IPV6_CIDR = 
ExprEval.ofString("201:ef:168::/32").toExpr();
  
   public IPv6AddressMatchExprMacroTest()
   {
@@ -50,7 +50,7 @@ public class IPv6AddressMatchExprMacroTest extends 
MacroTestBase
   @Test
   public void testTooManyArgs()
   {
-    Expr extraArgument = ExprEval.of("An extra argument").toExpr();
+    Expr extraArgument = ExprEval.ofString("An extra argument").toExpr();
     expectException(ExpressionValidationException.class, "requires 2 
arguments");
     apply(Arrays.asList(IPV6, IPV6_CIDR, extraArgument));
   }
@@ -59,14 +59,14 @@ public class IPv6AddressMatchExprMacroTest extends 
MacroTestBase
   public void testSubnetArgInvalid()
   {
     expectException(ExpressionProcessingException.class, "Function[ipv6_match] 
failed to parse address");
-    Expr invalidSubnet = ExprEval.of("201:ef:168::/invalid").toExpr();
+    Expr invalidSubnet = ExprEval.ofString("201:ef:168::/invalid").toExpr();
     apply(Arrays.asList(IPV6, invalidSubnet));
   }
 
   @Test
   public void testNullStringArg()
   {
-    Expr nullString = ExprEval.of(null).toExpr();
+    Expr nullString = ExprEval.ofString(null).toExpr();
     Assert.assertFalse(eval(nullString, IPV6_CIDR));
   }
 
@@ -79,14 +79,14 @@ public class IPv6AddressMatchExprMacroTest extends 
MacroTestBase
   @Test
   public void testNotMatchingStringArgIPv6()
   {
-    Expr nonMatchingIpv6 = ExprEval.of("2002:ef:168::").toExpr();
+    Expr nonMatchingIpv6 = ExprEval.ofString("2002:ef:168::").toExpr();
     Assert.assertFalse(eval(nonMatchingIpv6, IPV6_CIDR));
   }
  
   @Test
   public void testNotIpAddress()
   {
-    Expr notIpAddress = ExprEval.of("druid.apache.org").toExpr();
+    Expr notIpAddress = ExprEval.ofString("druid.apache.org").toExpr();
     Assert.assertFalse(eval(notIpAddress, IPV6_CIDR));
   }
  
diff --git 
a/processing/src/test/java/org/apache/druid/query/expression/LookupExprMacroTest.java
 
b/processing/src/test/java/org/apache/druid/query/expression/LookupExprMacroTest.java
index cef936729e7..5d1c477903a 100644
--- 
a/processing/src/test/java/org/apache/druid/query/expression/LookupExprMacroTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/expression/LookupExprMacroTest.java
@@ -93,7 +93,7 @@ public class LookupExprMacroTest extends MacroTestBase
   {
     return args.stream().map(a -> {
       if (a != null && a instanceof String) {
-        return ExprEval.of(a.toString()).toExpr();
+        return ExprEval.ofString(a.toString()).toExpr();
       }
       return ExprEval.bestEffortOf(null).toExpr();
     }).collect(Collectors.toList());
diff --git 
a/processing/src/test/java/org/apache/druid/query/expression/TimestampExtractExprMacroTest.java
 
b/processing/src/test/java/org/apache/druid/query/expression/TimestampExtractExprMacroTest.java
index 128a6fd5108..e5fdf67a252 100644
--- 
a/processing/src/test/java/org/apache/druid/query/expression/TimestampExtractExprMacroTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/expression/TimestampExtractExprMacroTest.java
@@ -49,8 +49,8 @@ public class TimestampExtractExprMacroTest
   {
     Expr expression = target.apply(
         ImmutableList.of(
-            ExprEval.of("2001-02-16").toExpr(),
-            
ExprEval.of(TimestampExtractExprMacro.Unit.DECADE.toString()).toExpr()
+            ExprEval.ofString("2001-02-16").toExpr(),
+            
ExprEval.ofString(TimestampExtractExprMacro.Unit.DECADE.toString()).toExpr()
         ));
     Assert.assertEquals(200, 
expression.eval(InputBindings.nilBindings()).asInt());
   }
@@ -60,8 +60,8 @@ public class TimestampExtractExprMacroTest
   {
     Expr expression = target.apply(
         ImmutableList.of(
-            ExprEval.of("2000-12-16").toExpr(),
-            
ExprEval.of(TimestampExtractExprMacro.Unit.CENTURY.toString()).toExpr()
+            ExprEval.ofString("2000-12-16").toExpr(),
+            
ExprEval.ofString(TimestampExtractExprMacro.Unit.CENTURY.toString()).toExpr()
         ));
     Assert.assertEquals(20, 
expression.eval(InputBindings.nilBindings()).asInt());
   }
@@ -71,8 +71,8 @@ public class TimestampExtractExprMacroTest
   {
     Expr expression = target.apply(
         ImmutableList.of(
-            ExprEval.of("2001-02-16").toExpr(),
-            
ExprEval.of(TimestampExtractExprMacro.Unit.CENTURY.toString()).toExpr()
+            ExprEval.ofString("2001-02-16").toExpr(),
+            
ExprEval.ofString(TimestampExtractExprMacro.Unit.CENTURY.toString()).toExpr()
         ));
     Assert.assertEquals(21, 
expression.eval(InputBindings.nilBindings()).asInt());
   }
@@ -82,8 +82,8 @@ public class TimestampExtractExprMacroTest
   {
     Expr expression = target.apply(
         ImmutableList.of(
-            ExprEval.of("2000-12-16").toExpr(),
-            
ExprEval.of(TimestampExtractExprMacro.Unit.MILLENNIUM.toString()).toExpr()
+            ExprEval.ofString("2000-12-16").toExpr(),
+            
ExprEval.ofString(TimestampExtractExprMacro.Unit.MILLENNIUM.toString()).toExpr()
         ));
     Assert.assertEquals(2, 
expression.eval(InputBindings.nilBindings()).asInt());
   }
@@ -93,8 +93,8 @@ public class TimestampExtractExprMacroTest
   {
     Expr expression = target.apply(
         ImmutableList.of(
-            ExprEval.of("2001-02-16").toExpr(),
-            
ExprEval.of(TimestampExtractExprMacro.Unit.MILLENNIUM.toString()).toExpr()
+            ExprEval.ofString("2001-02-16").toExpr(),
+            
ExprEval.ofString(TimestampExtractExprMacro.Unit.MILLENNIUM.toString()).toExpr()
         ));
     Assert.assertEquals(3, 
expression.eval(InputBindings.nilBindings()).asInt());
   }
@@ -104,9 +104,9 @@ public class TimestampExtractExprMacroTest
   {
     Expr expression = target.apply(
         ImmutableList.of(
-            ExprEval.of("2023-12-15").toExpr(),
-            
ExprEval.of(TimestampExtractExprMacro.Unit.DOW.toString()).toExpr(),
-            ExprEval.of("UTC").toExpr()
+            ExprEval.ofString("2023-12-15").toExpr(),
+            
ExprEval.ofString(TimestampExtractExprMacro.Unit.DOW.toString()).toExpr(),
+            ExprEval.ofString("UTC").toExpr()
         ));
     Assert.assertEquals(5, 
expression.eval(InputBindings.nilBindings()).asInt());
   }
diff --git 
a/processing/src/test/java/org/apache/druid/query/expression/TimestampShiftMacroTest.java
 
b/processing/src/test/java/org/apache/druid/query/expression/TimestampShiftMacroTest.java
index ef5985b440d..04e09bc9f8a 100644
--- 
a/processing/src/test/java/org/apache/druid/query/expression/TimestampShiftMacroTest.java
+++ 
b/processing/src/test/java/org/apache/druid/query/expression/TimestampShiftMacroTest.java
@@ -73,7 +73,7 @@ public class TimestampShiftMacroTest extends MacroTestBase
     apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr(),
-            ExprEval.of("P1M").toExpr()
+            ExprEval.ofString("P1M").toExpr()
         ));
   }
 
@@ -84,10 +84,10 @@ public class TimestampShiftMacroTest extends MacroTestBase
     apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr(),
-            ExprEval.of("P1M").toExpr(),
-            ExprEval.of("1").toExpr(),
-            ExprEval.of("+08:00").toExpr(),
-            ExprEval.of("extra").toExpr()
+            ExprEval.ofString("P1M").toExpr(),
+            ExprEval.ofString("1").toExpr(),
+            ExprEval.ofString("+08:00").toExpr(),
+            ExprEval.ofString("extra").toExpr()
         ));
   }
 
@@ -98,7 +98,7 @@ public class TimestampShiftMacroTest extends MacroTestBase
     Expr expr = apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr(),
-            ExprEval.of("P1M").toExpr(),
+            ExprEval.ofString("P1M").toExpr(),
             ExprEval.of(step).toExpr()
         ));
 
@@ -115,7 +115,7 @@ public class TimestampShiftMacroTest extends MacroTestBase
     Expr expr = apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr(),
-            ExprEval.of("P1M").toExpr(),
+            ExprEval.ofString("P1M").toExpr(),
             ExprEval.of(step).toExpr()
         ));
 
@@ -132,7 +132,7 @@ public class TimestampShiftMacroTest extends MacroTestBase
     Expr expr = apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr(),
-            ExprEval.of("P1M").toExpr(),
+            ExprEval.ofString("P1M").toExpr(),
             ExprEval.of(step).toExpr()
         ));
 
@@ -148,7 +148,7 @@ public class TimestampShiftMacroTest extends MacroTestBase
     Expr expr = apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr(),
-            ExprEval.of("PT1M").toExpr(),
+            ExprEval.ofString("PT1M").toExpr(),
             ExprEval.of(1).toExpr()
         ));
 
@@ -164,7 +164,7 @@ public class TimestampShiftMacroTest extends MacroTestBase
     Expr expr = apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr(),
-            ExprEval.of("P1D").toExpr(),
+            ExprEval.ofString("P1D").toExpr(),
             ExprEval.of(1).toExpr()
         ));
 
@@ -180,9 +180,9 @@ public class TimestampShiftMacroTest extends MacroTestBase
     Expr expr = apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr(),
-            ExprEval.of("P1Y").toExpr(),
+            ExprEval.ofString("P1Y").toExpr(),
             ExprEval.of(1).toExpr(),
-            ExprEval.of("America/Los_Angeles").toExpr()
+            ExprEval.ofString("America/Los_Angeles").toExpr()
         ));
 
     Assert.assertEquals(
@@ -198,9 +198,9 @@ public class TimestampShiftMacroTest extends MacroTestBase
     Expr expr = apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr(),
-            ExprEval.of("P1Y").toExpr(),
+            ExprEval.ofString("P1Y").toExpr(),
             Parser.parse("\"step\"", ExprMacroTable.nil()), // "step" is not a 
literal
-            ExprEval.of("America/Los_Angeles").toExpr()
+            ExprEval.ofString("America/Los_Angeles").toExpr()
         ));
 
     final int step = 3;
@@ -235,7 +235,7 @@ public class TimestampShiftMacroTest extends MacroTestBase
     Expr expr = apply(
         ImmutableList.of(
             ExprEval.ofLong(null).toExpr(),
-            ExprEval.of("P1M").toExpr(),
+            ExprEval.ofString("P1M").toExpr(),
             ExprEval.of(1L).toExpr()
         )
     );
diff --git 
a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java
 
b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java
index 2a155ffde9d..4946bc96258 100644
--- 
a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java
+++ 
b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java
@@ -1624,7 +1624,7 @@ public class EqualityFilterTests
                         "optimizedFilterNoIncludeUnknown"
                     )
                     .withPrefabValues(ColumnType.class, ColumnType.STRING, 
ColumnType.DOUBLE)
-                    .withPrefabValues(ExprEval.class, ExprEval.of("hello"), 
ExprEval.of(1.0))
+                    .withPrefabValues(ExprEval.class, 
ExprEval.ofString("hello"), ExprEval.of(1.0))
                     .withIgnoredFields(
                         "predicateFactory",
                         "optimizedFilterIncludeUnknown",
diff --git 
a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java
 
b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java
index dee5b524794..bf1a8794d1c 100644
--- 
a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java
+++ 
b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java
@@ -213,7 +213,7 @@ public class ExpressionFilterTest extends BaseFilterTest
   {
     assertFilterMatchesSkipVectorize(edf("mv_overlap(dim4, '1')"), 
List.of("0"));
     assertFilterMatchesSkipVectorize(edf("mv_overlap(dim4, '4')"), 
List.of("4", "5"));
-    assertFilterMatchesSkipVectorize(edf("mv_overlap(dim4, array(1, 2, 3, 
4)"), List.of("0", "3", "4", "5"));
+    assertFilterMatchesSkipVectorize(edf("mv_overlap(dim4, array(1, 2, 3, 
4))"), List.of("0", "3", "4", "5"));
     assertFilterMatchesSkipVectorize(edf("mv_overlap(dim4, dim3)"), 
List.of("5", "9"));
     assertFilterMatchesSkipVectorize(edf("mv_overlap(dim4, null)"), 
List.of("1", "6", "7", "8"));
     assertFilterMatchesSkipVectorize(edf("mv_overlap(dim4, [])"), List.of());
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java 
b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java
index 06c2241643c..980ff808928 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java
@@ -43,6 +43,7 @@ import org.apache.druid.query.JoinDataSource;
 import org.apache.druid.query.LookupDataSource;
 import org.apache.druid.query.Order;
 import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContext;
 import org.apache.druid.query.QueryContexts;
 import org.apache.druid.query.QueryDataSource;
 import org.apache.druid.query.QueryException;
@@ -4541,11 +4542,15 @@ public class CalciteJoinQueryTest extends 
BaseCalciteQueryTest
   @ParameterizedTest(name = "{0}")
   public void testTopNFilterJoinWithProjection(Map<String, Object> 
queryContext)
   {
-    // Cannot vectorize JOIN operator.
-    cannotVectorize();
+    if (QueryContext.of(queryContext).getEnableRewriteJoinToFilter()) {
+      // Join is eliminated. Cannot vectorize substring function unless 
fallback vectorization is on.
+      cannotVectorizeUnlessFallback();
+    } else {
+      // Cannot vectorize the join or the substring function.
+      cannotVectorize();
+    }
 
     // Filters on top N values of some dimension by using an inner join. Also 
projects the outer dimension.
-
     testQuery(
         "SELECT SUBSTRING(t1.dim1, 1, 10), SUM(t1.cnt)\n"
         + "FROM druid.foo t1\n"
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java
 
b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java
index 0553164a312..f4b65da3885 100644
--- 
a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java
+++ 
b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteLookupFunctionQueryTest.java
@@ -102,7 +102,7 @@ public class CalciteLookupFunctionQueryTest extends 
BaseCalciteQueryTest
   @Test
   public void testFilterLookupOfFunction()
   {
-    cannotVectorize();
+    cannotVectorizeUnlessFallback();
 
     testQuery(
         buildFilterTestSql("LOOKUP(LOWER(dim1), 'lookyloo') = 'xabc'"),
@@ -118,7 +118,7 @@ public class CalciteLookupFunctionQueryTest extends 
BaseCalciteQueryTest
   @Test
   public void testFilterFunctionOfLookup()
   {
-    cannotVectorize();
+    cannotVectorizeUnlessFallback();
 
     testQuery(
         buildFilterTestSql("LOWER(LOOKUP(dim1, 'lookyloo')) = 'xabc'"),
@@ -686,7 +686,7 @@ public class CalciteLookupFunctionQueryTest extends 
BaseCalciteQueryTest
   @Test
   public void testFilterMvContainsNull()
   {
-    cannotVectorize();
+    cannotVectorizeUnlessFallback();
 
     testQuery(
         buildFilterTestSql("MV_CONTAINS(LOOKUP(dim1, 'lookyloo'), NULL)"),
@@ -700,7 +700,7 @@ public class CalciteLookupFunctionQueryTest extends 
BaseCalciteQueryTest
   @Test
   public void testFilterMvContainsNullInjective()
   {
-    cannotVectorize();
+    cannotVectorizeUnlessFallback();
 
     testQuery(
         buildFilterTestSql("MV_CONTAINS(LOOKUP(dim1, 'lookyloo121'), NULL)"),
@@ -724,7 +724,7 @@ public class CalciteLookupFunctionQueryTest extends 
BaseCalciteQueryTest
   @Test
   public void testFilterMvOverlapNull()
   {
-    cannotVectorize();
+    cannotVectorizeUnlessFallback();
 
     testQuery(
         buildFilterTestSql("MV_OVERLAP(lookup(dim1, 'lookyloo'), ARRAY['xabc', 
'x6', 'nonexistent', NULL])"),
@@ -755,7 +755,7 @@ public class CalciteLookupFunctionQueryTest extends 
BaseCalciteQueryTest
   @Test
   public void testFilterNotMvContains()
   {
-    cannotVectorize();
+    cannotVectorizeUnlessFallback();
 
     testQuery(
         buildFilterTestSql("NOT MV_CONTAINS(lookup(dim1, 'lookyloo'), 
'xabc')"),
@@ -797,7 +797,7 @@ public class CalciteLookupFunctionQueryTest extends 
BaseCalciteQueryTest
   @Test
   public void testFilterNotMvOverlap()
   {
-    cannotVectorize();
+    cannotVectorizeUnlessFallback();
 
     testQuery(
         buildFilterTestSql("NOT MV_OVERLAP(lookup(dim1, 'lookyloo'), 
ARRAY['xabc', 'x6', 'nonexistent'])"),
@@ -1119,7 +1119,7 @@ public class CalciteLookupFunctionQueryTest extends 
BaseCalciteQueryTest
   @Test
   public void testFilterMvContainsCoalesceSameLiteral()
   {
-    cannotVectorize();
+    cannotVectorizeUnlessFallback();
 
     testQuery(
         buildFilterTestSql("MV_CONTAINS(COALESCE(LOOKUP(dim1, 'lookyloo'), 
'x6'), 'x6')"),
@@ -1134,7 +1134,7 @@ public class CalciteLookupFunctionQueryTest extends 
BaseCalciteQueryTest
   @Test
   public void testFilterMvOverlapCoalesceSameLiteral()
   {
-    cannotVectorize();
+    cannotVectorizeUnlessFallback();
 
     testQuery(
         buildFilterTestSql("MV_OVERLAP(COALESCE(LOOKUP(dim1, 'lookyloo'), 
'x6'), ARRAY['xabc', 'x6', 'nonexistent'])"),
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java 
b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
index bf1ba524072..593d8535663 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java
@@ -7502,11 +7502,9 @@ public class CalciteQueryTest extends 
BaseCalciteQueryTest
   @Test
   public void testApproxCountDistinct()
   {
+    cannotVectorizeUnlessFallback();
     msqIncompatible();
 
-    // Cannot vectorize due to multi-valued dim2.
-    cannotVectorize();
-
     testQuery(
         "SELECT\n"
         + "  SUM(cnt),\n"
@@ -8167,8 +8165,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
   @Test
   public void testCountDistinctOfSubstring()
   {
-    // Cannot vectorize due to extraction dimension spec.
-    cannotVectorize();
+    // Cannot vectorize due to substring function.
+    cannotVectorizeUnlessFallback();
 
     testQuery(
         "SELECT COUNT(DISTINCT SUBSTRING(dim1, 1, 1)) FROM druid.foo WHERE 
dim1 <> ''",
@@ -8312,7 +8310,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
   public void testRegexpExtract()
   {
     // Cannot vectorize due to regexp_extract function.
-    cannotVectorize();
+    cannotVectorizeUnlessFallback();
 
     testQuery(
         "SELECT DISTINCT\n"
diff --git 
a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java
 
b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java
index a04617b8513..b8f66a8439c 100644
--- 
a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java
+++ 
b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java
@@ -2246,27 +2246,6 @@ public class ExpressionsTest extends CalciteTestBase
     );
   }
 
-  @Test
-  public void testAbnormalReverseWithWrongType()
-  {
-    Throwable t = Assert.assertThrows(
-        DruidException.class,
-        () -> testHelper.testExpression(
-            new ReverseOperatorConversion().calciteOperator(),
-            testHelper.makeInputRef("a"),
-            DruidExpression.ofExpression(
-                ColumnType.STRING,
-                DruidExpression.functionCall("reverse"),
-                ImmutableList.of(
-                    DruidExpression.ofColumn(ColumnType.LONG, "a")
-                )
-            ),
-            null
-        )
-    );
-    Assert.assertEquals("Function[reverse] needs a STRING argument but got 
LONG instead", t.getMessage());
-  }
-
   @Test
   public void testRight()
   {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to