You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by vo...@apache.org on 2022/09/07 16:06:40 UTC

[druid] branch 24.0.0 updated: more consistent expression error messages (#12995) (#13042)

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

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


The following commit(s) were added to refs/heads/24.0.0 by this push:
     new 7f4e034930 more consistent expression error messages (#12995) (#13042)
7f4e034930 is described below

commit 7f4e034930fa861b16c886b058d69ff4ecb94fb3
Author: Vadim Ogievetsky <va...@ogievetsky.com>
AuthorDate: Wed Sep 7 09:06:31 2022 -0700

    more consistent expression error messages (#12995) (#13042)
    
    * more consistent expression error messages
    
    * review stuff
    
    * add NamedFunction for Function, ApplyFunction, and ExprMacro to share common stuff
    
    * fixes
    
    * add expression transform name to transformer failure, better parse_json error messaging
    
    Co-authored-by: Clint Wylie <cw...@apache.org>
---
 .../druid/benchmark/IPv4AddressBenchmark.java      |   3 +-
 .../org/apache/druid/math/expr/ApplyFunction.java  |  62 +----
 .../apache/druid/math/expr/BinaryOperatorExpr.java |  12 +-
 .../org/apache/druid/math/expr/ConstantExpr.java   |   2 +-
 .../java/org/apache/druid/math/expr/ExprEval.java  |   4 +-
 .../org/apache/druid/math/expr/ExprMacroTable.java |   4 +-
 .../math/expr/ExpressionProcessingException.java   |  36 +++
 .../druid/math/expr/ExpressionTypeFactory.java     |   2 +-
 .../math/expr/ExpressionValidationException.java   |  36 +++
 .../java/org/apache/druid/math/expr/Function.java  | 251 ++++++++-------------
 .../org/apache/druid/math/expr/FunctionalExpr.java |   2 +-
 .../org/apache/druid/math/expr/NamedFunction.java  | 175 ++++++++++++++
 .../druid/math/expr/vector/VectorProcessors.java   |   8 +-
 .../apache/druid/math/expr/ApplyFunctionTest.java  |  57 ++++-
 .../org/apache/druid/math/expr/FunctionTest.java   |  86 +++----
 .../org/apache/druid/math/expr/ParserTest.java     |   4 +-
 docs/misc/math-expr.md                             |   4 +-
 .../query/expressions/BloomFilterExpressions.java  |  27 +--
 .../expressions/BloomFilterExpressionsTest.java    |   6 +-
 .../druid/query/expressions/SleepExprMacro.java    |   8 +-
 .../apache/druid/testing/tools/SleepExprMacro.java |   8 +-
 .../CaseInsensitiveContainsExprMacro.java          |   5 +-
 .../druid/query/expression/ContainsExprMacro.java  |   5 +-
 .../apache/druid/query/expression/ExprUtils.java   |  12 -
 .../query/expression/HyperUniqueExpressions.java   |  38 ++--
 .../expression/IPv4AddressMatchExprMacro.java      |  11 +-
 .../expression/IPv4AddressParseExprMacro.java      |   5 +-
 .../expression/IPv4AddressStringifyExprMacro.java  |   5 +-
 .../druid/query/expression/LikeExprMacro.java      |  12 +-
 .../druid/query/expression/LookupExprMacro.java    |  10 +-
 .../query/expression/NestedDataExpressions.java    |  47 ++--
 .../query/expression/RegexpExtractExprMacro.java   |   9 +-
 .../query/expression/RegexpLikeExprMacro.java      |   7 +-
 .../query/expression/TimestampCeilExprMacro.java   |   5 +-
 .../expression/TimestampExtractExprMacro.java      |  14 +-
 .../query/expression/TimestampFloorExprMacro.java  |   5 +-
 .../query/expression/TimestampFormatExprMacro.java |   8 +-
 .../query/expression/TimestampParseExprMacro.java  |   5 +-
 .../query/expression/TimestampShiftExprMacro.java  |   5 +-
 .../druid/query/expression/TrimExprMacro.java      |   5 +-
 .../segment/transform/ExpressionTransform.java     |  30 ++-
 .../expression/CaseInsensitiveExprMacroTest.java   |   4 +-
 .../query/expression/ContainsExprMacroTest.java    |   4 +-
 .../expression/HyperUniqueExpressionsTest.java     |  14 +-
 .../expression/IPv4AddressMatchExprMacroTest.java  |   7 +-
 .../expression/IPv4AddressParseExprMacroTest.java  |   4 +-
 .../IPv4AddressStringifyExprMacroTest.java         |   4 +-
 .../expression/NestedDataExpressionsTest.java      |   6 +-
 .../expression/RegexpExtractExprMacroTest.java     |   4 +-
 .../query/expression/RegexpLikeExprMacroTest.java  |  10 +-
 .../query/expression/TimestampShiftMacroTest.java  |   8 +-
 .../server/coordinator/duty/CompactSegments.java   |   3 +-
 .../sql/calcite/expression/ExpressionsTest.java    |  40 +++-
 .../expression/IPv4AddressMatchExpressionTest.java |   7 +-
 .../expression/IPv4AddressParseExpressionTest.java |   5 +-
 .../IPv4AddressStringifyExpressionTest.java        |   5 +-
 56 files changed, 660 insertions(+), 505 deletions(-)

diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/IPv4AddressBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/IPv4AddressBenchmark.java
index 156d54c03c..87d8d6eae1 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/IPv4AddressBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/IPv4AddressBenchmark.java
@@ -26,7 +26,6 @@ import inet.ipaddr.IPAddressString;
 import inet.ipaddr.ipv4.IPv4Address;
 import org.apache.commons.net.util.SubnetUtils;
 import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.query.expression.ExprUtils;
 import org.apache.druid.query.expression.IPv4AddressExprUtils;
 import org.openjdk.jmh.annotations.Benchmark;
 import org.openjdk.jmh.annotations.BenchmarkMode;
@@ -293,7 +292,7 @@ public class IPv4AddressBenchmark
       subnetUtils = new SubnetUtils(subnet);
     }
     catch (IllegalArgumentException e) {
-      throw new IAE(e, ExprUtils.createErrMsg("getSubnetInfo()", " arg has an invalid format: " + subnet));
+      throw new IAE(e, "getSubnetInfo() arg has an invalid format: " + subnet);
     }
     subnetUtils.setInclusiveHostCount(true);  // make network and broadcast addresses match
     return subnetUtils.getInfo();
diff --git a/core/src/main/java/org/apache/druid/math/expr/ApplyFunction.java b/core/src/main/java/org/apache/druid/math/expr/ApplyFunction.java
index ab4386b011..9f5bbffcc2 100644
--- a/core/src/main/java/org/apache/druid/math/expr/ApplyFunction.java
+++ b/core/src/main/java/org/apache/druid/math/expr/ApplyFunction.java
@@ -19,13 +19,10 @@
 
 package org.apache.druid.math.expr;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableSet;
 import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
 import it.unimi.dsi.fastutil.objects.Object2IntMap;
 import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
-import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.UOE;
 import org.apache.druid.math.expr.vector.ExprVectorProcessor;
 
@@ -43,13 +40,8 @@ import java.util.stream.Stream;
  * Base interface describing the mechanism used to evaluate an {@link ApplyFunctionExpr}, which 'applies' a
  * {@link LambdaExpr} to one or more array {@link Expr}.  All {@link ApplyFunction} implementations are immutable.
  */
-public interface ApplyFunction
+public interface ApplyFunction extends NamedFunction
 {
-  /**
-   * Name of the function
-   */
-  String name();
-
   /**
    * Check if an apply function can be 'vectorized', for a given {@link LambdaExpr} and set of {@link Expr} inputs.
    * If this method returns true, {@link #asVectorProcessor} is expected to produce a {@link ExprVectorProcessor} which
@@ -101,7 +93,9 @@ public interface ApplyFunction
   }
 
   /**
-   * Validate apply function arguments, throwing an exception if incorrect
+   * Validate function arguments. This method is called whenever a {@link ApplyFunctionExpr} is created, and should
+   * validate everything that is feasible up front. Note that input type information is typically unavailable at the
+   * time {@link Expr} are parsed, and so this method is incapable of performing complete validation.
    */
   void validateArguments(LambdaExpr lambdaExpr, List<Expr> args);
 
@@ -194,13 +188,8 @@ public interface ApplyFunction
     @Override
     public void validateArguments(LambdaExpr lambdaExpr, List<Expr> args)
     {
-      Preconditions.checkArgument(args.size() == 1);
-      if (lambdaExpr.identifierCount() > 0) {
-        Preconditions.checkArgument(
-            args.size() == lambdaExpr.identifierCount(),
-            StringUtils.format("lambda expression argument count does not match %s argument count", name())
-        );
-      }
+      validationHelperCheckArgumentCount(lambdaExpr, args, 1);
+
     }
   }
 
@@ -260,13 +249,7 @@ public interface ApplyFunction
     @Override
     public void validateArguments(LambdaExpr lambdaExpr, List<Expr> args)
     {
-      Preconditions.checkArgument(args.size() > 0);
-      if (lambdaExpr.identifierCount() > 0) {
-        Preconditions.checkArgument(
-            args.size() == lambdaExpr.identifierCount(),
-            StringUtils.format("lambda expression argument count does not match %s argument count", name())
-        );
-      }
+      validationHelperCheckMinArgumentCount(lambdaExpr, args, 1);
     }
   }
 
@@ -358,11 +341,7 @@ public interface ApplyFunction
     @Override
     public void validateArguments(LambdaExpr lambdaExpr, List<Expr> args)
     {
-      Preconditions.checkArgument(args.size() == 2);
-      Preconditions.checkArgument(
-          args.size() == lambdaExpr.identifierCount(),
-          StringUtils.format("lambda expression argument count does not match %s argument count", name())
-      );
+      validationHelperCheckArgumentCount(lambdaExpr, args, 2);
     }
   }
 
@@ -432,10 +411,7 @@ public interface ApplyFunction
     @Override
     public void validateArguments(LambdaExpr lambdaExpr, List<Expr> args)
     {
-      Preconditions.checkArgument(
-          args.size() == lambdaExpr.identifierCount(),
-          StringUtils.format("lambda expression argument count does not match %s argument count", name())
-      );
+      validationHelperCheckMinArgumentCount(lambdaExpr, args, 1);
     }
   }
 
@@ -477,21 +453,13 @@ public interface ApplyFunction
     @Override
     public Set<Expr> getArrayInputs(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("ApplyFunction[%s] needs 1 argument", name());
-      }
-
       return ImmutableSet.of(args.get(0));
     }
 
     @Override
     public void validateArguments(LambdaExpr lambdaExpr, List<Expr> args)
     {
-      Preconditions.checkArgument(args.size() == 1);
-      Preconditions.checkArgument(
-          args.size() == lambdaExpr.identifierCount(),
-          StringUtils.format("lambda expression argument count does not match %s argument count", name())
-      );
+      validationHelperCheckArgumentCount(lambdaExpr, args, 1);
     }
 
     @Nullable
@@ -532,21 +500,13 @@ public interface ApplyFunction
     @Override
     public Set<Expr> getArrayInputs(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("ApplyFunction[%s] needs 1 argument", name());
-      }
-
       return ImmutableSet.of(args.get(0));
     }
 
     @Override
     public void validateArguments(LambdaExpr lambdaExpr, List<Expr> args)
     {
-      Preconditions.checkArgument(args.size() == 1);
-      Preconditions.checkArgument(
-          args.size() == lambdaExpr.identifierCount(),
-          StringUtils.format("lambda expression argument count does not match %s argument count", name())
-      );
+      validationHelperCheckArgumentCount(lambdaExpr, args, 1);
     }
 
     @Nullable
diff --git a/core/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java b/core/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java
index 8932a46811..64acb04bb5 100644
--- a/core/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java
+++ b/core/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java
@@ -152,11 +152,13 @@ abstract class BinaryEvalOpExprBase extends BinaryOpExprBase
 
   protected ExprEval evalString(@Nullable String left, @Nullable String right)
   {
-    throw new IAE("operator '%s' in expression (%s %s %s) is not supported on type 'string'.",
-                  this.op,
-                  this.left.stringify(),
-                  this.op,
-                  this.right.stringify());
+    throw new IAE(
+        "operator '%s' in expression (%s %s %s) is not supported on type STRING.",
+        this.op,
+        this.left.stringify(),
+        this.op,
+        this.right.stringify()
+    );
   }
 
   protected abstract long evalLong(long left, long right);
diff --git a/core/src/main/java/org/apache/druid/math/expr/ConstantExpr.java b/core/src/main/java/org/apache/druid/math/expr/ConstantExpr.java
index 5533e03642..087c0e169e 100644
--- a/core/src/main/java/org/apache/druid/math/expr/ConstantExpr.java
+++ b/core/src/main/java/org/apache/druid/math/expr/ConstantExpr.java
@@ -324,7 +324,7 @@ class ArrayExpr extends ConstantExpr<Object[]>
   public ArrayExpr(ExpressionType outputType, @Nullable Object[] value)
   {
     super(outputType, value);
-    Preconditions.checkArgument(outputType.isArray());
+    Preconditions.checkArgument(outputType.isArray(), "Output type %s is not an array", outputType);
     ExpressionType.checkNestedArrayAllowed(outputType);
   }
 
diff --git a/core/src/main/java/org/apache/druid/math/expr/ExprEval.java b/core/src/main/java/org/apache/druid/math/expr/ExprEval.java
index e628a9a39a..e83644d99c 100644
--- a/core/src/main/java/org/apache/druid/math/expr/ExprEval.java
+++ b/core/src/main/java/org/apache/druid/math/expr/ExprEval.java
@@ -330,7 +330,7 @@ public abstract class ExprEval<T>
 
   public static ExprEval ofArray(ExpressionType outputType, Object[] value)
   {
-    Preconditions.checkArgument(outputType.isArray());
+    Preconditions.checkArgument(outputType.isArray(), "Output type %s is not an array", outputType);
     return new ArrayExprEval(outputType, value);
   }
 
@@ -1021,7 +1021,7 @@ public abstract class ExprEval<T>
     {
       super(value);
       this.arrayType = arrayType;
-      Preconditions.checkArgument(arrayType.isArray());
+      Preconditions.checkArgument(arrayType.isArray(), "Output type %s is not an array", arrayType);
       ExpressionType.checkNestedArrayAllowed(arrayType);
     }
 
diff --git a/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java b/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java
index 94d9729631..ec7c2036c9 100644
--- a/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java
+++ b/core/src/main/java/org/apache/druid/math/expr/ExprMacroTable.java
@@ -86,10 +86,8 @@ public class ExprMacroTable
     return exprMacro.apply(args);
   }
 
