You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2020/08/21 06:30:25 UTC

[calcite] 01/02: [CALCITE-2082] Do not store types or type factories inside operators

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

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

commit 39cf82b8bf94b9240fe4592706d8eeb2952712f0
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Tue Aug 18 15:46:23 2020 -0700

    [CALCITE-2082] Do not store types or type factories inside operators
    
    Rationale: A SqlOperator has a lifetime that spans several
    statements; but a type factory is only for one statement, and
    each type belongs to that factory. We want to share
    SqlOperator instances across statements and connections,
    therefore we need to create them before there is a type
    factory.
    
    In particular:
    * Remove (deprecate) method
      `List<RelDataType> SqlOperator.getParamTypes()`;
    * Remove `RelDataTypeFactory` argument from
      `SqlUserDefinedAggFunction` constructor, and remove its
      `typeFactory` field.
    * Add `interface SqlOperandMetadata extends
      SqlOperatorTypeChecker`, which has new methods
      `List<RelDataType>> paramTypes(RelDataTypeFactory)`
      and `List<String> paramNames()`.
    
    The `SqlOperandMetadata` interface will typically be
    implemented only for user-defined functions. Unlike SQL
    built-in functions, UDFs have a fixed set of parameters
    (although some of them may be optional), and the parameters
    have names.
    
    In `interface SqlOperandTypeChecker`, add method `boolean
    isFixedParameters()`. Will typically return true for UDFs,
    false for built-in functions. Returns false for table window
    functions (e.g. HOP), even though these have named parameters
    (which tends to make them look a bit like UDFs).
    
    Following [CALCITE-4171], change `instanceof SqlWindowTableFunction`
    to `checker.isFixedParameters()`, because it is less specific.
    
    Add `SqlKind` argument to UDF constructors; we will populate it
    using Java annotations in [CALCITE-1861].
---
 .../calcite/prepare/CalciteCatalogReader.java      | 138 ++++++++-------
 .../org/apache/calcite/sql/SqlAggFunction.java     |   2 +-
 .../org/apache/calcite/sql/SqlCallBinding.java     |  38 ++--
 .../java/org/apache/calcite/sql/SqlFunction.java   |  56 +++---
 .../apache/calcite/sql/SqlHopTableFunction.java    |  43 ++---
 .../java/org/apache/calcite/sql/SqlOperator.java   |   3 +-
 .../calcite/sql/SqlSessionTableFunction.java       |  39 +----
 .../apache/calcite/sql/SqlTumbleTableFunction.java |  49 ++----
 .../main/java/org/apache/calcite/sql/SqlUtil.java  |  74 ++++----
 .../apache/calcite/sql/SqlWindowTableFunction.java | 191 +++++++++++++--------
 .../sql/type/ExplicitOperandTypeChecker.java       |   5 +-
 .../calcite/sql/type/OperandMetadataImpl.java      |  69 ++++++++
 .../org/apache/calcite/sql/type/OperandTypes.java  |  20 +++
 .../calcite/sql/type/SqlOperandMetadata.java       |  43 +++++
 .../calcite/sql/type/SqlOperandTypeChecker.java    |   9 +
 .../sql/validate/SqlUserDefinedAggFunction.java    |  74 +++-----
 .../sql/validate/SqlUserDefinedFunction.java       |  35 +++-
 .../sql/validate/SqlUserDefinedTableFunction.java  |  22 ++-
 .../sql/validate/SqlUserDefinedTableMacro.java     |  26 ++-
 .../sql/validate/implicit/TypeCoercionImpl.java    |  12 +-
 .../apache/calcite/test/MockSqlOperatorTable.java  |  11 +-
 .../org/apache/calcite/piglet/PigRelSqlUdfs.java   |  51 ++++--
 .../calcite/piglet/PigUserDefinedFunction.java     |  30 +---
 23 files changed, 627 insertions(+), 413 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
index f77679c..b1d1d20 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -25,10 +25,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
 import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.schema.AggregateFunction;
-import org.apache.calcite.schema.Function;
-import org.apache.calcite.schema.FunctionParameter;
 import org.apache.calcite.schema.ScalarFunction;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableFunction;
@@ -37,16 +34,17 @@ import org.apache.calcite.schema.Wrapper;
 import org.apache.calcite.schema.impl.ScalarFunctionImpl;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.SqlSyntax;
 import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.type.FamilyOperandTypeChecker;
 import org.apache.calcite.sql.type.InferTypes;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
+import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
-import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ListSqlOperatorTable;
@@ -65,7 +63,6 @@ import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -74,6 +71,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
 import java.util.Objects;
+import java.util.function.Function;
 import java.util.function.Predicate;
 
 /**
@@ -137,8 +135,10 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
     return config;
   }
 
-  private Collection<Function> getFunctionsFrom(List<String> names) {
-    final List<Function> functions2 = new ArrayList<>();
+  private Collection<org.apache.calcite.schema.Function> getFunctionsFrom(
+      List<String> names) {
+    final List<org.apache.calcite.schema.Function> functions2 =
+        new ArrayList<>();
     final List<List<String>> schemaNameList = new ArrayList<>();
     if (names.size() > 1) {
       // Name qualified: ignore path. But we do look in "/catalog" and "/",
@@ -253,7 +253,7 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
       return;
     }
 
-    final Predicate<Function> predicate;
+    final Predicate<org.apache.calcite.schema.Function> predicate;
     if (category == null) {
       predicate = function -> true;
     } else if (category.isTableFunction()) {
@@ -285,61 +285,73 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
           className, "*", true);
     }
 
-    // The following is technical debt; see [CALCITE-2082] Remove
-    // RelDataTypeFactory argument from SqlUserDefinedAggFunction constructor
-    final SqlTypeFactoryImpl typeFactory =
-        new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
-
     final ListSqlOperatorTable table = new ListSqlOperatorTable();
     for (String name : schema.getFunctionNames()) {
-      for (Function function : schema.getFunctions(name, true)) {
+      schema.getFunctions(name, true).forEach(function -> {
         final SqlIdentifier id = new SqlIdentifier(name, SqlParserPos.ZERO);
-        table.add(
-            toOp(typeFactory, id, function));
-      }
+        table.add(toOp(id, function));
+      });
     }
     return table;
   }
 
-  private SqlOperator toOp(SqlIdentifier name, final Function function) {
-    return toOp(typeFactory, name, function);
-  }
-
-  /** Converts a function to a {@link org.apache.calcite.sql.SqlOperator}.
-   *
-   * <p>The {@code typeFactory} argument is technical debt; see [CALCITE-2082]
-   * Remove RelDataTypeFactory argument from SqlUserDefinedAggFunction
-   * constructor. */
-  private static SqlOperator toOp(RelDataTypeFactory typeFactory,
-      SqlIdentifier name, final Function function) {
-    List<RelDataType> argTypes = new ArrayList<>();
-    List<SqlTypeFamily> typeFamilies = new ArrayList<>();
-    for (FunctionParameter o : function.getParameters()) {
-      final RelDataType type = o.getType(typeFactory);
-      argTypes.add(type);
-      typeFamilies.add(
-          Util.first(type.getSqlTypeName().getFamily(), SqlTypeFamily.ANY));
-    }
-    final FamilyOperandTypeChecker typeChecker =
-        OperandTypes.family(typeFamilies, i ->
-            function.getParameters().get(i).isOptional());
-    final List<RelDataType> paramTypes = toSql(typeFactory, argTypes);
+  /** Converts a function to a {@link org.apache.calcite.sql.SqlOperator}. */
+  private static SqlOperator toOp(SqlIdentifier name,
+      final org.apache.calcite.schema.Function function) {
+    final Function<RelDataTypeFactory, List<RelDataType>> argTypesFactory =
+        typeFactory -> function.getParameters()
+            .stream()
+            .map(o -> o.getType(typeFactory))
+            .collect(Util.toImmutableList());
+    final Function<RelDataTypeFactory, List<SqlTypeFamily>> typeFamiliesFactory =
+        typeFactory -> argTypesFactory.apply(typeFactory)
+            .stream()
+            .map(type ->
+                Util.first(type.getSqlTypeName().getFamily(),
+                    SqlTypeFamily.ANY))
+            .collect(Util.toImmutableList());
+    final Function<RelDataTypeFactory, List<RelDataType>> paramTypesFactory =
+        typeFactory ->
+            argTypesFactory.apply(typeFactory)
+                .stream()
+                .map(type -> toSql(typeFactory, type))
+                .collect(Util.toImmutableList());
+
+    // Use a short-lived type factory to populate "typeFamilies" and "argTypes".
+    // SqlOperandMetadata.paramTypes will use the real type factory, during
+    // validation.
+    final RelDataTypeFactory dummyTypeFactory = new JavaTypeFactoryImpl();
+    final List<RelDataType> argTypes = argTypesFactory.apply(dummyTypeFactory);
+    final List<SqlTypeFamily> typeFamilies =
+        typeFamiliesFactory.apply(dummyTypeFactory);
+
+    final SqlOperandTypeInference operandTypeInference =
+        InferTypes.explicit(argTypes);
+
+    final SqlOperandMetadata operandMetadata =
+        OperandTypes.operandMetadata(typeFamilies, paramTypesFactory,
+            i -> function.getParameters().get(i).getName(),
+            i -> function.getParameters().get(i).isOptional());
+
+    final SqlKind kind = SqlKind.OTHER_FUNCTION;
     if (function instanceof ScalarFunction) {
-      return new SqlUserDefinedFunction(name, infer((ScalarFunction) function),
-          InferTypes.explicit(argTypes), typeChecker, paramTypes, function);
+      final SqlReturnTypeInference returnTypeInference =
+          infer((ScalarFunction) function);
+      return new SqlUserDefinedFunction(name, kind, returnTypeInference,
+          operandTypeInference, operandMetadata, function);
     } else if (function instanceof AggregateFunction) {
-      return new SqlUserDefinedAggFunction(name,
-          infer((AggregateFunction) function), InferTypes.explicit(argTypes),
-          typeChecker, (AggregateFunction) function, false, false,
-          Optionality.FORBIDDEN, typeFactory);
+      final SqlReturnTypeInference returnTypeInference =
+          infer((AggregateFunction) function);
+      return new SqlUserDefinedAggFunction(name, kind,
+          returnTypeInference, operandTypeInference,
+          operandMetadata, (AggregateFunction) function, false, false,
+          Optionality.FORBIDDEN);
     } else if (function instanceof TableMacro) {
-      return new SqlUserDefinedTableMacro(name, ReturnTypes.CURSOR,
-          InferTypes.explicit(argTypes), typeChecker, paramTypes,
-          (TableMacro) function);
+      return new SqlUserDefinedTableMacro(name, kind, ReturnTypes.CURSOR,
+          operandTypeInference, operandMetadata, (TableMacro) function);
     } else if (function instanceof TableFunction) {
-      return new SqlUserDefinedTableFunction(name, ReturnTypes.CURSOR,
-          InferTypes.explicit(argTypes), typeChecker, paramTypes,
-          (TableFunction) function);
+      return new SqlUserDefinedTableFunction(name, kind, ReturnTypes.CURSOR,
+          operandTypeInference, operandMetadata, (TableFunction) function);
     } else {
       throw new AssertionError("unknown function type " + function);
     }
@@ -368,11 +380,6 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
     };
   }
 
