diff --git a/core/src/main/java/org/opensearch/sql/calcite/CalcitePlanContext.java b/core/src/main/java/org/opensearch/sql/calcite/CalcitePlanContext.java index d9cc7251e14..2eb73d69a61 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/CalcitePlanContext.java +++ b/core/src/main/java/org/opensearch/sql/calcite/CalcitePlanContext.java @@ -8,6 +8,7 @@ import static org.opensearch.sql.calcite.utils.OpenSearchTypeFactory.TYPE_FACTORY; import java.sql.Connection; +import java.sql.SQLException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -17,13 +18,21 @@ import java.util.function.BiFunction; import lombok.Getter; import lombok.Setter; +import org.apache.calcite.config.NullCollation; import org.apache.calcite.rex.RexCorrelVariable; import org.apache.calcite.rex.RexLambdaRef; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.server.CalciteServerStatement; +import org.apache.calcite.sql.validate.SqlValidator; import org.apache.calcite.tools.FrameworkConfig; import org.apache.calcite.tools.RelBuilder; import org.opensearch.sql.ast.expression.UnresolvedExpression; import org.opensearch.sql.calcite.utils.CalciteToolsHelper; +import org.opensearch.sql.calcite.validate.OpenSearchSparkSqlDialect; +import org.opensearch.sql.calcite.validate.PplTypeCoercion; +import org.opensearch.sql.calcite.validate.PplTypeCoercionRule; +import org.opensearch.sql.calcite.validate.PplValidator; +import org.opensearch.sql.calcite.validate.SqlOperatorTableProvider; import org.opensearch.sql.common.setting.Settings; import org.opensearch.sql.executor.QueryType; import org.opensearch.sql.expression.function.FunctionProperties; @@ -72,6 +81,25 @@ public class CalcitePlanContext { /** Whether we're currently inside a lambda context. */ @Getter @Setter private boolean inLambdaContext = false; + /** + * -- SETTER -- Sets the SQL operator table provider. This must be called during initialization by + * the opensearch module. + * + * @param provider the provider to use for obtaining operator tables + */ + @Setter private static SqlOperatorTableProvider operatorTableProvider; + + /** Cached SqlValidator instance (lazy initialized). */ + private SqlValidator validator; + + /** + * Initialize a new CalcitePlanContext and set up Calcite connection, builders, function + * properties, and lambda/capture state. + * + * @param config the Calcite FrameworkConfig to use for planning + * @param sysLimit system limits governing planning behaviour + * @param queryType the QueryType for this planning context + */ private CalcitePlanContext(FrameworkConfig config, SysLimit sysLimit, QueryType queryType) { this.config = config; this.sysLimit = sysLimit; @@ -85,8 +113,11 @@ private CalcitePlanContext(FrameworkConfig config, SysLimit sysLimit, QueryType } /** - * Private constructor for creating a context that shares relBuilder with parent. Used by clone() - * to create lambda contexts that can resolve fields from the parent context. + * Creates a child CalcitePlanContext that shares relational builders with the given parent while isolating lambda-specific state. + * + *
The child context reuses the parent's relBuilder and rexBuilder so it can resolve fields against the parent, but it initializes independent maps/lists for lambda references and captured variables and marks the context as a lambda context.
+ * + * @param parent the parent context whose builders and configuration are reused */ private CalcitePlanContext(CalcitePlanContext parent) { this.config = parent.config; @@ -101,6 +132,59 @@ private CalcitePlanContext(CalcitePlanContext parent) { this.inLambdaContext = true; // Mark that we're inside a lambda } + /** + * Provides the lazily initialized SqlValidator configured for PPL within this context. + * + * @return the SqlValidator configured with PPL type coercion rules, Spark SQL conformance, + * Spark NULL collation, and identifier expansion + * @throws IllegalStateException if the global SqlOperatorTableProvider has not been set + * @throws RuntimeException if creating or unwrapping the underlying Calcite statement fails + */ + public SqlValidator getValidator() { + if (validator == null) { + final CalciteServerStatement statement; + try { + statement = connection.createStatement().unwrap(CalciteServerStatement.class); + } catch (SQLException e) { + throw new RuntimeException(e); + } + if (operatorTableProvider == null) { + throw new IllegalStateException( + "SqlOperatorTableProvider must be set before creating CalcitePlanContext"); + } + SqlValidator.Config validatorConfig = + SqlValidator.Config.DEFAULT + .withTypeCoercionRules(PplTypeCoercionRule.instance()) + .withTypeCoercionFactory(PplTypeCoercion::create) + // Use lenient conformance for PPL compatibility + .withConformance(OpenSearchSparkSqlDialect.DEFAULT.getConformance()) + // Use Spark SQL's NULL collation (NULLs sorted LOW/FIRST) + .withDefaultNullCollation(NullCollation.LOW) + // This ensures that coerced arguments are replaced with cast version in sql select + // list because coercion is performed during select list expansion during sql + // validation. Affects 4356.yml + // See SqlValidatorImpl#validateSelectList and AggConverter#translateAgg + .withIdentifierExpansion(true); + validator = + PplValidator.create( + statement, + config, + operatorTableProvider.getOperatorTable(), + TYPE_FACTORY, + validatorConfig); + } + return validator; + } + + /** + * Temporarily marks the context as resolving a join condition and applies a transformation to the given unresolved expression. + * + *The context flag indicating join-condition resolution is set to true before invoking the transform and reset to false afterwards.
+ *
+ * @param expr the unresolved expression representing the join condition
+ * @param transformFunction a function that converts the unresolved expression into a {@code RexNode} using this context
+ * @return the {@code RexNode} produced by applying {@code transformFunction} to {@code expr}
+ */
public RexNode resolveJoinCondition(
UnresolvedExpression expr,
BiFunction Window frames such as ROWS n PRECEDING require an ORDER BY to define row order; this method
+ * adds the RelBuilder's existing collation as the OVER clause's ORDER BY for each encountered
+ * RexOver.
+ *
+ * @param overExpressions window function expressions that may contain nested {@link RexOver}
+ * @param context plan context used to obtain the current RelBuilder and RexBuilder
+ * @return a list of expressions where each {@link RexOver} has its ORDER BY populated from the
+ * current plan collation (unchanged expressions are returned as-is)
+ */
+ private List Each expression is transformed to: CASE groupNotNull WHEN TRUE THEN Behavior highlights:
+ * - Converts character literal "1"/"0" to boolean `true`/`false`; converts exact numeric to boolean
+ * by comparing against zero.
+ * - For user-defined types (UDT), delegates conversion to corresponding PPL builtin operators
+ * (DATE, TIME, TIMESTAMP, IP) when applicable; preserves IP values and converts STRING to IP.
+ * - Converts floating point/decimal numbers to character using the `NUMBER_TO_STRING` operator.
+ *
+ * @param pos the parser position associated with this cast
+ * @param type the target relational data type to cast to
+ * @param exp the expression to be cast
+ * @param matchNullability whether the resulting type's nullability should match the target
+ * @param safe whether the cast should be performed in a "safe" mode (no runtime errors)
+ * @param format optional literal specifying a target format (may be null)
+ * @return a RexNode representing the cast or an equivalent expression implementing the conversion
+ * @throws ExpressionEvaluationException if converting to IP from an unsupported argument type
+ * @throws SemanticCheckException if casting to a user-defined type is unsupported for the argument
+ */
@Override
public RexNode makeCast(
SqlParserPos pos,
@@ -146,7 +172,7 @@ public RexNode makeCast(
// SqlStdOperatorTable.NOT_EQUALS,
// ImmutableList.of(exp, makeZeroLiteral(sourceType)));
}
- } else if (OpenSearchTypeFactory.isUserDefinedType(type)) {
+ } else if (OpenSearchTypeUtil.isUserDefinedType(type)) {
if (RexLiteral.isNullLiteral(exp)) {
return super.makeCast(pos, type, exp, matchNullability, safe, format);
}
@@ -185,4 +211,29 @@ else if ((SqlTypeUtil.isApproximateNumeric(sourceType) || SqlTypeUtil.isDecimal(
}
return super.makeCast(pos, type, exp, matchNullability, safe, format);
}
-}
+
+ /**
+ * Infers the result type for a call to the given operator.
+ *
+ * When the operator is a binary arithmetic and one operand is numeric while the other is a
+ * character type, returns the numeric operand's type. For all other cases, delegates to the
+ * superclass implementation.
+ *
+ * @param op the operator being called
+ * @param exprs the actual operand expressions
+ * @return the inferred relational data type for the operator call
+ */
+ @Override
+ public RelDataType deriveReturnType(SqlOperator op, List extends RexNode> exprs) {
+ if (op.getKind().belongsTo(SqlKind.BINARY_ARITHMETIC) && exprs.size() == 2) {
+ final RelDataType type1 = exprs.get(0).getType();
+ final RelDataType type2 = exprs.get(1).getType();
+ if (SqlTypeUtil.isNumeric(type1) && OpenSearchTypeUtil.isCharacter(type2)) {
+ return type1;
+ } else if (OpenSearchTypeUtil.isCharacter(type1) && SqlTypeUtil.isNumeric(type2)) {
+ return type2;
+ }
+ }
+ return super.deriveReturnType(op, exprs);
+ }
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/calcite/utils/CalciteToolsHelper.java b/core/src/main/java/org/opensearch/sql/calcite/utils/CalciteToolsHelper.java
index a5cdf0f45f0..b3ca8966271 100644
--- a/core/src/main/java/org/opensearch/sql/calcite/utils/CalciteToolsHelper.java
+++ b/core/src/main/java/org/opensearch/sql/calcite/utils/CalciteToolsHelper.java
@@ -236,11 +236,19 @@ private void registerCustomizedRules(RelOptPlanner planner) {
}
/**
- * Customize CalcitePreparingStmt. Override {@link CalcitePrepareImpl#getPreparingStmt} and
- * return {@link OpenSearchCalcitePreparingStmt}
+ * Create an OpenSearch-specific CalcitePreparingStmt configured for the provided prepare context.
+ *
+ * The returned preparing statement is constructed to match the requested element type and planner,
+ * selecting an appropriate element preference and result convention.
+ *
+ * @param context the prepare context containing type factory and root schema
+ * @param elementType the expected element Java type for result rows (e.g., Object[].class for arrays)
+ * @param catalogReader the catalog reader for resolving schemas and objects
+ * @param planner the relational optimizer planner used to create the execution cluster
+ * @return an OpenSearchCalcitePreparingStmt configured for the given context, element type, catalog reader, and planner
*/
@Override
- protected CalcitePrepareImpl.CalcitePreparingStmt getPreparingStmt(
+ public CalcitePrepareImpl.CalcitePreparingStmt getPreparingStmt(
CalcitePrepare.Context context,
Type elementType,
CalciteCatalogReader catalogReader,
@@ -336,8 +344,13 @@ public Type getElementType() {
public static class OpenSearchRelRunners {
/**
- * Runs a relational expression by existing connection. This class copied from {@link
- * org.apache.calcite.tools.RelRunners#run(RelNode)}
+ * Prepare a JDBC PreparedStatement for the given relational expression using the connection from the provided CalcitePlanContext.
+ *
+ * @param context holds the JDBC Connection used to create the PreparedStatement; its connection is unwrapped to a RelRunner
+ * @param rel the relational expression to prepare; table scans may be rewritten to BindableTableScan before preparation
+ * @return a PreparedStatement that executes the prepared relational expression
+ * @throws UnsupportedOperationException if preparation fails due to WIDTH_BUCKET usage on timestamp fields (indicates an unsupported bins-on-timestamp case)
+ * @throws RuntimeException if a SQLException occurs during preparation and is rethrown as an unchecked exception
*/
public static PreparedStatement run(CalcitePlanContext context, RelNode rel) {
final RelShuttle shuttle =
@@ -369,10 +382,11 @@ public RelNode visit(TableScan scan) {
"The 'bins' parameter on timestamp fields requires: (1) pushdown to be enabled"
+ " (controlled by plugins.calcite.pushdown.enabled, enabled by default), and"
+ " (2) the timestamp field to be used as an aggregation bucket (e.g., 'stats"
- + " count() by @timestamp').");
+ + " count() by @timestamp').",
+ e);
}
throw Util.throwAsRuntime(e);
}
}
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/calcite/utils/OpenSearchTypeFactory.java b/core/src/main/java/org/opensearch/sql/calcite/utils/OpenSearchTypeFactory.java
index 17d99fb4fbb..0ab39487daf 100644
--- a/core/src/main/java/org/opensearch/sql/calcite/utils/OpenSearchTypeFactory.java
+++ b/core/src/main/java/org/opensearch/sql/calcite/utils/OpenSearchTypeFactory.java
@@ -5,9 +5,6 @@
package org.opensearch.sql.calcite.utils;
-import static org.opensearch.sql.calcite.utils.OpenSearchTypeFactory.ExprUDT.EXPR_DATE;
-import static org.opensearch.sql.calcite.utils.OpenSearchTypeFactory.ExprUDT.EXPR_TIME;
-import static org.opensearch.sql.calcite.utils.OpenSearchTypeFactory.ExprUDT.EXPR_TIMESTAMP;
import static org.opensearch.sql.data.type.ExprCoreType.ARRAY;
import static org.opensearch.sql.data.type.ExprCoreType.BINARY;
import static org.opensearch.sql.data.type.ExprCoreType.BOOLEAN;
@@ -45,12 +42,14 @@
import org.apache.calcite.sql.SqlCollation;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.checkerframework.checker.nullness.qual.Nullable;
import org.opensearch.sql.calcite.type.AbstractExprRelDataType;
import org.opensearch.sql.calcite.type.ExprBinaryType;
import org.opensearch.sql.calcite.type.ExprDateType;
import org.opensearch.sql.calcite.type.ExprIPType;
import org.opensearch.sql.calcite.type.ExprTimeStampType;
import org.opensearch.sql.calcite.type.ExprTimeType;
+import org.opensearch.sql.calcite.validate.PplTypeCoercionRule;
import org.opensearch.sql.data.model.ExprValue;
import org.opensearch.sql.data.model.ExprValueUtils;
import org.opensearch.sql.data.type.ExprCoreType;
@@ -257,9 +256,15 @@ public static String getLegacyTypeName(RelDataType relDataType, QueryType queryT
.toUpperCase(Locale.ROOT);
}
- /** Converts a Calcite data type to OpenSearch ExprCoreType. */
+ /**
+ * Maps a Calcite RelDataType to the corresponding OpenSearch ExprType.
+ *
+ * @param type Calcite relational type to convert
+ * @return the corresponding ExprType
+ * @throws IllegalArgumentException if the relational type has no corresponding ExprType
+ */
public static ExprType convertRelDataTypeToExprType(RelDataType type) {
- if (isUserDefinedType(type)) {
+ if (OpenSearchTypeUtil.isUserDefinedType(type)) {
AbstractExprRelDataType> udt = (AbstractExprRelDataType>) type;
return udt.getExprType();
}
@@ -321,7 +326,12 @@ public static RelDataType convertSchema(Table table) {
return TYPE_FACTORY.createStructType(typeList, fieldNameList, true);
}
- /** not in use for now, but let's keep this code for future reference. */
+ /**
+ * Obtain the Java Class corresponding to a given RelDataType, preferring an ExprRelDataType's specific Java type when available.
+ *
+ * @param type the relational type for which to determine the Java Class
+ * @return the Class representing the Java type for the provided RelDataType; if `type` is an ExprRelDataType, returns its declared Java type, otherwise returns the superclass result
+ */
@Override
public Type getJavaClass(RelDataType type) {
if (type instanceof AbstractExprRelDataType> exprRelDataType) {
@@ -331,81 +341,75 @@ public Type getJavaClass(RelDataType type) {
}
/**
- * Whether a given RelDataType is a user-defined type (UDT)
+ * Determine the least-restrictive relational type for a list of types, with special handling for OpenSearch user-defined types (UDTs).
*
- * @param type the RelDataType to check
- * @return true if the type is a user-defined type, false otherwise
- */
- public static boolean isUserDefinedType(RelDataType type) {
- return type instanceof AbstractExprRelDataType>;
- }
-
- /**
- * Checks if the RelDataType represents a numeric type. Supports standard SQL numeric types
- * (INTEGER, BIGINT, SMALLINT, TINYINT, FLOAT, DOUBLE, DECIMAL, REAL), OpenSearch UDT numeric
- * types, and string types (VARCHAR, CHAR).
- *
- * @param fieldType the RelDataType to check
- * @return true if the type is numeric or string, false otherwise
- */
- public static boolean isNumericType(RelDataType fieldType) {
- // Check standard SQL numeric types
- SqlTypeName sqlType = fieldType.getSqlTypeName();
- if (sqlType == SqlTypeName.INTEGER
- || sqlType == SqlTypeName.BIGINT
- || sqlType == SqlTypeName.SMALLINT
- || sqlType == SqlTypeName.TINYINT
- || sqlType == SqlTypeName.FLOAT
- || sqlType == SqlTypeName.DOUBLE
- || sqlType == SqlTypeName.DECIMAL
- || sqlType == SqlTypeName.REAL) {
- return true;
- }
-
- // Check string types (VARCHAR, CHAR)
- if (sqlType == SqlTypeName.VARCHAR || sqlType == SqlTypeName.CHAR) {
- return true;
- }
-
- // Check for OpenSearch UDT numeric types
- if (isUserDefinedType(fieldType)) {
- AbstractExprRelDataType> exprType = (AbstractExprRelDataType>) fieldType;
- ExprType udtType = exprType.getExprType();
- return ExprCoreType.numberTypes().contains(udtType);
- }
-
- return false;
- }
-
- /**
- * Checks if the RelDataType represents a time-based field (timestamp, date, or time). Supports
- * both standard SQL time types (including TIMESTAMP, TIMESTAMP_WITH_LOCAL_TIME_ZONE, DATE, TIME,
- * and their timezone variants) and OpenSearch UDT time types.
+ * When UDTs are present among the candidates, this method prefers an appropriate OpenSearch UDT (date, time, timestamp, ip, or binary)
+ * instead of promoting to VARCHAR. If no UDT-specific resolution applies, or no UDTs are present, it falls back to standard coercion rules.
+ * The method also converts `CHAR(precision)` results to `VARCHAR` to avoid padded results.
*
- * @param fieldType the RelDataType to check
- * @return true if the type is time-based, false otherwise
+ * @param types list of candidate relational types to reconcile
+ * @return the least-restrictive {@link RelDataType} that can represent all input types, or `null` if no common supertype exists
*/
- public static boolean isTimeBasedType(RelDataType fieldType) {
- // Check standard SQL time types
- SqlTypeName sqlType = fieldType.getSqlTypeName();
- if (sqlType == SqlTypeName.TIMESTAMP
- || sqlType == SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE
- || sqlType == SqlTypeName.DATE
- || sqlType == SqlTypeName.TIME
- || sqlType == SqlTypeName.TIME_WITH_LOCAL_TIME_ZONE) {
- return true;
+ @Override
+ public @Nullable RelDataType leastRestrictive(List This method returns true for both Calcite's built-in {@link SqlTypeName#DATE} type and
+ * OpenSearch's user-defined date type {@link OpenSearchTypeFactory.ExprUDT#EXPR_DATE}.
+ *
+ * @param type the type to check
+ * @return true if the type is a date type (built-in or user-defined), false otherwise
+ */
+ public static boolean isDate(RelDataType type) {
+ if (isUserDefinedType(type)) {
+ if (((AbstractExprRelDataType>) type).getUdt() == OpenSearchTypeFactory.ExprUDT.EXPR_DATE) {
+ return true;
+ }
+ }
+ return SqlTypeName.DATE.equals(type.getSqlTypeName());
+ }
+
+ /**
+ * Checks whether a {@link RelDataType} represents a timestamp type.
+ *
+ * This method returns true for both Calcite's built-in {@link SqlTypeName#TIMESTAMP} type and
+ * OpenSearch's user-defined timestamp type {@link OpenSearchTypeFactory.ExprUDT#EXPR_TIMESTAMP}.
+ *
+ * @param type the type to check
+ * @return true if the type is a timestamp type (built-in or user-defined), false otherwise
+ */
+ public static boolean isTimestamp(RelDataType type) {
+ if (isUserDefinedType(type)) {
+ if (((AbstractExprRelDataType>) type).getUdt()
+ == OpenSearchTypeFactory.ExprUDT.EXPR_TIMESTAMP) {
+ return true;
+ }
+ }
+ return SqlTypeName.TIMESTAMP.equals(type.getSqlTypeName());
+ }
+
+ /**
+ * Checks whether a {@link RelDataType} represents a time type.
+ *
+ * This method returns true for both Calcite's built-in {@link SqlTypeName#TIME} type and
+ * OpenSearch's user-defined time type {@link OpenSearchTypeFactory.ExprUDT#EXPR_TIME}.
+ *
+ * @param type the type to check
+ * @return true if the type is a time type (built-in or user-defined), false otherwise
+ */
+ public static boolean isTime(RelDataType type) {
+ if (isUserDefinedType(type)) {
+ if (((AbstractExprRelDataType>) type).getUdt() == OpenSearchTypeFactory.ExprUDT.EXPR_TIME) {
+ return true;
+ }
+ }
+ return SqlTypeName.TIME.equals(type.getSqlTypeName());
+ }
+
+ /**
+ * Determine whether the given RelDataType represents a SQL character type while excluding OpenSearch user-defined types.
+ *
+ * @param type the Calcite RelDataType to check
+ * @return `true` if the type is a SQL character type and not a user-defined OpenSearch type, `false` otherwise
+ */
+ public static boolean isCharacter(RelDataType type) {
+ return !isUserDefinedType(type) && SqlTypeUtil.isCharacter(type);
+ }
+
+ /**
+ * Checks whether a {@link RelDataType} represents an IP address type.
+ *
+ * This method returns true only for OpenSearch's user-defined IP type {@link
+ * OpenSearchTypeFactory.ExprUDT#EXPR_IP}.
+ *
+ * @param type the type to check
+ * @return true if the type is an IP address type, false otherwise
+ */
+ public static boolean isIp(RelDataType type) {
+ if (isUserDefinedType(type)) {
+ return ((AbstractExprRelDataType>) type).getUdt() == OpenSearchTypeFactory.ExprUDT.EXPR_IP;
+ }
+ return false;
+ }
+
+ /**
+ * Checks whether a {@link RelDataType} represents a binary type.
+ *
+ * This method returns true for both Calcite's built-in binary types (BINARY, VARBINARY) and
+ * OpenSearch's user-defined binary type {@link OpenSearchTypeFactory.ExprUDT#EXPR_BINARY}.
+ *
+ * @param type the type to check
+ * @return true if the type is a binary type (built-in or user-defined), false otherwise
+ */
+ public static boolean isBinary(RelDataType type) {
+ if (isUserDefinedType(type)) {
+ return ((AbstractExprRelDataType>) type).getUdt()
+ == OpenSearchTypeFactory.ExprUDT.EXPR_BINARY;
+ }
+ return SqlTypeName.BINARY_TYPES.contains(type.getSqlTypeName());
+ }
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/calcite/utils/binning/BinnableField.java b/core/src/main/java/org/opensearch/sql/calcite/utils/binning/BinnableField.java
index a4e924b631c..37274dc2cb9 100644
--- a/core/src/main/java/org/opensearch/sql/calcite/utils/binning/BinnableField.java
+++ b/core/src/main/java/org/opensearch/sql/calcite/utils/binning/BinnableField.java
@@ -8,7 +8,7 @@
import lombok.Getter;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rex.RexNode;
-import org.opensearch.sql.calcite.utils.OpenSearchTypeFactory;
+import org.opensearch.sql.calcite.utils.OpenSearchTypeUtil;
import org.opensearch.sql.exception.SemanticCheckException;
/** Represents a field that supports binning operations. */
@@ -21,20 +21,20 @@ public class BinnableField {
private final boolean isNumeric;
/**
- * Creates a BinnableField. Validates that the field type is compatible with binning operations.
+ * Create a BinnableField and validate that its type supports binning operations.
*
- * @param fieldExpr The Rex expression for the field
- * @param fieldType The relational data type of the field
- * @param fieldName The name of the field (for error messages)
- * @throws SemanticCheckException if the field type is not supported for binning
+ * @param fieldExpr the Rex expression representing the field
+ * @param fieldType the relational data type of the field
+ * @param fieldName the field name used in error messages
+ * @throws SemanticCheckException if the field type is not supported for binning (neither datetime nor numeric/character)
*/
public BinnableField(RexNode fieldExpr, RelDataType fieldType, String fieldName) {
this.fieldExpr = fieldExpr;
this.fieldType = fieldType;
this.fieldName = fieldName;
- this.isTimeBased = OpenSearchTypeFactory.isTimeBasedType(fieldType);
- this.isNumeric = OpenSearchTypeFactory.isNumericType(fieldType);
+ this.isTimeBased = OpenSearchTypeUtil.isDatetime(fieldType);
+ this.isNumeric = OpenSearchTypeUtil.isNumericOrCharacter(fieldType);
// Reject truly unsupported types (e.g., BOOLEAN, ARRAY, MAP)
if (!isNumeric && !isTimeBased) {
@@ -51,4 +51,4 @@ public BinnableField(RexNode fieldExpr, RelDataType fieldType, String fieldName)
public boolean requiresNumericBinning() {
return !isTimeBased;
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/calcite/validate/OpenSearchSparkSqlDialect.java b/core/src/main/java/org/opensearch/sql/calcite/validate/OpenSearchSparkSqlDialect.java
new file mode 100644
index 00000000000..66c7466d78c
--- /dev/null
+++ b/core/src/main/java/org/opensearch/sql/calcite/validate/OpenSearchSparkSqlDialect.java
@@ -0,0 +1,158 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.sql.calcite.validate;
+
+import com.google.common.collect.ImmutableMap;
+import java.util.Map;
+import lombok.experimental.Delegate;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlAlienSystemTypeNameSpec;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.dialect.SparkSqlDialect;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.opensearch.sql.calcite.utils.OpenSearchTypeUtil;
+
+/**
+ * Custom Spark SQL dialect that extends Calcite's SparkSqlDialect to handle OpenSearch-specific
+ * function translations. This dialect ensures that functions are translated to their correct Spark
+ * SQL equivalents.
+ */
+public class OpenSearchSparkSqlDialect extends SparkSqlDialect {
+
+ /** Singleton instance of the OpenSearch Spark SQL dialect. */
+ public static final OpenSearchSparkSqlDialect DEFAULT = new OpenSearchSparkSqlDialect();
+
+ private static final Map Registers IP-aware convertlets for standard comparison operators so expressions
+ * involving OpenSearch IP types use the corresponding PPL IP operators. Also registers
+ * a special convertlet for the PPL ATAN operator that remaps it to the standard SQL
+ * ATAN operator before conversion.
+ */
+ private PplConvertletTable() {
+ super();
+ registerOperator(SqlStdOperatorTable.EQUALS, ipConvertlet(PPLBuiltinOperators.EQUALS_IP));
+ registerOperator(
+ SqlStdOperatorTable.NOT_EQUALS, ipConvertlet(PPLBuiltinOperators.NOT_EQUALS_IP));
+ registerOperator(
+ SqlStdOperatorTable.GREATER_THAN, ipConvertlet(PPLBuiltinOperators.GREATER_IP));
+ registerOperator(
+ SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, ipConvertlet(PPLBuiltinOperators.GTE_IP));
+ registerOperator(SqlStdOperatorTable.LESS_THAN, ipConvertlet(PPLBuiltinOperators.LESS_IP));
+ registerOperator(
+ SqlStdOperatorTable.LESS_THAN_OR_EQUAL, ipConvertlet(PPLBuiltinOperators.LTE_IP));
+ // There is no implementation for PPLBuiltinOperators.ATAN. It needs to be replaced to
+ // SqlStdOperatorTable.ATAN when converted to RelNode
+ registerOperator(
+ PPLBuiltinOperators.ATAN,
+ (cx, call) -> {
+ ((SqlBasicCall) call).setOperator(SqlStdOperatorTable.ATAN);
+ return StandardConvertletTable.INSTANCE.convertCall(cx, call);
+ });
+ }
+
+ /**
+ * Selects the appropriate convertlet for the given SQL call, preferring registered custom convertlets.
+ *
+ * @param call the SQL call whose operator determines which convertlet to use
+ * @return the custom convertlet registered for the call's operator, or the convertlet from StandardConvertletTable if none was registered; may be `null` if no convertlet is available
+ */
+ @Override
+ public @Nullable SqlRexConvertlet get(SqlCall call) {
+ SqlRexConvertlet custom = map.get(call.getOperator());
+ if (custom != null) return custom;
+ return StandardConvertletTable.INSTANCE.get(call);
+ }
+
+ /**
+ * Registers a convertlet for the specified SQL operator by storing it in the internal map.
+ *
+ * @param op the SQL operator to associate the convertlet with
+ * @param convertlet the convertlet to register for the operator
+ */
+ private void registerOperator(
+ @UnderInitialization PplConvertletTable this, SqlOperator op, SqlRexConvertlet convertlet) {
+ map.put(op, convertlet);
+ }
+
+ /**
+ * Create a convertlet that substitutes a binary operator with the given PPL IP-specific function when either operand is an IP type.
+ *
+ * @param substitute the PPL `SqlFunction` to use as a replacement when an operand is an IP type
+ * @return a `SqlRexConvertlet` that converts the call to `substitute` if either operand's type is IP, otherwise returns the standard converted `RexCall`
+ */
+ private SqlRexConvertlet ipConvertlet(SqlFunction substitute) {
+ return (cx, call) -> {
+ final RexCall e = (RexCall) StandardConvertletTable.INSTANCE.convertCall(cx, call);
+ RelDataType type1 = e.getOperands().get(0).getType();
+ RelDataType type2 = e.getOperands().get(1).getType();
+ if (OpenSearchTypeUtil.isIp(type1) || OpenSearchTypeUtil.isIp(type2)) {
+ return StandardConvertletTable.INSTANCE.convertFunction(cx, substitute, call);
+ }
+ return e;
+ };
+ }
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/calcite/validate/PplTypeCoercion.java b/core/src/main/java/org/opensearch/sql/calcite/validate/PplTypeCoercion.java
new file mode 100644
index 00000000000..f90953454cb
--- /dev/null
+++ b/core/src/main/java/org/opensearch/sql/calcite/validate/PplTypeCoercion.java
@@ -0,0 +1,276 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.sql.calcite.validate;
+
+import static java.util.Objects.requireNonNull;
+import static org.opensearch.sql.calcite.validate.ValidationUtils.createUDTWithAttributes;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+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.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.fun.SqlLibraryOperators;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeMappingRule;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.sql.validate.implicit.TypeCoercion;
+import org.apache.calcite.sql.validate.implicit.TypeCoercionImpl;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.opensearch.sql.calcite.utils.OpenSearchTypeFactory;
+import org.opensearch.sql.calcite.utils.OpenSearchTypeUtil;
+import org.opensearch.sql.data.type.ExprCoreType;
+import org.opensearch.sql.data.type.ExprType;
+import org.opensearch.sql.expression.function.PPLBuiltinOperators;
+
+/**
+ * Custom type coercion implementation for PPL that extends Calcite's default type coercion with
+ * additional restrictions.
+ *
+ * This class implements a blacklist approach to prevent certain implicit type conversions that
+ * are not allowed in PPL semantics.
+ */
+public class PplTypeCoercion extends TypeCoercionImpl {
+
+ /**
+ * Create a PPL-specific TypeCoercion instance usable as a TypeCoercionFactory.
+ *
+ * @param typeFactory the RelDataTypeFactory used to create SQL types
+ * @param validator the SqlValidator used by the coercion
+ * @return the PPL TypeCoercion instance
+ */
+ public static TypeCoercion create(RelDataTypeFactory typeFactory, SqlValidator validator) {
+ return new PplTypeCoercion(typeFactory, validator);
+ }
+
+ /**
+ * Constructs a PplTypeCoercion using the given type factory and SQL validator.
+ *
+ * @param typeFactory factory used to create and manipulate SQL types
+ * @param validator SQL validator used for type resolution and scope information
+ */
+ public PplTypeCoercion(RelDataTypeFactory typeFactory, SqlValidator validator) {
+ super(typeFactory, validator);
+ }
+
+ /**
+ * Determines the implicit cast target for the given input type toward the expected SQL type family,
+ * applying PPL-specific user-defined-type (UDT) rules for datetime and binary targets.
+ *
+ * @param in the input type to be cast
+ * @param expected the expected SQL type family
+ * @return the resulting RelDataType to use for implicit casting, or `null` if no implicit cast is available;
+ * if no standard implicit cast exists and `in` is a character type while `expected` is DATETIME,
+ * returns a timestamp UDT derived from `in`; if a standard implicit cast yields DATE, TIME,
+ * TIMESTAMP, or BINARY, returns a UDT with attributes based on that target type and the original input;
+ * otherwise returns the standard cast target type
+ */
+ @Override
+ public @Nullable RelDataType implicitCast(RelDataType in, SqlTypeFamily expected) {
+ RelDataType casted = super.implicitCast(in, expected);
+ if (casted == null) {
+ // String -> DATETIME is converted to String -> TIMESTAMP
+ if (OpenSearchTypeUtil.isCharacter(in) && expected == SqlTypeFamily.DATETIME) {
+ return createUDTWithAttributes(factory, in, OpenSearchTypeFactory.ExprUDT.EXPR_TIMESTAMP);
+ }
+ return null;
+ }
+ return switch (casted.getSqlTypeName()) {
+ case SqlTypeName.DATE, SqlTypeName.TIME, SqlTypeName.TIMESTAMP, SqlTypeName.BINARY ->
+ createUDTWithAttributes(factory, in, casted.getSqlTypeName());
+ default -> casted;
+ };
+ }
+
+ /**
+ * Determines whether the given node must be cast to the target type, with special handling for user-defined types.
+ *
+ * Forces a cast when the target type is a user-defined type and the node's inferred type is a character type;
+ * in all other cases, returns the default decision.
+ *
+ * @param scope validation scope for deriving the node's type
+ * @param node expression to check for casting necessity
+ * @param toType target type to which the node may be cast
+ * @param mappingRule rule guiding type mapping decisions
+ * @return `true` if the node must be cast to `toType`, `false` otherwise
+ */
+ @Override
+ protected boolean needToCast(
+ SqlValidatorScope scope, SqlNode node, RelDataType toType, SqlTypeMappingRule mappingRule) {
+ boolean need = super.needToCast(scope, node, toType, mappingRule);
+ RelDataType fromType = validator.deriveType(scope, node);
+ if (OpenSearchTypeUtil.isUserDefinedType(toType) && OpenSearchTypeUtil.isCharacter(fromType)) {
+ need = true;
+ }
+ return need;
+ }
+
+ /**
+ * Coerces a string operand to a datetime user-defined type when comparing string and datetime operands.
+ *
+ * @param binding binding providing the call and scope for coercion
+ * @param left left operand type of the comparison
+ * @param right right operand type of the comparison
+ * @return `true` if the character operand was coerced to a datetime UDT, `false` otherwise
+ */
+ @Override
+ protected boolean dateTimeStringEquality(
+ SqlCallBinding binding, RelDataType left, RelDataType right) {
+ if (OpenSearchTypeUtil.isCharacter(left) && OpenSearchTypeUtil.isDatetime(right)) {
+ // Use user-defined types in place of inbuilt datetime types
+ RelDataType r =
+ OpenSearchTypeUtil.isUserDefinedType(right)
+ ? right
+ : ValidationUtils.createUDTWithAttributes(factory, right, right.getSqlTypeName());
+ return coerceOperandType(binding.getScope(), binding.getCall(), 0, r);
+ }
+ if (OpenSearchTypeUtil.isCharacter(right) && OpenSearchTypeUtil.isDatetime(left)) {
+ RelDataType l =
+ OpenSearchTypeUtil.isUserDefinedType(left)
+ ? left
+ : ValidationUtils.createUDTWithAttributes(factory, left, left.getSqlTypeName());
+ return coerceOperandType(binding.getScope(), binding.getCall(), 1, l);
+ }
+ return false;
+ }
+
+ /**
+ * Determine the common type used for binary comparisons between two operand types,
+ * applying PPL-specific rules for datetime and IP/string combinations.
+ *
+ * Special rules:
+ * - date with time (in either order) -> TIMESTAMP
+ * - time with timestamp -> TIMESTAMP (preserving the timestamp side)
+ * - IP with character/string (in either order) -> IP
+ *
+ * Nullability of the returned type is set to "nullable" if either input type is nullable.
+ *
+ * @param type1 the left operand type, or {@code null} if unknown
+ * @param type2 the right operand type, or {@code null} if unknown
+ * @return the resolved common type for comparison according to the above rules,
+ * or the superclass's common type resolution when no rule applies
+ */
+ @Override
+ public @Nullable RelDataType commonTypeForBinaryComparison(
+ @Nullable RelDataType type1, @Nullable RelDataType type2) {
+ // Prepend following rules for datetime comparisons:
+ // - (date, time) -> timestamp
+ // - (time, timestamp) -> timestamp
+ // - (ip, string) -> ip
+ if (type1 != null && type2 != null) {
+ boolean anyNullable = type1.isNullable() || type2.isNullable();
+ if ((SqlTypeUtil.isDate(type1) && OpenSearchTypeUtil.isTime(type2))
+ || (OpenSearchTypeUtil.isTime(type1) && SqlTypeUtil.isDate(type2))) {
+ return factory.createTypeWithNullability(
+ factory.createSqlType(SqlTypeName.TIMESTAMP), anyNullable);
+ }
+ if (OpenSearchTypeUtil.isTime(type1) && SqlTypeUtil.isTimestamp(type2)) {
+ return factory.createTypeWithNullability(type2, anyNullable);
+ }
+ if (SqlTypeUtil.isTimestamp(type1) && OpenSearchTypeUtil.isTime(type2)) {
+ return factory.createTypeWithNullability(type1, anyNullable);
+ }
+ if (OpenSearchTypeUtil.isIp(type1) && OpenSearchTypeUtil.isCharacter(type2)) {
+ return factory.createTypeWithNullability(type1, anyNullable);
+ }
+ if (OpenSearchTypeUtil.isCharacter(type1) && OpenSearchTypeUtil.isIp(type2)) {
+ return factory.createTypeWithNullability(type2, anyNullable);
+ }
+ }
+ return super.commonTypeForBinaryComparison(type1, type2);
+ }
+
+ /**
+ * Attempts to coerce the operand at the given index of the call to the specified target type and,
+ * if coercion is applied, replaces the operand with the corresponding cast expression.
+ *
+ * @param scope the validator scope used to derive the operand's current type; must not be null
+ * @param call the SqlCall containing the operand to coerce
+ * @param index the zero-based index of the operand within the call to coerce
+ * @param targetType the desired target SQL type for the operand
+ * @return `true` if the operand was coerced and replaced in the call, `false` otherwise
+ *
+ * Notes:
+ * - Dynamic parameters are not coerced; the method returns `false` for SqlDynamicParam operands.
+ * - If no cast is necessary according to coercion rules, the method returns `false`. This class defines additional type mapping rules for PPL, particularly for handling custom
+ * types like IP addresses and number-to-string coercion. These additional rules are merged with
+ * Calcite's built-in type coercion rules.
+ *
+ * The additional mappings defined include:
+ *
+ * Three variants of type coercion rules are provided:
+ *
+ * This validator extends Calcite's default SqlValidatorImpl to provide PPL-specific validation
+ * behavior. Currently, it uses the default implementation but can be extended in the future to add
+ * PPL-specific validation rules.
+ */
+public class PplValidator extends SqlValidatorImpl {
+ /**
+ * Tracks whether the current deriveType call is at the top level (true) or a recursive call
+ * (false). Top-level calls return user-defined types, while recursive calls return SQL types for
+ * internal validation.
+ */
+ private boolean top;
+
+ /**
+ * Create a PPL-specific SQL validator configured with the given catalog, operator table, type factory, and validator settings.
+ *
+ * @param statement Calcite server statement used to build a prepare context and resolve the root schema
+ * @param frameworkConfig Framework configuration providing the default schema
+ * @param operatorTable SQL operator table to use for validation
+ * @param typeFactory type factory for constructing SQL types and catalog reader
+ * @param validatorConfig validator configuration to pass to the created PPL validator
+ * @return a configured PplValidator instance
+ */
+ public static PplValidator create(
+ CalciteServerStatement statement,
+ FrameworkConfig frameworkConfig,
+ SqlOperatorTable operatorTable,
+ RelDataTypeFactory typeFactory,
+ SqlValidator.Config validatorConfig) {
+ SchemaPlus defaultSchema = frameworkConfig.getDefaultSchema();
+
+ final CalcitePrepare.Context prepareContext = statement.createPrepareContext();
+ final CalciteSchema schema =
+ defaultSchema != null ? CalciteSchema.from(defaultSchema) : prepareContext.getRootSchema();
+ CalciteCatalogReader catalogReader =
+ new CalciteCatalogReader(
+ schema.root(), schema.path(null), typeFactory, prepareContext.config());
+ return new PplValidator(operatorTable, catalogReader, typeFactory, validatorConfig);
+ }
+
+ /**
+ * Constructs a PPL-specific SqlValidator with the given operator table, catalog reader, type factory, and configuration,
+ * and initializes the validator to treat subsequent type derivations as top-level.
+ *
+ * @param opTab operator table containing PPL operators
+ * @param catalogReader catalog reader used to resolve schema and object information
+ * @param typeFactory factory used to create and manipulate SQL type instances
+ * @param config validator configuration options
+ */
+ protected PplValidator(
+ SqlOperatorTable opTab,
+ SqlValidatorCatalogReader catalogReader,
+ RelDataTypeFactory typeFactory,
+ Config config) {
+ super(opTab, catalogReader, typeFactory, config);
+ top = true;
+ }
+
+ /**
+ * Derives the type of a SQL expression while mapping between SQL types and OpenSearch user-defined types (UDTs) for PPL validation.
+ *
+ * @param scope the validation scope in which the expression is resolved
+ * @param expr the SQL expression whose type is being derived
+ * @return the derived RelDataType; at top-level derivations this is converted to a user-defined type, otherwise it is converted to a SQL type used for internal validation
+ */
+ @Override
+ public RelDataType deriveType(SqlValidatorScope scope, SqlNode expr) {
+ // The type has to be sql type during type derivation & validation
+ boolean original = top;
+ top = false;
+ RelDataType type = super.deriveType(scope, expr);
+ top = original;
+ if (top) {
+ return sqlTypeToUserDefinedType(type);
+ }
+ return userDefinedTypeToSqlType(type);
+ }
+
+ /**
+ * Get the validated type for a SQL node, mapped to OpenSearch user-defined types.
+ *
+ * @param node the SQL node whose validated type to retrieve
+ * @return the node's validated {@code RelDataType} with SQL types converted to OpenSearch user-defined types, or {@code null} if the type is unknown
+ */
+ @Override
+ public @Nullable RelDataType getValidatedNodeTypeIfKnown(SqlNode node) {
+ RelDataType type = super.getValidatedNodeTypeIfKnown(node);
+ return sqlTypeToUserDefinedType(type);
+ }
+
+ /**
+ * Disables nullary calls to avoid treating identifiers like field references as function calls.
+ *
+ * PPL does not use functions without parentheses (nullary functions). Allowing nullary calls
+ * would cause identifiers such as USER or LOCALTIME to be resolved as function calls before being
+ * treated as field references.
+ *
+ * @see SqlValidatorImpl.Expander#visit(SqlIdentifier)
+ * @return always {@code null}
+ */
+ @Override
+ public @Nullable SqlCall makeNullaryCall(SqlIdentifier id) {
+ return null;
+ }
+
+ /**
+ * Converts OpenSearch user-defined RelDataType instances to their equivalent SQL types.
+ *
+ * Maps UDTs for timestamp, time, date, and binary to the corresponding SQL types preserving
+ * nullability; maps the OpenSearch IP UDT to the nullable IP UDT constant; returns the input
+ * type unchanged if no mapping applies.
+ *
+ * @param type the input RelDataType, possibly a user-defined type
+ * @return the corresponding SQL RelDataType or the original type if no conversion is required
+ */
+ private RelDataType userDefinedTypeToSqlType(RelDataType type) {
+ return convertType(
+ type,
+ t -> {
+ if (OpenSearchTypeUtil.isUserDefinedType(t)) {
+ AbstractExprRelDataType> exprType = (AbstractExprRelDataType>) t;
+ ExprType udtType = exprType.getExprType();
+ OpenSearchTypeFactory typeFactory = (OpenSearchTypeFactory) this.getTypeFactory();
+ return switch (udtType) {
+ case ExprCoreType.TIMESTAMP ->
+ typeFactory.createSqlType(SqlTypeName.TIMESTAMP, t.isNullable());
+ case ExprCoreType.TIME -> typeFactory.createSqlType(SqlTypeName.TIME, t.isNullable());
+ case ExprCoreType.DATE -> typeFactory.createSqlType(SqlTypeName.DATE, t.isNullable());
+ case ExprCoreType.BINARY ->
+ typeFactory.createSqlType(SqlTypeName.BINARY, t.isNullable());
+ case ExprCoreType.IP -> UserDefinedFunctionUtils.NULLABLE_IP_UDT;
+ default -> t;
+ };
+ }
+ return t;
+ });
+ }
+
+ /**
+ * Convert SQL types to OpenSearch user-defined types (UDTs) where applicable.
+ *
+ * Recursively traverses the input type and replaces SQL TIMESTAMP, TIME, DATE, and BINARY
+ * types with corresponding OpenSearch UDTs that carry the original type's attributes; all other
+ * types are preserved.
+ *
+ * @param type the input type to convert; may be a complex/compound RelDataType
+ * @return a RelDataType with SQL temporal and binary types mapped to OpenSearch UDTs, or the
+ * original type structure if no mappings apply
+ */
+ private RelDataType sqlTypeToUserDefinedType(RelDataType type) {
+ return convertType(
+ type,
+ t -> {
+ OpenSearchTypeFactory typeFactory = (OpenSearchTypeFactory) this.getTypeFactory();
+ return switch (t.getSqlTypeName()) {
+ case TIMESTAMP -> createUDTWithAttributes(typeFactory, t, ExprUDT.EXPR_TIMESTAMP);
+ case TIME -> createUDTWithAttributes(typeFactory, t, ExprUDT.EXPR_TIME);
+ case DATE -> createUDTWithAttributes(typeFactory, t, ExprUDT.EXPR_DATE);
+ case BINARY -> createUDTWithAttributes(typeFactory, t, ExprUDT.EXPR_BINARY);
+ default -> t;
+ };
+ });
+ }
+
+ /**
+ * Recursively applies a conversion function to a RelDataType and its nested component types, preserving structure and nullability.
+ *
+ * For record types, each field type is converted and a new struct is created with the original field
+ * names and the record's nullability. For array, map, and multiset types, their component/key/value
+ * types are converted and the original collection nullability is preserved. For other types, the
+ * conversion function is applied directly.
+ *
+ * @param type the input type to convert; may be {@code null}
+ * @param convert function that maps a RelDataType to another RelDataType
+ * @return the converted RelDataType with the same composite structure and nullability as {@code type},
+ * or {@code null} if {@code type} is {@code null}
+ */
+ private RelDataType convertType(RelDataType type, Function This interface breaks the circular dependency between core and opensearch modules by allowing
+ * the opensearch module to provide its operator table implementation to the core module through
+ * dependency injection.
+ */
+@FunctionalInterface
+public interface SqlOperatorTableProvider {
+ /**
+ * Provides the SqlOperatorTable used for SQL validation and query processing.
+ *
+ * @return the SqlOperatorTable instance used for validation and query processing
+ */
+ SqlOperatorTable getOperatorTable();
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/calcite/validate/ValidationUtils.java b/core/src/main/java/org/opensearch/sql/calcite/validate/ValidationUtils.java
new file mode 100644
index 00000000000..0397a5bf40b
--- /dev/null
+++ b/core/src/main/java/org/opensearch/sql/calcite/validate/ValidationUtils.java
@@ -0,0 +1,113 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.sql.calcite.validate;
+
+import java.nio.charset.Charset;
+import java.util.List;
+import lombok.experimental.UtilityClass;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.type.NonNullableAccessors;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.opensearch.sql.calcite.utils.OpenSearchTypeFactory;
+
+@UtilityClass
+public class ValidationUtils {
+ /**
+ * Copy nullability, and for character or binary types also charset and collation, from a source
+ * type onto a target type.
+ *
+ * @param factory the type factory used to create adjusted types
+ * @param fromType the source type whose attributes (nullability, charset, collation) will be copied;
+ * may be null, in which case the target type is returned unchanged
+ * @param toType the target type to receive attributes from {@code fromType}
+ * @return the resulting {@link RelDataType} based on {@code toType} with attributes synchronized
+ * from {@code fromType}
+ */
+ public static RelDataType syncAttributes(
+ RelDataTypeFactory factory, RelDataType fromType, RelDataType toType) {
+ RelDataType syncedType = toType;
+ if (fromType != null) {
+ syncedType = factory.createTypeWithNullability(syncedType, fromType.isNullable());
+ if (SqlTypeUtil.inCharOrBinaryFamilies(fromType)
+ && SqlTypeUtil.inCharOrBinaryFamilies(toType)) {
+ Charset charset = fromType.getCharset();
+ if (charset != null && SqlTypeUtil.inCharFamily(syncedType)) {
+ SqlCollation collation = NonNullableAccessors.getCollation(fromType);
+ syncedType = factory.createTypeWithCharsetAndCollation(syncedType, charset, collation);
+ }
+ }
+ }
+ return syncedType;
+ }
+
+ /**
+ * Create a user-defined RelDataType and copy nullability, charset, and collation from another type.
+ *
+ * @param factory the type factory used to create the UDT; must be an instance of OpenSearchTypeFactory
+ * @param fromType the source type whose nullability, charset, and collation will be copied (may be null)
+ * @param userDefinedType the expression-defined UDT to create
+ * @return the created UDT with attributes copied from {@code fromType}
+ * @throws IllegalArgumentException if {@code factory} is not an instance of OpenSearchTypeFactory
+ */
+ public static RelDataType createUDTWithAttributes(
+ RelDataTypeFactory factory,
+ RelDataType fromType,
+ OpenSearchTypeFactory.ExprUDT userDefinedType) {
+ if (!(factory instanceof OpenSearchTypeFactory typeFactory)) {
+ throw new IllegalArgumentException("factory must be an instance of OpenSearchTypeFactory");
+ }
+ RelDataType type = typeFactory.createUDT(userDefinedType);
+ return syncAttributes(typeFactory, fromType, type);
+ }
+
+ /**
+ * Creates a user-defined type by mapping a SQL type name to the corresponding UDT, with
+ * attributes copied from another type.
+ *
+ * @param factory the type factory used to create the UDT
+ * @param fromType the source type to copy attributes from
+ * @param sqlTypeName the SQL type name to map to a UDT (DATE, TIME, TIMESTAMP, or BINARY)
+ * @return a new RelDataType representing the UDT with attributes from fromType
+ * @throws IllegalArgumentException if the sqlTypeName is not supported
+ */
+ public static RelDataType createUDTWithAttributes(
+ RelDataTypeFactory factory, RelDataType fromType, SqlTypeName sqlTypeName) {
+ return switch (sqlTypeName) {
+ case SqlTypeName.DATE ->
+ createUDTWithAttributes(factory, fromType, OpenSearchTypeFactory.ExprUDT.EXPR_DATE);
+ case SqlTypeName.TIME ->
+ createUDTWithAttributes(factory, fromType, OpenSearchTypeFactory.ExprUDT.EXPR_TIME);
+ case SqlTypeName.TIMESTAMP ->
+ createUDTWithAttributes(factory, fromType, OpenSearchTypeFactory.ExprUDT.EXPR_TIMESTAMP);
+ case SqlTypeName.BINARY ->
+ createUDTWithAttributes(factory, fromType, OpenSearchTypeFactory.ExprUDT.EXPR_BINARY);
+ default -> throw new IllegalArgumentException("Unsupported type: " + sqlTypeName);
+ };
+ }
+
+ /**
+ * Determines whether an exception matches known Calcite v1.41 validation errors related to
+ * nested window functions and window functions used as function operands.
+ *
+ * Used to tolerate specific Calcite validation failures (for example, incorrect detection of
+ * window functions inside CASE expressions or failure to push projections containing OVER into
+ * subqueries) that are treated as non-fatal/workaroundable within this codebase.
+ *
+ * @param e the exception whose message will be inspected for known Calcite error patterns
+ * @return {@code true} if the exception's message contains a known Calcite validation error, {@code false} otherwise
+ */
+ public static boolean tolerantValidationException(Exception e) {
+ List This converter is used during the validation phase to convert RelNode back to SqlNode for
+ * validation and type checking using Calcite's SqlValidator.
+ */
+public class PplRelToSqlNodeConverter extends RelToSqlConverter {
+ /**
+ * Initializes a RelToSqlConverter customized for PPL using the specified SQL dialect.
+ *
+ * @param dialect the SQL dialect to apply for SQL generation and dialect-specific behavior
+ */
+ public PplRelToSqlNodeConverter(SqlDialect dialect) {
+ super(dialect);
+ }
+
+ /**
+ * Propagates a Correlate node's join type into the generated SQL when the FROM clause is a SqlJoin.
+ *
+ * If the correlate's join type is not INNER and the result's FROM is a SqlJoin, attempts to map
+ * the relational join type to a SQL join type and apply it to the SqlJoin. When applied, the join
+ * condition is set to ON TRUE so SQL syntax remains valid (the actual correlation predicate is kept
+ * inside the subquery's WHERE clause). If the join type cannot be mapped, the result is returned
+ * unchanged.
+ *
+ * @return the original or modified Result reflecting the correlate's join type in the SQL
+ */
+ @Override
+ public Result visit(Correlate e) {
+ Result result = super.visit(e);
+ SqlNode from = result.asSelect().getFrom();
+ if (e.getJoinType() != JoinRelType.INNER && from instanceof SqlJoin join) {
+ JoinType joinType;
+ try {
+ joinType = JoinType.valueOf(e.getJoinType().name());
+ } catch (IllegalArgumentException ignored) {
+ return result;
+ }
+ join.setOperand(2, joinType.symbol(POS));
+ // INNER, LEFT, RIGHT, FULL, or ASOF join requires a condition
+ // Use ON TRUE to satisfy SQL syntax because the actual correlation condition logic is inside
+ // the subquery's WHERE clause
+ join.setOperand(4, JoinConditionType.ON.symbol(POS));
+ join.setOperand(5, SqlLiteral.createBoolean(true, POS));
+ }
+ return result;
+ }
+
+ /**
+ * Translate SEMI and ANTI joins into Spark SQL native LEFT_SEMI_JOIN or LEFT_ANTI_JOIN
+ * semantics by producing a SqlJoin with an ON condition instead of EXISTS/NOT EXISTS subqueries.
+ *
+ * This avoids a Calcite issue where correlation references inside generated subqueries can
+ * become unqualified and resolve to the wrong scope.
+ *
+ * @return a Result whose FROM node is a SqlJoin using LEFT_SEMI_JOIN or LEFT_ANTI_JOIN with the
+ * converted join condition, and which composes the left and right input results
+ */
+ @Override
+ protected Result visitAntiOrSemiJoin(Join e) {
+ final Result leftResult = visitInput(e, 0).resetAlias();
+ final Result rightResult = visitInput(e, 1).resetAlias();
+ final Context leftContext = leftResult.qualifiedContext();
+ final Context rightContext = rightResult.qualifiedContext();
+
+ JoinType joinType =
+ e.getJoinType() == JoinRelType.ANTI ? JoinType.LEFT_ANTI_JOIN : JoinType.LEFT_SEMI_JOIN;
+ SqlNode sqlCondition = convertConditionToSqlNode(e.getCondition(), leftContext, rightContext);
+ SqlNode join =
+ new SqlJoin(
+ POS,
+ leftResult.asFrom(),
+ SqlLiteral.createBoolean(false, POS),
+ joinType.symbol(POS), // LEFT ANTI JOIN or LEFT SEMI JOIN
+ rightResult.asFrom(),
+ JoinConditionType.ON.symbol(POS),
+ sqlCondition);
+
+ return result(join, leftResult, rightResult);
+ }
+
+ /**
+ * Converts an Aggregate relational node to SQL and attaches any relational hints to the resulting SELECT node.
+ *
+ * If the aggregate contains RelHint entries, they are converted to SqlHint instances and set on the Result's
+ * SELECT node as a hint list.
+ *
+ * @param e the Aggregate relational node to convert
+ * @return the conversion Result whose SELECT node includes converted hints when present
+ */
+ @Override
+ public Result visit(Aggregate e) {
+ Result r = super.visit(e);
+ if (!e.getHints().isEmpty()) {
+ List The default implementation in {@link SqlToRelConverter#convertJoinType} does not expect
+ * LEFT_ANTI_JOIN and LEFT_SEMI_JOIN. This override works around the limitation by first
+ * temporarily changing LEFT_ANTI_JOIN/LEFT_SEMI_JOIN to LEFT join in the SqlJoin node, then
+ * calling {@code super.convertFrom()} to perform normal conversion, finally substituting the join
+ * type in the resulting RelNode to ANTI/SEMI.
+ *
+ * @param bb Scope within which to resolve identifiers
+ * @param from FROM clause of a query.
+ * @param fieldNames Field aliases, usually come from AS clause, or null
+ */
+ @Override
+ protected void convertFrom(
+ Blackboard bb, @Nullable SqlNode from, @Nullable List This shuttle extends RelShuttleImpl to ensure it visits the entire RelNode tree recursively,
+ * applying the interval literal fixes at each node.
+ */
+public class PplRelToSqlRelShuttle extends RelShuttleImpl {
+ private final RexShuttle rexShuttle;
+
+ /**
+ * Constructs a shuttle that visits relational expressions and applies fixes to literal
+ * expressions during conversion to SQL.
+ *
+ * The created shuttle contains an internal {@code RexShuttle} that ensures float/real
+ * literals are emitted with an explicit cast and that interval literals are adjusted
+ * according to their interval qualifiers.
+ *
+ * @param rexBuilder builder used to create replacement {@code RexNode} instances
+ * @param forward if {@code true}, interval literal values are converted forward (multiplied
+ * by the unit multiplier); if {@code false}, values are converted backward
+ * (divided by the unit multiplier)
+ */
+ public PplRelToSqlRelShuttle(RexBuilder rexBuilder, boolean forward) {
+ this.rexShuttle =
+ new RexShuttle() {
+ /**
+ * This visitor fixes: 1. float literal: when converting logical plan to sql node, float
+ * information is missing. All floats will be treated as double. A compulsory cast is
+ * inserted here to ensure a cast presents in the generated SQL 2. interval literal: we
+ * create and read the interval literal in a different way that how Calcite originally
+ * expected it to be.
+ */
+ @Override
+ public RexNode visitLiteral(RexLiteral literal) {
+ // 1. Fix float literal
+ SqlTypeName literalType = literal.getType().getSqlTypeName();
+ if (SqlTypeName.REAL.equals(literalType) || SqlTypeName.FLOAT.equals(literalType)) {
+ return rexBuilder.makeCall(
+ literal.getType(), SqlLibraryOperators.SAFE_CAST, List.of(literal));
+ }
+
+ // 2. Fix interval literal
+ SqlIntervalQualifier qualifier = literal.getType().getIntervalQualifier();
+ if (qualifier == null) {
+ return literal;
+ }
+ BigDecimal value = literal.getValueAs(BigDecimal.class);
+ if (value == null) {
+ return literal;
+ }
+ TimeUnit unit = qualifier.getUnit();
+ // An ad-hoc fix to a Calcite bug in RexLiteral#intervalString -- quarter type does not
+ // exist in SqlTypeName, rendering it return number of months instead of number of
+ // quarters.
+ BigDecimal forwardMultiplier =
+ TimeUnit.QUARTER.equals(unit) ? BigDecimal.valueOf(1) : unit.multiplier;
+
+ // QUARTER intervals are stored as INTERVAL_MONTH in Calcite's type system
+ // but the qualifier preserves the actual unit (QUARTER vs MONTH).
+ // The multiplier for QUARTER is 3 (months), for MONTH is 1.
+ BigDecimal newValue =
+ forward
+ ? value.multiply(forwardMultiplier)
+ : value.divideToIntegralValue(unit.multiplier);
+ return rexBuilder.makeIntervalLiteral(newValue, qualifier);
+ }
+ };
+ }
+
+ /**
+ * Visit the specified child relational node and apply the configured RexShuttle to its expression tree.
+ *
+ * @return the child RelNode after visiting, with its RexNode expressions transformed by the internal RexShuttle
+ */
+ @Override
+ protected RelNode visitChild(RelNode parent, int i, RelNode child) {
+ RelNode newChild = super.visitChild(parent, i, child);
+ return newChild.accept(rexShuttle);
+ }
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/calcite/validate/shuttles/SkipRelValidationShuttle.java b/core/src/main/java/org/opensearch/sql/calcite/validate/shuttles/SkipRelValidationShuttle.java
new file mode 100644
index 00000000000..ab93080f4e3
--- /dev/null
+++ b/core/src/main/java/org/opensearch/sql/calcite/validate/shuttles/SkipRelValidationShuttle.java
@@ -0,0 +1,205 @@
+/*
+ * Copyright OpenSearch Contributors
+ * SPDX-License-Identifier: Apache-2.0
+ */
+
+package org.opensearch.sql.calcite.validate.shuttles;
+
+import java.util.List;
+import java.util.function.Predicate;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.sql.SqlKind;
+import org.opensearch.sql.calcite.utils.OpenSearchTypeUtil;
+
+/**
+ * A RelShuttle that detects if validation should be skipped for certain operations. Currently, it
+ * detects the following patterns:
+ *
+ * When grouping by multiple CASE expressions, a Calcite 1.41 bug causes field references to
+ * become invalid during SQL-to-Rel conversion. This affects queries in {@code
+ * testCaseCanBePushedDownAsCompositeRangeQuery} 2.4 and {@code testCaseCanBePushedDownAsRangeQuery}
+ * 1.3. E.g. for the following query:
+ *
+ * During validation, this PPL query is converted to SQL:
+ *
+ * When Calcite converts this SQL back to RelNode, it processes GROUP BY expressions
+ * sequentially, making field references in the second CASE expression invalid.
+ *
+ * Generate empty row with LogicalValues
+ *
+ * Types in the rows generated with {@code VALUES} will not be preserved, causing validation
+ * issues when converting SQL back to a logical plan.
+ *
+ * For example, in {@code CalcitePPLAggregationIT.testSumEmpty}, the query {@code
+ * source=opensearch-sql_test_index_bank_with_null_values | where 1=2 | stats sum(balance)} will be
+ * converted to the following SQL:
+ *
+ * Note for developers: when validations fail during developing new features, please try
+ * to solve the root cause instead of adding skipping rules here. Under rare cases when you have to
+ * skip validation, please document the exact reason.
+ *
+ * WARNING: When a skip pattern is detected, we bypass the entire validation pipeline,
+ * skipping potentially useful transformation relying on rewriting SQL node
+ */
+public class SkipRelValidationShuttle extends RelShuttleImpl {
+ private boolean shouldSkip = false;
+ private final RexShuttle rexShuttle;
+
+ /** Predicates about patterns of calls that should not be validated. */
+ public static final List The installed RexShuttle tests each visited {@code RexCall} against the {@code SKIP_CALLS} predicates;
+ * when a predicate matches, {@code shouldSkip} is set to {@code true} and the original call is returned.
+ */
+ public SkipRelValidationShuttle() {
+ this.rexShuttle =
+ new RexShuttle() {
+ @Override
+ public RexNode visitCall(RexCall call) {
+ for (Predicate Performs two targeted rewrites:
+ * If Calcite-based explanation fails and fallback is allowed by configuration or by the
+ * exception type, the method delegates to the legacy explain path and forwards the original
+ * Calcite failure. Otherwise the listener is notified of the failure (Calcite errors are wrapped
+ * as {@code CalciteUnsupportedException} for {@code Error} instances).
+ *
+ * @param plan the unresolved query plan to explain
+ * @param queryType the type of the query (used to select planning behavior)
+ * @param listener listener to receive the explain response or failure
+ * @param format the desired explain output format
+ */
public void explainWithCalcite(
UnresolvedPlan plan,
QueryType queryType,
@@ -144,7 +187,8 @@ public void explainWithCalcite(
() -> {
RelNode relNode = analyze(plan, context);
relNode = mergeAdjacentFilters(relNode);
- RelNode optimized = optimize(relNode, context);
+ RelNode validated = validate(relNode, context);
+ RelNode optimized = optimize(validated, context);
RelNode calcitePlan = convertToCalcitePlan(optimized);
executionEngine.explain(calcitePlan, format, context, listener);
},
@@ -266,11 +310,80 @@ private RelNode mergeAdjacentFilters(RelNode relNode) {
return planner.findBestExp();
}
- /** Analyze {@link UnresolvedPlan}. */
+ /**
+ * Converts an UnresolvedPlan into a LogicalPlan using the provided query type.
+ *
+ * @param plan the unresolved query plan to analyze
+ * @param queryType the type of the query which influences analysis rules
+ * @return the analyzed LogicalPlan
+ */
public LogicalPlan analyze(UnresolvedPlan plan, QueryType queryType) {
return analyzer.analyze(plan, new AnalysisContext(queryType));
}
+ /**
+ * Perform Calcite validation on a relational expression and return a relational expression
+ * that reflects any type coercions or other adjustments introduced by validation.
+ *
+ * This enables Calcite's type validation and implicit casting behavior for PPL queries.
+ *
+ * @param relNode the relational expression to validate; may be returned unchanged if validation is skipped
+ * or if tolerant validation detects issues that should not be escalated
+ * @param context the Calcite plan context supplying the validator, relBuilder, and conversion/configuration
+ * used during validation
+ * @return a RelNode that incorporates any validation-driven modifications (or the original relNode when
+ * validation is skipped or tolerated)
+ */
+ private RelNode validate(RelNode relNode, CalcitePlanContext context) {
+ SkipRelValidationShuttle skipShuttle = new SkipRelValidationShuttle();
+ relNode.accept(skipShuttle);
+ if (skipShuttle.shouldSkipValidation()) {
+ return relNode;
+ }
+ // Fix interval literals before conversion to SQL
+ RelNode sqlRelNode = relNode.accept(new PplRelToSqlRelShuttle(context.rexBuilder, true));
+
+ // Convert RelNode to SqlNode for validation
+ RelToSqlConverter rel2sql = new PplRelToSqlNodeConverter(OpenSearchSparkSqlDialect.DEFAULT);
+ SqlImplementor.Result result = rel2sql.visitRoot(sqlRelNode);
+ SqlNode root = result.asStatement();
+
+ // Rewrite SqlNode to remove database qualifiers
+ SqlNode rewritten = root.accept(new SqlRewriteShuttle());
+ SqlValidator validator = context.getValidator();
+ try {
+ validator.validate(Objects.requireNonNull(rewritten));
+ } catch (CalciteContextException e) {
+ if (ValidationUtils.tolerantValidationException(e)) {
+ return relNode;
+ }
+ throw new ExpressionEvaluationException(e.getMessage(), e);
+ }
+
+ SqlToRelConverter.Config sql2relConfig =
+ SqlToRelConverter.config()
+ // Do not remove sort in subqueries so that the orders for queries like `... | sort a
+ // | fields b` is preserved
+ .withRemoveSortInSubQuery(false)
+ // Disable automatic JSON_TYPE_OPERATOR wrapping for nested JSON functions.
+ // See CALCITE-4989: Calcite wraps nested JSON functions with JSON_TYPE by default
+ .withAddJsonTypeOperatorEnabled(false)
+ // Set hint strategy so that hints can be properly propagated.
+ // See SqlToRelConverter.java#convertSelectImpl
+ .withHintStrategyTable(PPLHintStrategyTable.getHintStrategyTable());
+ SqlToRelConverter sql2rel =
+ new PplSqlToRelConverter(
+ context.config.getViewExpander(),
+ validator,
+ validator.getCatalogReader().unwrap(CalciteCatalogReader.class),
+ context.relBuilder.getCluster(),
+ PplConvertletTable.INSTANCE,
+ sql2relConfig);
+ // Convert the validated SqlNode back to RelNode
+ RelNode validatedRel = sql2rel.convertQuery(rewritten, false, true).project();
+ return validatedRel.accept(new PplRelToSqlRelShuttle(context.rexBuilder, false));
+ }
+
/** Translate {@link LogicalPlan} to {@link PhysicalPlan}. */
public PhysicalPlan plan(LogicalPlan plan) {
return planner.plan(plan);
@@ -356,4 +469,4 @@ private static RelNode convertToCalcitePlan(RelNode osPlan) {
}
return calcitePlan;
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ArrayFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ArrayFunctionImpl.java
index 2e9e53b9ac2..921f52aac27 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ArrayFunctionImpl.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ArrayFunctionImpl.java
@@ -22,6 +22,7 @@
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.sql.fun.SqlLibraryOperators;
+import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.apache.calcite.sql.type.SqlTypeName;
import org.opensearch.sql.expression.function.ImplementorUDF;
@@ -58,9 +59,14 @@ public SqlReturnTypeInference getReturnTypeInference() {
};
}
+ /**
+ * Provide operand metadata indicating the UDF accepts array-style operands.
+ *
+ * @return a {@link UDFOperandMetadata} describing operands as {@link OperandTypes#ARRAY_FUNCTION}
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
- return null;
+ return UDFOperandMetadata.wrap(OperandTypes.ARRAY_FUNCTION);
}
public static class ArrayImplementor implements NotNullImplementor {
@@ -121,4 +127,4 @@ public static Object internalCast(Object... args) {
}
return result;
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ExistsFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ExistsFunctionImpl.java
index 4b1c9586773..52cad6d68e4 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ExistsFunctionImpl.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ExistsFunctionImpl.java
@@ -14,6 +14,7 @@
import org.apache.calcite.linq4j.tree.Types;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.opensearch.sql.expression.function.ImplementorUDF;
@@ -34,9 +35,16 @@ public SqlReturnTypeInference getReturnTypeInference() {
return ReturnTypes.BOOLEAN;
}
+ /**
+ * Provide operand metadata for the EXISTS user-defined function.
+ *
+ * Wraps the standard EXISTS operand type into a UDFOperandMetadata instance.
+ *
+ * @return a UDFOperandMetadata describing the operand types required by the EXISTS function
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
- return null;
+ return UDFOperandMetadata.wrap(OperandTypes.EXISTS);
}
public static class ExistsImplementor implements NotNullImplementor {
@@ -66,4 +74,4 @@ public static Object eval(Object... args) {
}
return false;
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/FilterFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/FilterFunctionImpl.java
index 953b75303db..7b1b159d53e 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/FilterFunctionImpl.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/FilterFunctionImpl.java
@@ -15,6 +15,7 @@
import org.apache.calcite.linq4j.tree.Types;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.opensearch.sql.expression.function.ImplementorUDF;
@@ -34,9 +35,14 @@ public SqlReturnTypeInference getReturnTypeInference() {
return ReturnTypes.ARG0;
}
+ /**
+ * Declare the operand metadata indicating this UDF expects a MAP_FUNCTION operand.
+ *
+ * @return a {@link UDFOperandMetadata} that wraps {@link OperandTypes#MAP_FUNCTION}
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
- return null;
+ return UDFOperandMetadata.wrap(OperandTypes.MAP_FUNCTION);
}
public static class FilterImplementor implements NotNullImplementor {
@@ -67,4 +73,4 @@ public static Object eval(Object... args) {
}
return results;
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ForallFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ForallFunctionImpl.java
index 720eee841ab..28a081f57dc 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ForallFunctionImpl.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ForallFunctionImpl.java
@@ -14,6 +14,7 @@
import org.apache.calcite.linq4j.tree.Types;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.opensearch.sql.expression.function.ImplementorUDF;
@@ -33,9 +34,14 @@ public SqlReturnTypeInference getReturnTypeInference() {
return ReturnTypes.BOOLEAN;
}
+ /**
+ * Provide operand metadata specifying that this UDF expects a map-function operand.
+ *
+ * @return a UDFOperandMetadata instance wrapping OperandTypes.MAP_FUNCTION
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
- return null;
+ return UDFOperandMetadata.wrap(OperandTypes.MAP_FUNCTION);
}
public static class ForallImplementor implements NotNullImplementor {
@@ -65,4 +71,4 @@ public static Object eval(Object... args) {
}
return true;
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/LambdaUtils.java b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/LambdaUtils.java
index 314ac3ad945..05ffe3121ae 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/LambdaUtils.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/LambdaUtils.java
@@ -6,20 +6,20 @@
package org.opensearch.sql.expression.function.CollectionUDF;
import java.math.BigDecimal;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexCallBinding;
-import org.apache.calcite.rex.RexLambda;
-import org.apache.calcite.rex.RexLambdaRef;
-import org.apache.calcite.rex.RexNode;
-import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.apache.calcite.sql.type.SqlTypeName;
public class LambdaUtils {
+ /**
+ * Convert a BigDecimal lambda result to the specified SQL target type.
+ *
+ * If `candidate` is a BigDecimal, convert it to `Integer`, `Double`, or `Float` when
+ * `targetType` is INTEGER, DOUBLE, or FLOAT respectively; otherwise return the BigDecimal.
+ * If `candidate` is not a BigDecimal, return it unchanged.
+ *
+ * @param candidate the lambda-produced value to convert
+ * @param targetType the desired SQL target type; supported conversions: INTEGER, DOUBLE, FLOAT
+ * @return the converted value: an `Integer`, `Double`, `Float`, the original `BigDecimal`, or the original `candidate` if no conversion was performed
+ */
public static Object transferLambdaOutputToTargetType(Object candidate, SqlTypeName targetType) {
if (candidate instanceof BigDecimal) {
BigDecimal bd = (BigDecimal) candidate;
@@ -37,58 +37,4 @@ public static Object transferLambdaOutputToTargetType(Object candidate, SqlTypeN
return candidate;
}
}
-
- public static RelDataType inferReturnTypeFromLambda(
- RexLambda rexLambda, Map Examines each operand's component type and yields the most general common element type
+ * across operands. If no element type can be determined, returns the SQL `NULL` type.
+ *
+ * @param sqlOperatorBinding provides access to the function call's operand types
+ * @param typeFactory used to construct the SQL `NULL` type when no element type is found
+ * @return the most general element {@link RelDataType} among operands, or a SQL `NULL` {@link RelDataType} if none
+ */
private static RelDataType determineElementType(
SqlOperatorBinding sqlOperatorBinding, RelDataTypeFactory typeFactory) {
RelDataType mostGeneralType = null;
@@ -99,4 +115,4 @@ public Expression implement(
public static Object mvappend(Object... args) {
return MVAppendCore.collectElements(args);
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MapAppendFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MapAppendFunctionImpl.java
index 4cb0acae612..4f7057ee20c 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MapAppendFunctionImpl.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MapAppendFunctionImpl.java
@@ -18,7 +18,9 @@
import org.apache.calcite.linq4j.tree.Types;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.opensearch.sql.expression.function.ImplementorUDF;
import org.opensearch.sql.expression.function.UDFOperandMetadata;
@@ -43,9 +45,14 @@ public SqlReturnTypeInference getReturnTypeInference() {
};
}
+ /**
+ * Provide operand metadata that requires exactly two MAP-typed operands.
+ *
+ * @return UDFOperandMetadata enforcing two operands of SQL MAP type
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
- return null;
+ return UDFOperandMetadata.wrap(OperandTypes.family(SqlTypeFamily.MAP, SqlTypeFamily.MAP));
}
public static class MapAppendImplementor implements NotNullImplementor {
@@ -112,4 +119,4 @@ private static Set For a RexCallBinding, the inference is the type of the last operand. For a SqlCallBinding, the inference:
+ * derives element and accumulator types from the array and base operands, re-evaluates the first reduce lambda to
+ * obtain its result type, and if a second (final) lambda is supplied, derives the final return type from that lambda.
+ *
+ * @return a SqlReturnTypeInference that returns the resolved RelDataType according to the binding and lambda operands
+ * @throws IllegalStateException if the provided SqlOperatorBinding is neither a RexCallBinding nor a SqlCallBinding
+ */
@Override
public SqlReturnTypeInference getReturnTypeInference() {
return sqlOperatorBinding -> {
- RelDataTypeFactory typeFactory = sqlOperatorBinding.getTypeFactory();
- RexCallBinding rexCallBinding = (RexCallBinding) sqlOperatorBinding;
- List Accepts either three operands: (ARRAY, ANY, FUNCTION) or four operands:
+ * (ARRAY, ANY, FUNCTION, FUNCTION).
+ *
+ * @return a UDFOperandMetadata describing the allowed operand families for this function
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
- return null;
+ return UDFOperandMetadata.wrap(
+ OperandTypes.family(SqlTypeFamily.ARRAY, SqlTypeFamily.ANY, SqlTypeFamily.FUNCTION)
+ .or(
+ OperandTypes.family(
+ SqlTypeFamily.ARRAY,
+ SqlTypeFamily.ANY,
+ SqlTypeFamily.FUNCTION,
+ SqlTypeFamily.FUNCTION)));
}
public static class ReduceImplementor implements NotNullImplementor {
@@ -116,4 +155,4 @@ public static Object eval(Object... args) {
throw new IllegalArgumentException("wrong lambda function input");
}
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/TransformFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/TransformFunctionImpl.java
index d2184e12e8c..e44c7c10ae0 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/TransformFunctionImpl.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/TransformFunctionImpl.java
@@ -6,10 +6,12 @@
package org.opensearch.sql.expression.function.CollectionUDF;
import static org.apache.calcite.sql.type.SqlTypeUtil.createArrayType;
+import static org.apache.calcite.util.Static.RESOURCE;
import static org.opensearch.sql.expression.function.CollectionUDF.LambdaUtils.transferLambdaOutputToTargetType;
import java.util.ArrayList;
import java.util.List;
+import java.util.stream.IntStream;
import org.apache.calcite.adapter.enumerable.NotNullImplementor;
import org.apache.calcite.adapter.enumerable.NullPolicy;
import org.apache.calcite.adapter.enumerable.RexToLixTranslator;
@@ -19,12 +21,19 @@
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rel.type.RelDataTypeFactory;
import org.apache.calcite.rex.RexCall;
-import org.apache.calcite.rex.RexCallBinding;
-import org.apache.calcite.rex.RexLambda;
-import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlUtil;
import org.apache.calcite.sql.type.ArraySqlType;
+import org.apache.calcite.sql.type.FamilyOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlOperandCountRanges;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlSingleOperandTypeChecker;
+import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
import org.opensearch.sql.expression.function.ImplementorUDF;
import org.opensearch.sql.expression.function.UDFOperandMetadata;
@@ -39,21 +48,127 @@ public TransformFunctionImpl() {
super(new TransformImplementor(), NullPolicy.ANY);
}
+ /**
+ * Produces a return-type inference that derives a nullable array type from the lambda's return type.
+ *
+ * The inference uses the second operand (index 1) as the lambda return type, makes that component type nullable,
+ * and returns an array type whose component is that nullable type; the array itself is nullable.
+ *
+ * @return a SqlReturnTypeInference that yields an array type whose component is the lambda operand's return type made nullable and with the array marked nullable
+ */
@Override
public SqlReturnTypeInference getReturnTypeInference() {
return sqlOperatorBinding -> {
RelDataTypeFactory typeFactory = sqlOperatorBinding.getTypeFactory();
- RexCallBinding rexCallBinding = (RexCallBinding) sqlOperatorBinding;
- List The returned metadata wraps a SqlSingleOperandTypeChecker that:
+ * - enforces the first operand belongs to the ARRAY family and the second to the FUNCTION family;
+ * - accepts null literals when type coercion is enabled and otherwise signals validation failure;
+ * - rejects CURSOR typed operands even when the expected family is ANY;
+ * - treats SqlTypeFamily.ANY operands as permissive; and
+ * - performs strict validation for the first two operands while allowing arbitrary extra operands.
+ *
+ * @return a UDFOperandMetadata wrapping a SqlSingleOperandTypeChecker with the described behavior
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
- return null;
+ // Only checks the first two arguments as it allows arbitrary number of arguments to follow them
+ return UDFOperandMetadata.wrap(
+ new SqlSingleOperandTypeChecker() {
+ private static final List Sets the call's operator to `ATAN2` when the call has two operands, otherwise to `ATAN`.
+ *
+ * @param call the SQL call to rewrite; its operator will be modified in-place
+ * @return the rewritten SQL node (the same `call` instance)
+ @Override
+ public SqlNode rewriteCall(SqlValidator validator, SqlCall call) {
+ SqlOperator op =
+ call.getOperandList().size() == 2
+ ? SqlStdOperatorTable.ATAN2
+ : SqlStdOperatorTable.ATAN;
+ ((SqlBasicCall) call).setOperator(op);
+ return call;
+ }
+ };
+ // SPARK dialect is not included in lookup table to resolve overrides issues (e.g. reverse
+ // function won't work if spark is included because there are multiple overrides of reverse, and
+ // it will choose none of them in the end.) Therefore, SPARK functions used are explicitly
+ // declared here for lookup.
+ public static final SqlFunction REGEXP = SqlLibraryOperators.REGEXP;
/**
* Returns the PPL specific operator table, creating it if necessary.
@@ -493,12 +530,21 @@ public static PPLBuiltinOperators instance() {
}
/**
- * Invoking an implementor registered in {@link RexImpTable}, need to use reflection since they're
- * all private Use method directly in {@link BuiltInMethod} if possible, most operators'
- * implementor could be substituted by a single method.
+ * Invoke the Calcite implementor registered in RexImpTable for the given operator and return its generated Expression.
+ *
+ * This method uses reflection to call the implementor's private `implementSafe(RexToLixTranslator, RexCall, List)` method.
+ *
+ * @param translator the RexToLixTranslator used to translate Rex expressions
+ * @param call the RexCall representing the operator invocation
+ * @param operator the SqlFunction whose implementor is looked up in RexImpTable
+ * @param field the expression to pass as the single argument list element to the implementor
+ * @return the Expression produced by the implementor for the provided call and field
+ * @throws NoSuchMethodException if the implementor does not declare an `implementSafe` method with the expected signature
+ * @throws IllegalAccessException if the `implementSafe` method cannot be made accessible
+ * @throws InvocationTargetException if the underlying `implementSafe` invocation throws an exception
*/
private static Expression invokeCalciteImplementor(
- RexToLixTranslator translator, RexCall call, SqlOperator operator, Expression field)
+ RexToLixTranslator translator, RexCall call, SqlFunction operator, Expression field)
throws NoSuchMethodException, IllegalAccessException, InvocationTargetException {
RexCallImplementor rexCallImplementor = RexImpTable.INSTANCE.get(operator);
Method method =
@@ -509,4 +555,4 @@ private static Expression invokeCalciteImplementor(
method.setAccessible(true);
return (Expression) method.invoke(rexCallImplementor, translator, call, List.of(field));
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/PPLFuncImpTable.java b/core/src/main/java/org/opensearch/sql/expression/function/PPLFuncImpTable.java
index 205f3a0f2e1..feab49665e0 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/PPLFuncImpTable.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/PPLFuncImpTable.java
@@ -250,19 +250,17 @@
import com.google.common.collect.ImmutableMap;
import java.math.BigDecimal;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
+import java.util.Locale;
import java.util.Map;
import java.util.Optional;
-import java.util.StringJoiner;
import java.util.concurrent.ConcurrentHashMap;
import java.util.regex.Pattern;
import java.util.regex.PatternSyntaxException;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-import javax.annotation.Nullable;
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexLambda;
@@ -273,25 +271,20 @@
import org.apache.calcite.sql.fun.SqlLibraryOperators;
import org.apache.calcite.sql.fun.SqlStdOperatorTable;
import org.apache.calcite.sql.fun.SqlTrimFunction.Flag;
-import org.apache.calcite.sql.type.CompositeOperandTypeChecker;
-import org.apache.calcite.sql.type.FamilyOperandTypeChecker;
-import org.apache.calcite.sql.type.ImplicitCastOperandTypeChecker;
import org.apache.calcite.sql.type.OperandTypes;
-import org.apache.calcite.sql.type.SameOperandTypeChecker;
import org.apache.calcite.sql.type.SqlOperandTypeChecker;
import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
import org.apache.calcite.tools.RelBuilder;
-import org.apache.commons.lang3.tuple.Pair;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.opensearch.sql.calcite.CalcitePlanContext;
+import org.opensearch.sql.calcite.utils.OpenSearchTypeUtil;
import org.opensearch.sql.calcite.utils.PPLOperandTypes;
import org.opensearch.sql.calcite.utils.PlanUtils;
import org.opensearch.sql.calcite.utils.UserDefinedFunctionUtils;
-import org.opensearch.sql.exception.ExpressionEvaluationException;
import org.opensearch.sql.executor.QueryType;
import org.opensearch.sql.expression.function.CollectionUDF.MVIndexFunctionImp;
@@ -363,98 +356,107 @@ default RexNode resolve(RexBuilder builder, RexNode... args) {
* implementations are independent of any specific data storage, should be registered here
* internally.
*/
- private final ImmutableMap The provided UDF will be used to construct aggregate calls for that function name and
+ * overrides any built-in implementation. If the function name was already registered externally,
+ * a warning is logged and the new implementation replaces the previous one.
+ *
+ * @param functionName the built-in function name to override
+ * @param aggFunction the user-defined aggregate function used to build aggregate calls
*/
public void registerExternalAggOperator(
BuiltinFunctionName functionName, SqlUserDefinedAggFunction aggFunction) {
- PPLTypeChecker typeChecker =
- wrapSqlOperandTypeChecker(aggFunction.getOperandTypeChecker(), functionName.name(), true);
- CalciteFuncSignature signature = new CalciteFuncSignature(functionName.getName(), typeChecker);
+ if (aggExternalFunctionRegistry.containsKey(functionName)) {
+ logger.warn(
+ String.format(
+ Locale.ROOT, "Aggregate function %s is registered multiple times", functionName));
+ }
AggHandler handler =
(distinct, field, argList, ctx) ->
UserDefinedFunctionUtils.makeAggregateCall(
aggFunction, List.of(field), argList, ctx.relBuilder);
- aggExternalFunctionRegistry.put(functionName, Pair.of(signature, handler));
- }
-
- public List Currently only applies to {@code REDUCE}: casts the second argument (args[1]) to the
+ * return type of the lambda supplied as the third argument (args[2]) so code generation can
+ * correctly accumulate the result. The provided {@code args} array is modified in place.
+ *
+ * @param builder a {@code RexBuilder} used to create cast expressions
+ * @param functionName the built-in function being resolved
+ * @param args the function arguments; may be mutated to insert required casts
*/
private void compulsoryCast(
final RexBuilder builder, final BuiltinFunctionName functionName, RexNode... args) {
@@ -609,93 +549,64 @@ private void compulsoryCast(
}
}
- private @Nullable RexNode resolveWithCoercion(
- final RexBuilder builder,
- final BuiltinFunctionName functionName,
- List When a function is called, the system will try each registered operator in sequence,
- * checking if the provided arguments match the operator's type requirements. The first operator
- * whose type checker accepts the arguments will be used to execute the function.
+ * For user-defined functions, extracts the inner operand type checker before registration.
*
- * @param functionName the built-in function name under which to register the operators
- * @param operators the operators to associate with this function name, tried in sequence until
- * one matches the argument types during resolution
+ * @param functionName the built-in function name to register the operator under
+ * @param operator the SQL operator to register; its operand type checker will be used for resolution
*/
- protected void registerOperator(BuiltinFunctionName functionName, SqlOperator... operators) {
- for (SqlOperator operator : operators) {
- SqlOperandTypeChecker typeChecker;
- if (operator instanceof SqlUserDefinedFunction udfOperator) {
- typeChecker = extractTypeCheckerFromUDF(udfOperator);
- } else {
- typeChecker = operator.getOperandTypeChecker();
- }
- PPLTypeChecker pplTypeChecker =
- wrapSqlOperandTypeChecker(
- typeChecker, operator.getName(), operator instanceof SqlUserDefinedFunction);
- registerOperator(functionName, operator, pplTypeChecker);
+ protected void registerOperator(BuiltinFunctionName functionName, SqlOperator operator) {
+ SqlOperandTypeChecker typeChecker;
+ if (operator instanceof SqlUserDefinedFunction udfOperator) {
+ typeChecker = extractTypeCheckerFromUDF(udfOperator);
+ } else {
+ typeChecker = operator.getOperandTypeChecker();
}
+ registerOperator(functionName, operator, typeChecker);
}
/**
- * Registers an operator for a built-in function name with a specified {@link PPLTypeChecker}.
- * This allows custom type checking logic to be associated with the operator.
+ * Registers an operator for a built-in function name with a specified {@link
+ * SqlOperandTypeChecker}. This allows custom type checking logic to be associated with the
+ * operator.
*
* @param functionName the built-in function name
* @param operator the SQL operator to register
* @param typeChecker the type checker to use for validating argument types
*/
protected void registerOperator(
- BuiltinFunctionName functionName, SqlOperator operator, PPLTypeChecker typeChecker) {
+ BuiltinFunctionName functionName, SqlOperator operator, SqlOperandTypeChecker typeChecker) {
register(
functionName,
(RexBuilder builder, RexNode... args) -> builder.makeCall(operator, args),
typeChecker);
}
+ /**
+ * Register a binary division implementation for the given built-in function name.
+ *
+ * The registered implementation accepts two numeric operands and produces a division call.
+ *
+ * @param functionName the built-in function name to register the division implementation for
+ */
protected void registerDivideFunction(BuiltinFunctionName functionName) {
register(
functionName,
@@ -707,38 +618,52 @@ protected void registerDivideFunction(BuiltinFunctionName functionName) {
: SqlLibraryOperators.SAFE_DIVIDE;
return builder.makeCall(operator, left, right);
},
- PPLTypeChecker.family(SqlTypeFamily.NUMERIC, SqlTypeFamily.NUMERIC));
+ OperandTypes.family(SqlTypeFamily.NUMERIC, SqlTypeFamily.NUMERIC));
}
+ /**
+ * Registers all built-in scalar and library function implementations and their operand type checkers.
+ *
+ * Populates the builder's internal registry with comparison, logical, numeric, string,
+ * datetime, array/map, JSON, and PPL-specific operators, including several custom implementations
+ * and type-checker overrides required by the planner.
+ *
+ * Notable behaviours documented here:
+ * - REPLACE: validates regex patterns at planning time and converts PCRE backreferences (e.g. \1)
+ * to Java-style ($1) when the replacement is a literal.
+ * - SPLIT: uses REGEXP_EXTRACT_ALL('.') when the delimiter is the empty string, otherwise uses
+ * standard SPLIT.
+ * - ADD/ADDFUNCTION: selects CONCAT for character operands and PLUS for numeric operands at
+ * runtime; registered with the PLUS operand checker.
+ * - TOSTRING: casts single-argument calls to VARCHAR; preserves original operator for other arities.
+ * - MVJOIN: maps to Calcite's ARRAY_JOIN.
+ * - MVINDEX: uses a custom implementation to normalize indexes for ITEM/ARRAY_SLICE semantics.
+ * - JSON_ARRAY/JSON_OBJECT: prepends the NULL_ON_NULL flag to JSON array/object construction calls.
+ *
+ * The method also registers many standard Calcite and PPL operators (e.g., arithmetic,
+ * trig, rounding, datetime functions, regex operators, array/map functions, and UDFs) and
+ * adjusts operand type checkers for operators that require non-default typing.
+ */
void populate() {
// register operators for comparison
- registerOperator(NOTEQUAL, PPLBuiltinOperators.NOT_EQUALS_IP, SqlStdOperatorTable.NOT_EQUALS);
- registerOperator(EQUAL, PPLBuiltinOperators.EQUALS_IP, SqlStdOperatorTable.EQUALS);
- registerOperator(GREATER, PPLBuiltinOperators.GREATER_IP, SqlStdOperatorTable.GREATER_THAN);
- registerOperator(GTE, PPLBuiltinOperators.GTE_IP, SqlStdOperatorTable.GREATER_THAN_OR_EQUAL);
- registerOperator(LESS, PPLBuiltinOperators.LESS_IP, SqlStdOperatorTable.LESS_THAN);
- registerOperator(LTE, PPLBuiltinOperators.LTE_IP, SqlStdOperatorTable.LESS_THAN_OR_EQUAL);
+ registerOperator(NOTEQUAL, SqlStdOperatorTable.NOT_EQUALS);
+ registerOperator(EQUAL, SqlStdOperatorTable.EQUALS);
+ registerOperator(GREATER, SqlStdOperatorTable.GREATER_THAN);
+ registerOperator(GTE, SqlStdOperatorTable.GREATER_THAN_OR_EQUAL);
+ registerOperator(LESS, SqlStdOperatorTable.LESS_THAN);
+ registerOperator(LTE, SqlStdOperatorTable.LESS_THAN_OR_EQUAL);
// Register std operator
registerOperator(AND, SqlStdOperatorTable.AND);
registerOperator(OR, SqlStdOperatorTable.OR);
registerOperator(NOT, SqlStdOperatorTable.NOT);
-
- // Register ADDFUNCTION for numeric addition only
- registerOperator(ADDFUNCTION, SqlStdOperatorTable.PLUS);
- registerOperator(
- SUBTRACTFUNCTION,
- SqlStdOperatorTable.MINUS,
- PPLTypeChecker.wrapFamily((FamilyOperandTypeChecker) OperandTypes.NUMERIC_NUMERIC));
- registerOperator(
- SUBTRACT,
- SqlStdOperatorTable.MINUS,
- PPLTypeChecker.wrapFamily((FamilyOperandTypeChecker) OperandTypes.NUMERIC_NUMERIC));
+ registerOperator(SUBTRACTFUNCTION, SqlStdOperatorTable.MINUS, OperandTypes.NUMERIC_NUMERIC);
// Add DATETIME-DATETIME variant for timestamp binning support
registerOperator(
SUBTRACT,
SqlStdOperatorTable.MINUS,
- PPLTypeChecker.family(SqlTypeFamily.DATETIME, SqlTypeFamily.DATETIME));
+ OperandTypes.NUMERIC_NUMERIC.or(
+ OperandTypes.family(SqlTypeFamily.DATETIME, SqlTypeFamily.DATETIME)));
registerOperator(MULTIPLY, SqlStdOperatorTable.MULTIPLY);
registerOperator(MULTIPLYFUNCTION, SqlStdOperatorTable.MULTIPLY);
registerOperator(TRUNCATE, SqlStdOperatorTable.TRUNCATE);
@@ -788,32 +713,25 @@ void populate() {
}
return builder.makeCall(SqlLibraryOperators.REGEXP_REPLACE_3, args);
},
- wrapSqlOperandTypeChecker(
- SqlLibraryOperators.REGEXP_REPLACE_3.getOperandTypeChecker(), REPLACE.name(), false));
+ SqlLibraryOperators.REGEXP_REPLACE_3.getOperandTypeChecker());
registerOperator(UPPER, SqlStdOperatorTable.UPPER);
registerOperator(ABS, SqlStdOperatorTable.ABS);
registerOperator(ACOS, SqlStdOperatorTable.ACOS);
registerOperator(ASIN, SqlStdOperatorTable.ASIN);
- registerOperator(ATAN, SqlStdOperatorTable.ATAN);
+ registerOperator(ATAN, PPLBuiltinOperators.ATAN);
registerOperator(ATAN2, SqlStdOperatorTable.ATAN2);
// TODO, workaround to support sequence CompositeOperandTypeChecker.
registerOperator(
CEIL,
SqlStdOperatorTable.CEIL,
- PPLTypeChecker.wrapComposite(
- (CompositeOperandTypeChecker)
- OperandTypes.NUMERIC_OR_INTERVAL.or(
- OperandTypes.family(SqlTypeFamily.DATETIME, SqlTypeFamily.ANY)),
- false));
+ OperandTypes.NUMERIC_OR_INTERVAL.or(
+ OperandTypes.family(SqlTypeFamily.DATETIME, SqlTypeFamily.ANY)));
// TODO, workaround to support sequence CompositeOperandTypeChecker.
registerOperator(
CEILING,
SqlStdOperatorTable.CEIL,
- PPLTypeChecker.wrapComposite(
- (CompositeOperandTypeChecker)
- OperandTypes.NUMERIC_OR_INTERVAL.or(
- OperandTypes.family(SqlTypeFamily.DATETIME, SqlTypeFamily.ANY)),
- false));
+ OperandTypes.NUMERIC_OR_INTERVAL.or(
+ OperandTypes.family(SqlTypeFamily.DATETIME, SqlTypeFamily.ANY)));
registerOperator(COS, SqlStdOperatorTable.COS);
registerOperator(COT, SqlStdOperatorTable.COT);
registerOperator(DEGREES, SqlStdOperatorTable.DEGREES);
@@ -822,11 +740,8 @@ void populate() {
registerOperator(
FLOOR,
SqlStdOperatorTable.FLOOR,
- PPLTypeChecker.wrapComposite(
- (CompositeOperandTypeChecker)
- OperandTypes.NUMERIC_OR_INTERVAL.or(
- OperandTypes.family(SqlTypeFamily.DATETIME, SqlTypeFamily.ANY)),
- false));
+ OperandTypes.NUMERIC_OR_INTERVAL.or(
+ OperandTypes.family(SqlTypeFamily.DATETIME, SqlTypeFamily.ANY)));
registerOperator(LN, SqlStdOperatorTable.LN);
registerOperator(LOG10, SqlStdOperatorTable.LOG10);
registerOperator(PI, SqlStdOperatorTable.PI);
@@ -838,11 +753,8 @@ void populate() {
registerOperator(
ROUND,
SqlStdOperatorTable.ROUND,
- PPLTypeChecker.wrapComposite(
- (CompositeOperandTypeChecker)
- OperandTypes.NUMERIC.or(
- OperandTypes.family(SqlTypeFamily.NUMERIC, SqlTypeFamily.INTEGER)),
- false));
+ OperandTypes.NUMERIC.or(
+ OperandTypes.family(SqlTypeFamily.NUMERIC, SqlTypeFamily.INTEGER)));
registerOperator(SIGN, SqlStdOperatorTable.SIGN);
registerOperator(SIGNUM, SqlStdOperatorTable.SIGN);
registerOperator(SIN, SqlStdOperatorTable.SIN);
@@ -851,17 +763,17 @@ void populate() {
registerOperator(IFNULL, SqlStdOperatorTable.COALESCE);
registerOperator(EARLIEST, PPLBuiltinOperators.EARLIEST);
registerOperator(LATEST, PPLBuiltinOperators.LATEST);
- registerOperator(COALESCE, PPLBuiltinOperators.ENHANCED_COALESCE);
+ registerOperator(COALESCE, SqlStdOperatorTable.COALESCE);
// Register library operator
- registerOperator(REGEXP, SqlLibraryOperators.REGEXP);
+ registerOperator(REGEXP, PPLBuiltinOperators.REGEXP);
registerOperator(REGEXP_MATCH, SqlLibraryOperators.REGEXP_CONTAINS);
registerOperator(CONCAT, SqlLibraryOperators.CONCAT_FUNCTION);
registerOperator(CONCAT_WS, SqlLibraryOperators.CONCAT_WS);
- registerOperator(CONCAT_WS, SqlLibraryOperators.CONCAT_WS);
registerOperator(REVERSE, SqlLibraryOperators.REVERSE);
registerOperator(RIGHT, SqlLibraryOperators.RIGHT);
registerOperator(LEFT, SqlLibraryOperators.LEFT);
+ registerOperator(LOG, SqlLibraryOperators.LOG_MYSQL);
registerOperator(LOG2, SqlLibraryOperators.LOG2);
registerOperator(MD5, SqlLibraryOperators.MD5);
registerOperator(SHA1, SqlLibraryOperators.SHA1);
@@ -981,13 +893,17 @@ void populate() {
registerOperator(INTERNAL_PATTERN_PARSER, PPLBuiltinOperators.PATTERN_PARSER);
registerOperator(TONUMBER, PPLBuiltinOperators.TONUMBER);
- registerOperator(TOSTRING, PPLBuiltinOperators.TOSTRING);
register(
TOSTRING,
- (FunctionImp1)
- (builder, source) ->
- builder.makeCast(TYPE_FACTORY.createSqlType(SqlTypeName.VARCHAR, true), source),
- PPLTypeChecker.family(SqlTypeFamily.ANY));
+ (builder, args) -> {
+ if (args.length == 1) {
+ return builder.makeCast(
+ TYPE_FACTORY.createSqlType(SqlTypeName.VARCHAR, true), args[0]);
+ }
+ return builder.makeCall(PPLBuiltinOperators.TOSTRING, args);
+ },
+ OperandTypes.family(SqlTypeFamily.ANY)
+ .or(OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER)));
// Register MVJOIN to use Calcite's ARRAY_JOIN
register(
@@ -995,7 +911,7 @@ void populate() {
(FunctionImp2)
(builder, array, delimiter) ->
builder.makeCall(SqlLibraryOperators.ARRAY_JOIN, array, delimiter),
- PPLTypeChecker.family(SqlTypeFamily.ARRAY, SqlTypeFamily.CHARACTER));
+ OperandTypes.family(SqlTypeFamily.ARRAY, SqlTypeFamily.CHARACTER));
// Register SPLIT with custom logic for empty delimiter
// Case 1: Delimiter is not empty string, use SPLIT
@@ -1023,19 +939,16 @@ void populate() {
return builder.makeCall(
SqlStdOperatorTable.CASE, isEmptyDelimiter, splitChars, normalSplit);
},
- PPLTypeChecker.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER));
+ OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER));
// Register MVINDEX to use Calcite's ITEM/ARRAY_SLICE with index normalization
register(
MVINDEX,
new MVIndexFunctionImp(),
- PPLTypeChecker.wrapComposite(
- (CompositeOperandTypeChecker)
- OperandTypes.family(SqlTypeFamily.ARRAY, SqlTypeFamily.INTEGER)
- .or(
- OperandTypes.family(
- SqlTypeFamily.ARRAY, SqlTypeFamily.INTEGER, SqlTypeFamily.INTEGER)),
- false));
+ OperandTypes.family(SqlTypeFamily.ARRAY, SqlTypeFamily.INTEGER)
+ .or(
+ OperandTypes.family(
+ SqlTypeFamily.ARRAY, SqlTypeFamily.INTEGER, SqlTypeFamily.INTEGER)));
registerOperator(ARRAY, PPLBuiltinOperators.ARRAY);
registerOperator(MVAPPEND, PPLBuiltinOperators.MVAPPEND);
@@ -1083,52 +996,45 @@ void populate() {
registerOperator(JSON_EXTRACT_ALL, PPLBuiltinOperators.JSON_EXTRACT_ALL); // internal
// Register operators with a different type checker
-
- // Register ADD (+ symbol) for string concatenation
- // Replaced type checker since CONCAT also supports array concatenation
- registerOperator(
- ADD,
- SqlStdOperatorTable.CONCAT,
- PPLTypeChecker.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER));
- // Register ADD (+ symbol) for numeric addition
- // Replace type checker since PLUS also supports binary addition
- registerOperator(
- ADD,
- SqlStdOperatorTable.PLUS,
- PPLTypeChecker.family(SqlTypeFamily.NUMERIC, SqlTypeFamily.NUMERIC));
+ // Register ADD (+ symbol) for string concatenation and numeric addition
+ // Not creating PPL builtin operator as it will cause confusion during function resolution
+ FunctionImp add =
+ (builder, args) -> {
+ SqlOperator op =
+ (Stream.of(args).map(RexNode::getType).allMatch(OpenSearchTypeUtil::isCharacter))
+ ? SqlStdOperatorTable.CONCAT
+ : SqlStdOperatorTable.PLUS;
+ return builder.makeCall(op, args);
+ };
+ register(ADD, add, SqlStdOperatorTable.PLUS.getOperandTypeChecker());
+ register(ADDFUNCTION, add, SqlStdOperatorTable.PLUS.getOperandTypeChecker());
// Replace with a custom CompositeOperandTypeChecker to check both operands as
// SqlStdOperatorTable.ITEM.getOperandTypeChecker() checks only the first
- // operand instead
- // of all operands.
+ // operand instead of all operands.
registerOperator(
INTERNAL_ITEM,
SqlStdOperatorTable.ITEM,
- PPLTypeChecker.wrapComposite(
- (CompositeOperandTypeChecker)
- OperandTypes.family(SqlTypeFamily.ARRAY, SqlTypeFamily.INTEGER)
- .or(OperandTypes.family(SqlTypeFamily.MAP, SqlTypeFamily.ANY)),
- false));
+ OperandTypes.family(SqlTypeFamily.ARRAY, SqlTypeFamily.INTEGER)
+ .or(OperandTypes.family(SqlTypeFamily.MAP, SqlTypeFamily.ANY)));
registerOperator(
XOR,
SqlStdOperatorTable.NOT_EQUALS,
- PPLTypeChecker.family(SqlTypeFamily.BOOLEAN, SqlTypeFamily.BOOLEAN));
+ OperandTypes.family(SqlTypeFamily.BOOLEAN, SqlTypeFamily.BOOLEAN));
// SqlStdOperatorTable.CASE.getOperandTypeChecker is null. We manually create a type checker
// for it. The second and third operands are required to be of the same type. If not, it will
// throw an IllegalArgumentException with information Can't find leastRestrictive type
registerOperator(
IF,
SqlStdOperatorTable.CASE,
- PPLTypeChecker.family(SqlTypeFamily.BOOLEAN, SqlTypeFamily.ANY, SqlTypeFamily.ANY));
+ OperandTypes.family(SqlTypeFamily.BOOLEAN, SqlTypeFamily.ANY, SqlTypeFamily.ANY));
// Re-define the type checker for is not null, is present, and is null since
// their original type checker ANY isn't compatible with struct types.
registerOperator(
- IS_NOT_NULL,
- SqlStdOperatorTable.IS_NOT_NULL,
- PPLTypeChecker.family(SqlTypeFamily.IGNORE));
+ IS_NOT_NULL, SqlStdOperatorTable.IS_NOT_NULL, OperandTypes.family(SqlTypeFamily.IGNORE));
registerOperator(
- IS_PRESENT, SqlStdOperatorTable.IS_NOT_NULL, PPLTypeChecker.family(SqlTypeFamily.IGNORE));
+ IS_PRESENT, SqlStdOperatorTable.IS_NOT_NULL, OperandTypes.family(SqlTypeFamily.IGNORE));
registerOperator(
- IS_NULL, SqlStdOperatorTable.IS_NULL, PPLTypeChecker.family(SqlTypeFamily.IGNORE));
+ IS_NULL, SqlStdOperatorTable.IS_NULL, OperandTypes.family(SqlTypeFamily.IGNORE));
// Register implementation.
// Note, make the implementation an individual class if too complex.
@@ -1141,7 +1047,7 @@ void populate() {
builder.makeFlag(Flag.BOTH),
builder.makeLiteral(" "),
arg),
- PPLTypeChecker.family(SqlTypeFamily.CHARACTER));
+ OperandTypes.family(SqlTypeFamily.CHARACTER));
register(
LTRIM,
@@ -1152,7 +1058,7 @@ void populate() {
builder.makeFlag(Flag.LEADING),
builder.makeLiteral(" "),
arg),
- PPLTypeChecker.family(SqlTypeFamily.CHARACTER));
+ OperandTypes.family(SqlTypeFamily.CHARACTER));
register(
RTRIM,
(FunctionImp1)
@@ -1162,59 +1068,30 @@ void populate() {
builder.makeFlag(Flag.TRAILING),
builder.makeLiteral(" "),
arg),
- PPLTypeChecker.family(SqlTypeFamily.CHARACTER));
- registerOperator(
- ATAN,
- SqlStdOperatorTable.ATAN2,
- PPLTypeChecker.family(SqlTypeFamily.NUMERIC, SqlTypeFamily.NUMERIC));
+ OperandTypes.family(SqlTypeFamily.CHARACTER));
register(
STRCMP,
(FunctionImp2)
(builder, arg1, arg2) -> builder.makeCall(SqlLibraryOperators.STRCMP, arg2, arg1),
- PPLTypeChecker.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER));
+ OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER));
// SqlStdOperatorTable.SUBSTRING.getOperandTypeChecker is null. We manually
- // create a type
- // checker for it.
+ // create a type checker for it.
register(
SUBSTRING,
(RexBuilder builder, RexNode... args) ->
builder.makeCall(SqlStdOperatorTable.SUBSTRING, args),
- PPLTypeChecker.wrapComposite(
- (CompositeOperandTypeChecker)
- OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER)
- .or(
- OperandTypes.family(
- SqlTypeFamily.CHARACTER,
- SqlTypeFamily.INTEGER,
- SqlTypeFamily.INTEGER)),
- false));
+ OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER)
+ .or(
+ OperandTypes.family(
+ SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.INTEGER)));
register(
SUBSTR,
(RexBuilder builder, RexNode... args) ->
builder.makeCall(SqlStdOperatorTable.SUBSTRING, args),
- PPLTypeChecker.wrapComposite(
- (CompositeOperandTypeChecker)
- OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER)
- .or(
- OperandTypes.family(
- SqlTypeFamily.CHARACTER,
- SqlTypeFamily.INTEGER,
- SqlTypeFamily.INTEGER)),
- false));
- register(
- LOG,
- (FunctionImp2)
- (builder, arg1, arg2) -> builder.makeCall(SqlLibraryOperators.LOG, arg2, arg1),
- PPLTypeChecker.family(SqlTypeFamily.NUMERIC, SqlTypeFamily.NUMERIC));
- register(
- LOG,
- (FunctionImp1)
- (builder, arg) ->
- builder.makeCall(
- SqlLibraryOperators.LOG,
- arg,
- builder.makeApproxLiteral(BigDecimal.valueOf(Math.E))),
- PPLTypeChecker.family(SqlTypeFamily.NUMERIC));
+ OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER)
+ .or(
+ OperandTypes.family(
+ SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.INTEGER)));
// SqlStdOperatorTable.SQRT is declared but not implemented. The call to SQRT in Calcite is
// converted to POWER(x, 0.5).
register(
@@ -1225,7 +1102,7 @@ void populate() {
SqlStdOperatorTable.POWER,
arg,
builder.makeApproxLiteral(BigDecimal.valueOf(0.5))),
- PPLTypeChecker.family(SqlTypeFamily.NUMERIC));
+ OperandTypes.family(SqlTypeFamily.NUMERIC));
register(
TYPEOF,
(FunctionImp1)
@@ -1241,7 +1118,7 @@ void populate() {
builder.makeCall(SqlStdOperatorTable.EQUALS, arg1, arg2),
builder.makeNullLiteral(arg1.getType()),
arg1),
- PPLTypeChecker.wrapComparable((SameOperandTypeChecker) OperandTypes.SAME_SAME));
+ OperandTypes.SAME_SAME);
register(
IS_EMPTY,
(FunctionImp1)
@@ -1249,8 +1126,8 @@ void populate() {
builder.makeCall(
SqlStdOperatorTable.OR,
builder.makeCall(SqlStdOperatorTable.IS_NULL, arg),
- builder.makeCall(SqlStdOperatorTable.IS_EMPTY, arg)),
- PPLTypeChecker.family(SqlTypeFamily.ANY));
+ builder.makeCall(SqlStdOperatorTable.EQUALS, arg, builder.makeLiteral(""))),
+ OperandTypes.family(SqlTypeFamily.ANY));
register(
IS_BLANK,
(FunctionImp1)
@@ -1259,20 +1136,21 @@ void populate() {
SqlStdOperatorTable.OR,
builder.makeCall(SqlStdOperatorTable.IS_NULL, arg),
builder.makeCall(
- SqlStdOperatorTable.IS_EMPTY,
+ SqlStdOperatorTable.EQUALS,
builder.makeCall(
SqlStdOperatorTable.TRIM,
builder.makeFlag(Flag.BOTH),
builder.makeLiteral(" "),
- arg))),
- PPLTypeChecker.family(SqlTypeFamily.ANY));
+ arg),
+ builder.makeLiteral(""))),
+ OperandTypes.family(SqlTypeFamily.ANY));
register(
ILIKE,
(FunctionImp2)
(builder, arg1, arg2) ->
builder.makeCall(
SqlLibraryOperators.ILIKE, arg1, arg2, builder.makeLiteral("\\")),
- PPLTypeChecker.family(SqlTypeFamily.STRING, SqlTypeFamily.STRING));
+ OperandTypes.family(SqlTypeFamily.STRING, SqlTypeFamily.STRING));
register(
LIKE,
(FunctionImp3)
@@ -1282,43 +1160,71 @@ void populate() {
SqlStdOperatorTable.LIKE, arg1, arg2, builder.makeLiteral("\\"))
: builder.makeCall(
SqlLibraryOperators.ILIKE, arg1, arg2, builder.makeLiteral("\\")),
- PPLTypeChecker.family(SqlTypeFamily.STRING, SqlTypeFamily.STRING, SqlTypeFamily.BOOLEAN));
+ OperandTypes.family(SqlTypeFamily.STRING, SqlTypeFamily.STRING, SqlTypeFamily.BOOLEAN));
}
}
private static class Builder extends AbstractBuilder {
- private final Map Handlers registered include common aggregates (MAX, MIN, SUM, AVG, COUNT), variance/stddev
+ * variants, list/values/take helpers, percentile/median, time-based aggregates (EARLIEST, LATEST),
+ * and document-order FIRST/LAST.
+ *
+ * @throws IllegalArgumentException if a function-specific validation fails when the handler is
+ * invoked, for example:
+ * Delegates operand type checks, operand count range, and allowed-signature formatting to the
+ * provided checker. Parameter type and name lists are empty in the returned metadata.
+ *
+ * @param typeChecker the operand type checker to delegate to
+ * @return a UDFOperandMetadata view that delegates behavior to the given type checker
+ */
+ static UDFOperandMetadata wrap(SqlOperandTypeChecker typeChecker) {
return new UDFOperandMetadata() {
@Override
public SqlOperandTypeChecker getInnerTypeChecker() {
@@ -131,14 +101,38 @@ public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFail
return false;
}
+ /**
+ * Compute the allowed operand count range from the configured UDT signatures.
+ *
+ * Determines the smallest and largest number of parameters among all entries in
+ * {@code allowedParamTypes} and returns a range spanning those values.
+ *
+ * @return a {@link SqlOperandCountRange} that spans the minimum and maximum parameter
+ * counts found in {@code allowedParamTypes}
+ */
@Override
public SqlOperandCountRange getOperandCountRange() {
- return null;
+ int max = Integer.MIN_VALUE;
+ int min = Integer.MAX_VALUE;
+ for (List This implementation does not supply any signature information.
+ *
+ * @param op the operator for which signatures would be described
+ * @param opName the display name of the operator
+ * @return an empty string indicating no allowed-signature information is available
+ */
@Override
public String getAllowedSignatures(SqlOperator op, String opName) {
return "";
}
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/UserDefinedFunctionBuilder.java b/core/src/main/java/org/opensearch/sql/expression/function/UserDefinedFunctionBuilder.java
index f52c6b69f0e..5193cfb64d1 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/UserDefinedFunctionBuilder.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/UserDefinedFunctionBuilder.java
@@ -9,6 +9,7 @@
import org.apache.calcite.schema.ImplementableFunction;
import org.apache.calcite.sql.SqlIdentifier;
import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperandCountRange;
import org.apache.calcite.sql.parser.SqlParserPos;
import org.apache.calcite.sql.type.InferTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
@@ -30,27 +31,49 @@ public interface UserDefinedFunctionBuilder {
SqlReturnTypeInference getReturnTypeInference();
- UDFOperandMetadata getOperandMetadata();
+ /**
+ * Provides operand metadata describing the UDF's expected arguments for validation and type checking.
+ *
+ * Implementations may return {@code null} if the function does not expose operand metadata.
+ *
+ * @return the UDFOperandMetadata for this function, or {@code null} when no metadata is supplied
+ */
+UDFOperandMetadata getOperandMetadata();
+
+ /**
+ * Specifies the SQL kind to assign to the constructed user-defined function.
+ *
+ * @return the SqlKind to use when creating the SqlUserDefinedFunction, defaults to {@link SqlKind#OTHER_FUNCTION}.
+ */
+ default SqlKind getKind() {
+ return SqlKind.OTHER_FUNCTION;
+ }
+ /**
+ * Create a SqlUserDefinedFunction for the given function name using the builder's default determinism.
+ *
+ * @param functionName the name of the function as it will appear in SQL
+ * @return the SqlUserDefinedFunction for the specified name using the builder's default determinism
+ */
default SqlUserDefinedFunction toUDF(String functionName) {
return toUDF(functionName, true);
}
/**
- * In some rare cases, we need to call out the UDF to be not deterministic to avoid Volcano
- * planner over-optimization. For example, we don't need ReduceExpressionsRule to optimize
- * relevance query UDF.
+ * Create a Calcite SqlUserDefinedFunction for the given UDF name with the specified determinism.
*
- * @param functionName UDF name to be registered
- * @param isDeterministic Specified isDeterministic flag
- * @return Calcite SqlUserDefinedFunction
+ * The produced function uses this builder's return type inference, operand metadata, and implementation.
+ *
+ * @param functionName the UDF name to register
+ * @param isDeterministic whether the created function should be treated as deterministic
+ * @return a configured SqlUserDefinedFunction instance
*/
default SqlUserDefinedFunction toUDF(String functionName, boolean isDeterministic) {
SqlIdentifier udfLtrimIdentifier =
new SqlIdentifier(Collections.singletonList(functionName), null, SqlParserPos.ZERO, null);
return new SqlUserDefinedFunction(
udfLtrimIdentifier,
- SqlKind.OTHER_FUNCTION,
+ getKind(),
getReturnTypeInference(),
InferTypes.ANY_NULLABLE,
getOperandMetadata(),
@@ -60,12 +83,27 @@ public boolean isDeterministic() {
return isDeterministic;
}
+ /**
+ * Indicates the UDF does not expose a SQL identifier and should be rendered using function syntax or a keyword.
+ *
+ * @return {@code null} to signal there is no identifier and to trigger function-syntax/keyword unparsing
+ */
@Override
public SqlIdentifier getSqlIdentifier() {
// to avoid convert to sql dialog as identifier, use keyword instead
// check the code SqlUtil.unparseFunctionSyntax()
return null;
}
+
+ /**
+ * Provide the allowed operand count range for this function.
+ *
+ * @return the SqlOperandCountRange that specifies the minimum and maximum number of operands permitted for this function
+ */
+ @Override
+ public SqlOperandCountRange getOperandCountRange() {
+ return getOperandMetadata().getOperandCountRange();
+ }
};
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonAppendFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonAppendFunctionImpl.java
index dd76a002e06..b966c7e1d1b 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonAppendFunctionImpl.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonAppendFunctionImpl.java
@@ -22,6 +22,7 @@
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.runtime.JsonFunctions;
import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.opensearch.sql.expression.function.ImplementorUDF;
import org.opensearch.sql.expression.function.UDFOperandMetadata;
@@ -36,9 +37,14 @@ public SqlReturnTypeInference getReturnTypeInference() {
return STRING_FORCE_NULLABLE;
}
+ /**
+ * Provide operand metadata indicating the function accepts one or more operands.
+ *
+ * @return UDFOperandMetadata describing that the function accepts one or more operands
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
- return null;
+ return UDFOperandMetadata.wrap(OperandTypes.ONE_OR_MORE);
}
public static class JsonAppendImplementor implements NotNullImplementor {
@@ -76,4 +82,4 @@ public static Object eval(Object... args) throws JsonProcessingException {
}
return JsonFunctions.jsonInsert(jsonStr, expands.toArray());
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonDeleteFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonDeleteFunctionImpl.java
index b3a884a4f17..30e93a1f30c 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonDeleteFunctionImpl.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonDeleteFunctionImpl.java
@@ -20,6 +20,7 @@
import org.apache.calcite.linq4j.tree.Types;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.opensearch.sql.expression.function.ImplementorUDF;
import org.opensearch.sql.expression.function.UDFOperandMetadata;
@@ -34,9 +35,14 @@ public SqlReturnTypeInference getReturnTypeInference() {
return STRING_FORCE_NULLABLE;
}
+ /**
+ * Declares that this UDF accepts one or more operands.
+ *
+ * @return operand metadata indicating the function accepts one or more operands
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
- return null;
+ return UDFOperandMetadata.wrap(OperandTypes.ONE_OR_MORE);
}
public static class JsonDeleteImplementor implements NotNullImplementor {
@@ -60,4 +66,4 @@ public static Object eval(Object... args) throws JsonProcessingException {
.toArray(String[]::new);
return jsonRemove(args[0].toString(), pathSpecs);
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtendFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtendFunctionImpl.java
index dd91f1d95bd..96e6ba2399c 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtendFunctionImpl.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtendFunctionImpl.java
@@ -22,6 +22,7 @@
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.runtime.JsonFunctions;
import org.apache.calcite.schema.impl.ScalarFunctionImpl;
+import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.opensearch.sql.expression.function.ImplementorUDF;
import org.opensearch.sql.expression.function.UDFOperandMetadata;
@@ -36,9 +37,14 @@ public SqlReturnTypeInference getReturnTypeInference() {
return STRING_FORCE_NULLABLE;
}
+ /**
+ * Declares the function's operand arity as one or more operands.
+ *
+ * @return UDFOperandMetadata specifying the function accepts one or more operands
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
- return null;
+ return UDFOperandMetadata.wrap(OperandTypes.ONE_OR_MORE);
}
public static class JsonExtendImplementor implements NotNullImplementor {
@@ -91,4 +97,4 @@ public static Object eval(Object... args) throws JsonProcessingException {
}
return JsonFunctions.jsonInsert(jsonStr, expands.toArray());
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtractAllFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtractAllFunctionImpl.java
index 1f91c87bb77..6f950c56e97 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtractAllFunctionImpl.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtractAllFunctionImpl.java
@@ -28,7 +28,6 @@
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.ReturnTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
-import org.apache.calcite.sql.type.SqlTypeFamily;
import org.apache.calcite.sql.type.SqlTypeName;
import org.opensearch.sql.expression.function.ImplementorUDF;
import org.opensearch.sql.expression.function.UDFOperandMetadata;
@@ -55,9 +54,14 @@ public SqlReturnTypeInference getReturnTypeInference() {
true));
}
+ /**
+ * Specify that this UDF accepts a single CHARACTER (string) operand.
+ *
+ * @return a UDFOperandMetadata instance describing one CHARACTER argument
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
- return UDFOperandMetadata.wrap(OperandTypes.family(SqlTypeFamily.STRING));
+ return UDFOperandMetadata.wrap(OperandTypes.CHARACTER);
}
public static class JsonExtractAllImplementor implements NotNullImplementor {
@@ -215,4 +219,4 @@ private static String buildPath(Collection The function requires at least one operand and accepts up to 25 operands total.
+ * The first operand is always required (either fields or the query). Operands 2–25 are
+ * optional and are expected to be map-typed parameters used by the relevance query
+ * builders. The query argument itself is required and must not be null.
+ *
+ * @return a UDFOperandMetadata indicating between 1 and 25 operands where operands 2–25
+ * are map-typed and optional
*/
@Override
public UDFOperandMetadata getOperandMetadata() {
return UDFOperandMetadata.wrap(
- (CompositeOperandTypeChecker)
- OperandTypes.family(
- ImmutableList.of(
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP),
- i -> i > 0 && i < 14) // Parameters 3-14 are optional
- .or(
- OperandTypes.family(
- ImmutableList.of(
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP,
- SqlTypeFamily.MAP),
- i -> i > 0 && i < 25))); // Parameters 3-25 are optional
+ OperandTypes.repeat(
+ SqlOperandCountRanges.between(1, 25),
+ OperandTypes.MAP)); // Parameters 2-25 are optional
}
public static class RelevanceQueryImplementor implements NotNullImplementor {
@@ -98,4 +56,4 @@ public Expression implement(
"Relevance search query functions are only supported when they are pushed down");
}
}
-}
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/expression/function/udf/SpanFunction.java b/core/src/main/java/org/opensearch/sql/expression/function/udf/SpanFunction.java
index f28f12e30b9..79e38056037 100644
--- a/core/src/main/java/org/opensearch/sql/expression/function/udf/SpanFunction.java
+++ b/core/src/main/java/org/opensearch/sql/expression/function/udf/SpanFunction.java
@@ -18,10 +18,10 @@
import org.apache.calcite.rel.type.RelDataType;
import org.apache.calcite.rex.RexCall;
import org.apache.calcite.schema.impl.ScalarFunctionImpl;
-import org.apache.calcite.sql.type.CompositeOperandTypeChecker;
import org.apache.calcite.sql.type.OperandTypes;
import org.apache.calcite.sql.type.SqlReturnTypeInference;
import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.calcite.sql.type.SqlTypeUtil;
import org.apache.calcite.util.BuiltInMethod;
import org.opensearch.sql.calcite.type.ExprSqlType;
@@ -56,21 +56,49 @@ public SqlReturnTypeInference getReturnTypeInference() {
};
}
+ /**
+ * Describe valid operand-type combinations accepted by the SPAN UDF.
+ *
+ * Accepted signatures:
+ * Behavior:
+ * - If the interval type is decimal, the interval operand is converted to double.
+ * - If the unit operand is null or has SQL type ANY, returns a numeric rounding expression:
+ * integral SQL return types use (field / interval) * interval; other numeric types use
+ * floor(field / interval) * interval.
+ * - If the field type is an ExprSqlType, delegates to the appropriate eval method
+ * (evalDate, evalTime, evalTimestamp) and returns that implementation's expression.
+ * - Otherwise, throws IllegalArgumentException for unsupported field expression types.
+ *
+ * @param translator the RexToLixTranslator used to build expressions
+ * @param call the original RexCall for the SPAN invocation
+ * @param translatedOperands the already-translated operand expressions (expected size 3)
+ * @return an Expression that evaluates the SPAN operation
+ * @throws IllegalArgumentException if the field expression type is unsupported or cannot be handled
+ */
@Override
public Expression implement(
RexToLixTranslator translator, RexCall call, List The test is skipped when Calcite is not enabled.
+ *
+ * @throws IOException if reading the query or expected plan files fails
+ */
+ @Test
+ public void testQ41() throws IOException {
+ Assume.assumeTrue(isCalciteEnabled());
+ logger.info("Running Query 41");
+ String ppl = sanitize(loadFromFile("clickbench/queries/q41.ppl"));
+ String expected = loadExpectedPlan("clickbench/q41.yaml");
+ String alternative = loadExpectedPlan("clickbench/q41_alternative.yaml");
+ assertYamlEqualsIgnoreId(expected, alternative, explainQueryYaml(ppl));
+ timing(summary, "q" + 41, ppl);
+ }
+}
\ No newline at end of file
diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteArrayFunctionIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteArrayFunctionIT.java
index 1b32b76b611..402b04203c2 100644
--- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteArrayFunctionIT.java
+++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteArrayFunctionIT.java
@@ -51,6 +51,12 @@ public void testArrayWithString() throws IOException {
verifyDataRows(actual, rows(List.of("1", "demo")));
}
+ /**
+ * Verifies that creating an array with incompatible element types fails with the expected error.
+ *
+ * Asserts that executing the query `array(1, true)` raises an exception and that the exception
+ * message contains "Cannot infer return type for array; operand types: [INTEGER, BOOLEAN]".
+ */
@Test
public void testArrayWithMix() {
Class extends Exception> expectedException =
@@ -65,9 +71,7 @@ public void testArrayWithMix() {
TEST_INDEX_BANK)));
verifyErrorMessageContains(
- e,
- "Cannot resolve function: ARRAY, arguments: [INTEGER,BOOLEAN], caused by: fail to create"
- + " array with fixed type");
+ e, "Cannot infer return type for array; operand types: [INTEGER, BOOLEAN]");
}
@Test
@@ -867,4 +871,4 @@ public void testMvmapWithEvalFieldReference() throws IOException {
verifySchema(actual, schema("result", "array"));
verifyDataRows(actual, rows(List.of(10, 20, 30)));
}
-}
+}
\ No newline at end of file
diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteExplainIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteExplainIT.java
index cdea1738eb0..5265ff6885e 100644
--- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteExplainIT.java
+++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteExplainIT.java
@@ -60,15 +60,20 @@ public void supportSearchSargPushDown_singleRange() throws IOException {
assertYamlEqualsIgnoreId(expected, result);
}
- // Only for Calcite
+ /**
+ * Verifies that Calcite generates correct SARG pushdown for a WHERE clause containing multiple
+ * numeric ranges and an equality condition.
+ *
+ * @throws IOException if reading the expected plan or executing the explain query fails
+ */
@Test
public void supportSearchSargPushDown_multiRange() throws IOException {
String query =
"source=opensearch-sql_test_index_account | where (age > 20 and age < 28) or (age > 25 and"
+ " age < 30) or (age >= 1 and age <= 10) or age = 0 | fields age";
- var result = explainQueryToString(query);
- String expected = loadExpectedPlan("explain_sarg_filter_push_multi_range.json");
- assertJsonEqualsIgnoreId(expected, result);
+ var result = explainQueryYaml(query);
+ String expected = loadExpectedPlan("explain_sarg_filter_push_multi_range.yaml");
+ assertYamlEqualsIgnoreId(expected, result);
}
// Only for Calcite
@@ -335,14 +340,20 @@ public void testExplainIsEmptyOrOthers() throws IOException {
+ " isnull(firstname)"));
}
- // Only for Calcite
+ /**
+ * Verifies that an IS NULL predicate combined with another condition is represented as a pushdown in the Calcite explain plan.
+ *
+ * Loads the expected YAML plan and compares it to the explain output for a query combining `isnull(firstname)` with `gender = 'M'`.
+ *
+ * @throws IOException if reading the expected plan or producing the explain output fails
+ */
@Test
public void testExplainIsNullOrOthers() throws IOException {
// pushdown should work
- String expected = loadExpectedPlan("explain_isnull_or_others.json");
- assertJsonEqualsIgnoreId(
+ String expected = loadExpectedPlan("explain_isnull_or_others.yaml");
+ assertYamlEqualsIgnoreId(
expected,
- explainQueryToString(
+ explainQueryYaml(
"source=opensearch-sql_test_index_account | where isnull(firstname) or gender = 'M'"));
}
@@ -370,9 +381,9 @@ public void testPartialPushdownFilterWithIsNull() throws IOException {
Locale.ROOT,
"source=%s | where isnull(address) and name='david'",
TEST_INDEX_NESTED_SIMPLE);
- var result = explainQueryToString(query);
- String expected = loadExpectedPlan("explain_partial_filter_isnull.json");
- assertJsonEqualsIgnoreId(expected, result);
+ var result = explainQueryYaml(query);
+ String expected = loadExpectedPlan("explain_partial_filter_isnull.yaml");
+ assertYamlEqualsIgnoreId(expected, result);
}
@Test
@@ -381,9 +392,9 @@ public void testSkipScriptEncodingOnExtendedFormat() throws IOException {
String query =
"source=opensearch-sql_test_index_account | where address = '671 Bristol Street' and age -"
+ " 2 = 30 | fields firstname, age, address";
- var result = explainQueryToString(query, true);
- String expected = loadFromFile("expectedOutput/calcite/explain_skip_script_encoding.json");
- assertJsonEqualsIgnoreId(expected, result);
+ var result = explainQueryYaml(query);
+ String expected = loadExpectedPlan("explain_skip_script_encoding.yaml");
+ assertYamlEqualsIgnoreId(expected, result);
}
// Only for Calcite, as v2 gets unstable serialized string for function
@@ -441,42 +452,50 @@ public void testExplainWithTimechartCount() throws IOException {
@Test
public void testExplainTimechartPerSecond() throws IOException {
- var result = explainQueryToString("source=events | timechart span=2m per_second(cpu_usage)");
+ var result = explainQueryYaml("source=events | timechart span=2m per_second(cpu_usage)");
assertTrue(
result.contains(
- "per_second(cpu_usage)=[DIVIDE(*($1, 1000.0E0), TIMESTAMPDIFF('MILLISECOND':VARCHAR,"
- + " $0, TIMESTAMPADD('MINUTE':VARCHAR, 2, $0)))]"));
- assertTrue(result.contains("per_second(cpu_usage)=[SUM($0)]"));
+ "per_second(cpu_usage)=[DIVIDE(*($1, 1000.0E0), TIMESTAMPDIFF('MILLISECOND', $0,"
+ + " TIMESTAMPADD('MINUTE', 2, $0)))]"));
+ assertTrue(result.contains("per_second(cpu_usage)=[SUM($1)]"));
}
+ /**
+ * Verifies the explain plan for a timechart with a 2-minute span and a per_minute aggregation on cpu_usage.
+ *
+ * Confirms the generated explain YAML includes the expected per_minute expression (the division form over the
+ * span interval) and the corresponding SUM aggregation form.
+ *
+ * @throws IOException if reading or generating the explain output fails
+ */
@Test
public void testExplainTimechartPerMinute() throws IOException {
- var result = explainQueryToString("source=events | timechart span=2m per_minute(cpu_usage)");
+ var result = explainQueryYaml("source=events | timechart span=2m per_minute(cpu_usage)");
assertTrue(
result.contains(
- "per_minute(cpu_usage)=[DIVIDE(*($1, 60000.0E0), TIMESTAMPDIFF('MILLISECOND':VARCHAR,"
- + " $0, TIMESTAMPADD('MINUTE':VARCHAR, 2, $0)))]"));
- assertTrue(result.contains("per_minute(cpu_usage)=[SUM($0)]"));
+ "per_minute(cpu_usage)=[DIVIDE(*($1, 60000.0E0), TIMESTAMPDIFF('MILLISECOND', $0,"
+ + " TIMESTAMPADD('MINUTE', 2, $0)))]"));
+ assertTrue(result.contains("per_minute(cpu_usage)=[SUM($1)]"));
}
@Test
public void testExplainTimechartPerHour() throws IOException {
- var result = explainQueryToString("source=events | timechart span=2m per_hour(cpu_usage)");
+ var result = explainQueryYaml("source=events | timechart span=2m per_hour(cpu_usage)");
assertTrue(
result.contains(
- "per_hour(cpu_usage)=[DIVIDE(*($1, 3600000.0E0), TIMESTAMPDIFF('MILLISECOND':VARCHAR,"
- + " $0, TIMESTAMPADD('MINUTE':VARCHAR, 2, $0)))]"));
- assertTrue(result.contains("per_hour(cpu_usage)=[SUM($0)]"));
+ "per_hour(cpu_usage)=[DIVIDE(*($1, 3600000.0E0), TIMESTAMPDIFF('MILLISECOND', $0,"
+ + " TIMESTAMPADD('MINUTE', 2, $0)))]"));
+ assertTrue(result.contains("per_hour(cpu_usage)=[SUM($1)]"));
}
@Test
public void testExplainTimechartPerDay() throws IOException {
- var result = explainQueryToString("source=events | timechart span=2m per_day(cpu_usage)");
+ var result = explainQueryYaml("source=events | timechart span=2m per_day(cpu_usage)");
assertTrue(
result.contains(
- "per_day(cpu_usage)=[DIVIDE(*($1, 8.64E7), TIMESTAMPDIFF('MILLISECOND':VARCHAR, $0,"
- + " TIMESTAMPADD('MINUTE':VARCHAR, 2, $0)))]"));
- assertTrue(result.contains("per_day(cpu_usage)=[SUM($0)]"));
+ "per_day(cpu_usage)=[DIVIDE(*($1, 8.64E7), TIMESTAMPDIFF('MILLISECOND', $0,"
+ + " TIMESTAMPADD('MINUTE', 2, $0)))]"));
+ assertTrue(result.contains("per_day(cpu_usage)=[SUM($1)]"));
}
@Test
@@ -504,12 +523,20 @@ public void supportPushDownScriptOnTextField() throws IOException {
@Test
public void testExplainBinWithBins() throws IOException {
- String expected = loadExpectedPlan("explain_bin_bins.json");
- assertJsonEqualsIgnoreId(
+ String expected = loadExpectedPlan("explain_bin_bins.yaml");
+ assertYamlEqualsIgnoreId(
expected,
- explainQueryToString("source=opensearch-sql_test_index_account | bin age bins=3 | head 5"));
+ explainQueryYaml("source=opensearch-sql_test_index_account | bin age bins=3 | head 5"));
}
+ /**
+ * Verifies explain plans for statistics computed after binning on a time field.
+ *
+ * Requires pushdown to be enabled. Compares the YAML explain output of two queries
+ * (count and avg over binned @timestamp) against their corresponding expected YAML plans.
+ *
+ * @throws IOException if loading the expected YAML plan files fails
+ */
@Test
public void testExplainStatsWithBinsOnTimeField() throws IOException {
// TODO: Remove this after addressing https://github.com/opensearch-project/sql/issues/4317
@@ -552,21 +579,29 @@ public void testExplainBinWithSpan() throws IOException {
explainQueryYaml("source=opensearch-sql_test_index_account | bin age span=10 | head 5"));
}
+ /**
+ * Verifies the explain plan for a `bin` command using the `minspan` option.
+ *
+ * Loads the expected YAML plan "explain_bin_minspan.yaml" and asserts that the
+ * explain output for `source=opensearch-sql_test_index_account | bin age minspan=5 | head 5`
+ * matches the expected plan (ignoring IDs).
+ *
+ * @throws IOException if reading the expected plan file or generating the explain output fails
+ */
@Test
public void testExplainBinWithMinspan() throws IOException {
- String expected = loadExpectedPlan("explain_bin_minspan.json");
- assertJsonEqualsIgnoreId(
+ String expected = loadExpectedPlan("explain_bin_minspan.yaml");
+ assertYamlEqualsIgnoreId(
expected,
- explainQueryToString(
- "source=opensearch-sql_test_index_account | bin age minspan=5 | head 5"));
+ explainQueryYaml("source=opensearch-sql_test_index_account | bin age minspan=5 | head 5"));
}
@Test
public void testExplainBinWithStartEnd() throws IOException {
- String expected = loadExpectedPlan("explain_bin_start_end.json");
- assertJsonEqualsIgnoreId(
+ String expected = loadExpectedPlan("explain_bin_start_end.yaml");
+ assertYamlEqualsIgnoreId(
expected,
- explainQueryToString(
+ explainQueryYaml(
"source=opensearch-sql_test_index_account | bin balance start=0 end=100001 | head 5"));
}
@@ -584,9 +619,9 @@ public void testExplainBinWithAligntime() throws IOException {
public void testExplainCountEval() throws IOException {
String query =
"source=opensearch-sql_test_index_bank | stats count(eval(age > 30)) as mature_count";
- var result = explainQueryToString(query);
- String expected = loadExpectedPlan("explain_count_eval_push.json");
- assertJsonEqualsIgnoreId(expected, result);
+ var result = explainQueryYaml(query);
+ String expected = loadExpectedPlan("explain_count_eval_push.yaml");
+ assertYamlEqualsIgnoreId(expected, result);
}
@Test
@@ -594,11 +629,16 @@ public void testExplainCountEvalComplex() throws IOException {
String query =
"source=opensearch-sql_test_index_bank | stats count(eval(age > 30 and age < 50)) as"
+ " mature_count";
- var result = explainQueryToString(query);
- String expected = loadExpectedPlan("explain_count_eval_complex_push.json");
- assertJsonEqualsIgnoreId(expected, result);
+ var result = explainQueryYaml(query);
+ String expected = loadExpectedPlan("explain_count_eval_complex_push.yaml");
+ assertYamlEqualsIgnoreId(expected, result);
}
+ /**
+ * Verifies the Calcite explain plan for an eventstats distinct-count (`dc(state)`) operation when pushdown is enabled.
+ *
+ * @throws IOException if the expected plan file cannot be read
+ */
@Test
public void testEventstatsDistinctCountExplain() throws IOException {
enabledOnlyWhenPushdownIsEnabled();
@@ -755,6 +795,13 @@ public void testExplainRegexMatchInWhereWithScriptPushdown() throws IOException
assertYamlEqualsIgnoreId(expected, result);
}
+ /**
+ * Verifies the explain plan for using `regexp_match` in an `eval` expression when the expression is not pushed down as a script.
+ *
+ * The test compares the generated YAML explain plan against the expected plan file.
+ *
+ * @throws IOException if loading the expected plan or producing the explain output fails
+ */
@Test
public void testExplainRegexMatchInEvalWithOutScriptPushdown() throws IOException {
enabledOnlyWhenPushdownIsEnabled();
@@ -762,9 +809,8 @@ public void testExplainRegexMatchInEvalWithOutScriptPushdown() throws IOExceptio
String.format(
"source=%s |eval has_hello = regexp_match(name, 'hello') | fields has_hello",
TEST_INDEX_STRINGS);
- var result = explainQueryToString(query);
- String expected = loadFromFile("expectedOutput/calcite/explain_regexp_match_in_eval.json");
- assertJsonEqualsIgnoreId(expected, result);
+ var result = explainQueryYaml(query);
+ assertYamlEqualsIgnoreId(loadExpectedPlan("explain_regexp_match_in_eval.yaml"), result);
}
// Only for Calcite
@@ -1076,10 +1122,10 @@ public void testExplainAppendCommand() throws IOException {
@Test
public void testExplainAppendPipeCommand() throws IOException {
- String expected = loadExpectedPlan("explain_appendpipe_command.json");
- assertJsonEqualsIgnoreId(
+ String expected = loadExpectedPlan("explain_appendpipe_command.yaml");
+ assertYamlEqualsIgnoreId(
expected,
- explainQueryToString(
+ explainQueryYaml(
String.format(
Locale.ROOT,
"source=%s | appendpipe [ stats count(balance) as cnt by gender ]",
@@ -1091,11 +1137,19 @@ public void testMvjoinExplain() throws IOException {
String query =
"source=opensearch-sql_test_index_account | eval result = mvjoin(array('a', 'b', 'c'), ',')"
+ " | fields result | head 1";
- var result = explainQueryToString(query);
- String expected = loadExpectedPlan("explain_mvjoin.json");
- assertJsonEqualsIgnoreId(expected, result);
+ assertYamlEqualsIgnoreId(loadExpectedPlan("explain_mvjoin.yaml"), explainQueryYaml(query));
}
+ /**
+ * Verifies that a LIMIT (head with offset) is not pushed down when the index's max_result_window
+ * is smaller than the requested offset.
+ *
+ * This test enables pushdown mode, sets the index max_result_window to 1, runs an explain for a
+ * query with head 1 from 1, and asserts the produced YAML plan matches the expected plan that
+ * prevents limit pushdown. The index max_result_window is reset after the assertion.
+ *
+ * @throws IOException if loading the expected plan or executing the explain query fails
+ */
@Test
public void testPreventLimitPushdown() throws IOException {
enabledOnlyWhenPushdownIsEnabled();
@@ -1110,50 +1164,43 @@ public void testPreventLimitPushdown() throws IOException {
@Test
public void testPushdownLimitIntoAggregation() throws IOException {
enabledOnlyWhenPushdownIsEnabled();
- String expected = loadExpectedPlan("explain_limit_agg_pushdown.json");
- assertJsonEqualsIgnoreId(
- expected,
- explainQueryToString("source=opensearch-sql_test_index_account | stats count() by state"));
+ assertYamlEqualsIgnoreId(
+ loadExpectedPlan("explain_limit_agg_pushdown.yaml"),
+ explainQueryYaml("source=opensearch-sql_test_index_account | stats count() by state"));
- expected = loadExpectedPlan("explain_limit_agg_pushdown2.yaml");
assertYamlEqualsIgnoreId(
- expected,
+ loadExpectedPlan("explain_limit_agg_pushdown2.yaml"),
explainQueryYaml(
"source=opensearch-sql_test_index_account | stats count() by state | head 100"));
- expected = loadExpectedPlan("explain_limit_agg_pushdown3.json");
- assertJsonEqualsIgnoreId(
- expected,
- explainQueryToString(
+ assertYamlEqualsIgnoreId(
+ loadExpectedPlan("explain_limit_agg_pushdown3.yaml"),
+ explainQueryYaml(
"source=opensearch-sql_test_index_account | stats count() by state | head 100 | head 10"
+ " from 10 "));
- expected = loadExpectedPlan("explain_limit_agg_pushdown4.yaml");
assertYamlEqualsIgnoreId(
- expected,
+ loadExpectedPlan("explain_limit_agg_pushdown4.yaml"),
explainQueryYaml(
"source=opensearch-sql_test_index_account | stats count() by state | sort state | head"
+ " 100 | head 10 from 10 "));
- expected = loadExpectedPlan("explain_limit_agg_pushdown_bucket_nullable1.yaml");
assertYamlEqualsIgnoreId(
- expected,
+ loadExpectedPlan("explain_limit_agg_pushdown_bucket_nullable1.yaml"),
explainQueryYaml(
"source=opensearch-sql_test_index_account | stats bucket_nullable=false count() by"
+ " state | head 100 | head 10 from 10 "));
- expected = loadExpectedPlan("explain_limit_agg_pushdown_bucket_nullable2.yaml");
assertYamlEqualsIgnoreId(
- expected,
+ loadExpectedPlan("explain_limit_agg_pushdown_bucket_nullable2.yaml"),
explainQueryYaml(
"source=opensearch-sql_test_index_account | stats bucket_nullable=false count() by"
+ " state | sort state | head 100 | head 10 from 10 "));
- // Don't pushdown the combination of limit and sort
- expected = loadExpectedPlan("explain_limit_agg_pushdown5.json");
- assertJsonEqualsIgnoreId(
- expected,
- explainQueryToString(
+ // Don't push down the combination of limit and sort
+ assertYamlEqualsIgnoreId(
+ loadExpectedPlan("explain_limit_agg_pushdown5.yaml"),
+ explainQueryYaml(
"source=opensearch-sql_test_index_account | stats count() by state | sort `count()` |"
+ " head 100 | head 10 from 10 "));
}
@@ -1620,15 +1667,20 @@ private String executeWithReplace(String ppl) throws IOException {
return result.replace("\\r\\n", "\\n");
}
+ /**
+ * Verifies the explain plan produced for a query that uses the `strftime` function.
+ *
+ * @throws IOException if loading the expected plan file fails
+ */
@Test
public void testStrftimeFunctionExplain() throws IOException {
// Test explain for strftime function
String query =
"source=opensearch-sql_test_index_account | eval formatted_date = strftime(1521467703,"
+ " '%Y-%m-%d') | fields formatted_date | head 1";
- var result = explainQueryToString(query);
- String expected = loadExpectedPlan("explain_strftime_function.json");
- assertJsonEqualsIgnoreId(expected, result);
+ var result = explainQueryYaml(query);
+ String expected = loadExpectedPlan("explain_strftime_function.yaml");
+ assertYamlEqualsIgnoreId(expected, result);
}
// Script generation is not stable in v2
@@ -1817,6 +1869,7 @@ public void testExplainChartWithLimits() throws IOException {
String.format(
"source=%s | chart limit=0 avg(balance) over state by gender", TEST_INDEX_BANK)));
+ // TODO: Fix -- now it gets away from push-down
assertYamlEqualsIgnoreId(
loadExpectedPlan("chart_use_other.yaml"),
explainQueryYaml(
@@ -2206,13 +2259,12 @@ public void testAliasTypeField() throws IOException {
@Test
public void testRexStandardizationForScript() throws IOException {
enabledOnlyWhenPushdownIsEnabled();
- assertJsonEqualsIgnoreId(
- loadExpectedPlan("explain_extended_for_standardization.json"),
- explainQueryToString(
+ assertYamlEqualsIgnoreId(
+ loadExpectedPlan("explain_extended_for_standardization.yaml"),
+ explainQueryYaml(
String.format(
"source=%s | eval age_range = case(age < 30, 'u30', age >= 30 and age <= 40, 'u40'"
+ " else 'u100') | stats avg(age) as avg_age by age_range",
- TEST_INDEX_BANK),
- true));
+ TEST_INDEX_BANK)));
}
-}
+}
\ No newline at end of file
diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteMultisearchCommandIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteMultisearchCommandIT.java
index 393b0a4a501..57ff006022b 100644
--- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteMultisearchCommandIT.java
+++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteMultisearchCommandIT.java
@@ -141,6 +141,15 @@ public void testMultisearchWithFieldsProjection() throws IOException {
rows("Mcgee", "Mooney", 18612L));
}
+ /**
+ * Verifies multisearch correctly interleaves timestamped results from two time-series indices.
+ *
+ * The test runs a multisearch across two time-indexed test indices with category filters,
+ * limits the output to the first 10 rows, and asserts the resulting schema and ordered rows
+ * contain both indices' timestamped records interleaved by time.
+ *
+ * @throws IOException if query execution or response parsing fails
+ */
@Test
public void testMultisearchWithTimestampInterleaving() throws IOException {
JSONObject result =
@@ -152,10 +161,10 @@ public void testMultisearchWithTimestampInterleaving() throws IOException {
verifySchema(
result,
- schema("@timestamp", null, "string"),
+ schema("@timestamp", null, "timestamp"),
schema("category", null, "string"),
schema("value", null, "int"),
- schema("timestamp", null, "string"));
+ schema("timestamp", null, "timestamp"));
verifyDataRows(
result,
@@ -364,4 +373,4 @@ public void testMultisearchTypeConflictWithStats() {
.getMessage()
.contains("Unable to process column 'age' due to incompatible types:"));
}
-}
+}
\ No newline at end of file
diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLAppendCommandIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLAppendCommandIT.java
index d01ddfb2a44..d7bb34046cc 100644
--- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLAppendCommandIT.java
+++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLAppendCommandIT.java
@@ -237,6 +237,14 @@ public void testAppendWithConflictTypeColumn() {
.contains("Unable to process column 'sum' due to incompatible types:"));
}
+ /**
+ * Verifies that appending the BANK source to the ACCOUNT source merges their schemas and preserves `birthdate` as a timestamp UDT.
+ *
+ * Asserts the merged schema contains `account_number` (bigint), `firstname` (string), `age` (int), and `birthdate` (timestamp),
+ * and that the query returns the expected row filtered by a non-null `birthdate` and `account_number > 30`.
+ *
+ * @throws IOException if query execution fails
+ */
@Test
public void testAppendSchemaMergeWithTimestampUDT() throws IOException {
JSONObject actual =
@@ -253,7 +261,7 @@ public void testAppendSchemaMergeWithTimestampUDT() throws IOException {
schema("account_number", "bigint"),
schema("firstname", "string"),
schema("age", "int"),
- schema("birthdate", "string"));
+ schema("birthdate", "timestamp"));
verifyDataRows(actual, rows(32, null, 34, "2018-08-11 00:00:00"));
}
@@ -271,4 +279,4 @@ public void testAppendSchemaMergeWithIpUDT() throws IOException {
actual, schema("account_number", "bigint"), schema("age", "bigint"), schema("host", "ip"));
verifyDataRows(actual, rows(null, null, "0.0.0.2"));
}
-}
+}
\ No newline at end of file
diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLAppendcolIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLAppendcolIT.java
index 877c10947b8..facb09b7369 100644
--- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLAppendcolIT.java
+++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLAppendcolIT.java
@@ -24,8 +24,21 @@ public void init() throws Exception {
loadIndex(Index.ACCOUNT);
}
+ /**
+ * Verifies that appendcol produces the expected schema and rows when appending a per-gender count
+ * to aggregated age sums by gender and state with pushdown enabled.
+ *
+ * Executes a PPL query against the ACCOUNT test index that computes sum(age) by gender and state,
+ * appends a per-gender count column via an inner stats plan, and asserts the resulting schema and
+ * first 10 rows (including expected nulls when values are not present).
+ *
+ * @throws IOException if query execution or result parsing fails
+ */
@Test
public void testAppendCol() throws IOException {
+ // Although the plans are identical, not pushing down resulting the cnt in the first two rows
+ // being null
+ enabledOnlyWhenPushdownIsEnabled();
JSONObject actual =
executeQuery(
String.format(
@@ -53,8 +66,21 @@ public void testAppendCol() throws IOException {
rows("F", "FL", 310, null));
}
+ /**
+ * Verifies that APPENDCOL with `override = true` merges the inner aggregated `cnt` column
+ * into the outer aggregation results and that the resulting schema and rows match expectations.
+ *
+ * Runs a PPL query against the ACCOUNT test index that computes a sum by gender and state,
+ * appends a per-gender count (with override enabled), and asserts the returned schema and first
+ * ten rows. The test is executed only when pushdown is enabled.
+ *
+ * @throws IOException if an I/O error occurs during query execution
+ */
@Test
public void testAppendColOverride() throws IOException {
+ // Although the plans are identical, not pushing down resulting the cnt in the first two rows
+ // being null
+ enabledOnlyWhenPushdownIsEnabled();
JSONObject actual =
executeQuery(
String.format(
@@ -81,4 +107,4 @@ public void testAppendColOverride() throws IOException {
rows("F", "DE", 101, null),
rows("F", "FL", 310, null));
}
-}
+}
\ No newline at end of file
diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLBasicIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLBasicIT.java
index 5f69159fec5..abbfbd1bf46 100644
--- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLBasicIT.java
+++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLBasicIT.java
@@ -10,6 +10,7 @@
import static org.opensearch.sql.util.MatcherUtils.schema;
import static org.opensearch.sql.util.MatcherUtils.verifyDataRows;
import static org.opensearch.sql.util.MatcherUtils.verifyErrorMessageContains;
+import static org.opensearch.sql.util.MatcherUtils.verifyNumOfRows;
import static org.opensearch.sql.util.MatcherUtils.verifySchema;
import java.io.IOException;
@@ -17,7 +18,6 @@
import org.junit.jupiter.api.Test;
import org.opensearch.client.Request;
import org.opensearch.client.ResponseException;
-import org.opensearch.sql.exception.SemanticCheckException;
import org.opensearch.sql.ppl.PPLIntegTestCase;
public class CalcitePPLBasicIT extends PPLIntegTestCase {
@@ -443,6 +443,13 @@ public void testBetweenWithDifferentTypes() throws IOException {
verifyDataRows(actual, rows("Hattie", 36), rows("Elinor", 36));
}
+ /**
+ * Verifies that a BETWEEN filter with mixed numeric bounds (integer field with integer lower and decimal upper) matches the expected integer rows.
+ *
+ * Runs a PPL query filtering rows where age between 35 and 38.5, and asserts the result schema contains firstname (string) and age (int) and that the expected rows are returned.
+ *
+ * @throws IOException if an I/O error occurs while executing the query
+ */
@Test
public void testBetweenWithDifferentTypes2() throws IOException {
JSONObject actual =
@@ -454,17 +461,44 @@ public void testBetweenWithDifferentTypes2() throws IOException {
verifyDataRows(actual, rows("Hattie", 36), rows("Elinor", 36));
}
+ /**
+ * Verifies that BETWEEN handles mixed-type bounds where one bound is a string and the other is numeric.
+ *
+ * Executes a PPL query filtering rows with age between '35' and 38 and asserts that the returned
+ * rows contain the expected first names and ages ("Hattie", 36) and ("Elinor", 36).
+ *
+ * @throws IOException if executing the query or reading the response fails
+ */
@Test
- public void testBetweenWithIncompatibleTypes() {
- Throwable e =
- assertThrowsWithReplace(
- SemanticCheckException.class,
- () ->
- executeQuery(
- String.format(
- "source=%s | where age between '35' and 38.5 | fields firstname, age",
- TEST_INDEX_BANK)));
- verifyErrorMessageContains(e, "BETWEEN expression types are incompatible");
+ public void testBetweenWithMixedTypes() throws IOException {
+ JSONObject actual =
+ executeQuery(
+ String.format(
+ "source=%s | where age between '35' and 38 | fields firstname, age",
+ TEST_INDEX_BANK));
+ verifyDataRows(actual, rows("Hattie", 36), rows("Elinor", 36));
+ }
+
+ /**
+ * Verifies that a BETWEEN expression mixing a string and a decimal produces no matching rows.
+ *
+ * Executes a query with age between '35' and 38.5 against the bank test index and asserts
+ * that the returned schema contains firstname (string) and age (int) and that the result set is empty.
+ *
+ * @throws IOException if executing the query or reading the response fails
+ */
+ @Test
+ public void testBetweenWithIncompatibleTypes() throws IOException {
+ // Plan: SAFE_CAST(NUMBER_TO_STRING(38.5:DECIMAL(3, 1))). The least restrictive type between
+ // int, decimal, and varchar is resolved to varchar. between '35' and '38.5' is then optimized
+ // to empty rows
+ JSONObject actual =
+ executeQuery(
+ String.format(
+ "source=%s | where age between '35' and 38.5 | fields firstname, age",
+ TEST_INDEX_BANK));
+ verifySchema(actual, schema("firstname", "string"), schema("age", "int"));
+ verifyNumOfRows(actual, 0);
}
@Test
@@ -654,4 +688,4 @@ public void testDecimalLiteral() throws IOException {
null,
null));
}
-}
+}
\ No newline at end of file
diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLExplainIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLExplainIT.java
index 674a7d96f8d..d54df31d90f 100644
--- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLExplainIT.java
+++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLExplainIT.java
@@ -6,6 +6,7 @@
package org.opensearch.sql.calcite.remote;
import static org.opensearch.sql.util.MatcherUtils.assertJsonEquals;
+import static org.opensearch.sql.util.MatcherUtils.assertYamlEqualsIgnoreId;
import java.io.IOException;
import org.junit.jupiter.api.Test;
@@ -31,17 +32,28 @@ public void init() throws Exception {
client().performRequest(request3);
}
+ /**
+ * Verifies that the YAML explain plan for a PPL query with a filter and selected fields
+ * matches the expected plan defined in "explain_filter.yaml".
+ *
+ * @throws IOException if executing the explain query or loading the expected plan fails
+ */
@Test
public void testExplainCommand() throws IOException {
- var result = explainQueryToString("source=test | where age = 20 | fields name, age");
- String expected =
- !isPushdownDisabled()
- ? loadFromFile("expectedOutput/calcite/explain_filter_w_pushdown.json")
- : loadFromFile("expectedOutput/calcite/explain_filter_wo_pushdown.json");
-
- assertJsonEquals(expected, result);
+ var result = explainQueryYaml("source=test | where age = 20 | fields name, age");
+ String expected = loadExpectedPlan("explain_filter.yaml");
+ assertYamlEqualsIgnoreId(expected, result);
}
+ /**
+ * Verifies that an extended EXPLAIN with code generation emits a Java binding method for Calcite's
+ * DataContext.
+ *
+ * Executes an extended EXPLAIN on a PPL query that includes a join and asserts the generated
+ * explanation contains the expected `bind(org.apache.calcite.DataContext)` method signature.
+ *
+ * @throws IOException if executing the PPL query or reading/normalizing the result fails
+ */
@Test
public void testExplainCommandExtendedWithCodegen() throws IOException {
var result =
@@ -86,4 +98,4 @@ private String executeWithReplace(String ppl) throws IOException {
var result = executeQueryToString(ppl);
return result.replace("\\r\\n", "\\n");
}
-}
+}
\ No newline at end of file
diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLPatternsIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLPatternsIT.java
index 46df914e611..07360218053 100644
--- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLPatternsIT.java
+++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalcitePPLPatternsIT.java
@@ -18,6 +18,7 @@
import com.google.common.collect.ImmutableMap;
import java.io.IOException;
import org.json.JSONObject;
+import org.junit.Ignore;
import org.junit.Test;
import org.opensearch.sql.ppl.PPLIntegTestCase;
@@ -287,6 +288,18 @@ public void testBrainLabelMode_ShowNumberedToken() throws IOException {
ImmutableList.of("-6620182933895093708"))));
}
+ /**
+ * Integration test for BRAIN pattern aggregation on HDFS logs that verifies aggregated
+ * pattern templates, their counts, and representative sample logs when numbered tokens
+ * are not exposed.
+ *
+ * Verifies the result schema contains `patterns_field` (string), `pattern_count` (bigint),
+ * and `sample_logs` (array), and asserts specific aggregated pattern rows with their
+ * corresponding counts and sample log examples.
+ *
+ * @throws IOException if the query execution or result parsing fails
+ */
+ @Ignore("To be fixed in https://github.com/opensearch-project/sql/issues/4968")
@Test
public void testBrainAggregationMode_NotShowNumberedToken() throws IOException {
JSONObject result =
@@ -337,6 +350,17 @@ public void testBrainAggregationMode_NotShowNumberedToken() throws IOException {
"PacketResponder failed for blk_-1547954353065580372")));
}
+ /**
+ * Verifies that Brain pattern matching in aggregation mode with numbered tokens produces the expected
+ * aggregated patterns, token mappings, and sample logs for HDFS log data.
+ *
+ * The test asserts the result schema contains `patterns_field`, `pattern_count`, `tokens`, and
+ * `sample_logs`, and verifies several aggregated pattern rows with their corresponding numbered-token
+ * mappings and sample log entries.
+ *
+ * @throws IOException if executing the test query or reading the results fails
+ */
+ @Ignore("To be fixed in https://github.com/opensearch-project/sql/issues/4968")
@Test
public void testBrainAggregationMode_ShowNumberedToken() throws IOException {
JSONObject result =
@@ -420,6 +444,17 @@ public void testBrainAggregationMode_ShowNumberedToken() throws IOException {
"PacketResponder failed for blk_-1547954353065580372")));
}
+ /**
+ * Integration test that verifies BRAIN pattern aggregation grouped by `level` produces
+ * expected patterns, token mappings, and sample logs when numbered tokens are shown.
+ *
+ * Verifies the result schema contains: level, patterns_field, pattern_count, tokens, and sample_logs,
+ * and asserts the aggregated rows contain the expected pattern labels, token-to-values mappings,
+ * and representative sample log entries.
+ *
+ * @throws IOException if query execution or result processing fails
+ */
+ @Ignore("To be fixed in https://github.com/opensearch-project/sql/issues/4968")
@Test
public void testBrainAggregationModeWithGroupByClause_ShowNumberedToken() throws IOException {
JSONObject result =
@@ -530,4 +565,4 @@ public void testBrainParseWithUUID_ShowNumberedToken() throws IOException {
"[PlaceOrder] user_id= This test loads the expected YAML plan and compares it to the explain output for a
+ * patterns query that invokes `method=brain` with aggregation mode and numbered tokens shown.
+ * The test is currently disabled pending resolution of https://github.com/opensearch-project/sql/issues/4968.
+ *
+ * @throws IOException if loading the expected plan file fails
+ */
+ @Ignore("Enable after fixing https://github.com/opensearch-project/sql/issues/4968")
@Test
public void testPatternsBrainMethodWithAggPushDownExplain() throws IOException {
// TODO: Correct calcite expected result once pushdown is supported
@@ -442,21 +479,35 @@ public void testPatternsBrainMethodWithAggPushDownExplain() throws IOException {
+ "| patterns email method=brain mode=aggregation show_numbered_token=true"));
}
+ /**
+ * Verifies the explain plan for a `stats count() by span(age,10)` query on the BANK test index.
+ *
+ * Compares the actual explain YAML to the expected plan loaded from "explain_stats_by_span.yaml".
+ *
+ * @throws IOException if reading the expected plan file fails
+ */
@Test
public void testStatsBySpan() throws IOException {
- String expected = loadExpectedPlan("explain_stats_by_span.json");
- assertJsonEqualsIgnoreId(
+ String expected = loadExpectedPlan("explain_stats_by_span.yaml");
+ assertYamlEqualsIgnoreId(
expected,
- explainQueryToString(
+ explainQueryYaml(
String.format("source=%s | stats count() by span(age,10)", TEST_INDEX_BANK)));
}
+ /**
+ * Verifies the explain plan for a stats-by-span query when `bucket_nullable=false`.
+ *
+ * Loads the expected YAML plan and asserts the generated explain YAML (ignoring IDs) matches it.
+ *
+ * @throws IOException if the expected plan file cannot be read
+ */
@Test
public void testStatsBySpanNonBucketNullable() throws IOException {
- String expected = loadExpectedPlan("explain_stats_by_span_non_bucket_nullable.json");
- assertJsonEqualsIgnoreId(
+ String expected = loadExpectedPlan("explain_stats_by_span_non_bucket_nullable.yaml");
+ assertYamlEqualsIgnoreId(
expected,
- explainQueryToString(
+ explainQueryYaml(
String.format(
"source=%s | stats bucket_nullable=false count() by span(age,10)",
TEST_INDEX_BANK)));
@@ -518,14 +569,9 @@ public void testDedupKeepEmptyFalsePushdown() throws IOException {
@Test
public void testSingleFieldRelevanceQueryFunctionExplain() throws IOException {
enabledOnlyWhenPushdownIsEnabled();
- String expected =
- isCalciteEnabled()
- ? loadFromFile("expectedOutput/calcite/explain_single_field_relevance_push.json")
- : loadFromFile("expectedOutput/ppl/explain_single_field_relevance_push.json");
-
- assertJsonEqualsIgnoreId(
- expected,
- explainQueryToString(
+ assertYamlEqualsIgnoreId(
+ loadExpectedPlan("explain_single_field_relevance_push.yaml"),
+ explainQueryYaml(
"source=opensearch-sql_test_index_account"
+ "| where match(email, '*@gmail.com', boost=1.0)"));
}
@@ -533,14 +579,9 @@ public void testSingleFieldRelevanceQueryFunctionExplain() throws IOException {
@Test
public void testMultiFieldsRelevanceQueryFunctionExplain() throws IOException {
enabledOnlyWhenPushdownIsEnabled();
- String expected =
- isCalciteEnabled()
- ? loadFromFile("expectedOutput/calcite/explain_multi_fields_relevance_push.json")
- : loadFromFile("expectedOutput/ppl/explain_multi_fields_relevance_push.json");
-
- assertJsonEqualsIgnoreId(
- expected,
- explainQueryToString(
+ assertYamlEqualsIgnoreId(
+ loadExpectedPlan("explain_multi_fields_relevance_push.yaml"),
+ explainQueryYaml(
"source=opensearch-sql_test_index_account"
+ "| where simple_query_string(['email', name 4.0], 'gmail',"
+ " default_operator='or', analyzer=english)"));
@@ -661,10 +702,10 @@ public void testExplainOnTake() throws IOException {
@Test
public void testExplainOnPercentile() throws IOException {
- String expected = loadExpectedPlan("explain_percentile.json");
- assertJsonEqualsIgnoreId(
+ String expected = loadExpectedPlan("explain_percentile.yaml");
+ assertYamlEqualsIgnoreId(
expected,
- explainQueryToString(
+ explainQueryYaml(
"source=opensearch-sql_test_index_account | stats percentile(balance, 50) as p50,"
+ " percentile(balance, 90) as p90"));
}
@@ -729,29 +770,37 @@ public void testSearchCommandWithChainedTimeModifier() throws IOException {
@Test
public void testExplainSearchBasicText() throws IOException {
// Example 1: Basic text search without field specification
- String expected = loadExpectedPlan("explain_search_basic_text.json");
- assertJsonEqualsIgnoreId(
- expected,
- explainQueryToString(String.format("search source=%s ERROR", TEST_INDEX_OTEL_LOGS)));
+ String expected = loadExpectedPlan("explain_search_basic_text.yaml");
+ assertYamlEqualsIgnoreId(
+ expected, explainQueryYaml(String.format("search source=%s ERROR", TEST_INDEX_OTEL_LOGS)));
}
+ /**
+ * Verifies the explain plan for a search query that uses a numeric '>' comparison on the
+ * `severityNumber` field.
+ *
+ * Loads the expected YAML plan and asserts that the actual explain output matches it,
+ * ignoring plan identifiers.
+ *
+ * @throws IOException if loading the expected plan file or retrieving the explain output fails
+ */
@Test
public void testExplainSearchNumericComparison() throws IOException {
// Example 2: Numeric field comparison with greater than
- String expected = loadExpectedPlan("explain_search_numeric_comparison.json");
- assertJsonEqualsIgnoreId(
+ String expected = loadExpectedPlan("explain_search_numeric_comparison.yaml");
+ assertYamlEqualsIgnoreId(
expected,
- explainQueryToString(
+ explainQueryYaml(
String.format("search source=%s severityNumber>15", TEST_INDEX_OTEL_LOGS)));
}
@Test
public void testExplainSearchWildcardStar() throws IOException {
// Example 3: Wildcard search with asterisk for pattern matching
- String expected = loadExpectedPlan("explain_search_wildcard_star.json");
- assertJsonEqualsIgnoreId(
+ String expected = loadExpectedPlan("explain_search_wildcard_star.yaml");
+ assertYamlEqualsIgnoreId(
expected,
- explainQueryToString(
+ explainQueryYaml(
String.format("search source=%s severityText=ERR*", TEST_INDEX_OTEL_LOGS)));
}
@@ -765,4 +814,4 @@ public void testStatsByDependentGroupFieldsExplain() throws IOException {
+ "| eval age1 = age * 10, age2 = age + 10, age3 = 10"
+ "| stats count() by age1, age2, age3, age"));
}
-}
+}
\ No newline at end of file
+ *
+ *
+ *
+ *
+ *
+ * @see SqlTypeCoercionRule
+ * @see PplTypeCoercion
+ */
+public class PplTypeCoercionRule {
+ /**
+ * PPL-specific additional type mapping rules
+ *
+ *
+ *
+ */
+ private static final Map
+ *
+ *
+ * Group by multiple CASE statements
+ *
+ * {@code
+ * source=opensearch-sql_test_index_bank
+ * | eval age_range = case(age < 30, 'u30', age < 40, 'u40' else 'u100'),
+ * balance_range = case(balance < 20000, 'medium' else 'high')
+ * | stats avg(balance) as avg_balance by age_range, balance_range
+ * }
+ *
+ * {@code
+ * SELECT AVG(`balance`) AS `avg_balance`,
+ * CASE WHEN `age` < 30 THEN 'u30' WHEN `age` < 40 THEN 'u40' ELSE 'u100' END AS `age_range`,
+ * CASE WHEN `balance` < 20000 THEN 'medium' ELSE 'high' END AS `balance_range`
+ * FROM `OpenSearch`.`opensearch-sql_test_index_bank`
+ * GROUP BY CASE WHEN `age` < 30 THEN 'u30' WHEN `age` < 40 THEN 'u40' ELSE 'u100' END,
+ * CASE WHEN `balance` < 20000 THEN 'medium' ELSE 'high' END
+ * }
+ *
+ * {@code
+ * SELECT SUM(CAST(`balance` AS DECIMAL(38, 19))) AS `sum(balance)`
+ * FROM (VALUES (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)) AS `t` (`account_number`, `firstname`, `address`, `balance`, `gender`, `age`, `lastname`, `_id`, `_index`, `_score`, `_maxscore`, `_sort`, `_routing`)
+ * WHERE 1 = 0
+ * }
+ *
+ * When converted back to logical plan, {@code CAST(`balance` AS DECIMAL(38, 19))} will fail because
+ * the type of balance is lost.
+ *
+ *
+ *
+ *
+ * @param call the SQL call to potentially rewrite and visit
+ * @return the node returned by the superclass visitor after applying any rewrites; may be `null`
+ */
+ @Override
+ public @org.checkerframework.checker.nullness.qual.Nullable SqlNode visit(SqlCall call) {
+ if (call.getOperator() instanceof SqlCountAggFunction && call.getOperandList().isEmpty()) {
+ // Convert COUNT() to COUNT(*) so that SqlCall.isCountStar() resolves to True
+ // This is useful when deriving the return types in SqlCountAggFunction#deriveType
+ call =
+ new SqlBasicCall(
+ SqlStdOperatorTable.COUNT,
+ List.of(SqlIdentifier.STAR),
+ call.getParserPosition(),
+ call.getFunctionQuantifier());
+ } else if (call.getKind() == SqlKind.IN || call.getKind() == SqlKind.NOT_IN) {
+ // Fix for tuple IN / NOT IN queries: Convert SqlNodeList to ROW SqlCall
+ //
+ // When RelToSqlConverter converts a tuple expression like (id, name) back to
+ // SqlNode, it generates a bare SqlNodeList instead of wrapping it in a ROW
+ // operator. This causes validation to fail because:
+ // 1. SqlValidator.deriveType() doesn't know how to handle SqlNodeList
+ // 2. SqlToRelConverter.visit(SqlNodeList) throws UnsupportedOperationException
+ //
+ // For example, the query:
+ // WHERE (id, name) NOT IN (SELECT uid, name FROM ...)
+ //
+ // After Rel-to-SQL conversion becomes:
+ // IN operator with operands: [SqlNodeList[id, name], SqlSelect[...]]
+ //
+ // But it should be:
+ // IN operator with operands: [ROW(id, name), SqlSelect[...]]
+ //
+ // This fix wraps the SqlNodeList in a ROW SqlCall before validation,
+ // ensuring proper type derivation and subsequent SQL-to-Rel conversion.
+ if (!call.getOperandList().isEmpty()
+ && call.getOperandList().get(0) instanceof SqlNodeList nodes) {
+ call.setOperand(0, SqlStdOperatorTable.ROW.createCall(nodes));
+ }
+ }
+ return super.visit(call);
+ }
+}
\ No newline at end of file
diff --git a/core/src/main/java/org/opensearch/sql/executor/QueryService.java b/core/src/main/java/org/opensearch/sql/executor/QueryService.java
index c85849df725..6fdb96daa8c 100644
--- a/core/src/main/java/org/opensearch/sql/executor/QueryService.java
+++ b/core/src/main/java/org/opensearch/sql/executor/QueryService.java
@@ -6,6 +6,7 @@
package org.opensearch.sql.executor;
import java.util.List;
+import java.util.Objects;
import java.util.Optional;
import javax.annotation.Nullable;
import lombok.AllArgsConstructor;
@@ -17,14 +18,21 @@
import org.apache.calcite.plan.hep.HepPlanner;
import org.apache.calcite.plan.hep.HepProgram;
import org.apache.calcite.plan.hep.HepProgramBuilder;
+import org.apache.calcite.prepare.CalciteCatalogReader;
import org.apache.calcite.rel.RelCollation;
import org.apache.calcite.rel.RelCollations;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.core.Sort;
import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.rel2sql.RelToSqlConverter;
+import org.apache.calcite.rel.rel2sql.SqlImplementor;
import org.apache.calcite.rel.rules.FilterMergeRule;
+import org.apache.calcite.runtime.CalciteContextException;
import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlNode;
import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql2rel.SqlToRelConverter;
import org.apache.calcite.tools.FrameworkConfig;
import org.apache.calcite.tools.Frameworks;
import org.apache.calcite.tools.Programs;
@@ -38,10 +46,20 @@
import org.opensearch.sql.calcite.SysLimit;
import org.opensearch.sql.calcite.plan.LogicalSystemLimit;
import org.opensearch.sql.calcite.plan.LogicalSystemLimit.SystemLimitType;
+import org.opensearch.sql.calcite.utils.PPLHintStrategyTable;
+import org.opensearch.sql.calcite.validate.OpenSearchSparkSqlDialect;
+import org.opensearch.sql.calcite.validate.PplConvertletTable;
+import org.opensearch.sql.calcite.validate.ValidationUtils;
+import org.opensearch.sql.calcite.validate.converters.PplRelToSqlNodeConverter;
+import org.opensearch.sql.calcite.validate.converters.PplSqlToRelConverter;
+import org.opensearch.sql.calcite.validate.shuttles.PplRelToSqlRelShuttle;
+import org.opensearch.sql.calcite.validate.shuttles.SkipRelValidationShuttle;
+import org.opensearch.sql.calcite.validate.shuttles.SqlRewriteShuttle;
import org.opensearch.sql.common.response.ResponseListener;
import org.opensearch.sql.common.setting.Settings;
import org.opensearch.sql.datasource.DataSourceService;
import org.opensearch.sql.exception.CalciteUnsupportedException;
+import org.opensearch.sql.exception.ExpressionEvaluationException;
import org.opensearch.sql.exception.NonFallbackCalciteException;
import org.opensearch.sql.planner.PlanContext;
import org.opensearch.sql.planner.Planner;
@@ -91,6 +109,17 @@ public void explain(
}
}
+ /**
+ * Execute an UnresolvedPlan using the Calcite planning and execution path, optionally falling back to the legacy engine on supported failures.
+ *
+ * This method runs the full Calcite pipeline (analysis, validation, optimization, conversion) and delegates execution to the ExecutionEngine.
+ * If a Calcite failure occurs and fallback is allowed (and the exception is not a NonFallbackCalciteException), the legacy execution path is invoked and the original Calcite failure is propagated to the legacy handler.
+ * Virtual machine errors (e.g., OutOfMemoryError) are rethrown and not reported to the listener.
+ *
+ * @param plan the unresolved query plan to execute
+ * @param queryType the type of the query (used to configure Calcite context)
+ * @param listener callback to receive the query response or failures
+ */
public void executeWithCalcite(
UnresolvedPlan plan,
QueryType queryType,
@@ -103,7 +132,8 @@ public void executeWithCalcite(
buildFrameworkConfig(), SysLimit.fromSettings(settings), queryType);
RelNode relNode = analyze(plan, context);
relNode = mergeAdjacentFilters(relNode);
- RelNode optimized = optimize(relNode, context);
+ RelNode validated = validate(relNode, context);
+ RelNode optimized = optimize(validated, context);
RelNode calcitePlan = convertToCalcitePlan(optimized);
executionEngine.execute(calcitePlan, context, listener);
} catch (Throwable t) {
@@ -129,6 +159,19 @@ public void executeWithCalcite(
settings);
}
+ /**
+ * Explains the given unresolved query using the Calcite-based planning and execution path.
+ *
+ *
+ *
+ */
void populate() {
registerOperator(MAX, SqlStdOperatorTable.MAX);
registerOperator(MIN, SqlStdOperatorTable.MIN);
@@ -1345,8 +1266,7 @@ void populate() {
register(
AVG,
(distinct, field, argList, ctx) -> ctx.relBuilder.avg(distinct, null, field),
- wrapSqlOperandTypeChecker(
- SqlStdOperatorTable.AVG.getOperandTypeChecker(), AVG.name(), false));
+ SqlStdOperatorTable.AVG.getOperandTypeChecker());
register(
COUNT,
@@ -1359,7 +1279,7 @@ void populate() {
return ctx.relBuilder.count(distinct, null, field);
}
},
- wrapSqlOperandTypeChecker(PPLOperandTypes.OPTIONAL_ANY, COUNT.name(), false));
+ OperandTypes.family(SqlTypeFamily.ANY).or(OperandTypes.family()));
register(
PERCENTILE_APPROX,
@@ -1373,10 +1293,7 @@ void populate() {
return UserDefinedFunctionUtils.makeAggregateCall(
PPLBuiltinOperators.PERCENTILE_APPROX, List.of(field), newArgList, ctx.relBuilder);
},
- wrapSqlOperandTypeChecker(
- extractTypeCheckerFromUDF(PPLBuiltinOperators.PERCENTILE_APPROX),
- PERCENTILE_APPROX.name(),
- false));
+ extractTypeCheckerFromUDF(PPLBuiltinOperators.PERCENTILE_APPROX));
register(
MEDIAN,
@@ -1400,8 +1317,7 @@ void populate() {
medianArgList,
ctx.relBuilder);
},
- wrapSqlOperandTypeChecker(
- PPLOperandTypes.NUMERIC.getInnerTypeChecker(), MEDIAN.name(), false));
+ OperandTypes.NUMERIC);
register(
EARLIEST,
@@ -1410,8 +1326,7 @@ void populate() {
return UserDefinedFunctionUtils.makeAggregateCall(
SqlStdOperatorTable.ARG_MIN, List.of(field), args, ctx.relBuilder);
},
- wrapSqlOperandTypeChecker(
- PPLOperandTypes.ANY_OPTIONAL_TIMESTAMP, EARLIEST.name(), false));
+ OperandTypes.ANY.or(OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.TIMESTAMP)));
register(
LATEST,
@@ -1420,8 +1335,7 @@ void populate() {
return UserDefinedFunctionUtils.makeAggregateCall(
SqlStdOperatorTable.ARG_MAX, List.of(field), args, ctx.relBuilder);
},
- wrapSqlOperandTypeChecker(
- PPLOperandTypes.ANY_OPTIONAL_TIMESTAMP, EARLIEST.name(), false));
+ PPLOperandTypes.ANY_OPTIONAL_TIMESTAMP);
// Register FIRST function - uses document order
register(
@@ -1430,8 +1344,7 @@ void populate() {
// Use our custom FirstAggFunction for document order aggregation
return ctx.relBuilder.aggregateCall(PPLBuiltinOperators.FIRST, field);
},
- wrapSqlOperandTypeChecker(
- PPLBuiltinOperators.FIRST.getOperandTypeChecker(), FIRST.name(), false));
+ PPLBuiltinOperators.FIRST.getOperandTypeChecker());
// Register LAST function - uses document order
register(
@@ -1440,11 +1353,21 @@ void populate() {
// Use our custom LastAggFunction for document order aggregation
return ctx.relBuilder.aggregateCall(PPLBuiltinOperators.LAST, field);
},
- wrapSqlOperandTypeChecker(
- PPLBuiltinOperators.LAST.getOperandTypeChecker(), LAST.name(), false));
+ PPLBuiltinOperators.LAST.getOperandTypeChecker());
}
}
+ /**
+ * Resolve the time field to use for time-based aggregations.
+ *
+ * If `argList` is empty, returns a single input reference to the row's `@timestamp` field.
+ * If `argList` contains elements, returns each argument after applying `PlanUtils.derefMapCall`.
+ *
+ * @param argList the provided time-field arguments (may be empty)
+ * @param ctx the planning context used to access the current row type and RexBuilder
+ * @return a list of `RexNode` representing the resolved time field(s)
+ * @throws IllegalArgumentException if `argList` is empty and the default `@timestamp` field cannot be found
+ */
static List
+ *
+ *
+ * @return a {@link UDFOperandMetadata} that enforces the above operand families for the SPAN function
+ */
@Override
public UDFOperandMetadata getOperandMetadata() {
return UDFOperandMetadata.wrap(
- (CompositeOperandTypeChecker)
- OperandTypes.family(
- SqlTypeFamily.CHARACTER, SqlTypeFamily.NUMERIC, SqlTypeFamily.CHARACTER)
- .or(
- OperandTypes.family(
- SqlTypeFamily.DATETIME, SqlTypeFamily.NUMERIC, SqlTypeFamily.CHARACTER))
- .or(
- OperandTypes.family(
- SqlTypeFamily.NUMERIC, SqlTypeFamily.NUMERIC, SqlTypeFamily.ANY)));
+ OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.NUMERIC, SqlTypeFamily.CHARACTER)
+ .or(
+ OperandTypes.family(
+ SqlTypeFamily.DATETIME, SqlTypeFamily.NUMERIC, SqlTypeFamily.CHARACTER))
+ .or(
+ OperandTypes.family(
+ SqlTypeFamily.NUMERIC, SqlTypeFamily.NUMERIC, SqlTypeFamily.ANY)));
}
public static class SpanImplementor implements NotNullImplementor {
+ /**
+ * Translates a SPAN RexCall into an executable expression for code generation.
+ *
+ *