-  public interface ExprMacro
+  public interface ExprMacro extends NamedFunction
   {
-    String name();
-
     Expr apply(List<Expr> args);
   }
 
diff --git a/core/src/main/java/org/apache/druid/math/expr/ExpressionProcessingException.java b/core/src/main/java/org/apache/druid/math/expr/ExpressionProcessingException.java
new file mode 100644
index 0000000000..bec051f1d1
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/math/expr/ExpressionProcessingException.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.math.expr;
+
+import org.apache.druid.java.util.common.ISE;
+import org.apache.druid.java.util.common.StringUtils;
+
+public class ExpressionProcessingException extends ISE
+{
+  public ExpressionProcessingException(NamedFunction fn, String msg, Object... formatArgs)
+  {
+    super("Function[%s] %s", fn.name(), StringUtils.format(msg, formatArgs));
+  }
+
+  public ExpressionProcessingException(NamedFunction fn, Throwable e, String msg, Object... formatArgs)
+  {
+    super(e, "Function[%s] %s", fn.name(), StringUtils.format(msg, formatArgs));
+  }
+}
diff --git a/core/src/main/java/org/apache/druid/math/expr/ExpressionTypeFactory.java b/core/src/main/java/org/apache/druid/math/expr/ExpressionTypeFactory.java
index 29fec291bd..29b8edd5ae 100644
--- a/core/src/main/java/org/apache/druid/math/expr/ExpressionTypeFactory.java
+++ b/core/src/main/java/org/apache/druid/math/expr/ExpressionTypeFactory.java
@@ -115,7 +115,7 @@ public class ExpressionTypeFactory implements TypeFactory<ExpressionType>
         strategy = complexStrategy;
         break;
       default:
-        throw new ISE("Unsupported column type[%s]", expressionType.getType());
+        throw new ISE("Unsupported expression type[%s]", expressionType.getType());
     }
     return strategy;
   }
diff --git a/core/src/main/java/org/apache/druid/math/expr/ExpressionValidationException.java b/core/src/main/java/org/apache/druid/math/expr/ExpressionValidationException.java
new file mode 100644
index 0000000000..c63f853301
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/math/expr/ExpressionValidationException.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.math.expr;
+
+import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.java.util.common.StringUtils;
+
+public class ExpressionValidationException extends IAE
+{
+  public ExpressionValidationException(NamedFunction fn, String msg, Object... formatArgs)
+  {
+    super("Function[%s] %s", fn.name(), StringUtils.format(msg, formatArgs));
+  }
+
+  public ExpressionValidationException(NamedFunction fn, Throwable e, String msg, Object... formatArgs)
+  {
+    super(e, "Function[%s] %s", fn.name(), StringUtils.format(msg, formatArgs));
+  }
+}
diff --git a/core/src/main/java/org/apache/druid/math/expr/Function.java b/core/src/main/java/org/apache/druid/math/expr/Function.java
index 02c0f3d3aa..62e7ab9b80 100644
--- a/core/src/main/java/org/apache/druid/math/expr/Function.java
+++ b/core/src/main/java/org/apache/druid/math/expr/Function.java
@@ -23,7 +23,6 @@ import com.google.common.collect.ImmutableSet;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.HumanReadableBytes;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.java.util.common.UOE;
 import org.apache.druid.math.expr.vector.CastToTypeVectorProcessor;
@@ -62,13 +61,8 @@ import java.util.stream.Collectors;
  * Do NOT remove "unused" members in this class. They are used by generated Antlr
  */
 @SuppressWarnings("unused")
-public interface Function
+public interface Function extends NamedFunction
 {
-  /**
-   * Name of the function.
-   */
-  String name();
-
   /**
    * Evaluate the function, given a list of arguments and a set of bindings to provide values for {@link IdentifierExpr}.
    */
@@ -109,10 +103,13 @@ public interface Function
   }
 
   /**
-   * Validate function arguments
+   * Validate function arguments. This method is called whenever a {@link FunctionExpr} is created, and should validate
+   * everything that is feasible up front. Note that input type information is typically unavailable at the time
+   * {@link Expr} are parsed, and so this method is incapable of performing complete validation.
    */
   void validateArguments(List<Expr> args);
 