-  private static List<RelDataType> toSql(
-      final RelDataTypeFactory typeFactory, List<RelDataType> types) {
-    return Lists.transform(types, type -> toSql(typeFactory, type));
-  }
-
   private static RelDataType toSql(RelDataTypeFactory typeFactory,
       RelDataType type) {
     if (type instanceof RelDataTypeFactoryImpl.JavaType
@@ -385,7 +392,18 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
   }
 
   public List<SqlOperator> getOperatorList() {
-    return null;
+    final ImmutableList.Builder<SqlOperator> b = ImmutableList.builder();
+    for (List<String> schemaPath : schemaPaths) {
+      CalciteSchema schema =
+          SqlValidatorUtil.getSchema(rootSchema, schemaPath, nameMatcher);
+      if (schema != null) {
+        for (String name : schema.getFunctionNames()) {
+          schema.getFunctions(name, true).forEach(f ->
+              b.add(toOp(new SqlIdentifier(name, SqlParserPos.ZERO), f)));
+        }
+      }
+    }
+    return b.build();
   }
 
   public CalciteSchema getRootSchema() {
@@ -396,7 +414,7 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
     return typeFactory;
   }
 
-  public void registerRules(RelOptPlanner planner) throws Exception {
+  public void registerRules(RelOptPlanner planner) {
   }
 
   @SuppressWarnings("deprecation")
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
index 4f9aac3..9555a0b 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAggFunction.java
@@ -103,7 +103,7 @@ public abstract class SqlAggFunction extends SqlFunction implements Context {
       boolean requiresOver,
       Optionality requiresGroupOrder) {
     super(name, sqlIdentifier, kind, returnTypeInference, operandTypeInference,
-        operandTypeChecker, null, funcType);
+        operandTypeChecker, funcType);
     this.requiresOrder = requiresOrder;
     this.requiresOver = requiresOver;
     this.requiresGroupOrder = Objects.requireNonNull(requiresGroupOrder);
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
index ea6dad8..5ab0bac 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
@@ -24,6 +24,8 @@ import org.apache.calcite.runtime.Resources;
 import org.apache.calcite.sql.fun.SqlLiteralChainOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
+import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SelectScope;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
@@ -129,22 +131,24 @@ public class SqlCallBinding extends SqlOperatorBinding {
   /** Returns the operands to a call permuted into the same order as the
    * formal parameters of the function. */
   public List<SqlNode> operands() {
-    if (hasAssignment() && !(call.getOperator() instanceof SqlUnresolvedFunction)) {
+    if (hasAssignment()
+        && !(call.getOperator() instanceof SqlUnresolvedFunction)) {
       return permutedOperands(call);
     } else {
       final List<SqlNode> operandList = call.getOperandList();
-      if (call.getOperator() instanceof SqlFunction) {
-        final List<RelDataType> paramTypes =
-            ((SqlFunction) call.getOperator()).getParamTypes();
-        if (paramTypes != null && operandList.size() < paramTypes.size()) {
-          final List<SqlNode> list = Lists.newArrayList(operandList);
-          while (list.size() < paramTypes.size()) {
-            list.add(DEFAULT_CALL);
-          }
-          return list;
-        }
+      final SqlOperandTypeChecker checker =
+          call.getOperator().getOperandTypeChecker();
+      if (checker == null) {
+        return operandList;
+      }
+      final SqlOperandCountRange range = checker.getOperandCountRange();
+      final List<SqlNode> list = Lists.newArrayList(operandList);
+      while (list.size() < range.getMax()
+          && checker.isOptional(list.size())
+          && checker.isFixedParameters()) {
+        list.add(DEFAULT_CALL);
       }
-      return operandList;
+      return list;
     }
   }
 
@@ -162,10 +166,12 @@ public class SqlCallBinding extends SqlOperatorBinding {
   /** Returns the operands to a call permuted into the same order as the
    * formal parameters of the function. */
   private List<SqlNode> permutedOperands(final SqlCall call) {
-    final SqlFunction operator = (SqlFunction) call.getOperator();
-    final List<String> paramNames = operator.getParamNames();
+    final SqlOperandMetadata operandMetadata =
+        (SqlOperandMetadata) call.getOperator().getOperandTypeChecker();
+    final List<String> paramNames = operandMetadata.paramNames();
     final List<SqlNode> permuted = new ArrayList<>();
-    final SqlNameMatcher nameMatcher = validator.getCatalogReader().nameMatcher();
+    final SqlNameMatcher nameMatcher =
+        validator.getCatalogReader().nameMatcher();
     for (final String paramName : paramNames) {
       Pair<String, SqlIdentifier> args = null;
       for (int j = 0; j < call.getOperandList().size(); j++) {
@@ -188,7 +194,7 @@ public class SqlCallBinding extends SqlOperatorBinding {
                 RESOURCE.paramNotFoundInFunctionDidYouMean(args.right.getSimple(),
                     call.getOperator().getName(), args.left));
           }
-          if (!(operator instanceof SqlWindowTableFunction)) {
+          if (operandMetadata.isFixedParameters()) {
             // Not like user defined functions, we do not patch up the operands
             // with DEFAULT and then convert to nulls during sql-to-rel conversion.
             // Thus, there is no need to show the optional operands in the plan and
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
index fa6b180..e2128e6 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
@@ -18,8 +18,10 @@ package org.apache.calcite.sql;
 
 import org.apache.calcite.linq4j.function.Functions;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
@@ -28,8 +30,6 @@ import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.implicit.TypeCoercion;
 import org.apache.calcite.util.Util;
 
-import com.google.common.collect.ImmutableList;
-
 import java.util.List;
 import java.util.Objects;
 import javax.annotation.Nonnull;
@@ -48,8 +48,6 @@ public class SqlFunction extends SqlOperator {
 
   private final SqlIdentifier sqlIdentifier;
 
-  private final List<RelDataType> paramTypes;
-
   //~ Constructors -----------------------------------------------------------
 
   /**
@@ -70,9 +68,9 @@ public class SqlFunction extends SqlOperator {
       SqlOperandTypeChecker operandTypeChecker,
       SqlFunctionCategory category) {
     // We leave sqlIdentifier as null to indicate
-    // that this is a built-in.  Same for paramTypes.
+    // that this is a built-in.
     this(name, null, kind, returnTypeInference, operandTypeInference,
-        operandTypeChecker, null, category);
+        operandTypeChecker, category);
 
     assert !((category == SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR)
         && (returnTypeInference == null));
@@ -102,6 +100,20 @@ public class SqlFunction extends SqlOperator {
         paramTypes, funcType);
   }
 
+  @Deprecated // to be removed before 2.0
+  protected SqlFunction(
+      String name,
+      SqlIdentifier sqlIdentifier,
+      SqlKind kind,
+      SqlReturnTypeInference returnTypeInference,
+      SqlOperandTypeInference operandTypeInference,
+      SqlOperandTypeChecker operandTypeChecker,
+      List<RelDataType> paramTypes,
+      SqlFunctionCategory category) {
+    this(name, sqlIdentifier, kind, returnTypeInference, operandTypeInference,
+        operandTypeChecker, category);
+  }
+
   /**
    * Internal constructor.
    */
@@ -112,15 +124,12 @@ public class SqlFunction extends SqlOperator {
       SqlReturnTypeInference returnTypeInference,
       SqlOperandTypeInference operandTypeInference,
       SqlOperandTypeChecker operandTypeChecker,
-      List<RelDataType> paramTypes,
       SqlFunctionCategory category) {
     super(name, kind, 100, 100, returnTypeInference, operandTypeInference,
         operandTypeChecker);
 
     this.sqlIdentifier = sqlIdentifier;
     this.category = Objects.requireNonNull(category);
-    this.paramTypes =
-        paramTypes == null ? null : ImmutableList.copyOf(paramTypes);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -144,20 +153,18 @@ public class SqlFunction extends SqlOperator {
     return super.getNameAsId();
   }
 
-  /**
-   * Return array of parameter types, or null for built-in function.
-   */
+  /** Use {@link SqlOperandMetadata#paramTypes(RelDataTypeFactory)} on the
+   * result of {@link #getOperandTypeChecker()}. */
+  @Deprecated // to be removed before 2.0
   public List<RelDataType> getParamTypes() {
-    return paramTypes;
+    return null;
   }
 
-  /**
-   * Returns a list of parameter names.
-   *
-   * <p>The default implementation returns {@code [arg0, arg1, ..., argN]}.
-   */
+  /** Use {@link SqlOperandMetadata#paramNames()} on the result of
+   * {@link #getOperandTypeChecker()}. */
+  @Deprecated // to be removed before 2.0
   public List<String> getParamNames() {
-    return Functions.generate(paramTypes.size(), i -> "arg" + i);
+    return Functions.generate(getParamTypes().size(), i -> "arg" + i);
   }
 
   public void unparse(
@@ -239,10 +246,9 @@ public class SqlFunction extends SqlOperator {
 
     SqlFunction function =
         (SqlFunction) SqlUtil.lookupRoutine(validator.getOperatorTable(),
-            getNameAsId(), argTypes, argNames, getFunctionType(),
-            SqlSyntax.FUNCTION, getKind(),
-            validator.getCatalogReader().nameMatcher(),
-            false);
+            validator.getTypeFactory(), getNameAsId(), argTypes, argNames,
+            getFunctionType(), SqlSyntax.FUNCTION, getKind(),
+            validator.getCatalogReader().nameMatcher(), false);
     try {
       // if we have a match on function name and parameter count, but
       // couldn't find a function with  a COLUMN_LIST type, retry, but
@@ -280,7 +286,9 @@ public class SqlFunction extends SqlOperator {
         if (validator.config().typeCoercionEnabled()) {
           // try again if implicit type coercion is allowed.
           function = (SqlFunction)
-              SqlUtil.lookupRoutine(validator.getOperatorTable(), getNameAsId(),
+              SqlUtil.lookupRoutine(validator.getOperatorTable(),
+                  validator.getTypeFactory(),
+                  getNameAsId(),
                   argTypes, argNames, getFunctionType(), SqlSyntax.FUNCTION,
                   getKind(), validator.getCatalogReader().nameMatcher(), true);
           // try to coerce the function arguments to the declared sql type name.
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlHopTableFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlHopTableFunction.java
index 816140b..d08dd2e 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlHopTableFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlHopTableFunction.java
@@ -16,13 +16,8 @@
  */
 package org.apache.calcite.sql;
 
-import org.apache.calcite.sql.type.SqlOperandCountRanges;
-import org.apache.calcite.sql.type.SqlOperandTypeChecker;
-
 import com.google.common.collect.ImmutableList;
 
-import java.util.List;
-
 /**
  * SqlHopTableFunction implements an operator for hopping.
  *
@@ -37,47 +32,31 @@ import java.util.List;
  */
 public class SqlHopTableFunction extends SqlWindowTableFunction {
   public SqlHopTableFunction() {
-    super(SqlKind.HOP.name(), OperandTypeCheckerImpl.INSTANCE);
-  }
-
-  @Override public List<String> getParamNames() {
-    return ImmutableList.of(PARAM_DATA, PARAM_TIMECOL, PARAM_SLIDE, PARAM_SIZE, PARAM_OFFSET);
+    super(SqlKind.HOP.name(), new OperandMetadataImpl());
   }
 
-  // -------------------------------------------------------------------------
-  //  Inner Class
-  // -------------------------------------------------------------------------
-
   /** Operand type checker for HOP. */
-  private static class OperandTypeCheckerImpl implements SqlOperandTypeChecker {
-    static final OperandTypeCheckerImpl INSTANCE = new OperandTypeCheckerImpl();
+  private static class OperandMetadataImpl extends AbstractOperandMetadata {
+    OperandMetadataImpl() {
+      super(
+          ImmutableList.of(PARAM_DATA, PARAM_TIMECOL, PARAM_SLIDE,
+              PARAM_SIZE, PARAM_OFFSET), 4);
+    }
 
-    @Override public boolean checkOperandTypes(
-        SqlCallBinding callBinding, boolean throwOnFailure) {
-      if (!validateTableWithFollowingDescriptors(callBinding, 1)) {
+    @Override public boolean checkOperandTypes(SqlCallBinding callBinding,
+        boolean throwOnFailure) {
+      if (!checkTableAndDescriptorOperands(callBinding, 1)) {
         return throwValidationSignatureErrorOrReturnFalse(callBinding, throwOnFailure);
       }
-      if (!validateTailingIntervals(callBinding, 2)) {
+      if (!checkIntervalOperands(callBinding, 2)) {
         return throwValidationSignatureErrorOrReturnFalse(callBinding, throwOnFailure);
       }
       return true;
     }
 
-    @Override public SqlOperandCountRange getOperandCountRange() {
-      return SqlOperandCountRanges.between(4, 5);
-    }
-
     @Override public String getAllowedSignatures(SqlOperator op, String opName) {
       return opName + "(TABLE table_name, DESCRIPTOR(timecol), "
           + "datetime interval, datetime interval[, datetime interval])";
     }
-
-    @Override public Consistency getConsistency() {
-      return Consistency.NONE;
-    }
-
-    @Override public boolean isOptional(int i) {
-      return i == 4;
-    }
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
index e1c2c2c..310b325 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
@@ -522,7 +522,8 @@ public abstract class SqlOperator {
     // Always disable type coercion for builtin operator operands,
     // they are handled by the TypeCoercion specifically.
     final SqlOperator sqlOperator =
-        SqlUtil.lookupRoutine(validator.getOperatorTable(), getNameAsId(),
+        SqlUtil.lookupRoutine(validator.getOperatorTable(),
+            validator.getTypeFactory(), getNameAsId(),
             argTypes, null, null, getSyntax(), getKind(),
             validator.getCatalogReader().nameMatcher(), false);
 
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSessionTableFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlSessionTableFunction.java
index 61a2ba6..f82560f 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSessionTableFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSessionTableFunction.java
@@ -17,15 +17,11 @@
 package org.apache.calcite.sql;
 
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.sql.type.SqlOperandCountRanges;
-import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlValidator;
 
 import com.google.common.collect.ImmutableList;
 
-import java.util.List;
-
 /**
  * SqlSessionTableFunction implements an operator for per-key sessionization. It allows
  * four parameters:
@@ -39,25 +35,20 @@ import java.util.List;
  */
 public class SqlSessionTableFunction extends SqlWindowTableFunction {
   public SqlSessionTableFunction() {
-    super(SqlKind.SESSION.name(), OperandTypeCheckerImpl.INSTANCE);
-  }
-
-  @Override public List<String> getParamNames() {
-    return ImmutableList.of(PARAM_DATA, PARAM_TIMECOL, PARAM_KEY, PARAM_SIZE);
+    super(SqlKind.SESSION.name(), new OperandMetadataImpl());
   }
 
-  // -------------------------------------------------------------------------
-  //  Inner Class
-  // -------------------------------------------------------------------------
-
   /** Operand type checker for SESSION. */
-  private static class OperandTypeCheckerImpl implements SqlOperandTypeChecker {
-    static final OperandTypeCheckerImpl INSTANCE = new OperandTypeCheckerImpl();
+  private static class OperandMetadataImpl extends AbstractOperandMetadata {
+    OperandMetadataImpl() {
+      super(ImmutableList.of(PARAM_DATA, PARAM_TIMECOL, PARAM_KEY, PARAM_SIZE),
+          4);
+    }
 
-    @Override public boolean checkOperandTypes(
-        SqlCallBinding callBinding, boolean throwOnFailure) {
+    @Override public boolean checkOperandTypes(SqlCallBinding callBinding,
+        boolean throwOnFailure) {
       final SqlValidator validator = callBinding.getValidator();
-      if (!validateTableWithFollowingDescriptors(callBinding, 2)) {
+      if (!checkTableAndDescriptorOperands(callBinding, 2)) {
         return throwValidationSignatureErrorOrReturnFalse(callBinding, throwOnFailure);
       }
       final RelDataType type3 = validator.getValidatedNodeType(callBinding.operand(3));
@@ -67,21 +58,9 @@ public class SqlSessionTableFunction extends SqlWindowTableFunction {
       return true;
     }
 
-    @Override public SqlOperandCountRange getOperandCountRange() {
-      return SqlOperandCountRanges.of(4);
-    }
-
     @Override public String getAllowedSignatures(SqlOperator op, String opName) {
       return opName + "(TABLE table_name, DESCRIPTOR(timecol), "
           + "DESCRIPTOR(key), datetime interval)";
     }
-
-    @Override public Consistency getConsistency() {
-      return Consistency.NONE;
-    }
-
-    @Override public boolean isOptional(int i) {
-      return false;
-    }
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlTumbleTableFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlTumbleTableFunction.java
index f32f13d..90d2193 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTumbleTableFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTumbleTableFunction.java
@@ -16,13 +16,8 @@
  */
 package org.apache.calcite.sql;
 
-import org.apache.calcite.sql.type.SqlOperandCountRanges;
-import org.apache.calcite.sql.type.SqlOperandTypeChecker;
-
 import com.google.common.collect.ImmutableList;
 
-import java.util.List;
-
 /**
  * SqlTumbleTableFunction implements an operator for tumbling.
  *
@@ -36,53 +31,33 @@ import java.util.List;
  */
 public class SqlTumbleTableFunction extends SqlWindowTableFunction {
   public SqlTumbleTableFunction() {
-    super(SqlKind.TUMBLE.name(), OperandTypeCheckerImpl.INSTANCE);
-  }
-
-  @Override public SqlOperandCountRange getOperandCountRange() {
-    return SqlOperandCountRanges.between(3, 4);
+    super(SqlKind.TUMBLE.name(), new OperandMetadataImpl());
   }
 
-  @Override public List<String> getParamNames() {
-    return ImmutableList.of(PARAM_DATA, PARAM_TIMECOL, PARAM_SIZE, PARAM_OFFSET);
-  }
-
-  // -------------------------------------------------------------------------
-  //  Inner Class
-  // -------------------------------------------------------------------------
-
-  /** Operand type checker for SESSION. */
-  private static class OperandTypeCheckerImpl implements SqlOperandTypeChecker {
-    static final OperandTypeCheckerImpl INSTANCE = new OperandTypeCheckerImpl();
+  /** Operand type checker for TUMBLE. */
+  private static class OperandMetadataImpl extends AbstractOperandMetadata {
+    OperandMetadataImpl() {
+      super(
+          ImmutableList.of(PARAM_DATA, PARAM_TIMECOL, PARAM_SIZE, PARAM_OFFSET),
+          3);
+    }
 
-    @Override public boolean checkOperandTypes(
-        SqlCallBinding callBinding, boolean throwOnFailure) {
+    @Override public boolean checkOperandTypes(SqlCallBinding callBinding,
+        boolean throwOnFailure) {
       // There should only be three operands, and number of operands are checked before
       // this call.
-      if (!validateTableWithFollowingDescriptors(callBinding, 1)) {
+      if (!checkTableAndDescriptorOperands(callBinding, 1)) {
         return throwValidationSignatureErrorOrReturnFalse(callBinding, throwOnFailure);
       }
-      if (!validateTailingIntervals(callBinding, 2)) {
+      if (!checkIntervalOperands(callBinding, 2)) {
         return throwValidationSignatureErrorOrReturnFalse(callBinding, throwOnFailure);
       }
       return true;
     }
 
-    @Override public SqlOperandCountRange getOperandCountRange() {
-      return SqlOperandCountRanges.of(4);
-    }
-
     @Override public String getAllowedSignatures(SqlOperator op, String opName) {
       return opName + "(TABLE table_name, DESCRIPTOR(col1, col2 ...), datetime interval"
           + "[, datetime interval])";
     }
-
-    @Override public Consistency getConsistency() {
-      return Consistency.NONE;
-    }
-
-    @Override public boolean isOptional(int i) {
-      return i == 3;
-    }
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
index 9bfdd92..13144d4 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -31,6 +31,7 @@ import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Resources;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
@@ -422,6 +423,7 @@ public abstract class SqlUtil {
    * types.
    *
    * @param opTab         operator table to search
+   * @param typeFactory   Type factory
    * @param funcName      name of function being invoked
    * @param argTypes      argument types
    * @param argNames      argument names, or null if call by position
@@ -435,6 +437,7 @@ public abstract class SqlUtil {
    * @see Glossary#SQL99 SQL:1999 Part 2 Section 10.4
    */
   public static SqlOperator lookupRoutine(SqlOperatorTable opTab,
+      RelDataTypeFactory typeFactory,
       SqlIdentifier funcName, List<RelDataType> argTypes,
       List<String> argNames, SqlFunctionCategory category,
       SqlSyntax syntax, SqlKind sqlKind, SqlNameMatcher nameMatcher,
@@ -442,6 +445,7 @@ public abstract class SqlUtil {
     Iterator<SqlOperator> list =
         lookupSubjectRoutines(
             opTab,
+            typeFactory,
             funcName,
             argTypes,
             argNames,
@@ -467,6 +471,7 @@ public abstract class SqlUtil {
    * Looks up all subject routines matching the given name and argument types.
    *
    * @param opTab       operator table to search
+   * @param typeFactory Type factory
    * @param funcName    name of function being invoked
    * @param argTypes    argument types
    * @param argNames    argument names, or null if call by position
@@ -480,14 +485,10 @@ public abstract class SqlUtil {
    * @see Glossary#SQL99 SQL:1999 Part 2 Section 10.4
    */
   public static Iterator<SqlOperator> lookupSubjectRoutines(
-      SqlOperatorTable opTab,
-      SqlIdentifier funcName,
-      List<RelDataType> argTypes,
-      List<String> argNames,
-      SqlSyntax sqlSyntax,
-      SqlKind sqlKind,
-      SqlFunctionCategory category,
-      SqlNameMatcher nameMatcher,
+      SqlOperatorTable opTab, RelDataTypeFactory typeFactory,
+      SqlIdentifier funcName, List<RelDataType> argTypes, List<String> argNames,
+      SqlSyntax sqlSyntax, SqlKind sqlKind,
+      SqlFunctionCategory category, SqlNameMatcher nameMatcher,
       boolean coerce) {
     // start with all routines matching by name
     Iterator<SqlOperator> routines =
@@ -507,7 +508,8 @@ public abstract class SqlUtil {
     // second pass:  eliminate routines which don't accept the given
     // argument types and parameter names if specified
     routines =
-        filterRoutinesByParameterTypeAndName(sqlSyntax, routines, argTypes, argNames, coerce);
+        filterRoutinesByParameterTypeAndName(typeFactory, sqlSyntax, routines,
+            argTypes, argNames, coerce);
 
     // see if we can stop now; this is necessary for the case
     // of builtin functions where we don't have param type info,
@@ -521,7 +523,7 @@ public abstract class SqlUtil {
     // third pass:  for each parameter from left to right, eliminate
     // all routines except those with the best precedence match for
     // the given arguments
-    routines = filterRoutinesByTypePrecedence(sqlSyntax, routines, argTypes, argNames);
+    routines = filterRoutinesByTypePrecedence(sqlSyntax, typeFactory, routines, argTypes, argNames);
 
     // fourth pass: eliminate routines which do not have the same
     // SqlKind as requested
@@ -591,11 +593,9 @@ public abstract class SqlUtil {
    * @see Glossary#SQL99 SQL:1999 Part 2 Section 10.4 Syntax Rule 6.b.iii.2.B
    */
   private static Iterator<SqlOperator> filterRoutinesByParameterTypeAndName(
-      SqlSyntax syntax,
-      final Iterator<SqlOperator> routines,
-      final List<RelDataType> argTypes,
-      final List<String> argNames,
-      final boolean coerce) {
+      RelDataTypeFactory typeFactory, SqlSyntax syntax,
+      final Iterator<SqlOperator> routines, final List<RelDataType> argTypes,
+      final List<String> argNames, final boolean coerce) {
     if (syntax != SqlSyntax.FUNCTION) {
       return routines;
     }
@@ -604,15 +604,20 @@ public abstract class SqlUtil {
     return (Iterator) Iterators.filter(
         Iterators.filter(routines, SqlFunction.class),
         function -> {
-          List<RelDataType> paramTypes = function.getParamTypes();
-          if (paramTypes == null) {
+          if (Objects.requireNonNull(function).getOperandTypeChecker() == null
+              || !function.getOperandTypeChecker().isFixedParameters()) {
             // no parameter information for builtins; keep for now,
             // the type coerce will not work here.
             return true;
           }
+          final SqlOperandMetadata operandMetadata =
+              (SqlOperandMetadata) function.getOperandTypeChecker();
+          final List<RelDataType> paramTypes =
+              operandMetadata.paramTypes(typeFactory);
           final List<RelDataType> permutedArgTypes;
           if (argNames != null) {
-            permutedArgTypes = permuteArgTypes(function, argNames, argTypes);
+            final List<String> paramNames = operandMetadata.paramNames();
+            permutedArgTypes = permuteArgTypes(paramNames, argNames, argTypes);
             if (permutedArgTypes == null) {
               return false;
             }
@@ -638,19 +643,19 @@ public abstract class SqlUtil {
   /**
    * Permutes argument types to correspond to the order of parameter names.
    */
-  private static List<RelDataType> permuteArgTypes(SqlFunction function,
+  private static List<RelDataType> permuteArgTypes(List<String> paramNames,
       List<String> argNames, List<RelDataType> argTypes) {
     // Arguments passed by name. Make sure that the function has
     // parameters of all of these names.
     Map<Integer, Integer> map = new HashMap<>();
     for (Ord<String> argName : Ord.zip(argNames)) {
-      int i = function.getParamNames().indexOf(argName.e);
+      int i = paramNames.indexOf(argName.e);
       if (i < 0) {
         return null;
       }
       map.put(i, argName.i);
     }
-    return Functions.generate(function.getParamTypes().size(), index -> {
+    return Functions.generate(paramNames.size(), index -> {
       if (map.containsKey(index)) {
         return argTypes.get(map.get(index));
       } else {
@@ -667,6 +672,7 @@ public abstract class SqlUtil {
    */
   private static Iterator<SqlOperator> filterRoutinesByTypePrecedence(
       SqlSyntax sqlSyntax,
+      RelDataTypeFactory typeFactory,
       Iterator<SqlOperator> routines,
       List<RelDataType> argTypes,
       List<String> argNames) {
@@ -680,16 +686,21 @@ public abstract class SqlUtil {
     for (final Ord<RelDataType> argType : Ord.zip(argTypes)) {
       final RelDataTypePrecedenceList precList =
           argType.e.getPrecedenceList();
-      final RelDataType bestMatch = bestMatch(sqlFunctions, argType.i, argNames, precList);
+      final RelDataType bestMatch =
+          bestMatch(typeFactory, sqlFunctions, argType.i, argNames, precList);
       if (bestMatch != null) {
         sqlFunctions = sqlFunctions.stream()
             .filter(function -> {
-              final List<RelDataType> paramTypes = function.getParamTypes();
-              if (paramTypes == null) {
+              if (!function.getOperandTypeChecker().isFixedParameters()) {
                 return false;
               }
+              final SqlOperandMetadata operandMetadata =
+                  (SqlOperandMetadata) function.getOperandTypeChecker();
+              final List<String> paramNames = operandMetadata.paramNames();
+              final List<RelDataType> paramTypes =
+                  operandMetadata.paramTypes(typeFactory);
               int index = argNames != null
-                  ? function.getParamNames().indexOf(argNames.get(argType.i))
+                  ? paramNames.indexOf(argNames.get(argType.i))
                   : argType.i;
               final RelDataType paramType = paramTypes.get(index);
               return precList.compareTypePrecedence(paramType, bestMatch) >= 0;
@@ -701,16 +712,21 @@ public abstract class SqlUtil {
     return (Iterator) sqlFunctions.iterator();
   }
 
-  private static RelDataType bestMatch(List<SqlFunction> sqlFunctions, int i,
+  private static RelDataType bestMatch(RelDataTypeFactory typeFactory,
+      List<SqlFunction> sqlFunctions, int i,
       List<String> argNames, RelDataTypePrecedenceList precList) {
     RelDataType bestMatch = null;
     for (SqlFunction function : sqlFunctions) {
-      List<RelDataType> paramTypes = function.getParamTypes();
-      if (paramTypes == null) {
+      if (!function.getOperandTypeChecker().isFixedParameters()) {
         continue;
       }
+      final SqlOperandMetadata operandMetadata =
+          (SqlOperandMetadata) function.getOperandTypeChecker();
+      final List<RelDataType> paramTypes =
+          operandMetadata.paramTypes(typeFactory);
+      final List<String> paramNames = operandMetadata.paramNames();
       final RelDataType paramType = argNames != null
-          ? paramTypes.get(function.getParamNames().indexOf(argNames.get(i)))
+          ? paramTypes.get(paramNames.indexOf(argNames.get(i)))
           : paramTypes.get(i);
       if (bestMatch == null) {
         bestMatch = paramType;
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
index 2b4dfd3..8af4b77 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWindowTableFunction.java
@@ -19,13 +19,18 @@ package org.apache.calcite.sql;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.type.ReturnTypes;
-import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlNameMatcher;
 import org.apache.calcite.sql.validate.SqlValidator;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.Collections;
 import java.util.List;
 
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -69,80 +74,17 @@ public class SqlWindowTableFunction extends SqlFunction
       SqlWindowTableFunction::inferRowType;
 
   /** Creates a window table function with a given name. */
-  public SqlWindowTableFunction(String name, SqlOperandTypeChecker operandTypeChecker) {
+  public SqlWindowTableFunction(String name, SqlOperandMetadata operandMetadata) {
     super(name, SqlKind.OTHER_FUNCTION, ReturnTypes.CURSOR, null,
-        operandTypeChecker, SqlFunctionCategory.SYSTEM);
-  }
-
-  @Override public SqlReturnTypeInference getRowTypeInference() {
-    return ARG0_TABLE_FUNCTION_WINDOWING;
+        operandMetadata, SqlFunctionCategory.SYSTEM);
   }
 
-  protected static boolean throwValidationSignatureErrorOrReturnFalse(SqlCallBinding callBinding,
-      boolean throwOnFailure) {
-    if (throwOnFailure) {
-      throw callBinding.newValidationSignatureError();
-    } else {
-      return false;
-    }
+  @Override public SqlOperandMetadata getOperandTypeChecker() {
+    return (SqlOperandMetadata) super.getOperandTypeChecker();
   }
 
-  /**
-   * Validate the heading operands are in the form:
-   * (ROW, DESCRIPTOR, DESCRIPTOR ..., other params).
-   *
-   * @param callBinding The call binding
-   * @param descriptors The number of descriptors following the first operand (e.g. the table)
-   *
-   * @return true if validation passes
-   */
-  protected static boolean validateTableWithFollowingDescriptors(
-      SqlCallBinding callBinding, int descriptors) {
-    final SqlNode operand0 = callBinding.operand(0);
-    final SqlValidator validator = callBinding.getValidator();
-    final RelDataType type = validator.getValidatedNodeType(operand0);
-    if (type.getSqlTypeName() != SqlTypeName.ROW) {
-      return false;
-    }
-    for (int i = 1; i < descriptors + 1; i++) {
-      final SqlNode operand = callBinding.operand(i);
-      if (operand.getKind() != SqlKind.DESCRIPTOR) {
-        return false;
-      }
-      validateColumnNames(validator, type.getFieldNames(), ((SqlCall) operand).getOperandList());
-    }
-    return true;
-  }
-
-  /**
-   * Validate the operands starting from position {@code startPos} are all INTERVAL.
-   *
-   * @param callBinding The call binding
-   * @param startPos    The start position to validate (starting index is 0)
-   *
-   * @return true if validation passes
-   */
-  protected static boolean validateTailingIntervals(SqlCallBinding callBinding, int startPos) {
-    final SqlValidator validator = callBinding.getValidator();
-    for (int i = startPos; i < callBinding.getOperandCount(); i++) {
-      final RelDataType type = validator.getValidatedNodeType(callBinding.operand(i));
-      if (!SqlTypeUtil.isInterval(type)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  private static void validateColumnNames(SqlValidator validator,
-      List<String> fieldNames, List<SqlNode> columnNames) {
-    final SqlNameMatcher matcher = validator.getCatalogReader().nameMatcher();
-    for (SqlNode columnName : columnNames) {
-      final String name = ((SqlIdentifier) columnName).getSimple();
-      if (matcher.indexOf(fieldNames, name) < 0) {
-        throw SqlUtil.newContextException(columnName.getParserPosition(),
-            RESOURCE.unknownIdentifier(name));
-      }
-    }
+  @Override public SqlReturnTypeInference getRowTypeInference() {
+    return ARG0_TABLE_FUNCTION_WINDOWING;
   }
 
   /**
@@ -169,4 +111,113 @@ public class SqlWindowTableFunction extends SqlFunction
         .add("window_end", timestampType)
         .build();
   }
+
+  /** Partial implementation of operand type checker. */
+  protected abstract static class AbstractOperandMetadata
+      implements SqlOperandMetadata {
+    final List<String> paramNames;
+    final int mandatoryParamCount;
+
+    AbstractOperandMetadata(List<String> paramNames,
+        int mandatoryParamCount) {
+      this.paramNames = ImmutableList.copyOf(paramNames);
+      this.mandatoryParamCount = mandatoryParamCount;
+      Preconditions.checkArgument(mandatoryParamCount >= 0
+          && mandatoryParamCount <= paramNames.size());
+    }
+
+    @Override public SqlOperandCountRange getOperandCountRange() {
+      return SqlOperandCountRanges.between(mandatoryParamCount,
+          paramNames.size());
+    }
+
+    @Override public List<RelDataType> paramTypes(RelDataTypeFactory typeFactory) {
+      return Collections.nCopies(paramNames.size(),
+          typeFactory.createSqlType(SqlTypeName.ANY));
+    }
+
+    @Override public List<String> paramNames() {
+      return paramNames;
+    }
+
+    @Override public Consistency getConsistency() {
+      return Consistency.NONE;
+    }
+
+    @Override public boolean isOptional(int i) {
+      return i > getOperandCountRange().getMin()
+          && i <= getOperandCountRange().getMax();
+    }
+
+    boolean throwValidationSignatureErrorOrReturnFalse(SqlCallBinding callBinding,
+        boolean throwOnFailure) {
+      if (throwOnFailure) {
+        throw callBinding.newValidationSignatureError();
+      } else {
+        return false;
+      }
+    }
+
+    /**
+     * Checks whether the heading operands are in the form
+     * {@code (ROW, DESCRIPTOR, DESCRIPTOR ..., other params)},
+     * returning whether successful, and throwing if any columns are not found.
+     *
+     * @param callBinding The call binding
+     * @param descriptorCount The number of descriptors following the first
+     * operand (e.g. the table)
+     *
+     * @return true if validation passes; throws if any columns are not found
+     */
+    boolean checkTableAndDescriptorOperands(SqlCallBinding callBinding,
+        int descriptorCount) {
+      final SqlNode operand0 = callBinding.operand(0);
+      final SqlValidator validator = callBinding.getValidator();
+      final RelDataType type = validator.getValidatedNodeType(operand0);
+      if (type.getSqlTypeName() != SqlTypeName.ROW) {
+        return false;
+      }
+      for (int i = 1; i < descriptorCount + 1; i++) {
+        final SqlNode operand = callBinding.operand(i);
+        if (operand.getKind() != SqlKind.DESCRIPTOR) {
+          return false;
+        }
+        validateColumnNames(validator, type.getFieldNames(),
+            ((SqlCall) operand).getOperandList());
+      }
+      return true;
+    }
+
+    /**
+     * Checks whether the operands starting from position {@code startPos} are
+     * all of type {@code INTERVAL}, returning whether successful.
+     *
+     * @param callBinding The call binding
+     * @param startPos    The start position to validate (starting index is 0)
+     *
+     * @return true if validation passes
+     */
+    boolean checkIntervalOperands(SqlCallBinding callBinding, int startPos) {
+      final SqlValidator validator = callBinding.getValidator();
+      for (int i = startPos; i < callBinding.getOperandCount(); i++) {
+        final RelDataType type = validator.getValidatedNodeType(callBinding.operand(i));
+        if (!SqlTypeUtil.isInterval(type)) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    void validateColumnNames(SqlValidator validator,
+        List<String> fieldNames, List<SqlNode> columnNames) {
+      final SqlNameMatcher matcher = validator.getCatalogReader().nameMatcher();
+      for (SqlNode columnName : columnNames) {
+        final String name = ((SqlIdentifier) columnName).getSimple();
+        if (matcher.indexOf(fieldNames, name) < 0) {
+          throw SqlUtil.newContextException(columnName.getParserPosition(),
+              RESOURCE.unknownIdentifier(name));
+        }
+      }
+    }
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ExplicitOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/ExplicitOperandTypeChecker.java
index d0e5988..f817724 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ExplicitOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ExplicitOperandTypeChecker.java
@@ -24,6 +24,7 @@ import org.apache.calcite.sql.SqlOperator;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
 /**
  * Parameter type-checking strategy for Explicit Type.
@@ -31,10 +32,10 @@ import java.util.List;
 public class ExplicitOperandTypeChecker implements SqlOperandTypeChecker {
   //~ Methods ----------------------------------------------------------------
 
-  private RelDataType type;
+  private final RelDataType type;
 
   public ExplicitOperandTypeChecker(RelDataType type) {
-    this.type = type;
+    this.type = Objects.requireNonNull(type);
   }
 
   public boolean isOptional(int i) {
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandMetadataImpl.java b/core/src/main/java/org/apache/calcite/sql/type/OperandMetadataImpl.java
new file mode 100644
index 0000000..3a5acca
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandMetadataImpl.java
@@ -0,0 +1,69 @@
+/*
+ * 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.calcite.sql.type;
+
+import org.apache.calcite.linq4j.function.Functions;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.function.Function;
+import java.util.function.IntFunction;
+import java.util.function.Predicate;
+
+/**
+ * Operand type-checking strategy user-defined functions (including user-defined
+ * aggregate functions, table functions, and table macros).
+ *
+ * <p>UDFs have a fixed number of parameters is fixed. Per
+ * {@link SqlOperandMetadata}, this interface provides the name and types of
+ * each parameter.
+ *
+ * @see OperandTypes#operandMetadata
+ */
+public class OperandMetadataImpl extends FamilyOperandTypeChecker
+    implements SqlOperandMetadata {
+  private final Function<RelDataTypeFactory, List<RelDataType>>
+      paramTypesFactory;
+  private final IntFunction<String> paramNameFn;
+
+  //~ Constructors -----------------------------------------------------------
+
+  /** Package private. Create using {@link OperandTypes#operandMetadata}. */
+  OperandMetadataImpl(List<SqlTypeFamily> families,
+      Function<RelDataTypeFactory, List<RelDataType>> paramTypesFactory,
+      IntFunction<String> paramNameFn, Predicate<Integer> optional) {
+    super(families, optional);
+    this.paramTypesFactory = Objects.requireNonNull(paramTypesFactory);
+    this.paramNameFn = paramNameFn;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public boolean isFixedParameters() {
+    return true;
+  }
+
+  @Override public List<RelDataType> paramTypes(RelDataTypeFactory typeFactory) {
+    return paramTypesFactory.apply(typeFactory);
+  }
+
+  @Override public List<String> paramNames() {
+    return Functions.generate(families.size(), paramNameFn);
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
index 37d0250..9e4181b 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
@@ -18,6 +18,7 @@ package org.apache.calcite.sql.type;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeComparability;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlCallBinding;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
@@ -29,6 +30,8 @@ import com.google.common.collect.ImmutableList;
 
 import java.math.BigDecimal;
 import java.util.List;
+import java.util.function.Function;
+import java.util.function.IntFunction;
 import java.util.function.Predicate;
 
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -81,6 +84,23 @@ public abstract class OperandTypes {
   }
 
   /**
+   * Creates a checker for user-defined functions (including user-defined
+   * aggregate functions, table functions, and table macros).
+   *
+   * <p>Unlike built-in functions, there is a fixed number of parameters,
+   * and the parameters have names. You can ask for the type of a parameter
+   * without providing a particular call (and with it actual arguments) but you
+   * do need to provide a type factory, and therefore the types are only good
+   * for the duration of the current statement.
+   */
+  public static SqlOperandMetadata operandMetadata(List<SqlTypeFamily> families,
+      Function<RelDataTypeFactory, List<RelDataType>> typesFactory,
+      IntFunction<String> operandName, Predicate<Integer> optional) {
+    return new OperandMetadataImpl(families, typesFactory, operandName,
+        optional);
+  }
+
+  /**
    * Creates a checker that passes if any one of the rules passes.
    */
   public static SqlOperandTypeChecker or(SqlOperandTypeChecker... rules) {
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandMetadata.java b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandMetadata.java
new file mode 100644
index 0000000..41bbe45
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandMetadata.java
@@ -0,0 +1,43 @@
+/*
+ * 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.calcite.sql.type;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+
+import java.util.List;
+import javax.annotation.Nonnull;
+
+/**
+ * Extension to {@link SqlOperandTypeChecker} that also provides
+ * names and types of particular operands.
+ *
+ * <p>It is intended for user-defined functions (UDFs), and therefore the number
+ * of parameters is fixed.
+ *
+ * @see OperandTypes
+ */
+@Nonnull
+public interface SqlOperandMetadata extends SqlOperandTypeChecker {
+  //~ Methods ----------------------------------------------------------------
+
+  /** Returns the types of the parameters. */
+  List<RelDataType> paramTypes(RelDataTypeFactory typeFactory);
+
+  /** Returns the names of the parameters. */
+  List<String> paramNames();
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
index 7f15371..2a42280 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlOperandTypeChecker.java
@@ -62,6 +62,15 @@ public interface SqlOperandTypeChecker {
   /** Returns whether the {@code i}th operand is optional. */
   boolean isOptional(int i);
 
+  /** Returns whether the list of parameters is fixed-length. In standard SQL,
+   * user-defined functions are fixed-length.
+   *
+   * <p>If true, the validator should expand calls, supplying a {@code DEFAULT}
+   * value for each parameter for which an argument is not supplied. */
+  default boolean isFixedParameters() {
+    return false;
+  }
+
   /** Strategy used to make arguments consistent. */
   enum Consistency {
     /** Do not try to make arguments consistent. */
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java
index af9099c..cf16d3d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedAggFunction.java
@@ -16,29 +16,19 @@
  */
 package org.apache.calcite.sql.validate;
 
-import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.calcite.linq4j.function.Experimental;
-import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
 import org.apache.calcite.schema.AggregateFunction;
-import org.apache.calcite.schema.FunctionParameter;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
-import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.Optionality;
 import org.apache.calcite.util.Util;
 
-import com.google.common.collect.Lists;
-
-import java.util.ArrayList;
-import java.util.List;
-
 /**
  * User-defined aggregate function.
  *
@@ -48,58 +38,36 @@ import java.util.List;
 public class SqlUserDefinedAggFunction extends SqlAggFunction {
   public final AggregateFunction function;
 
-  /** This field is is technical debt; see [CALCITE-2082] Remove
-   * RelDataTypeFactory argument from SqlUserDefinedAggFunction constructor. */
-  @Experimental
-  public final RelDataTypeFactory typeFactory;
-
-  /** Creates a SqlUserDefinedAggFunction. */
+  @Deprecated // to be removed before 2.0
   public SqlUserDefinedAggFunction(SqlIdentifier opName,
       SqlReturnTypeInference returnTypeInference,
       SqlOperandTypeInference operandTypeInference,
       SqlOperandTypeChecker operandTypeChecker, AggregateFunction function,
       boolean requiresOrder, boolean requiresOver,
       Optionality requiresGroupOrder, RelDataTypeFactory typeFactory) {
-    super(Util.last(opName.names), opName, SqlKind.OTHER_FUNCTION,
-        returnTypeInference, operandTypeInference, operandTypeChecker,
+    this(opName, SqlKind.OTHER_FUNCTION, returnTypeInference,
+        operandTypeInference,
+        operandTypeChecker instanceof SqlOperandMetadata
+            ? (SqlOperandMetadata) operandTypeChecker : null, function,
+        requiresOrder, requiresOver, requiresGroupOrder);
+    Util.discard(typeFactory); // no longer used
+  }
+
+  /** Creates a SqlUserDefinedAggFunction. */
+  public SqlUserDefinedAggFunction(SqlIdentifier opName, SqlKind kind,
+      SqlReturnTypeInference returnTypeInference,
+      SqlOperandTypeInference operandTypeInference,
+      SqlOperandMetadata operandMetadata, AggregateFunction function,
+      boolean requiresOrder, boolean requiresOver,
+      Optionality requiresGroupOrder) {
+    super(Util.last(opName.names), opName, kind,
+        returnTypeInference, operandTypeInference, operandMetadata,
         SqlFunctionCategory.USER_DEFINED_FUNCTION, requiresOrder, requiresOver,
         requiresGroupOrder);
     this.function = function;
-    this.typeFactory = typeFactory;
-  }
-
-  @Override public List<RelDataType> getParamTypes() {
-    List<RelDataType> argTypes = new ArrayList<>();
-    for (FunctionParameter o : function.getParameters()) {
-      final RelDataType type = o.getType(typeFactory);
-      argTypes.add(type);
-    }
-    return toSql(argTypes);
-  }
-
-  private List<RelDataType> toSql(List<RelDataType> types) {
-    return Lists.transform(types, this::toSql);
-  }
-
-  private RelDataType toSql(RelDataType type) {
-    if (type instanceof RelDataTypeFactoryImpl.JavaType
-        && ((RelDataTypeFactoryImpl.JavaType) type).getJavaClass()
-        == Object.class) {
-      return typeFactory.createTypeWithNullability(
-          typeFactory.createSqlType(SqlTypeName.ANY), true);
-    }
-    return JavaTypeFactoryImpl.toSql(typeFactory, type);
-  }
-
-  @SuppressWarnings("deprecation")
-  public List<RelDataType> getParameterTypes(
-      final RelDataTypeFactory typeFactory) {
-    return Lists.transform(function.getParameters(),
-        parameter -> parameter.getType(typeFactory));
   }
 
-  @SuppressWarnings("deprecation")
-  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
-    return function.getReturnType(typeFactory);
+  @Override public SqlOperandMetadata getOperandTypeChecker() {
+    return (SqlOperandMetadata) super.getOperandTypeChecker();
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java
index ac78949..813658b 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedFunction.java
@@ -23,6 +23,7 @@ import org.apache.calcite.sql.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
@@ -41,31 +42,46 @@ import java.util.List;
 public class SqlUserDefinedFunction extends SqlFunction {
   public final Function function;
 
-  /** Creates a {@link SqlUserDefinedFunction}. */
+  @Deprecated // to be removed before 2.0
   public SqlUserDefinedFunction(SqlIdentifier opName,
       SqlReturnTypeInference returnTypeInference,
       SqlOperandTypeInference operandTypeInference,
       SqlOperandTypeChecker operandTypeChecker,
       List<RelDataType> paramTypes,
       Function function) {
-    this(opName, returnTypeInference, operandTypeInference, operandTypeChecker,
-        paramTypes, function, SqlFunctionCategory.USER_DEFINED_FUNCTION);
+    this(opName, SqlKind.OTHER_FUNCTION, returnTypeInference,
+        operandTypeInference,
+        operandTypeChecker instanceof SqlOperandMetadata
+            ? (SqlOperandMetadata) operandTypeChecker : null, function);
+    Util.discard(paramTypes); // no longer used
+  }
+
+  /** Creates a {@link SqlUserDefinedFunction}. */
+  public SqlUserDefinedFunction(SqlIdentifier opName, SqlKind kind,
+      SqlReturnTypeInference returnTypeInference,
+      SqlOperandTypeInference operandTypeInference,
+      SqlOperandMetadata operandMetadata,
+      Function function) {
+    this(opName, kind, returnTypeInference, operandTypeInference,
+        operandMetadata, function, SqlFunctionCategory.USER_DEFINED_FUNCTION);
   }
 
   /** Constructor used internally and by derived classes. */
-  protected SqlUserDefinedFunction(SqlIdentifier opName,
+  protected SqlUserDefinedFunction(SqlIdentifier opName, SqlKind kind,
       SqlReturnTypeInference returnTypeInference,
       SqlOperandTypeInference operandTypeInference,
-      SqlOperandTypeChecker operandTypeChecker,
-      List<RelDataType> paramTypes,
+      SqlOperandMetadata operandMetadata,
       Function function,
       SqlFunctionCategory category) {
-    super(Util.last(opName.names), opName, SqlKind.OTHER_FUNCTION,
-        returnTypeInference, operandTypeInference, operandTypeChecker,
-        paramTypes, category);
+    super(Util.last(opName.names), opName, kind, returnTypeInference,
+        operandTypeInference, operandMetadata, category);
     this.function = function;
   }
 
+  @Override public SqlOperandMetadata getOperandTypeChecker() {
+    return (SqlOperandMetadata) super.getOperandTypeChecker();
+  }
+
   /**
    * Returns function that implements given operator call.
    * @return function that implements given operator call
@@ -74,6 +90,7 @@ public class SqlUserDefinedFunction extends SqlFunction {
     return function;
   }
 
+  @SuppressWarnings("deprecation")
   @Override public List<String> getParamNames() {
     return Lists.transform(function.getParameters(),
         FunctionParameter::getName);
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
index 735d299..ce5f4d9 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableFunction.java
@@ -20,8 +20,10 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.schema.TableFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlTableFunction;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
@@ -37,14 +39,28 @@ import java.util.List;
 */
 public class SqlUserDefinedTableFunction extends SqlUserDefinedFunction
     implements SqlTableFunction {
+  @Deprecated // to be removed before 2.0
   public SqlUserDefinedTableFunction(SqlIdentifier opName,
       SqlReturnTypeInference returnTypeInference,
       SqlOperandTypeInference operandTypeInference,
       SqlOperandTypeChecker operandTypeChecker,
-      List<RelDataType> paramTypes,
+      List<RelDataType> paramTypes, // no longer used
       TableFunction function) {
-    super(opName, returnTypeInference, operandTypeInference, operandTypeChecker,
-        paramTypes, function, SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION);
+    this(opName, SqlKind.OTHER_FUNCTION, returnTypeInference,
+        operandTypeInference,
+        operandTypeChecker instanceof SqlOperandMetadata
+            ? (SqlOperandMetadata) operandTypeChecker : null, function);
+  }
+
+  /** Creates a user-defined table function. */
+  public SqlUserDefinedTableFunction(SqlIdentifier opName, SqlKind kind,
+      SqlReturnTypeInference returnTypeInference,
+      SqlOperandTypeInference operandTypeInference,
+      SqlOperandMetadata operandMetadata,
+      TableFunction function) {
+    super(opName, kind, returnTypeInference, operandTypeInference,
+        operandMetadata, function,
+        SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION);
   }
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
index b9066dd..307bf75 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlUserDefinedTableMacro.java
@@ -29,6 +29,7 @@ import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlTableFunction;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
@@ -38,7 +39,6 @@ import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Objects;
 
 /**
  * User-defined table macro.
@@ -50,18 +50,36 @@ public class SqlUserDefinedTableMacro extends SqlFunction
     implements SqlTableFunction {
   private final TableMacro tableMacro;
 
+  @Deprecated // to be removed before 2.0
   public SqlUserDefinedTableMacro(SqlIdentifier opName,
       SqlReturnTypeInference returnTypeInference,
       SqlOperandTypeInference operandTypeInference,
       SqlOperandTypeChecker operandTypeChecker, List<RelDataType> paramTypes,
       TableMacro tableMacro) {
-    super(Util.last(opName.names), opName, SqlKind.OTHER_FUNCTION,
-        returnTypeInference, operandTypeInference, operandTypeChecker,
-        Objects.requireNonNull(paramTypes),
+    this(opName, SqlKind.OTHER_FUNCTION, returnTypeInference,
+        operandTypeInference,
+        operandTypeChecker instanceof SqlOperandMetadata
+            ? (SqlOperandMetadata) operandTypeChecker : null, tableMacro);
+    Util.discard(paramTypes); // no longer used
+  }
+
+  /** Creates a user-defined table macro. */
+  public SqlUserDefinedTableMacro(SqlIdentifier opName, SqlKind kind,
+      SqlReturnTypeInference returnTypeInference,
+      SqlOperandTypeInference operandTypeInference,
+      SqlOperandMetadata operandMetadata,
+      TableMacro tableMacro) {
+    super(Util.last(opName.names), opName, kind,
+        returnTypeInference, operandTypeInference, operandMetadata,
         SqlFunctionCategory.USER_DEFINED_TABLE_FUNCTION);
     this.tableMacro = tableMacro;
   }
 
+  @Override public SqlOperandMetadata getOperandTypeChecker() {
+    return (SqlOperandMetadata) super.getOperandTypeChecker();
+  }
+
+  @SuppressWarnings("deprecation")
   @Override public List<String> getParamNames() {
     return Lists.transform(tableMacro.getParameters(),
         FunctionParameter::getName);
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java
index 4018b22..66ed010 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/implicit/TypeCoercionImpl.java
@@ -34,6 +34,7 @@ import org.apache.calcite.sql.SqlUpdate;
 import org.apache.calcite.sql.SqlWith;
 import org.apache.calcite.sql.fun.SqlCase;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlValidator;
@@ -561,19 +562,22 @@ public class TypeCoercionImpl extends AbstractTypeCoercion {
   }
 
   /**
-   * Type coercion for user defined functions(UDFs).
+   * Type coercion for user-defined functions (UDFs).
    */
   public boolean userDefinedFunctionCoercion(SqlValidatorScope scope,
       SqlCall call, SqlFunction function) {
-    final List<RelDataType> paramTypes = function.getParamTypes();
-    assert paramTypes != null;
+    final SqlOperandMetadata operandMetadata =
+        (SqlOperandMetadata) function.getOperandTypeChecker();
+    final List<RelDataType> paramTypes =
+        operandMetadata.paramTypes(scope.getValidator().getTypeFactory());
     boolean coerced = false;
     for (int i = 0; i < call.operandCount(); i++) {
       SqlNode operand = call.operand(i);
       if (operand.getKind() == SqlKind.ARGUMENT_ASSIGNMENT) {
         final List<SqlNode> operandList = ((SqlCall) operand).getOperandList();
         String name = ((SqlIdentifier) operandList.get(1)).getSimple();
-        int formalIndex = function.getParamNames().indexOf(name);
+        final List<String> paramNames = operandMetadata.paramNames();
+        int formalIndex = paramNames.indexOf(name);
         if (formalIndex < 0) {
           return false;
         }
diff --git a/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java b/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java
index 8ebcd19..4c8887f 100644
--- a/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java
+++ b/core/src/test/java/org/apache/calcite/test/MockSqlOperatorTable.java
@@ -157,7 +157,6 @@ public class MockSqlOperatorTable extends ChainedSqlOperatorTable {
           null,
           null,
           OperandTypes.NUMERIC,
-          null,
           SqlFunctionCategory.USER_DEFINED_FUNCTION);
     }
 
@@ -185,8 +184,9 @@ public class MockSqlOperatorTable extends ChainedSqlOperatorTable {
   public static class SplitFunction extends SqlFunction {
 
     public SplitFunction() {
-      super("SPLIT", new SqlIdentifier("SPLIT", SqlParserPos.ZERO), SqlKind.OTHER_FUNCTION, null,
-          null, OperandTypes.family(SqlTypeFamily.STRING, SqlTypeFamily.STRING), null,
+      super("SPLIT", new SqlIdentifier("SPLIT", SqlParserPos.ZERO),
+          SqlKind.OTHER_FUNCTION, null, null,
+          OperandTypes.family(SqlTypeFamily.STRING, SqlTypeFamily.STRING),
           SqlFunctionCategory.USER_DEFINED_FUNCTION);
     }
 
@@ -239,8 +239,9 @@ public class MockSqlOperatorTable extends ChainedSqlOperatorTable {
   /** "STRUCTURED_FUNC" user-defined function whose return type is structured type. */
   public static class StructuredFunction extends SqlFunction {
     StructuredFunction() {
-      super("STRUCTURED_FUNC", new SqlIdentifier("STRUCTURED_FUNC", SqlParserPos.ZERO),
-          SqlKind.OTHER_FUNCTION, null, null, OperandTypes.NILADIC, null,
+      super("STRUCTURED_FUNC",
+          new SqlIdentifier("STRUCTURED_FUNC", SqlParserPos.ZERO),
+          SqlKind.OTHER_FUNCTION, null, null, OperandTypes.NILADIC,
           SqlFunctionCategory.USER_DEFINED_FUNCTION);
     }
 
diff --git a/piglet/src/main/java/org/apache/calcite/piglet/PigRelSqlUdfs.java b/piglet/src/main/java/org/apache/calcite/piglet/PigRelSqlUdfs.java
index 421b82a..9ff48cb 100644
--- a/piglet/src/main/java/org/apache/calcite/piglet/PigRelSqlUdfs.java
+++ b/piglet/src/main/java/org/apache/calcite/piglet/PigRelSqlUdfs.java
@@ -17,8 +17,10 @@
 package org.apache.calcite.piglet;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.linq4j.function.Functions;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
@@ -30,6 +32,7 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.type.MultisetSqlType;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeFamily;
@@ -83,7 +86,7 @@ public class PigRelSqlUdfs {
    */
   static final SqlUserDefinedFunction MULTISET_PROJECTION =
       new PigUserDefinedFunction("MULTISET_PROJECTION",
-          multisetProjectionInfer(), multisetProjectionCheck(), null,
+          multisetProjectionInfer(), multisetProjectionCheck(),
           MULTISET_PROJECTION_FUNC);
 
   /**
@@ -95,8 +98,9 @@ public class PigRelSqlUdfs {
   static SqlUserDefinedFunction createPigTupleUDF(ImmutableList<RexNode> operands) {
     return new PigUserDefinedFunction("PIG_TUPLE",
         infer(PigRelSqlUdfs.PIG_TUPLE_FUNC),
-        OperandTypes.family(getTypeFamilies(operands)),
-        getRelDataTypes(operands),
+        OperandTypes.operandMetadata(getTypeFamilies(operands),
+            typeFactory -> getRelDataTypes(operands), i -> "arg" + i,
+            i -> false),
         PigRelSqlUdfs.PIG_TUPLE_FUNC);
   }
 
@@ -107,11 +111,12 @@ public class PigRelSqlUdfs {
    * @return Pig DataBag SqlUDF
    */
   static SqlUserDefinedFunction createPigBagUDF(ImmutableList<RexNode> operands) {
-    return new PigUserDefinedFunction(
-        "PIG_BAG",
-        infer(PigRelSqlUdfs.PIG_BAG_FUNC),
-        OperandTypes.family(getTypeFamilies(operands)),
-        getRelDataTypes(operands),
+    final SqlOperandMetadata operandMetadata =
+        OperandTypes.operandMetadata(getTypeFamilies(operands),
+            typeFactory -> getRelDataTypes(operands), i -> "arg" + i,
+            i -> false);
+    return new PigUserDefinedFunction("PIG_BAG",
+        infer(PigRelSqlUdfs.PIG_BAG_FUNC), operandMetadata,
         PigRelSqlUdfs.PIG_BAG_FUNC);
   }
 
@@ -127,9 +132,12 @@ public class PigRelSqlUdfs {
   static SqlUserDefinedFunction createGeneralPigUdf(String udfName,
       Method method, FuncSpec funcSpec, RelDataType inputType,
       RelDataType returnType) {
+    final SqlOperandMetadata operandMetadata =
+        OperandTypes.operandMetadata(ImmutableList.of(SqlTypeFamily.ANY),
+            typeFactory -> ImmutableList.of(inputType), i -> "arg" + i,
+            i -> false);
     return new PigUserDefinedFunction(udfName, opBinding -> returnType,
-        OperandTypes.ANY, Collections.singletonList(inputType),
-        ScalarFunctionImpl.createUnsafe(method), funcSpec);
+        operandMetadata, ScalarFunctionImpl.createUnsafe(method), funcSpec);
   }
 
   /**
@@ -166,8 +174,13 @@ public class PigRelSqlUdfs {
   /**
    * Returns a {@link SqlOperandTypeChecker} for multiset projection operator.
    */
-  private static SqlOperandTypeChecker multisetProjectionCheck() {
-    return new SqlOperandTypeChecker() {
+  private static SqlOperandMetadata multisetProjectionCheck() {
+    // This should not really be a UDF. A SQL UDF has a fixed number of named
+    // parameters, and this does not. But let's pretend that it has two
+    // parameters of type 'ANY'
+    final int paramCount = 2;
+
+    return new SqlOperandMetadata() {
       public boolean checkOperandTypes(
           SqlCallBinding callBinding, boolean throwOnFailure) {
         // Need at least two arguments
@@ -214,6 +227,20 @@ public class PigRelSqlUdfs {
       public Consistency getConsistency() {
         return Consistency.NONE;
       }
+
+      @Override public List<RelDataType> paramTypes(
+          RelDataTypeFactory typeFactory) {
+        return Functions.generate(paramCount,
+            i -> typeFactory.createSqlType(SqlTypeName.ANY));
+      }
+
+      @Override public List<String> paramNames() {
+        return Functions.generate(paramCount,  i -> "arg" + i);
+      }
+
+      @Override public boolean isFixedParameters() {
+        return true;
+      }
     };
   }
 
diff --git a/piglet/src/main/java/org/apache/calcite/piglet/PigUserDefinedFunction.java b/piglet/src/main/java/org/apache/calcite/piglet/PigUserDefinedFunction.java
index 8fcd708..acdb24a 100644
--- a/piglet/src/main/java/org/apache/calcite/piglet/PigUserDefinedFunction.java
+++ b/piglet/src/main/java/org/apache/calcite/piglet/PigUserDefinedFunction.java
@@ -16,12 +16,12 @@
  */
 package org.apache.calcite.piglet;
 
-import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandMetadata;
 import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
@@ -30,45 +30,33 @@ import org.apache.pig.FuncSpec;
 
 import com.google.common.collect.ImmutableList;
 
-import java.util.List;
-
 /** Pig user-defined function. */
 public class PigUserDefinedFunction extends SqlUserDefinedFunction {
   public final FuncSpec funcSpec;
   private PigUserDefinedFunction(SqlIdentifier opName,
       SqlReturnTypeInference returnTypeInference,
       SqlOperandTypeInference operandTypeInference,
-      SqlOperandTypeChecker operandTypeChecker,
-      List<RelDataType> paramTypes,
+      SqlOperandMetadata operandMetadata,
       Function function,
       FuncSpec funcSpec) {
-    super(opName, returnTypeInference, operandTypeInference, operandTypeChecker, paramTypes,
-        function,
+    super(opName, SqlKind.OTHER_FUNCTION, returnTypeInference,
+        operandTypeInference, operandMetadata, function,
         SqlFunctionCategory.USER_DEFINED_CONSTRUCTOR);
     this.funcSpec = funcSpec;
   }
 
   public PigUserDefinedFunction(String name,
       SqlReturnTypeInference returnTypeInference,
-      SqlOperandTypeChecker operandTypeChecker,
-      List<RelDataType> paramTypes,
-      Function function,
+      SqlOperandMetadata operandMetadata, Function function,
       FuncSpec funcSpec) {
     this(new SqlIdentifier(ImmutableList.of(name), SqlParserPos.ZERO),
-        returnTypeInference,
-        null,
-        operandTypeChecker,
-        paramTypes,
-        function,
-        funcSpec);
+        returnTypeInference, null, operandMetadata, function, funcSpec);
   }
 
   public PigUserDefinedFunction(String name,
       SqlReturnTypeInference returnTypeInference,
-      SqlOperandTypeChecker operandTypeChecker,
-      List<RelDataType> paramTypes,
-      Function function) {
-    this(name, returnTypeInference, operandTypeChecker, paramTypes, function, null);
+      SqlOperandMetadata operandMetadata, Function function) {
+    this(name, returnTypeInference, operandMetadata, function, null);
   }
 
 }