+
   /**
    * Compute the output type of this function for a given set of argument expression inputs.
    *
@@ -143,7 +140,7 @@ public interface Function
    */
   default <T> ExprVectorProcessor<T> asVectorProcessor(Expr.VectorInputBindingInspector inspector, List<Expr> args)
   {
-    throw new UOE("%s is not vectorized", name());
+    throw new UOE("Function[%s] is not vectorized", name());
   }
 
   /**
@@ -154,9 +151,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] needs 1 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 1);
     }
 
     @Override
@@ -177,9 +172,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 2) {
-        throw new IAE("Function[%s] needs 2 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 2);
     }
 
     @Override
@@ -221,7 +214,10 @@ public interface Function
     protected ExprEval eval(double param)
     {
       if (param < Long.MIN_VALUE || param > Long.MAX_VALUE) {
-        throw new IAE("Possible data truncation, param [%f] is out of long value range", param);
+        throw validationFailed(
+            "Possible data truncation, param [%f] is out of LONG value range",
+            param
+        );
       }
       return eval((long) param);
     }
@@ -363,10 +359,7 @@ public interface Function
     protected final ExprEval eval(ExprEval x, ExprEval y)
     {
       if (!x.type().is(ExprType.STRING) || !y.type().is(ExprType.LONG)) {
-        throw new IAE(
-            "Function[%s] needs a string as first argument and an integer as second argument",
-            name()
-        );
+        throw validationFailed("needs a STRING as first argument and a LONG as second argument");
       }
       return eval(x.asString(), y.asInt());
     }
@@ -382,9 +375,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 2) {
-        throw new IAE("Function[%s] needs 2 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 2);
     }
 
     @Override
@@ -437,9 +428,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 2) {
-        throw new IAE("Function[%s] needs 2 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 2);
     }
 
     @Override
@@ -656,7 +645,7 @@ public interface Function
         case STRING:
           return true;
         default:
-          throw new IAE("Function[%s] does not accept %s types", name(), exprType);
+          throw validationFailed("does not accept %s types", exprType);
       }
     }
   }
@@ -674,9 +663,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 1 && args.size() != 2) {
-        throw new IAE("Function[%s] needs 1 or 2 arguments", name());
-      }
+      validationHelperCheckAnyOfArgumentCount(args, 1, 2);
     }
 
     @Nullable
@@ -751,9 +738,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() > 0) {
-        throw new IAE("Function[%s] needs 0 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 0);
     }
 
     @Nullable
@@ -1469,9 +1454,8 @@ public interface Function
       }
 
       if (!value1.type().anyOf(ExprType.LONG, ExprType.DOUBLE)) {
-        throw new IAE(
-            "The first argument to the function[%s] should be integer or double type but got the type: %s",
-            name(),
+        throw validationFailed(
+            "first argument should be a LONG or DOUBLE but got %s instead",
             value1.type()
         );
       }
@@ -1481,9 +1465,8 @@ public interface Function
       } else {
         ExprEval value2 = args.get(1).eval(bindings);
         if (!value2.type().is(ExprType.LONG)) {
-          throw new IAE(
-              "The second argument to the function[%s] should be integer type but got the type: %s",
-              name(),
+          throw validationFailed(
+              "second argument should be a LONG but got %s instead",
               value2.type()
           );
         }
@@ -1494,9 +1477,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 1 && args.size() != 2) {
-        throw new IAE("Function[%s] needs 1 or 2 arguments", name());
-      }
+      validationHelperCheckAnyOfArgumentCount(args, 1, 2);
     }
 
     @Nullable
@@ -1971,7 +1952,7 @@ public interface Function
         castTo = ExpressionType.fromString(StringUtils.toUpperCase(y.asString()));
       }
       catch (IllegalArgumentException e) {
-        throw new IAE("invalid type '%s'", y.asString());
+        throw validationFailed("invalid type %s", y.asString());
       }
       return x.castTo(castTo);
     }
@@ -2099,9 +2080,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 3) {
-        throw new IAE("Function[%s] needs 3 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 3);
     }
 
     @Nullable
@@ -2142,9 +2121,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() < 2) {
-        throw new IAE("Function[%s] must have at least 2 arguments", name());
-      }
+      validationHelperCheckMinArgumentCount(args, 2);
     }
 
     @Nullable
@@ -2191,9 +2168,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() < 3) {
-        throw new IAE("Function[%s] must have at least 3 arguments", name());
-      }
+      validationHelperCheckMinArgumentCount(args, 3);
     }
 
     @Nullable
@@ -2228,9 +2203,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 2) {
-        throw new IAE("Function[%s] needs 2 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 2);
     }
 
     @Nullable
@@ -2271,9 +2244,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] needs 1 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 1);
     }
 
     @Nullable
@@ -2314,9 +2285,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] needs 1 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 1);
     }
 
     @Nullable
@@ -2423,9 +2392,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] needs 1 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 1);
     }
 
     @Nullable
@@ -2464,9 +2431,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() < 1) {
-        throw new IAE("Function[%s] needs 1 or more arguments", name());
-      }
+      validationHelperCheckMinArgumentCount(args, 1);
     }
 
     @Nullable
@@ -2509,9 +2474,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() < 2 || args.size() > 3) {
-        throw new IAE("Function[%s] needs 2 or 3 arguments", name());
-      }
+      validationHelperCheckAnyOfArgumentCount(args, 2, 3);
     }
 
     @Nullable
@@ -2559,9 +2522,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 3) {
-        throw new IAE("Function[%s] needs 3 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 3);
     }
 
     @Nullable
@@ -2591,10 +2552,7 @@ public interface Function
     protected ExprEval eval(@Nullable String x, int y)
     {
       if (y < 0) {
-        throw new IAE(
-            "Function[%s] needs a postive integer as second argument",
-            name()
-        );
+        throw validationFailed("needs a positive integer as the second argument");
       }
       if (x == null) {
         return ExprEval.of(null);
@@ -2623,10 +2581,7 @@ public interface Function
     protected ExprEval eval(@Nullable String x, int y)
     {
       if (y < 0) {
-        throw new IAE(
-            "Function[%s] needs a postive integer as second argument",
-            name()
-        );
+        throw validationFailed("needs a postive integer as second argument");
       }
       if (x == null) {
         return ExprEval.of(null);
@@ -2658,9 +2613,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 3) {
-        throw new IAE("Function[%s] needs 3 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 3);
     }
 
     @Nullable
@@ -2692,9 +2645,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] needs 1 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 1);
     }
 
     @Nullable
@@ -2726,9 +2677,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] needs 1 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 1);
     }
 
     @Nullable
@@ -2758,10 +2707,7 @@ public interface Function
     protected ExprEval eval(ExprEval param)
     {
       if (!param.type().is(ExprType.STRING)) {
-        throw new IAE(
-            "Function[%s] needs a string argument",
-            name()
-        );
+        throw validationFailed("needs a STRING argument but got %s instead", param.type());
       }
       final String arg = param.asString();
       return ExprEval.of(arg == null ? NullHandling.defaultStringValue() : new StringBuilder(arg).reverse().toString());
@@ -2819,9 +2765,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 3) {
-        throw new IAE("Function[%s] needs 3 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 3);
     }
 
     @Nullable
@@ -2858,9 +2802,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 3) {
-        throw new IAE("Function[%s] needs 3 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 3);
     }
 
     @Nullable
@@ -2884,14 +2826,20 @@ public interface Function
     {
       ExprEval value = args.get(0).eval(bindings);
       if (!value.type().is(ExprType.STRING)) {
-        throw new IAE("first argument should be string type but got %s type", value.type());
+        throw validationFailed(
+            "first argument should be a STRING but got %s instead",
+            value.type()
+        );
       }
 
       DateTimes.UtcFormatter formatter = DateTimes.ISO_DATE_OPTIONAL_TIME;
       if (args.size() > 1) {
         ExprEval format = args.get(1).eval(bindings);
         if (!format.type().is(ExprType.STRING)) {
-          throw new IAE("second argument should be string type but got %s type", format.type());
+          throw validationFailed(
+              "second argument should be STRING but got %s instead",
+              format.type()
+          );
         }
         formatter = DateTimes.wrapFormatter(DateTimeFormat.forPattern(format.asString()));
       }
@@ -2900,7 +2848,7 @@ public interface Function
         date = formatter.parse(value.asString());
       }
       catch (IllegalArgumentException e) {
-        throw new IAE(e, "invalid value %s", value.asString());
+        throw validationFailed(e, "invalid value %s", value.asString());
       }
       return toValue(date);
     }
@@ -2908,9 +2856,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 1 && args.size() != 2) {
-        throw new IAE("Function[%s] needs 1 or 2 arguments", name());
-      }
+      validationHelperCheckAnyOfArgumentCount(args, 1, 2);
     }
 
     @Nullable
@@ -2967,9 +2913,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 3) {
-        throw new IAE("Function[%s] needs 3 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 3);
     }
 
     @Nullable
@@ -2997,14 +2941,12 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] needs exactly 1 argument of type String", name());
-      }
+      validationHelperCheckArgumentCount(args, 1);
       IdentifierExpr expr = args.get(0).getIdentifierExprIfIdentifierExpr();
 
       if (expr == null) {
-        throw new IAE(
-            "Arg %s should be an identifier expression ie refer to columns directaly. Use array() instead",
+        throw validationFailed(
+            "argument %s should be an identifier expression. Use array() instead",
             args.get(0).toString()
         );
       }
@@ -3088,9 +3030,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.isEmpty()) {
-        throw new IAE("Function[%s] needs at least 1 argument", name());
-      }
+      validationHelperCheckMinArgumentCount(args, 1);
     }
 
     @Nullable
@@ -3152,9 +3092,6 @@ public interface Function
     @Override
     public Set<Expr> getArrayInputs(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] needs 1 argument", name());
-      }
       return ImmutableSet.of(args.get(0));
     }
 
@@ -3167,9 +3104,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] needs 1 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 1);
     }
 
     @Nullable
@@ -3197,9 +3132,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 2) {
-        throw new IAE("Function[%s] needs 2 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 2);
     }
 
     @Nullable
@@ -3353,7 +3286,10 @@ public interface Function
           }
           return index < 0 ? ExprEval.ofLong(NullHandling.replaceWithDefault() ? -1 : null) : ExprEval.ofLong(index);
         default:
-          throw new IAE("Function[%s] 2nd argument must be a a scalar type", name());
+          throw validationFailed(
+              "second argument must be a a scalar type but got %s instead",
+              scalarExpr.type()
+          );
       }
     }
   }
@@ -3392,7 +3328,10 @@ public interface Function
                  ? ExprEval.ofLong(NullHandling.replaceWithDefault() ? -1 : null)
                  : ExprEval.ofLong(index + 1);
         default:
-          throw new IAE("Function[%s] 2nd argument must be a a scalar type", name());
+          throw validationFailed(
+              "second argument must be a a scalar type but got %s instead",
+              scalarExpr.type()
+          );
       }
     }
   }
@@ -3576,9 +3515,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 2 && args.size() != 3) {
-        throw new IAE("Function[%s] needs 2 or 3 arguments", name());
-      }
+      validationHelperCheckAnyOfArgumentCount(args, 2, 3);
     }
 
     @Nullable
@@ -3658,7 +3595,10 @@ public interface Function
        * For a DOUBLE, it will be cast to LONG before format
        */
       if (valueParam.value() != null && !valueParam.type().anyOf(ExprType.LONG, ExprType.DOUBLE)) {
-        throw new IAE("Function[%s] needs a number as its first argument", name());
+        throw validationFailed(
+            "needs a number as its first argument but got %s instead",
+            valueParam.type()
+        );
       }
 
       /**
@@ -3668,11 +3608,17 @@ public interface Function
       if (args.size() > 1) {
         ExprEval precisionParam = args.get(1).eval(bindings);
         if (!precisionParam.type().is(ExprType.LONG)) {
-          throw new IAE("Function[%s] needs an integer as its second argument", name());
+          throw validationFailed(
+              "needs a LONG as its second argument but got %s instead",
+              precisionParam.type()
+          );
         }
         precision = precisionParam.asLong();
         if (precision < 0 || precision > 3) {
-          throw new IAE("Given precision[%d] of Function[%s] must be in the range of [0,3]", precision, name());
+          throw validationFailed(
+              "given precision[%d] must be in the range of [0,3]",
+              precision
+          );
         }
       }
 
@@ -3682,9 +3628,7 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() < 1 || args.size() > 2) {
-        throw new IAE("Function[%s] needs 1 or 2 arguments", name());
-      }
+      validationHelperCheckAnyOfArgumentCount(args, 1, 2);
     }
 
     @Nullable
@@ -3753,9 +3697,9 @@ public interface Function
     {
       ExprEval arg0 = args.get(0).eval(bindings);
       if (!arg0.type().is(ExprType.STRING)) {
-        throw new IAE(
-            "Function[%s] first argument must be constant 'STRING' expression containing a valid complex type name",
-            name()
+        throw validationFailed(
+            "first argument must be constant STRING expression containing a valid complex type name but got %s instead",
+            arg0.type()
         );
       }
       ExpressionType type = ExpressionTypeFactory.getInstance().ofComplex((String) args.get(0).getLiteralValue());
@@ -3763,18 +3707,17 @@ public interface Function
       try {
         strategy = type.getStrategy();
       }
-      catch (IAE illegal) {
-        throw new IAE(
-            "Function[%s] first argument must be a valid complex type name, unknown complex type [%s]",
-            name(),
+      catch (IllegalArgumentException illegal) {
+        throw validationFailed(
+            "first argument must be a valid COMPLEX type name, got unknown COMPLEX type [%s]",
             type.asTypeString()
         );
       }
       ExprEval base64String = args.get(1).eval(bindings);
       if (!base64String.type().is(ExprType.STRING)) {
-        throw new IAE(
-            "Function[%s] second argument must be a base64 encoded 'STRING' value",
-            name()
+        throw validationFailed(
+            "second argument must be a base64 encoded STRING value but got %s instead",
+            base64String.type()
         );
       }
       if (base64String.value() == null) {
@@ -3788,13 +3731,10 @@ public interface Function
     @Override
     public void validateArguments(List<Expr> args)
     {
-      if (args.size() != 2) {
-        throw new IAE("Function[%s] needs 2 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 2);
       if (!args.get(0).isLiteral() || args.get(0).isNullLiteral()) {
-        throw new IAE(
-            "Function[%s] first argument must be constant 'STRING' expression containing a valid complex type name",
-            name()
+        throw validationFailed(
+            "first argument must be constant STRING expression containing a valid COMPLEX type name"
         );
       }
     }
@@ -3808,9 +3748,8 @@ public interface Function
     {
       ExpressionType arg0Type = args.get(0).getOutputType(inspector);
       if (arg0Type == null || !arg0Type.is(ExprType.STRING)) {
-        throw new IAE(
-            "Function[%s] first argument must be constant 'STRING' expression containing a valid complex type name",
-            name()
+        throw validationFailed(
+            "first argument must be constant STRING expression containing a valid COMPLEX type name"
         );
       }
       return ExpressionTypeFactory.getInstance().ofComplex((String) args.get(0).getLiteralValue());
diff --git a/core/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java b/core/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java
index 1155ec90ea..5d0aee9d39 100644
--- a/core/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java
+++ b/core/src/main/java/org/apache/druid/math/expr/FunctionalExpr.java
@@ -56,7 +56,7 @@ class LambdaExpr implements Expr
   @Nullable
   public String getIdentifier()
   {
-    Preconditions.checkState(args.size() < 2, "LambdaExpr has multiple arguments");
+    Preconditions.checkState(args.size() < 2, "LambdaExpr has multiple arguments, use getIdentifiers");
     if (args.size() == 1) {
       return args.get(0).toString();
     }
diff --git a/core/src/main/java/org/apache/druid/math/expr/NamedFunction.java b/core/src/main/java/org/apache/druid/math/expr/NamedFunction.java
new file mode 100644
index 0000000000..574535ac68
--- /dev/null
+++ b/core/src/main/java/org/apache/druid/math/expr/NamedFunction.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.math.expr;
+
+import net.thisptr.jackson.jq.internal.misc.Strings;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Common stuff for "named" functions of "functional" expressions, such as {@link FunctionExpr},
+ * {@link ApplyFunctionExpr}, and {@link ExprMacroTable.ExprMacroFunctionExpr}.
+ *
+ * Provides helper methods for performing common validation operations to help reduce boilerplate for implementors and
+ * make it easier to provide consistent error messaging.
+ */
+public interface NamedFunction
+{
+  /**
+   * Name of the function
+   */
+  String name();
+
+  /**
+   * Helper method for creating a {@link ExpressionValidationException} with the specified reason
+   */
+  default ExpressionValidationException validationFailed(String reasonFormat, Object... args)
+  {
+    throw new ExpressionValidationException(this, reasonFormat, args);
+  }
+
+  default ExpressionValidationException validationFailed(Throwable e, String reasonFormat, Object... args)
+  {
+    throw new ExpressionValidationException(this, e, reasonFormat, args);
+  }
+
+  default ExpressionProcessingException processingFailed(Throwable e, String reasonFormat, Object... args)
+  {
+    throw new ExpressionProcessingException(this, e, reasonFormat, args);
+  }
+
+  /**
+   * Helper method for implementors performing validation to check if the argument count is expected
+   */
+  default void validationHelperCheckArgumentCount(List<Expr> args, int count)
+  {
+    if (args.size() != count) {
+      if (count == 0) {
+        throw validationFailed("does not accept arguments");
+      } else if (count == 1) {
+        throw validationFailed("requires 1 argument");
+      }
+      throw validationFailed("requires %d arguments", count);
+    }
+  }
+
+  /**
+   * Helper method for implementors performing validation to check if there are at least as many arguments as specified
+   */
+  default void validationHelperCheckMinArgumentCount(List<Expr> args, int count)
+  {
+    if (args.size() < count) {
+      if (count == 1) {
+        throw validationFailed("requires at least 1 argument");
+      }
+      throw validationFailed("requires at least %d arguments", count);
+    }
+  }
+
+  default void validationHelperCheckArgumentRange(List<Expr> args, int start, int end)
+  {
+    if (args.size() < start || args.size() > end) {
+      throw validationFailed("requires %d to %d arguments", start, end);
+    }
+  }
+
+  /**
+   * Helper method for implementors performing validation to check if argument count is any of specified counts
+   */
+  default void validationHelperCheckAnyOfArgumentCount(List<Expr> args, int... counts)
+  {
+    boolean satisfied = false;
+    for (int count : counts) {
+      if (args.size() == count) {
+        satisfied = true;
+        break;
+      }
+    }
+    if (!satisfied) {
+      throw validationFailed(
+          "requires %s arguments",
+          Strings.join(" or ", () -> Arrays.stream(counts).mapToObj(String::valueOf).iterator())
+      );
+    }
+  }
+
+  /**
+   * Helper method for implementors performing validation to check that an argument is a literal
+   */
+  default void validationHelperCheckArgIsLiteral(Expr arg, String argName)
+  {
+    if (!arg.isLiteral()) {
+      throw validationFailed(
+          "%s argument must be a literal",
+          argName
+      );
+    }
+  }
+
+
+  /**
+   * Helper method for implementors performing validation to check that the argument list is some expected size.
+   *
+   * The parser decomposes a function like 'fold((x, acc) -> x + acc, col, 0)' into the {@link LambdaExpr}
+   * '(x, acc) -> x + acc' and the list of arguments, ['col', 0], and so does not include the {@link LambdaExpr} here.
+   * To compensate for this, the error message will indicate that at least count + 1 arguments are required to count
+   * the lambda.
+   */
+  default void validationHelperCheckArgumentCount(LambdaExpr lambdaExpr, List<Expr> args, int count)
+  {
+    if (args.size() != count) {
+      throw validationFailed("requires %s arguments", count + 1);
+    }
+    validationHelperCheckLambaArgumentCount(lambdaExpr, args);
+  }
+
+  /**
+   * Helper method for implementors performing validation to check that the argument list is at least some
+   * expected size.
+   *
+   * The parser decomposes a function like 'fold((x, acc) -> x + acc, col, 0)' into the {@link LambdaExpr}
+   * '(x, acc) -> x + acc' and the list of arguments, ['col', 0], and so does not include the {@link LambdaExpr} here.
+   * To compensate for this, the error message will indicate that at least count + 1 arguments are required to count
+   * the lambda.
+   */
+  default void validationHelperCheckMinArgumentCount(LambdaExpr lambdaExpr, List<Expr> args, int count)
+  {
+    if (args.size() < count) {
+      throw validationFailed("requires at least %d arguments", count + 1);
+    }
+    validationHelperCheckLambaArgumentCount(lambdaExpr, args);
+  }
+
+  /**
+   * Helper method for implementors performing validation to check that the {@link LambdaExpr#identifierCount()}
+   * matches the number of arguments being passed to it
+   */
+  default void validationHelperCheckLambaArgumentCount(LambdaExpr lambdaExpr, List<Expr> args)
+  {
+    if (args.size() != lambdaExpr.identifierCount()) {
+      throw validationFailed(
+          "lambda expression argument count of %d does not match the %d arguments passed to it",
+          lambdaExpr.identifierCount(),
+          args.size()
+      );
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/druid/math/expr/vector/VectorProcessors.java b/core/src/main/java/org/apache/druid/math/expr/vector/VectorProcessors.java
index 47c7389805..4a548ca10d 100644
--- a/core/src/main/java/org/apache/druid/math/expr/vector/VectorProcessors.java
+++ b/core/src/main/java/org/apache/druid/math/expr/vector/VectorProcessors.java
@@ -53,11 +53,17 @@ public class VectorProcessors
   {
     final ExpressionType leftType = left.getOutputType(inspector);
 
+    // if type is null, it means the input is all nulls
     if (leftType == null) {
       return right.buildVectorized(inspector);
     }
 
-    Preconditions.checkArgument(inspector.areSameTypes(left, right));
+    Preconditions.checkArgument(
+        inspector.areSameTypes(left, right),
+        "%s and %s are not the same type",
+        leftType,
+        right.getOutputType(inspector)
+    );
 
     ExprVectorProcessor<?> processor = null;
     if (Types.is(leftType, ExprType.STRING)) {
diff --git a/core/src/test/java/org/apache/druid/math/expr/ApplyFunctionTest.java b/core/src/test/java/org/apache/druid/math/expr/ApplyFunctionTest.java
index d20fffee0f..d97055953a 100644
--- a/core/src/test/java/org/apache/druid/math/expr/ApplyFunctionTest.java
+++ b/core/src/test/java/org/apache/druid/math/expr/ApplyFunctionTest.java
@@ -65,7 +65,7 @@ public class ApplyFunctionTest extends InitializedNullHandlingTest
 
     assertExpr("map((x) -> x + 1, map((x) -> x + 1, [1, 2, 3, 4, 5]))", new Long[] {3L, 4L, 5L, 6L, 7L});
     assertExpr("map((x) -> x + 1, map((x) -> x + 1, b))", new Long[] {3L, 4L, 5L, 6L, 7L});
-    assertExpr("map(() -> 1, [1, 2, 3, 4, 5])", new Long[] {1L, 1L, 1L, 1L, 1L});
+    assertExpr("map((x) -> 1, [1, 2, 3, 4, 5])", new Long[] {1L, 1L, 1L, 1L, 1L});
   }
 
   @Test
@@ -73,7 +73,7 @@ public class ApplyFunctionTest extends InitializedNullHandlingTest
   {
     assertExpr("cartesian_map((x, y) -> concat(x, y), ['foo', 'bar', 'baz', 'foobar'], ['bar', 'baz'])", new String[] {"foobar", "foobaz", "barbar", "barbaz", "bazbar", "bazbaz", "foobarbar", "foobarbaz"});
     assertExpr("cartesian_map((x, y, z) -> concat(concat(x, y), z), ['foo', 'bar', 'baz', 'foobar'], ['bar', 'baz'], ['omg'])", new String[] {"foobaromg", "foobazomg", "barbaromg", "barbazomg", "bazbaromg", "bazbazomg", "foobarbaromg", "foobarbazomg"});
-    assertExpr("cartesian_map(() -> 1, [1, 2], [1, 2, 3])", new Long[] {1L, 1L, 1L, 1L, 1L, 1L});
+    assertExpr("cartesian_map((x, y) -> 1, [1, 2], [1, 2, 3])", new Long[] {1L, 1L, 1L, 1L, 1L, 1L});
     assertExpr("cartesian_map((x, y) -> concat(x, y), d, d)", new String[] {null});
     assertExpr("cartesian_map((x, y) -> concat(x, y), d, f)", new String[0]);
     if (NullHandling.replaceWithDefault()) {
@@ -140,21 +140,60 @@ public class ApplyFunctionTest extends InitializedNullHandlingTest
   }
 
   @Test
-  public void testInvalidArgCount()
+  public void testInvalidArgCountFold()
   {
-    expectedException.expect(IllegalArgumentException.class);
-    expectedException.expectMessage("lambda expression argument count does not match fold argument count");
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[fold] requires 3 arguments");
+    assertExpr("fold((x, y) -> x + 1, [1, 1, 1, 1, 1])", null);
+  }
+
+  @Test
+  public void testInvalidArgCountFoldLambda()
+  {
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[fold] lambda expression argument count of 0 does not match the 2 arguments passed to it");
     assertExpr("fold(() -> 1, [1, 1, 1, 1, 1], 0)", null);
 
-    expectedException.expectMessage("lambda expression argument count does not match cartesian_fold argument count");
+  }
+
+  @Test
+  public void testInvalidArgCountCartesianFoldLambda()
+  {
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[cartesian_fold] lambda expression argument count of 0 does not match the 3 arguments passed to it");
     assertExpr("cartesian_fold(() -> 1, [1, 1, 1, 1, 1], [1, 1], 0)", null);
+  }
 
-    expectedException.expectMessage("lambda expression argument count does not match any argument count");
+  @Test
+  public void testInvalidArgCountAny()
+  {
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[any] requires 2 arguments");
+    assertExpr("any((x) -> 1, [1, 2, 3, 4], y)", null);
+  }
+
+  @Test
+  public void testInvalidArgCountAnyLambda()
+  {
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[any] lambda expression argument count of 0 does not match the 1 arguments passed to it");
     assertExpr("any(() -> 1, [1, 2, 3, 4])", null);
+  }
 
-    expectedException.expectMessage("lambda expression argument count does not match all argument count");
-    assertExpr("all(() -> 0, [1, 2, 3, 4])", null);
+  @Test
+  public void testInvalidArgCountAll()
+  {
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[all] requires 2 arguments");
+    assertExpr("all((x) -> 0, [1, 2, 3, 4], y)", null);
+  }
 
+  @Test
+  public void testInvalidArgCountAllLambda()
+  {
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[all] lambda expression argument count of 0 does not match the 1 arguments passed to it");
+    assertExpr("all(() -> 0, [1, 2, 3, 4])", null);
   }
 
   private void assertExpr(final String expression, final Object expectedResult)
diff --git a/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java b/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java
index 26deec240c..0f9bba6652 100644
--- a/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java
+++ b/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java
@@ -500,10 +500,10 @@ public class FunctionTest extends InitializedNullHandlingTest
           assertExpr(StringUtils.format("round(%s)", argAndType.lhs), null);
           Assert.fail("Did not throw IllegalArgumentException");
         }
-        catch (IllegalArgumentException e) {
+        catch (ExpressionValidationException e) {
           Assert.assertEquals(
               StringUtils.format(
-                  "The first argument to the function[round] should be integer or double type but got the type: %s",
+                  "Function[round] first argument should be a LONG or DOUBLE but got %s instead",
                   argAndType.rhs
               ),
               e.getMessage()
@@ -528,10 +528,10 @@ public class FunctionTest extends InitializedNullHandlingTest
         assertExpr(String.format(Locale.ENGLISH, "round(d, %s)", argAndType.lhs), null);
         Assert.fail("Did not throw IllegalArgumentException");
       }
-      catch (IllegalArgumentException e) {
+      catch (ExpressionValidationException e) {
         Assert.assertEquals(
             StringUtils.format(
-                "The second argument to the function[round] should be integer type but got the type: %s",
+                "Function[round] second argument should be a LONG but got %s instead",
                 argAndType.rhs
             ),
             e.getMessage()
@@ -558,7 +558,7 @@ public class FunctionTest extends InitializedNullHandlingTest
       assertExpr("greatest(1, ['A'])", null);
       Assert.fail("Did not throw IllegalArgumentException");
     }
-    catch (IllegalArgumentException e) {
+    catch (ExpressionValidationException e) {
       Assert.assertEquals("Function[greatest] does not accept ARRAY<STRING> types", e.getMessage());
     }
 
@@ -586,7 +586,7 @@ public class FunctionTest extends InitializedNullHandlingTest
       assertExpr("least(1, [2, 3])", null);
       Assert.fail("Did not throw IllegalArgumentException");
     }
-    catch (IllegalArgumentException e) {
+    catch (ExpressionValidationException e) {
       Assert.assertEquals("Function[least] does not accept ARRAY<LONG> types", e.getMessage());
     }
 
@@ -678,9 +678,9 @@ public class FunctionTest extends InitializedNullHandlingTest
       // but for non-sqlCompatible, it must not go to here
       Assert.assertTrue(NullHandling.sqlCompatible() ? true : false);
     }
-    catch (IAE e) {
+    catch (ExpressionValidationException e) {
       Assert.assertEquals(
-          "Function[human_readable_binary_byte_format] needs a number as its first argument",
+          "Function[human_readable_binary_byte_format] needs a number as its first argument but got STRING instead",
           e.getMessage()
       );
     }
@@ -693,9 +693,9 @@ public class FunctionTest extends InitializedNullHandlingTest
       //must not go to here
       Assert.assertTrue(false);
     }
-    catch (IAE e) {
+    catch (ExpressionValidationException e) {
       Assert.assertEquals(
-          "Function[human_readable_binary_byte_format] needs an integer as its second argument",
+          "Function[human_readable_binary_byte_format] needs a LONG as its second argument but got STRING instead",
           e.getMessage()
       );
     }
@@ -708,9 +708,9 @@ public class FunctionTest extends InitializedNullHandlingTest
       //must not go to here
       Assert.assertTrue(false);
     }
-    catch (IAE e) {
+    catch (ExpressionValidationException e) {
       Assert.assertEquals(
-          "Function[human_readable_binary_byte_format] needs an integer as its second argument",
+          "Function[human_readable_binary_byte_format] needs a LONG as its second argument but got DOUBLE instead",
           e.getMessage()
       );
     }
@@ -723,9 +723,9 @@ public class FunctionTest extends InitializedNullHandlingTest
       //must not go to here
       Assert.assertTrue(false);
     }
-    catch (IAE e) {
+    catch (ExpressionValidationException e) {
       Assert.assertEquals(
-          "Function[human_readable_binary_byte_format] needs an integer as its second argument",
+          "Function[human_readable_binary_byte_format] needs a LONG as its second argument but got STRING instead",
           e.getMessage()
       );
     }
@@ -739,9 +739,9 @@ public class FunctionTest extends InitializedNullHandlingTest
             .eval(bindings);
       Assert.assertTrue(false);
     }
-    catch (IAE e) {
+    catch (ExpressionValidationException e) {
       Assert.assertEquals(
-          "Given precision[9223372036854775807] of Function[human_readable_binary_byte_format] must be in the range of [0,3]",
+          "Function[human_readable_binary_byte_format] given precision[9223372036854775807] must be in the range of [0,3]",
           e.getMessage()
       );
     }
@@ -751,9 +751,9 @@ public class FunctionTest extends InitializedNullHandlingTest
             .eval(bindings);
       Assert.assertTrue(false);
     }
-    catch (IAE e) {
+    catch (ExpressionValidationException e) {
       Assert.assertEquals(
-          "Given precision[-9223372036854775808] of Function[human_readable_binary_byte_format] must be in the range of [0,3]",
+          "Function[human_readable_binary_byte_format] given precision[-9223372036854775808] must be in the range of [0,3]",
           e.getMessage()
       );
     }
@@ -763,9 +763,9 @@ public class FunctionTest extends InitializedNullHandlingTest
             .eval(bindings);
       Assert.assertTrue(false);
     }
-    catch (IAE e) {
+    catch (ExpressionValidationException e) {
       Assert.assertEquals(
-          "Given precision[-1] of Function[human_readable_binary_byte_format] must be in the range of [0,3]",
+          "Function[human_readable_binary_byte_format] given precision[-1] must be in the range of [0,3]",
           e.getMessage()
       );
     }
@@ -775,9 +775,9 @@ public class FunctionTest extends InitializedNullHandlingTest
             .eval(bindings);
       Assert.assertTrue(false);
     }
-    catch (IAE e) {
+    catch (ExpressionValidationException e) {
       Assert.assertEquals(
-          "Given precision[4] of Function[human_readable_binary_byte_format] must be in the range of [0,3]",
+          "Function[human_readable_binary_byte_format] given precision[4] must be in the range of [0,3]",
           e.getMessage()
       );
     }
@@ -786,8 +786,8 @@ public class FunctionTest extends InitializedNullHandlingTest
   @Test
   public void testSizeFormatInvalidArgumentSize()
   {
-    expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[human_readable_binary_byte_format] needs 1 or 2 arguments");
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[human_readable_binary_byte_format] requires 1 or 2 arguments");
     Parser.parse("human_readable_binary_byte_format(1024, 2, 3)", ExprMacroTable.nil())
           .eval(bindings);
   }
@@ -837,9 +837,9 @@ public class FunctionTest extends InitializedNullHandlingTest
       assertExpr("bitwiseComplement(461168601842738800000000000000.000000)", null);
       Assert.fail("Did not throw IllegalArgumentException");
     }
-    catch (IllegalArgumentException e) {
+    catch (ExpressionValidationException e) {
       Assert.assertEquals(
-          "Possible data truncation, param [461168601842738800000000000000.000000] is out of long value range",
+          "Function[bitwiseComplement] Possible data truncation, param [461168601842738800000000000000.000000] is out of LONG value range",
           e.getMessage()
       );
     }
@@ -916,8 +916,8 @@ public class FunctionTest extends InitializedNullHandlingTest
   @Test
   public void testComplexDecodeBaseWrongArgCount()
   {
-    expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[complex_decode_base64] needs 2 arguments");
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[complex_decode_base64] requires 2 arguments");
     assertExpr(
         "complex_decode_base64(string)",
         null
@@ -927,9 +927,9 @@ public class FunctionTest extends InitializedNullHandlingTest
   @Test
   public void testComplexDecodeBaseArg0BadType()
   {
-    expectedException.expect(IAE.class);
+    expectedException.expect(ExpressionValidationException.class);
     expectedException.expectMessage(
-        "Function[complex_decode_base64] first argument must be constant 'STRING' expression containing a valid complex type name"
+        "Function[complex_decode_base64] first argument must be constant STRING expression containing a valid complex type name but got LONG instead"
     );
     assertExpr(
         "complex_decode_base64(1, string)",
@@ -940,9 +940,9 @@ public class FunctionTest extends InitializedNullHandlingTest
   @Test
   public void testComplexDecodeBaseArg0Unknown()
   {
-    expectedException.expect(IAE.class);
+    expectedException.expect(ExpressionValidationException.class);
     expectedException.expectMessage(
-        "Function[complex_decode_base64] first argument must be a valid complex type name, unknown complex type [COMPLEX<unknown>]"
+        "Function[complex_decode_base64] first argument must be a valid COMPLEX type name, got unknown COMPLEX type [COMPLEX<unknown>]"
     );
     assertExpr(
         "complex_decode_base64('unknown', string)",
@@ -960,32 +960,32 @@ public class FunctionTest extends InitializedNullHandlingTest
   @Test
   public void testMVToArrayWithConstantLiteral()
   {
-    expectedException.expect(IAE.class);
-    expectedException.expectMessage("should be an identifier expression");
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[mv_to_array] argument 1 should be an identifier expression. Use array() instead");
     assertArrayExpr("mv_to_array('1')", null);
   }
 
   @Test
   public void testMVToArrayWithFunction()
   {
-    expectedException.expect(IAE.class);
-    expectedException.expectMessage("should be an identifier expression");
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[mv_to_array] argument (repeat [hello, 2]) should be an identifier expression. Use array() instead");
     assertArrayExpr("mv_to_array(repeat('hello', 2))", null);
   }
 
   @Test
   public void testMVToArrayWithMoreArgs()
   {
-    expectedException.expect(IAE.class);
-    expectedException.expectMessage("needs exactly 1 argument of type String");
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[mv_to_array] requires 1 argument");
     assertArrayExpr("mv_to_array(x,y)", null);
   }
 
   @Test
   public void testMVToArrayWithNoArgs()
   {
-    expectedException.expect(IAE.class);
-    expectedException.expectMessage("needs exactly 1 argument of type String");
+    expectedException.expect(ExpressionValidationException.class);
+    expectedException.expectMessage("Function[mv_to_array] requires 1 argument");
     assertArrayExpr("mv_to_array()", null);
   }
 
@@ -999,7 +999,7 @@ public class FunctionTest extends InitializedNullHandlingTest
   public void testMultiplyOnString()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("operator '*' in expression (\"str1\" * \"str2\") is not supported on type 'string'.");
+    expectedException.expectMessage("operator '*' in expression (\"str1\" * \"str2\") is not supported on type STRING.");
     assertExpr("str1 * str2", null);
   }
 
@@ -1007,7 +1007,7 @@ public class FunctionTest extends InitializedNullHandlingTest
   public void testMinusOnString()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("operator '-' in expression (\"str1\" - \"str2\") is not supported on type 'string'.");
+    expectedException.expectMessage("operator '-' in expression (\"str1\" - \"str2\") is not supported on type STRING.");
     assertExpr("str1 - str2", null);
   }
 
@@ -1015,7 +1015,7 @@ public class FunctionTest extends InitializedNullHandlingTest
   public void testDivOnString()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("operator '/' in expression (\"str1\" / \"str2\") is not supported on type 'string'.");
+    expectedException.expectMessage("operator '/' in expression (\"str1\" / \"str2\") is not supported on type STRING.");
     assertExpr("str1 / str2", null);
   }
 
diff --git a/core/src/test/java/org/apache/druid/math/expr/ParserTest.java b/core/src/test/java/org/apache/druid/math/expr/ParserTest.java
index b997a53f91..749e38a83a 100644
--- a/core/src/test/java/org/apache/druid/math/expr/ParserTest.java
+++ b/core/src/test/java/org/apache/druid/math/expr/ParserTest.java
@@ -479,8 +479,8 @@ public class ParserTest extends InitializedNullHandlingTest
   public void testApplyFunctions()
   {
     validateParser(
-        "map(() -> 1, x)",
-        "(map ([] -> 1), [x])",
+        "map((x) -> 1, x)",
+        "(map ([x] -> 1), [x])",
         ImmutableList.of("x"),
         ImmutableSet.of(),
         ImmutableSet.of("x")
diff --git a/docs/misc/math-expr.md b/docs/misc/math-expr.md
index 1f85a59f3c..d58261e61e 100644
--- a/docs/misc/math-expr.md
+++ b/docs/misc/math-expr.md
@@ -200,8 +200,8 @@ Apply functions allow for special 'lambda' expressions to be defined and applied
 | map(lambda,arr) | applies a transform specified by a single argument lambda expression to all elements of arr, returning a new array |
 | cartesian_map(lambda,arr1,arr2,...) | applies a transform specified by a multi argument lambda expression to all elements of the Cartesian product of all input arrays, returning a new array; the number of lambda arguments and array inputs must be the same |
 | filter(lambda,arr) | filters arr by a single argument lambda, returning a new array with all matching elements, or null if no elements match |
-| fold(lambda,arr) | folds a 2 argument lambda across arr. The first argument of the lambda is the array element and the second the accumulator, returning a single accumulated value. |
-| cartesian_fold(lambda,arr1,arr2,...) | folds a multi argument lambda across the Cartesian product of all input arrays. The first arguments of the lambda is the array element and the last is the accumulator, returning a single accumulated value. |
+| fold(lambda,arr,acc) | folds a 2 argument lambda across arr using acc as the initial input value. The first argument of the lambda is the array element and the second the accumulator, returning a single accumulated value. |
+| cartesian_fold(lambda,arr1,arr2,...,acc) | folds a multi argument lambda across the Cartesian product of all input arrays using acc as the initial input value. The first arguments of the lambda are the array elements of each array and the last is the accumulator, returning a single accumulated value. |
 | any(lambda,arr) | returns 1 if any element in the array matches the lambda expression, else 0 |
 | all(lambda,arr) | returns 1 if all elements in the array matches the lambda expression, else 0 |
 
diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/expressions/BloomFilterExpressions.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/expressions/BloomFilterExpressions.java
index e26bcf730c..9461a3635d 100644
--- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/expressions/BloomFilterExpressions.java
+++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/expressions/BloomFilterExpressions.java
@@ -20,7 +20,6 @@
 package org.apache.druid.query.expressions;
 
 import org.apache.druid.guice.BloomFilterSerializersModule;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -55,14 +54,12 @@ public class BloomFilterExpressions
     @Override
     public Expr apply(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] must have 1 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 1);
 
       final Expr expectedSizeArg = args.get(0);
 
       if (!expectedSizeArg.isLiteral() || expectedSizeArg.getLiteralValue() == null) {
-        throw new IAE("Function[%s] argument must be an LONG constant", name());
+        throw validationFailed("argument must be a LONG constant");
       }
 
       class BloomExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
@@ -115,9 +112,7 @@ public class BloomFilterExpressions
     @Override
     public Expr apply(List<Expr> args)
     {
-      if (args.size() != 2) {
-        throw new IAE("Function[%s] must have 2 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 2);
 
       class BloomExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
       {
@@ -134,7 +129,7 @@ public class BloomFilterExpressions
           // type information everywhere
           if (!bloomy.type().equals(BLOOM_FILTER_TYPE) ||
               !bloomy.type().is(ExprType.COMPLEX) && bloomy.value() instanceof BloomKFilter) {
-            throw new IAE("Function[%s] must take a bloom filter as the second argument", FN_NAME);
+            throw AddExprMacro.this.validationFailed("must take a bloom filter as the second argument");
           }
           BloomKFilter filter = (BloomKFilter) bloomy.value();
           assert filter != null;
@@ -154,12 +149,13 @@ public class BloomFilterExpressions
                 filter.addLong(input.asLong());
                 break;
               case COMPLEX:
-                if (BLOOM_FILTER_TYPE.equals(input.type()) || (bloomy.type().is(ExprType.COMPLEX) && bloomy.value() instanceof BloomKFilter)) {
+                if (BLOOM_FILTER_TYPE.equals(input.type()) ||
+                    (bloomy.type().is(ExprType.COMPLEX) && bloomy.value() instanceof BloomKFilter)) {
                   filter.merge((BloomKFilter) input.value());
                   break;
                 }
               default:
-                throw new IAE("Function[%s] cannot add [%s] to a bloom filter", FN_NAME, input.type());
+                throw AddExprMacro.this.validationFailed("cannot add [%s] to a bloom filter", input.type());
             }
           }
 
@@ -198,9 +194,7 @@ public class BloomFilterExpressions
     @Override
     public Expr apply(List<Expr> args)
     {
-      if (args.size() != 2) {
-        throw new IAE("Function[%s] must have 2 arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 2);
 
       class BloomExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
       {
@@ -283,7 +277,7 @@ public class BloomFilterExpressions
           // type information everywhere
           if (!bloomy.type().equals(BLOOM_FILTER_TYPE) ||
               !bloomy.type().is(ExprType.COMPLEX) && bloomy.value() instanceof BloomKFilter) {
-            throw new IAE("Function[%s] must take a bloom filter as the second argument", FN_NAME);
+            throw TestExprMacro.this.validationFailed("must take a bloom filter as the second argument");
           }
           BloomKFilter filter = (BloomKFilter) bloomy.value();
           assert filter != null;
@@ -339,7 +333,6 @@ public class BloomFilterExpressions
         }
       }
 
-
       final Expr arg = args.get(0);
       final Expr filterExpr = args.get(1);
 
@@ -351,7 +344,7 @@ public class BloomFilterExpressions
           filter = BloomFilterSerializersModule.bloomKFilterFromBytes(decoded);
         }
         catch (IOException ioe) {
-          throw new RuntimeException("Failed to deserialize bloom filter", ioe);
+          throw processingFailed(ioe, "failed to deserialize bloom filter");
         }
         return new BloomExpr(filter, arg);
       } else {
diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/expressions/BloomFilterExpressionsTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/expressions/BloomFilterExpressionsTest.java
index 6a6be4e886..f2b4e4f074 100644
--- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/expressions/BloomFilterExpressionsTest.java
+++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/expressions/BloomFilterExpressionsTest.java
@@ -179,7 +179,7 @@ public class BloomFilterExpressionsTest extends InitializedNullHandlingTest
   public void testCreateWrongArgsCount()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[bloom_filter] must have 1 argument");
+    expectedException.expectMessage("Function[bloom_filter] requires 1 argument");
     Parser.parse("bloom_filter()", macroTable);
   }
 
@@ -187,7 +187,7 @@ public class BloomFilterExpressionsTest extends InitializedNullHandlingTest
   public void testAddWrongArgsCount()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[bloom_filter_add] must have 2 arguments");
+    expectedException.expectMessage("Function[bloom_filter_add] requires 2 arguments");
     Parser.parse("bloom_filter_add(1)", macroTable);
   }
 
@@ -212,7 +212,7 @@ public class BloomFilterExpressionsTest extends InitializedNullHandlingTest
   public void testTestWrongArgsCount()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[bloom_filter_test] must have 2 arguments");
+    expectedException.expectMessage("Function[bloom_filter_test] requires 2 arguments");
     Parser.parse("bloom_filter_test(1)", macroTable);
   }
 
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 0fd2aa1a64..e198354928 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
@@ -19,13 +19,11 @@
 
 package org.apache.druid.query.expressions;
 
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
 import org.apache.druid.math.expr.ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr;
 import org.apache.druid.math.expr.ExprMacroTable.ExprMacro;
 import org.apache.druid.math.expr.ExpressionType;
-import org.apache.druid.query.expression.ExprUtils;
 
 import java.util.List;
 
@@ -50,9 +48,7 @@ public class SleepExprMacro implements ExprMacro
   @Override
   public Expr apply(List<Expr> args)
   {
-    if (args.size() != 1) {
-      throw new IAE(ExprUtils.createErrMsg(name(), "must have 1 argument"));
-    }
+    validationHelperCheckArgumentCount(args, 1);
 
     Expr arg = args.get(0);
 
@@ -78,7 +74,7 @@ public class SleepExprMacro implements ExprMacro
         }
         catch (InterruptedException e) {
           Thread.currentThread().interrupt();
-          throw new RuntimeException(e);
+          throw processingFailed(e, "interrupted");
         }
       }
 
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 d278f6cda3..ff6882ff0c 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
@@ -19,13 +19,11 @@
 
 package org.apache.druid.testing.tools;
 
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
 import org.apache.druid.math.expr.ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr;
 import org.apache.druid.math.expr.ExprMacroTable.ExprMacro;
 import org.apache.druid.math.expr.ExpressionType;
-import org.apache.druid.query.expression.ExprUtils;
 
 import java.util.List;
 
@@ -50,9 +48,7 @@ public class SleepExprMacro implements ExprMacro
   @Override
   public Expr apply(List<Expr> args)
   {
-    if (args.size() != 1) {
-      throw new IAE(ExprUtils.createErrMsg(name(), "must have 1 argument"));
-    }
+    validationHelperCheckArgumentCount(args, 1);
 
     Expr arg = args.get(0);
 
@@ -78,7 +74,7 @@ public class SleepExprMacro implements ExprMacro
         }
         catch (InterruptedException e) {
           Thread.currentThread().interrupt();
-          throw new RuntimeException(e);
+          throw processingFailed(e, "interrupted");
         }
       }
 
diff --git a/processing/src/main/java/org/apache/druid/query/expression/CaseInsensitiveContainsExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/CaseInsensitiveContainsExprMacro.java
index 1fc9bf93e0..96dbdbbd80 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/CaseInsensitiveContainsExprMacro.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/CaseInsensitiveContainsExprMacro.java
@@ -19,7 +19,6 @@
 
 package org.apache.druid.query.expression;
 
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprMacroTable;
 
@@ -52,9 +51,7 @@ public class CaseInsensitiveContainsExprMacro implements ExprMacroTable.ExprMacr
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() != 2) {
-      throw new IAE("Function[%s] must have 2 arguments", name());
-    }
+    validationHelperCheckArgumentCount(args, 2);
 
     final Expr arg = args.get(0);
     final Expr searchStr = args.get(1);
diff --git a/processing/src/main/java/org/apache/druid/query/expression/ContainsExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/ContainsExprMacro.java
index 32c1b6c240..35ff176ae3 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/ContainsExprMacro.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/ContainsExprMacro.java
@@ -19,7 +19,6 @@
 
 package org.apache.druid.query.expression;
 
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprMacroTable;
 
@@ -51,9 +50,7 @@ public class ContainsExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() != 2) {
-      throw new IAE("Function[%s] must have 2 arguments", name());
-    }
+    validationHelperCheckArgumentCount(args, 2);
 
     final Expr arg = args.get(0);
     final Expr searchStr = args.get(1);
diff --git a/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java b/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java
index 4aa2317a2c..3a7224f59d 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java
@@ -19,7 +19,6 @@
 
 package org.apache.druid.query.expression;
 
-import com.google.common.base.Preconditions;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.IAE;
@@ -81,17 +80,6 @@ public class ExprUtils
     return new PeriodGranularity(period, origin, timeZone);
   }
 
-  public static String createErrMsg(String functionName, String msg)
-  {
-    String prefix = "Function[" + functionName + "] ";
-    return prefix + msg;
-  }
-
-  static void checkLiteralArgument(String functionName, Expr arg, String argName)
-  {
-    Preconditions.checkArgument(arg.isLiteral(), createErrMsg(functionName, argName + " arg must be a literal"));
-  }
-
   /**
    * True if Expr is a string literal.
    *
diff --git a/processing/src/main/java/org/apache/druid/query/expression/HyperUniqueExpressions.java b/processing/src/main/java/org/apache/druid/query/expression/HyperUniqueExpressions.java
index 1cdd7300c3..6b5cb5fba6 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/HyperUniqueExpressions.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/HyperUniqueExpressions.java
@@ -21,7 +21,6 @@ package org.apache.druid.query.expression;
 
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.hll.HyperLogLogCollector;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -54,9 +53,7 @@ public class HyperUniqueExpressions
     @Override
     public Expr apply(List<Expr> args)
     {
-      if (args.size() > 0) {
-        throw new IAE("Function[%s] must have no arguments", name());
-      }
+      validationHelperCheckArgumentCount(args, 0);
       final HyperLogLogCollector collector = HyperLogLogCollector.makeLatestCollector();
       class HllExpression implements ExprMacroTable.ExprMacroFunctionExpr
       {
@@ -138,10 +135,7 @@ public class HyperUniqueExpressions
     @Override
     public Expr apply(List<Expr> args)
     {
-      if (args.size() != 2) {
-        throw new IAE("Function[%s] must have 2 arguments", name());
-      }
-
+      validationHelperCheckArgumentCount(args, 2);
 
       class HllExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
       {
@@ -160,7 +154,9 @@ public class HyperUniqueExpressions
           if (!TYPE.equals(hllType) ||
               !(hllType.is(ExprType.COMPLEX) && hllCollector.value() instanceof HyperLogLogCollector)
           ) {
-            throw new IAE("Function[%s] must take a hyper-log-log collector as the second argument", NAME);
+            throw HllAddExprMacro.this.validationFailed(
+                "requires a hyper-log-log collector as the second argument"
+            );
           }
           HyperLogLogCollector collector = (HyperLogLogCollector) hllCollector.value();
           assert collector != null;
@@ -195,7 +191,10 @@ public class HyperUniqueExpressions
                 break;
               }
             default:
-              throw new IAE("Function[%s] cannot add [%s] to hyper-log-log collector", NAME, input.type());
+              throw HllAddExprMacro.this.validationFailed(
+                  "cannot add [%s] to hyper-log-log collector",
+                  input.type()
+              );
           }
 
           return ExprEval.ofComplex(TYPE, collector);
@@ -231,9 +230,8 @@ public class HyperUniqueExpressions
     @Override
     public Expr apply(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] must have 1 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 1);
+
       class HllExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
       {
         public HllExpr(Expr arg)
@@ -250,7 +248,10 @@ public class HyperUniqueExpressions
           if (!TYPE.equals(hllCollector.type()) ||
               !(hllCollector.type().is(ExprType.COMPLEX) && hllCollector.value() instanceof HyperLogLogCollector)
           ) {
-            throw new IAE("Function[%s] must take a hyper-log-log collector as input", NAME);
+            throw HllEstimateExprMacro.this.validationFailed(
+                "requires a hyper-log-log collector as input but got %s instead",
+                hllCollector.type()
+            );
           }
           HyperLogLogCollector collector = (HyperLogLogCollector) hllCollector.value();
           assert collector != null;
@@ -287,9 +288,7 @@ public class HyperUniqueExpressions
     @Override
     public Expr apply(List<Expr> args)
     {
-      if (args.size() != 1) {
-        throw new IAE("Function[%s] must have 1 argument", name());
-      }
+      validationHelperCheckArgumentCount(args, 1);
 
       class HllExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr
       {
@@ -303,7 +302,10 @@ public class HyperUniqueExpressions
         {
           ExprEval hllCollector = args.get(0).eval(bindings);
           if (!hllCollector.type().equals(TYPE)) {
-            throw new IAE("Function[%s] must take a hyper-log-log collector as input", NAME);
+            throw HllRoundEstimateExprMacro.this.validationFailed(
+                "requires a hyper-log-log collector as input but got %s instead",
+                hllCollector.type()
+            );
           }
           HyperLogLogCollector collector = (HyperLogLogCollector) hllCollector.value();
           assert collector != null;
diff --git a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacro.java
index 33273b2de7..6887ddc615 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacro.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacro.java
@@ -23,7 +23,6 @@ import inet.ipaddr.AddressStringException;
 import inet.ipaddr.IPAddress;
 import inet.ipaddr.IPAddressString;
 import inet.ipaddr.ipv4.IPv4Address;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -69,9 +68,7 @@ public class IPv4AddressMatchExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() != 2) {
-      throw new IAE(ExprUtils.createErrMsg(name(), "must have 2 arguments"));
-    }
+    validationHelperCheckArgumentCount(args, 2);
 
     try {
       final Expr arg = args.get(0);
@@ -142,7 +139,7 @@ public class IPv4AddressMatchExprMacro implements ExprMacroTable.ExprMacro
     }
 
     catch (AddressStringException e) {
-      throw new RuntimeException(e);
+      throw processingFailed(e, "failed to parse address");
     }
   }
 
@@ -150,10 +147,10 @@ public class IPv4AddressMatchExprMacro implements ExprMacroTable.ExprMacro
   {
     String subnetArgName = "subnet";
     Expr arg = args.get(ARG_SUBNET);
-    ExprUtils.checkLiteralArgument(name(), arg, subnetArgName);
+    validationHelperCheckArgIsLiteral(arg, subnetArgName);
     String subnet = (String) arg.getLiteralValue();
     if (!IPv4AddressExprUtils.isValidIPv4Subnet(subnet)) {
-      throw new IAE(ExprUtils.createErrMsg(name(), subnetArgName + " arg has an invalid format: " + subnet));
+      throw validationFailed(subnetArgName + " arg has an invalid format: " + subnet);
     }
     return new IPAddressString(subnet);
   }
diff --git a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressParseExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressParseExprMacro.java
index 9ccd2cd714..5b768b7f62 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressParseExprMacro.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/IPv4AddressParseExprMacro.java
@@ -20,7 +20,6 @@
 package org.apache.druid.query.expression;
 
 import inet.ipaddr.ipv4.IPv4Address;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
 import org.apache.druid.math.expr.ExprMacroTable;
@@ -60,9 +59,7 @@ public class IPv4AddressParseExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() != 1) {
-      throw new IAE(ExprUtils.createErrMsg(name(), "must have 1 argument"));
-    }
+    validationHelperCheckArgumentCount(args, 1);
 
     Expr arg = args.get(0);
 
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 b5c9a1166e..625fc48d91 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
@@ -20,7 +20,6 @@
 package org.apache.druid.query.expression;
 
 import inet.ipaddr.ipv4.IPv4Address;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
 import org.apache.druid.math.expr.ExprMacroTable;
@@ -59,9 +58,7 @@ public class IPv4AddressStringifyExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() != 1) {
-      throw new IAE(ExprUtils.createErrMsg(name(), "must have 1 argument"));
-    }
+    validationHelperCheckArgumentCount(args, 1);
 
     Expr arg = args.get(0);
 
diff --git a/processing/src/main/java/org/apache/druid/query/expression/LikeExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/LikeExprMacro.java
index 81b2fe263e..254d09f907 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/LikeExprMacro.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/LikeExprMacro.java
@@ -20,7 +20,6 @@
 package org.apache.druid.query.expression;
 
 import org.apache.druid.common.config.NullHandling;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -45,23 +44,22 @@ public class LikeExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() < 2 || args.size() > 3) {
-      throw new IAE("Function[%s] must have 2 or 3 arguments", name());
-    }
+    validationHelperCheckAnyOfArgumentCount(args, 2, 3);
 
     final Expr arg = args.get(0);
     final Expr patternExpr = args.get(1);
     final Expr escapeExpr = args.size() > 2 ? args.get(2) : null;
 
-    if (!patternExpr.isLiteral() || (escapeExpr != null && !escapeExpr.isLiteral())) {
-      throw new IAE("pattern and escape must be literals");
+    validationHelperCheckArgIsLiteral(patternExpr, "pattern");
+    if (escapeExpr != null) {
+      validationHelperCheckArgIsLiteral(escapeExpr, "escape");
     }
 
     final String escape = escapeExpr == null ? null : (String) escapeExpr.getLiteralValue();
     final Character escapeChar;
 
     if (escape != null && escape.length() != 1) {
-      throw new IllegalArgumentException("Escape must be null or a single character");
+      throw validationFailed("escape must be null or a single character");
     } else {
       escapeChar = escape == null ? null : escape.charAt(0);
     }
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 572cb4eee6..07fd7631d8 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
@@ -21,7 +21,6 @@ package org.apache.druid.query.expression;
 
 import com.google.inject.Inject;
 import org.apache.druid.common.config.NullHandling;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -56,15 +55,14 @@ public class LookupExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() != 2) {
-      throw new IAE("Function[%s] must have 2 arguments", name());
-    }
+    validationHelperCheckArgumentCount(args, 2);
 
     final Expr arg = args.get(0);
     final Expr lookupExpr = args.get(1);
 
-    if (!lookupExpr.isLiteral() || lookupExpr.getLiteralValue() == null) {
-      throw new IAE("Function[%s] second argument must be a registered lookup name", name());
+    validationHelperCheckArgIsLiteral(lookupExpr, "second argument");
+    if (lookupExpr.getLiteralValue() == null) {
+      throw validationFailed("second argument must be a registered lookup name");
     }
 
     final String lookupName = lookupExpr.getLiteralValue().toString();
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 768d519631..475ded6922 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
@@ -24,12 +24,12 @@ import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.common.base.Preconditions;
 import org.apache.druid.guice.annotations.Json;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
 import org.apache.druid.math.expr.ExprMacroTable;
 import org.apache.druid.math.expr.ExprType;
 import org.apache.druid.math.expr.ExpressionType;
+import org.apache.druid.math.expr.NamedFunction;
 import org.apache.druid.segment.nested.NestedDataComplexTypeSerde;
 import org.apache.druid.segment.nested.NestedPathFinder;
 import org.apache.druid.segment.nested.NestedPathPart;
@@ -62,7 +62,10 @@ public class NestedDataExpressions
     @Override
     public Expr apply(List<Expr> args)
     {
-      Preconditions.checkArgument(args.size() % 2 == 0);
+      if (args.size() % 2 != 0) {
+        throw validationFailed("must have an even number of arguments");
+      }
+
       class StructExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
       {
         public StructExpr(List<Expr> args)
@@ -78,7 +81,9 @@ public class NestedDataExpressions
             ExprEval field = args.get(i).eval(bindings);
             ExprEval value = args.get(i + 1).eval(bindings);
 
-            Preconditions.checkArgument(field.type().is(ExprType.STRING), "field name must be a STRING");
+            if (!field.type().is(ExprType.STRING)) {
+              throw JsonObjectExprMacro.this.validationFailed("field name must be a STRING");
+            }
             theMap.put(field.asString(), unwrap(value));
           }
 
@@ -146,7 +151,11 @@ public class NestedDataExpressions
             );
           }
           catch (JsonProcessingException e) {
-            throw new IAE(e, "Unable to stringify [%s] to JSON", input.value());
+            throw ToJsonStringExprMacro.this.processingFailed(
+                e,
+                "unable to stringify [%s] to JSON",
+                input.value()
+            );
           }
         }
 
@@ -213,15 +222,13 @@ public class NestedDataExpressions
               );
             }
             catch (JsonProcessingException e) {
-              throw new IAE("Bad string input [%s] to [%s]", arg.asString(), name());
+              throw ParseJsonExprMacro.this.processingFailed(e, "bad string input [%s]", arg.asString());
             }
           }
-          throw new IAE(
-              "Invalid input [%s] of type [%s] to [%s], expected [%s]",
-              arg.asString(),
-              arg.type(),
-              name(),
-              ExpressionType.STRING
+          throw ParseJsonExprMacro.this.validationFailed(
+              "invalid input expected %s but got %s instead",
+              ExpressionType.STRING,
+              arg.type()
           );
         }
 
@@ -328,11 +335,14 @@ public class NestedDataExpressions
     @Override
     public Expr apply(List<Expr> args)
     {
-      final List<NestedPathPart> parts = getJsonPathPartsFromLiteral(name(), args.get(1));
+      final List<NestedPathPart> parts = getJsonPathPartsFromLiteral(this, args.get(1));
       if (args.size() == 3 && args.get(2).isLiteral()) {
         final ExpressionType castTo = ExpressionType.fromString((String) args.get(2).getLiteralValue());
         if (castTo == null) {
-          throw new IAE("Invalid output type: [%s]", args.get(2).getLiteralValue());
+          throw JsonValueExprMacro.this.validationFailed(
+              "invalid output type: [%s]",
+              args.get(2).getLiteralValue()
+          );
         }
         final class JsonValueCastExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
         {
@@ -416,7 +426,7 @@ public class NestedDataExpressions
     @Override
     public Expr apply(List<Expr> args)
     {
-      final List<NestedPathPart> parts = getJsonPathPartsFromLiteral(name(), args.get(1));
+      final List<NestedPathPart> parts = getJsonPathPartsFromLiteral(this, args.get(1));
       final class JsonQueryExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
       {
         public JsonQueryExpr(List<Expr> args)
@@ -530,7 +540,7 @@ public class NestedDataExpressions
     @Override
     public Expr apply(List<Expr> args)
     {
-      final List<NestedPathPart> parts = getJsonPathPartsFromLiteral(name(), args.get(1));
+      final List<NestedPathPart> parts = getJsonPathPartsFromLiteral(this, args.get(1));
       final class JsonKeysExpr extends ExprMacroTable.BaseScalarMacroFunctionExpr
       {
         public JsonKeysExpr(List<Expr> args)
@@ -582,12 +592,11 @@ public class NestedDataExpressions
   }
 
 
-  static List<NestedPathPart> getJsonPathPartsFromLiteral(String fnName, Expr arg)
+  static List<NestedPathPart> getJsonPathPartsFromLiteral(NamedFunction fn, Expr arg)
   {
     if (!(arg.isLiteral() && arg.getLiteralValue() instanceof String)) {
-      throw new IAE(
-          "Function[%s] second argument [%s] must be a literal [%s] value",
-          fnName,
+      throw fn.validationFailed(
+          "second argument [%s] must be a literal [%s] value",
           arg.stringify(),
           ExpressionType.STRING
       );
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 a77d1dd2c5..c3506592ea 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
@@ -20,7 +20,6 @@
 package org.apache.druid.query.expression;
 
 import org.apache.druid.common.config.NullHandling;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -46,20 +45,18 @@ public class RegexpExtractExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() < 2 || args.size() > 3) {
-      throw new IAE("Function[%s] must have 2 to 3 arguments", name());
-    }
+    validationHelperCheckAnyOfArgumentCount(args, 2, 3);
 
     final Expr arg = args.get(0);
     final Expr patternExpr = args.get(1);
     final Expr indexExpr = args.size() > 2 ? args.get(2) : null;
 
     if (!ExprUtils.isStringLiteral(patternExpr)) {
-      throw new IAE("Function[%s] pattern must be a string literal", name());
+      throw validationFailed("pattern must be a string literal");
     }
 
     if (indexExpr != null && (!indexExpr.isLiteral() || !(indexExpr.getLiteralValue() instanceof Number))) {
-      throw new IAE("Function[%s] index must be a numeric literal", name());
+      throw validationFailed("index must be a numeric literal");
     }
 
     // Precompile the pattern.
diff --git a/processing/src/main/java/org/apache/druid/query/expression/RegexpLikeExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/RegexpLikeExprMacro.java
index 2ca6f9c7a2..b36440f1ee 100644
--- a/processing/src/main/java/org/apache/druid/query/expression/RegexpLikeExprMacro.java
+++ b/processing/src/main/java/org/apache/druid/query/expression/RegexpLikeExprMacro.java
@@ -20,7 +20,6 @@
 package org.apache.druid.query.expression;
 
 import org.apache.druid.common.config.NullHandling;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -46,15 +45,13 @@ public class RegexpLikeExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() != 2) {
-      throw new IAE("Function[%s] must have 2 arguments", name());
-    }
+    validationHelperCheckArgumentCount(args, 2);
 
     final Expr arg = args.get(0);
     final Expr patternExpr = args.get(1);
 
     if (!ExprUtils.isStringLiteral(patternExpr)) {
-      throw new IAE("Function[%s] pattern must be a string literal", name());
+      throw validationFailed("pattern must be a STRING literal");
     }
 
     // Precompile the pattern.
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 7833955951..c09f21e63c 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
@@ -20,7 +20,6 @@
 package org.apache.druid.query.expression;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.granularity.PeriodGranularity;
 import org.apache.druid.math.expr.Expr;
@@ -47,9 +46,7 @@ public class TimestampCeilExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() < 2 || args.size() > 4) {
-      throw new IAE("Function[%s] must have 2 to 4 arguments", name());
-    }
+    validationHelperCheckArgumentRange(args, 2, 4);
 
     if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
       return new TimestampCeilExpr(args);
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 0410fedf43..ca8b90f367 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
@@ -19,8 +19,6 @@
 
 package org.apache.druid.query.expression;
 
-import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -69,16 +67,14 @@ public class TimestampExtractExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() < 2 || args.size() > 3) {
-      throw new IAE("Function[%s] must have 2 to 3 arguments", name());
-    }
+    validationHelperCheckArgumentRange(args, 2, 3);
 
     if (!args.get(1).isLiteral() || args.get(1).getLiteralValue() == null) {
-      throw new IAE("Function[%s] unit arg must be literal", name());
+      throw validationFailed("unit arg must be literal");
     }
 
-    if (args.size() > 2 && !args.get(2).isLiteral()) {
-      throw new IAE("Function[%s] timezone arg must be literal", name());
+    if (args.size() > 2) {
+      validationHelperCheckArgIsLiteral(args.get(2), "timezone");
     }
 
     final Expr arg = args.get(0);
@@ -153,7 +149,7 @@ public class TimestampExtractExprMacro implements ExprMacroTable.ExprMacro
             // See https://www.postgresql.org/docs/10/functions-datetime.html
             return ExprEval.of(Math.ceil((double) dateTime.year().get() / 1000));
           default:
-            throw new ISE("Unhandled unit[%s]", unit);
+            throw TimestampExtractExprMacro.this.validationFailed("unhandled unit[%s]", unit);
         }
       }
 
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 f31dba9a7b..dea3604a6d 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
@@ -19,7 +19,6 @@
 
 package org.apache.druid.query.expression;
 
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.granularity.PeriodGranularity;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -48,9 +47,7 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() < 2 || args.size() > 4) {
-      throw new IAE("Function[%s] must have 2 to 4 arguments", name());
-    }
+    validationHelperCheckArgumentRange(args, 2, 4);
 
     if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
       return new TimestampFloorExpr(args);
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 bc62e50187..34a2eb6d4f 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
@@ -19,8 +19,6 @@
 
 package org.apache.druid.query.expression;
 
-import com.google.common.base.Preconditions;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -48,16 +46,14 @@ public class TimestampFormatExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() < 1 || args.size() > 3) {
-      throw new IAE("Function[%s] must have 1 to 3 arguments", name());
-    }
+    validationHelperCheckArgumentRange(args, 1, 3);
 
     final Expr arg = args.get(0);
     final String formatString;
     final DateTimeZone timeZone;
 
     if (args.size() > 1) {
-      Preconditions.checkArgument(args.get(1).isLiteral(), "Function[%s] format arg must be a literal", name());
+      validationHelperCheckArgIsLiteral(args.get(1), "format");
       formatString = (String) args.get(1).getLiteralValue();
     } else {
       formatString = null;
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 d1a1a1f416..ca6b655ec3 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
@@ -20,7 +20,6 @@
 package org.apache.druid.query.expression;
 
 import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -50,9 +49,7 @@ public class TimestampParseExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() < 1 || args.size() > 3) {
-      throw new IAE("Function[%s] must have 1 to 3 arguments", name());
-    }
+    validationHelperCheckArgumentRange(args, 1, 3);
 
     final Expr arg = args.get(0);
     final String formatString = args.size() > 1 ? (String) args.get(1).getLiteralValue() : 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 ac86428f97..875ed28a69 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
@@ -20,7 +20,6 @@
 package org.apache.druid.query.expression;
 
 import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
 import org.apache.druid.math.expr.ExprMacroTable;
@@ -50,9 +49,7 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() < 3 || args.size() > 4) {
-      throw new IAE("Function[%s] must have 3 to 4 arguments", name());
-    }
+    validationHelperCheckArgumentRange(args, 3, 4);
 
     if (args.stream().skip(1).allMatch(Expr::isLiteral)) {
       return new TimestampShiftExpr(args);
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 33b5cf2a8a..06b01fd153 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
@@ -21,7 +21,6 @@ package org.apache.druid.query.expression;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.StringUtils;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -90,9 +89,7 @@ public abstract class TrimExprMacro implements ExprMacroTable.ExprMacro
   @Override
   public Expr apply(final List<Expr> args)
   {
-    if (args.size() < 1 || args.size() > 2) {
-      throw new IAE("Function[%s] must have 1 or 2 arguments", name());
-    }
+    validationHelperCheckAnyOfArgumentCount(args, 1, 2);
 
     final Function<Expr.Shuttle, Expr> visitFn = shuttle -> shuttle.visit(apply(shuttle.visitAll(args)));
 
diff --git a/processing/src/main/java/org/apache/druid/segment/transform/ExpressionTransform.java b/processing/src/main/java/org/apache/druid/segment/transform/ExpressionTransform.java
index cf1ac50efc..98c6579790 100644
--- a/processing/src/main/java/org/apache/druid/segment/transform/ExpressionTransform.java
+++ b/processing/src/main/java/org/apache/druid/segment/transform/ExpressionTransform.java
@@ -25,6 +25,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Suppliers;
 import org.apache.druid.data.input.Row;
+import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.NonnullPair;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
@@ -78,7 +79,7 @@ public class ExpressionTransform implements Transform
   @Override
   public RowFunction getRowFunction()
   {
-    return new ExpressionRowFunction(parsedExpression.get());
+    return new ExpressionRowFunction(name, parsedExpression.get());
   }
 
   @Override
@@ -89,25 +90,32 @@ public class ExpressionTransform implements Transform
 
   static class ExpressionRowFunction implements RowFunction
   {
+    private final String name;
     private final Expr expr;
 
-    ExpressionRowFunction(final Expr expr)
+    ExpressionRowFunction(final String name, final Expr expr)
     {
+      this.name = name;
       this.expr = expr;
     }
 
     @Override
     public Object eval(final Row row)
     {
-      // this will need adjusted if we want to allow expression transforms to produce true arrays. Currently, calling
-      // this method will coerce any expression output into:
-      //    - the expression value if the value is not an array
-      //    - the single array element if the value is an array with 1 element
-      //    - a list with all of the array elements if the value is an array with more than 1 element
-      // and so is tuned towards multi-value strings
-      return ExpressionSelectors.coerceEvalToObjectOrList(
-          expr.eval(InputBindings.forFunction(name -> getValueFromRow(row, name)))
-      );
+      try {
+        // this will need adjusted if we want to allow expression transforms to produce true arrays. Currently, calling
+        // this method will coerce any expression output into:
+        //    - the expression value if the value is not an array
+        //    - the single array element if the value is an array with 1 element
+        //    - a list with all of the array elements if the value is an array with more than 1 element
+        // and so is tuned towards multi-value strings
+        return ExpressionSelectors.coerceEvalToObjectOrList(
+            expr.eval(InputBindings.forFunction(name -> getValueFromRow(row, name)))
+        );
+      }
+      catch (Throwable t) {
+        throw new ISE(t, "Could not transform value for %s reason: %s", name, t.getMessage());
+      }
     }
   }
 
diff --git a/processing/src/test/java/org/apache/druid/query/expression/CaseInsensitiveExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/CaseInsensitiveExprMacroTest.java
index ebc92b8666..db64a0727b 100644
--- a/processing/src/test/java/org/apache/druid/query/expression/CaseInsensitiveExprMacroTest.java
+++ b/processing/src/test/java/org/apache/druid/query/expression/CaseInsensitiveExprMacroTest.java
@@ -36,14 +36,14 @@ public class CaseInsensitiveExprMacroTest extends MacroTestBase
   @Test
   public void testErrorZeroArguments()
   {
-    expectException(IllegalArgumentException.class, "Function[icontains_string] must have 2 arguments");
+    expectException(IllegalArgumentException.class, "Function[icontains_string] requires 2 arguments");
     eval("icontains_string()", InputBindings.withMap(ImmutableMap.of()));
   }
 
   @Test
   public void testErrorThreeArguments()
   {
-    expectException(IllegalArgumentException.class, "Function[icontains_string] must have 2 arguments");
+    expectException(IllegalArgumentException.class, "Function[icontains_string] requires 2 arguments");
     eval("icontains_string('a', 'b', 'c')", InputBindings.withMap(ImmutableMap.of()));
   }
 
diff --git a/processing/src/test/java/org/apache/druid/query/expression/ContainsExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/ContainsExprMacroTest.java
index 4a29670540..aad2b1e3d9 100644
--- a/processing/src/test/java/org/apache/druid/query/expression/ContainsExprMacroTest.java
+++ b/processing/src/test/java/org/apache/druid/query/expression/ContainsExprMacroTest.java
@@ -36,14 +36,14 @@ public class ContainsExprMacroTest extends MacroTestBase
   @Test
   public void testErrorZeroArguments()
   {
-    expectException(IllegalArgumentException.class, "Function[contains_string] must have 2 arguments");
+    expectException(IllegalArgumentException.class, "Function[contains_string] requires 2 arguments");
     eval("contains_string()", InputBindings.withMap(ImmutableMap.of()));
   }
 
   @Test
   public void testErrorThreeArguments()
   {
-    expectException(IllegalArgumentException.class, "Function[contains_string] must have 2 arguments");
+    expectException(IllegalArgumentException.class, "Function[contains_string] requires 2 arguments");
     eval("contains_string('a', 'b', 'c')", InputBindings.withMap(ImmutableMap.of()));
   }
 
diff --git a/processing/src/test/java/org/apache/druid/query/expression/HyperUniqueExpressionsTest.java b/processing/src/test/java/org/apache/druid/query/expression/HyperUniqueExpressionsTest.java
index a20099bc52..7a3533a94b 100644
--- a/processing/src/test/java/org/apache/druid/query/expression/HyperUniqueExpressionsTest.java
+++ b/processing/src/test/java/org/apache/druid/query/expression/HyperUniqueExpressionsTest.java
@@ -199,7 +199,7 @@ public class HyperUniqueExpressionsTest extends InitializedNullHandlingTest
   public void testCreateWrongArgsCount()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[hyper_unique] must have no arguments");
+    expectedException.expectMessage("Function[hyper_unique] does not accept arguments");
     Parser.parse("hyper_unique(100)", MACRO_TABLE);
   }
 
@@ -207,7 +207,7 @@ public class HyperUniqueExpressionsTest extends InitializedNullHandlingTest
   public void testAddWrongArgsCount()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[hyper_unique_add] must have 2 arguments");
+    expectedException.expectMessage("Function[hyper_unique_add] requires 2 arguments");
     Parser.parse("hyper_unique_add(100, hyper_unique(), 100)", MACRO_TABLE);
   }
 
@@ -215,7 +215,7 @@ public class HyperUniqueExpressionsTest extends InitializedNullHandlingTest
   public void testAddWrongArgType()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[hyper_unique_add] must take a hyper-log-log collector as the second argument");
+    expectedException.expectMessage("Function[hyper_unique_add] requires a hyper-log-log collector as the second argument");
     Expr expr = Parser.parse("hyper_unique_add(long, string)", MACRO_TABLE);
     expr.eval(inputBindings);
   }
@@ -224,7 +224,7 @@ public class HyperUniqueExpressionsTest extends InitializedNullHandlingTest
   public void testEstimateWrongArgsCount()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[hyper_unique_estimate] must have 1 argument");
+    expectedException.expectMessage("Function[hyper_unique_estimate] requires 1 argument");
     Parser.parse("hyper_unique_estimate(hyper_unique(), 100)", MACRO_TABLE);
   }
 
@@ -232,7 +232,7 @@ public class HyperUniqueExpressionsTest extends InitializedNullHandlingTest
   public void testEstimateWrongArgTypes()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[hyper_unique_estimate] must take a hyper-log-log collector as input");
+    expectedException.expectMessage("Function[hyper_unique_estimate] requires a hyper-log-log collector as input");
     Expr expr = Parser.parse("hyper_unique_estimate(100)", MACRO_TABLE);
     expr.eval(inputBindings);
   }
@@ -241,7 +241,7 @@ public class HyperUniqueExpressionsTest extends InitializedNullHandlingTest
   public void testRoundEstimateWrongArgsCount()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[hyper_unique_round_estimate] must have 1 argument");
+    expectedException.expectMessage("Function[hyper_unique_round_estimate] requires 1 argument");
     Parser.parse("hyper_unique_round_estimate(hyper_unique(), 100)", MACRO_TABLE);
   }
 
@@ -249,7 +249,7 @@ public class HyperUniqueExpressionsTest extends InitializedNullHandlingTest
   public void testRoundEstimateWrongArgTypes()
   {
     expectedException.expect(IAE.class);
-    expectedException.expectMessage("Function[hyper_unique_round_estimate] must take a hyper-log-log collector as input");
+    expectedException.expectMessage("Function[hyper_unique_round_estimate] requires a hyper-log-log collector as input");
     Expr expr = Parser.parse("hyper_unique_round_estimate(string)", MACRO_TABLE);
     expr.eval(inputBindings);
   }
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 54bb71b298..6e47e9b630 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
@@ -22,6 +22,7 @@ package org.apache.druid.query.expression;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
 import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.math.expr.ExpressionValidationException;
 import org.apache.druid.math.expr.InputBindings;
 import org.junit.Assert;
 import org.junit.Test;
@@ -50,7 +51,7 @@ public class IPv4AddressMatchExprMacroTest extends MacroTestBase
   @Test
   public void testTooFewArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 2 arguments");
+    expectException(ExpressionValidationException.class, "requires 2 arguments");
 
     apply(Collections.emptyList());
   }
@@ -58,7 +59,7 @@ public class IPv4AddressMatchExprMacroTest extends MacroTestBase
   @Test
   public void testTooManyArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 2 arguments");
+    expectException(ExpressionValidationException.class, "requires 2 arguments");
 
     apply(Arrays.asList(IPV4, SUBNET_192_168, NOT_LITERAL));
   }
@@ -66,7 +67,7 @@ public class IPv4AddressMatchExprMacroTest extends MacroTestBase
   @Test
   public void testSubnetArgNotLiteral()
   {
-    expectException(IllegalArgumentException.class, "subnet arg must be a literal");
+    expectException(ExpressionValidationException.class, "subnet argument must be a literal");
 
     apply(Arrays.asList(IPV4, NOT_LITERAL));
   }
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 0bfaa5df85..19464b39c6 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
@@ -43,7 +43,7 @@ public class IPv4AddressParseExprMacroTest extends MacroTestBase
   @Test
   public void testTooFewArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 1 argument");
+    expectException(IllegalArgumentException.class, "requires 1 argument");
 
     apply(Collections.emptyList());
   }
@@ -51,7 +51,7 @@ public class IPv4AddressParseExprMacroTest extends MacroTestBase
   @Test
   public void testTooManyArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 1 argument");
+    expectException(IllegalArgumentException.class, "requires 1 argument");
 
     apply(Arrays.asList(VALID, VALID));
   }
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 fc6d893060..389acf5851 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
@@ -43,7 +43,7 @@ public class IPv4AddressStringifyExprMacroTest extends MacroTestBase
   @Test
   public void testTooFewArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 1 argument");
+    expectException(IllegalArgumentException.class, "requires 1 argument");
 
     apply(Collections.emptyList());
   }
@@ -51,7 +51,7 @@ public class IPv4AddressStringifyExprMacroTest extends MacroTestBase
   @Test
   public void testTooManyArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 1 argument");
+    expectException(IllegalArgumentException.class, "requires 1 argument");
 
     apply(Arrays.asList(VALID, VALID));
   }
diff --git a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java
index 82415f18c6..66c7257be9 100644
--- a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java
+++ b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java
@@ -26,11 +26,11 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.jackson.DefaultObjectMapper;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.Pair;
 import org.apache.druid.math.expr.Expr;
 import org.apache.druid.math.expr.ExprEval;
 import org.apache.druid.math.expr.ExprMacroTable;
+import org.apache.druid.math.expr.ExpressionProcessingException;
 import org.apache.druid.math.expr.ExpressionType;
 import org.apache.druid.math.expr.InputBindings;
 import org.apache.druid.math.expr.Parser;
@@ -264,13 +264,13 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest
     Assert.assertEquals(null, eval.value());
     Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
 
-    Assert.assertThrows(IAE.class, () -> Parser.parse("parse_json('{')", MACRO_TABLE));
+    Assert.assertThrows(ExpressionProcessingException.class, () -> Parser.parse("parse_json('{')", MACRO_TABLE));
     expr = Parser.parse("try_parse_json('{')", MACRO_TABLE);
     eval = expr.eval(inputBindings);
     Assert.assertEquals(null, eval.value());
     Assert.assertEquals(NestedDataExpressions.TYPE, eval.type());
 
-    Assert.assertThrows(IAE.class, () -> Parser.parse("parse_json('hello world')", MACRO_TABLE));
+    Assert.assertThrows(ExpressionProcessingException.class, () -> Parser.parse("parse_json('hello world')", MACRO_TABLE));
     expr = Parser.parse("try_parse_json('hello world')", MACRO_TABLE);
     eval = expr.eval(inputBindings);
     Assert.assertEquals(null, eval.value());
diff --git a/processing/src/test/java/org/apache/druid/query/expression/RegexpExtractExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/RegexpExtractExprMacroTest.java
index bf432a2fb9..610dfde77d 100644
--- a/processing/src/test/java/org/apache/druid/query/expression/RegexpExtractExprMacroTest.java
+++ b/processing/src/test/java/org/apache/druid/query/expression/RegexpExtractExprMacroTest.java
@@ -36,14 +36,14 @@ public class RegexpExtractExprMacroTest extends MacroTestBase
   @Test
   public void testErrorZeroArguments()
   {
-    expectException(IllegalArgumentException.class, "Function[regexp_extract] must have 2 to 3 arguments");
+    expectException(IllegalArgumentException.class, "Function[regexp_extract] requires 2 or 3 arguments");
     eval("regexp_extract()", InputBindings.withMap(ImmutableMap.of()));
   }
 
   @Test
   public void testErrorFourArguments()
   {
-    expectException(IllegalArgumentException.class, "Function[regexp_extract] must have 2 to 3 arguments");
+    expectException(IllegalArgumentException.class, "Function[regexp_extract] requires 2 or 3 arguments");
     eval("regexp_extract('a', 'b', 'c', 'd')", InputBindings.withMap(ImmutableMap.of()));
   }
 
diff --git a/processing/src/test/java/org/apache/druid/query/expression/RegexpLikeExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/RegexpLikeExprMacroTest.java
index 77eea92c2a..bab486276d 100644
--- a/processing/src/test/java/org/apache/druid/query/expression/RegexpLikeExprMacroTest.java
+++ b/processing/src/test/java/org/apache/druid/query/expression/RegexpLikeExprMacroTest.java
@@ -36,14 +36,14 @@ public class RegexpLikeExprMacroTest extends MacroTestBase
   @Test
   public void testErrorZeroArguments()
   {
-    expectException(IllegalArgumentException.class, "Function[regexp_like] must have 2 arguments");
+    expectException(IllegalArgumentException.class, "Function[regexp_like] requires 2 arguments");
     eval("regexp_like()", InputBindings.withMap(ImmutableMap.of()));
   }
 
   @Test
   public void testErrorThreeArguments()
   {
-    expectException(IllegalArgumentException.class, "Function[regexp_like] must have 2 arguments");
+    expectException(IllegalArgumentException.class, "Function[regexp_like] requires 2 arguments");
     eval("regexp_like('a', 'b', 'c')", InputBindings.withMap(ImmutableMap.of()));
   }
 
@@ -71,7 +71,7 @@ public class RegexpLikeExprMacroTest extends MacroTestBase
   public void testNullPattern()
   {
     if (NullHandling.sqlCompatible()) {
-      expectException(IllegalArgumentException.class, "Function[regexp_like] pattern must be a string literal");
+      expectException(IllegalArgumentException.class, "Function[regexp_like] pattern must be a STRING literal");
     }
 
     final ExprEval<?> result = eval("regexp_like(a, null)", InputBindings.withMap(ImmutableMap.of("a", "foo")));
@@ -95,7 +95,7 @@ public class RegexpLikeExprMacroTest extends MacroTestBase
   public void testNullPatternOnEmptyString()
   {
     if (NullHandling.sqlCompatible()) {
-      expectException(IllegalArgumentException.class, "Function[regexp_like] pattern must be a string literal");
+      expectException(IllegalArgumentException.class, "Function[regexp_like] pattern must be a STRING literal");
     }
 
     final ExprEval<?> result = eval("regexp_like(a, null)", InputBindings.withMap(ImmutableMap.of("a", "")));
@@ -119,7 +119,7 @@ public class RegexpLikeExprMacroTest extends MacroTestBase
   public void testNullPatternOnNull()
   {
     if (NullHandling.sqlCompatible()) {
-      expectException(IllegalArgumentException.class, "Function[regexp_like] pattern must be a string literal");
+      expectException(IllegalArgumentException.class, "Function[regexp_like] pattern must be a STRING literal");
     }
 
     final ExprEval<?> result = eval("regexp_like(a, null)", InputBindings.nilBindings());
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 44d94940b5..1dc87451c8 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
@@ -52,14 +52,14 @@ public class TimestampShiftMacroTest extends MacroTestBase
   @Test
   public void testZeroArguments()
   {
-    expectException(IAE.class, "Function[timestamp_shift] must have 3 to 4 arguments");
+    expectException(IAE.class, "Function[timestamp_shift] requires 3 to 4 arguments");
     apply(Collections.emptyList());
   }
 
   @Test
   public void testOneArguments()
   {
-    expectException(IAE.class, "Function[timestamp_shift] must have 3 to 4 arguments");
+    expectException(IAE.class, "Function[timestamp_shift] requires 3 to 4 arguments");
     apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr()
@@ -69,7 +69,7 @@ public class TimestampShiftMacroTest extends MacroTestBase
   @Test
   public void testTwoArguments()
   {
-    expectException(IAE.class, "Function[timestamp_shift] must have 3 to 4 arguments");
+    expectException(IAE.class, "Function[timestamp_shift] requires 3 to 4 arguments");
     apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr(),
@@ -80,7 +80,7 @@ public class TimestampShiftMacroTest extends MacroTestBase
   @Test
   public void testMoreThanFourArguments()
   {
-    expectException(IAE.class, "Function[timestamp_shift] must have 3 to 4 arguments");
+    expectException(IAE.class, "Function[timestamp_shift] requires 3 to 4 arguments");
     apply(
         ImmutableList.of(
             ExprEval.of(timestamp.getMillis()).toExpr(),
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java
index c809e8168f..87e79b1d4b 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java
@@ -33,7 +33,6 @@ import org.apache.druid.client.indexing.IndexingServiceClient;
 import org.apache.druid.client.indexing.TaskPayloadResponse;
 import org.apache.druid.indexer.TaskStatusPlus;
 import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
-import org.apache.druid.java.util.common.IAE;
 import org.apache.druid.java.util.common.ISE;
 import org.apache.druid.java.util.common.granularity.Granularity;
 import org.apache.druid.java.util.common.granularity.GranularityType;
@@ -386,7 +385,7 @@ public class CompactSegments implements CoordinatorCustomDuty
             try {
               segmentGranularityToUse = GranularityType.fromPeriod(interval.toPeriod()).getDefaultGranularity();
             }
-            catch (IAE iae) {
+            catch (IllegalArgumentException iae) {
               // This case can happen if the existing segment interval result in complicated periods.
               // Fall back to setting segmentGranularity as null
               LOG.warn("Cannot determine segmentGranularity from interval [%s]", interval);
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 7c4420edb8..3817d5e8f3 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
@@ -31,7 +31,7 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.druid.common.config.NullHandling;
 import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.IAE;
+import org.apache.druid.math.expr.ExpressionValidationException;
 import org.apache.druid.query.expression.TestExprMacroTable;
 import org.apache.druid.query.extraction.RegexDimExtractionFn;
 import org.apache.druid.query.filter.RegexDimFilter;
@@ -1099,8 +1099,8 @@ public class ExpressionsTest extends ExpressionTestBase
 
     if (!NullHandling.sqlCompatible()) {
       expectException(
-          IAE.class,
-          "The first argument to the function[round] should be integer or double type but got the type: STRING"
+          ExpressionValidationException.class,
+          "Function[round] first argument should be a LONG or DOUBLE but got STRING instead"
       );
     }
     testHelper.testExpression(
@@ -1123,8 +1123,8 @@ public class ExpressionsTest extends ExpressionTestBase
     final SqlFunction roundFunction = new RoundOperatorConversion().calciteOperator();
 
     expectException(
-        IAE.class,
-        "The second argument to the function[round] should be integer type but got the type: STRING"
+        ExpressionValidationException.class,
+        "Function[round] second argument should be a LONG but got STRING instead"
     );
     testHelper.testExpressionString(
         roundFunction,
@@ -2082,7 +2082,10 @@ public class ExpressionsTest extends ExpressionTestBase
   @Test
   public void testAbnormalReverseWithWrongType()
   {
-    expectException(IAE.class, "Function[reverse] needs a string argument");
+    expectException(
+        ExpressionValidationException.class,
+        "Function[reverse] needs a STRING argument but got LONG instead"
+    );
 
     testHelper.testExpression(
         new ReverseOperatorConversion().calciteOperator(),
@@ -2155,7 +2158,10 @@ public class ExpressionsTest extends ExpressionTestBase
   @Test
   public void testAbnormalRightWithNegativeNumber()
   {
-    expectException(IAE.class, "Function[right] needs a postive integer as second argument");
+    expectException(
+        ExpressionValidationException.class,
+        "Function[right] needs a positive integer as the second argument"
+    );
 
     testHelper.testExpressionString(
         new RightOperatorConversion().calciteOperator(),
@@ -2171,7 +2177,10 @@ public class ExpressionsTest extends ExpressionTestBase
   @Test
   public void testAbnormalRightWithWrongType()
   {
-    expectException(IAE.class, "Function[right] needs a string as first argument and an integer as second argument");
+    expectException(
+        ExpressionValidationException.class,
+        "Function[right] needs a STRING as first argument and a LONG as second argument"
+    );
 
     testHelper.testExpressionString(
         new RightOperatorConversion().calciteOperator(),
@@ -2241,7 +2250,10 @@ public class ExpressionsTest extends ExpressionTestBase
   @Test
   public void testAbnormalLeftWithNegativeNumber()
   {
-    expectException(IAE.class, "Function[left] needs a postive integer as second argument");
+    expectException(
+        ExpressionValidationException.class,
+        "Function[left] needs a postive integer as second argument"
+    );
 
     testHelper.testExpressionString(
         new LeftOperatorConversion().calciteOperator(),
@@ -2257,7 +2269,10 @@ public class ExpressionsTest extends ExpressionTestBase
   @Test
   public void testAbnormalLeftWithWrongType()
   {
-    expectException(IAE.class, "Function[left] needs a string as first argument and an integer as second argument");
+    expectException(
+        ExpressionValidationException.class,
+        "Function[left] needs a STRING as first argument and a LONG as second argument"
+    );
 
     testHelper.testExpressionString(
         new LeftOperatorConversion().calciteOperator(),
@@ -2307,7 +2322,10 @@ public class ExpressionsTest extends ExpressionTestBase
   @Test
   public void testAbnormalRepeatWithWrongType()
   {
-    expectException(IAE.class, "Function[repeat] needs a string as first argument and an integer as second argument");
+    expectException(
+        ExpressionValidationException.class,
+        "Function[repeat] needs a STRING as first argument and a LONG as second argument"
+    );
 
     testHelper.testExpressionString(
         new RepeatOperatorConversion().calciteOperator(),
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressMatchExpressionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressMatchExpressionTest.java
index 4fee9a16c9..9db4868c47 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressMatchExpressionTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressMatchExpressionTest.java
@@ -21,6 +21,7 @@ package org.apache.druid.sql.calcite.expression;
 
 import com.google.common.collect.ImmutableMap;
 import org.apache.calcite.rex.RexNode;
+import org.apache.druid.math.expr.ExpressionValidationException;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.sql.calcite.expression.builtin.IPv4AddressMatchOperatorConversion;
@@ -64,7 +65,7 @@ public class IPv4AddressMatchExpressionTest extends ExpressionTestBase
   @Test
   public void testTooFewArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 2 arguments");
+    expectException(IllegalArgumentException.class, "requires 2 arguments");
 
     testExpression(
         Collections.emptyList(),
@@ -76,7 +77,7 @@ public class IPv4AddressMatchExpressionTest extends ExpressionTestBase
   @Test
   public void testTooManyArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 2 arguments");
+    expectException(IllegalArgumentException.class, "requires 2 arguments");
 
     String address = IPV4;
     String subnet = SUBNET_192_168;
@@ -94,7 +95,7 @@ public class IPv4AddressMatchExpressionTest extends ExpressionTestBase
   @Test
   public void testSubnetArgNotLiteral()
   {
-    expectException(IllegalArgumentException.class, "subnet arg must be a literal");
+    expectException(ExpressionValidationException.class, "subnet argument must be a literal");
 
     String address = IPV4;
     String variableName = VAR;
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressParseExpressionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressParseExpressionTest.java
index 69a4628b56..63e01cf6d1 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressParseExpressionTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressParseExpressionTest.java
@@ -22,6 +22,7 @@ package org.apache.druid.sql.calcite.expression;
 import com.google.common.collect.ImmutableMap;
 import org.apache.calcite.rex.RexNode;
 import org.apache.druid.common.config.NullHandling;
+import org.apache.druid.math.expr.ExpressionValidationException;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.sql.calcite.expression.builtin.IPv4AddressParseOperatorConversion;
@@ -57,7 +58,7 @@ public class IPv4AddressParseExpressionTest extends ExpressionTestBase
   @Test
   public void testTooFewArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 1 argument");
+    expectException(ExpressionValidationException.class, "requires 1 argument");
 
     testExpression(
         Collections.emptyList(),
@@ -69,7 +70,7 @@ public class IPv4AddressParseExpressionTest extends ExpressionTestBase
   @Test
   public void testTooManyArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 1 argument");
+    expectException(ExpressionValidationException.class, "requires 1 argument");
 
     testExpression(
         Arrays.asList(
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressStringifyExpressionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressStringifyExpressionTest.java
index a93ff6372e..f434b7dca1 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressStringifyExpressionTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/IPv4AddressStringifyExpressionTest.java
@@ -21,6 +21,7 @@ package org.apache.druid.sql.calcite.expression;
 
 import com.google.common.collect.ImmutableMap;
 import org.apache.calcite.rex.RexNode;
+import org.apache.druid.math.expr.ExpressionValidationException;
 import org.apache.druid.segment.column.ColumnType;
 import org.apache.druid.segment.column.RowSignature;
 import org.apache.druid.sql.calcite.expression.builtin.IPv4AddressStringifyOperatorConversion;
@@ -56,7 +57,7 @@ public class IPv4AddressStringifyExpressionTest extends ExpressionTestBase
   @Test
   public void testTooFewArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 1 argument");
+    expectException(ExpressionValidationException.class, "requires 1 argument");
 
     testExpression(
         Collections.emptyList(),
@@ -68,7 +69,7 @@ public class IPv4AddressStringifyExpressionTest extends ExpressionTestBase
   @Test
   public void testTooManyArgs()
   {
-    expectException(IllegalArgumentException.class, "must have 1 argument");
+    expectException(ExpressionValidationException.class, "requires 1 argument");
 
     testExpression(
         Arrays.asList(


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