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 transformFunction) { @@ -206,4 +290,4 @@ public RexLambdaRef captureVariable(RexNode fieldRef, String fieldName) { return lambdaRef; } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java b/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java index f63b4b1e2e0..893cefd4d25 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java +++ b/core/src/main/java/org/opensearch/sql/calcite/CalciteRelNodeVisitor.java @@ -43,9 +43,12 @@ import java.util.stream.IntStream; import java.util.stream.Stream; import lombok.AllArgsConstructor; +import lombok.NonNull; import org.apache.calcite.adapter.enumerable.RexToLixTranslator; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.plan.ViewExpanders; +import org.apache.calcite.rel.RelCollation; +import org.apache.calcite.rel.RelFieldCollation; import org.apache.calcite.rel.RelNode; import org.apache.calcite.rel.core.Aggregate; import org.apache.calcite.rel.core.JoinRelType; @@ -55,10 +58,14 @@ import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexCorrelVariable; +import org.apache.calcite.rex.RexFieldCollation; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexOver; +import org.apache.calcite.rex.RexShuttle; import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.rex.RexWindow; import org.apache.calcite.rex.RexWindowBounds; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.fun.SqlStdOperatorTable; @@ -1678,6 +1685,17 @@ private void validateFillNullTypeCompatibility( } } + /** + * Builds a Calcite relational plan that implements the streaming window operation defined by the given AST node. + * + * The produced plan handles reset-before/reset-after semantics, global sliding windows, grouped windows, and + * per-window function expressions; it also embeds existing plan collations into window OVER clauses and + * adds any required helper columns (sequence/segment/reset flags) used by the stream-window logic. + * + * @param node the StreamWindow AST node describing window type, grouping, window size, reset options and functions + * @param context the planning context containing the RelBuilder, visitors, and planner state + * @return a RelNode representing the compiled stream window operation ready for integration into the surrounding plan + */ @Override public RelNode visitStreamWindow(StreamWindow node, CalcitePlanContext context) { visitChildren(node, context); @@ -1740,6 +1758,7 @@ public RelNode visitStreamWindow(StreamWindow node, CalcitePlanContext context) // Default: first get rawExpr List overExpressions = node.getWindowFunctionList().stream().map(w -> rexVisitor.analyze(w, context)).toList(); + overExpressions = embedExistingCollationsIntoOver(overExpressions, context); if (hasGroup) { // only build sequence when there is by condition @@ -1781,6 +1800,101 @@ public RelNode visitStreamWindow(StreamWindow node, CalcitePlanContext context) return context.relBuilder.peek(); } + /** + * Embed the current plan's collation into any RexOver window expressions. + * + *

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 embedExistingCollationsIntoOver( + List overExpressions, CalcitePlanContext context) { + RelCollation existingCollation = context.relBuilder.peek().getTraitSet().getCollation(); + List<@NonNull RelFieldCollation> relCollations = + existingCollation == null ? List.of() : existingCollation.getFieldCollations(); + ImmutableList<@NonNull RexFieldCollation> rexCollations = + relCollations.stream() + .map(f -> relCollationToRexCollation(f, context.relBuilder)) + .collect(ImmutableList.toImmutableList()); + return overExpressions.stream() + .map( + n -> + n.accept( + new RexShuttle() { + /** + * Rewrite the given windowed aggregation to attach the visitor's collected collations. + * + * @param over the original RexOver node to be rewritten + * @return a RexNode representing the same windowed aggregation with the visitor's + * accumulated RexFieldCollation ordering embedded into its OVER specification + */ + @Override + public RexNode visitOver(RexOver over) { + RexWindow window = over.getWindow(); + return context.rexBuilder.makeOver( + over.getType(), + over.getAggOperator(), + over.getOperands(), + window.partitionKeys, + rexCollations, + window.getLowerBound(), + window.getUpperBound(), + window.isRows(), + true, + false, + over.isDistinct(), + over.ignoreNulls()); + } + })) + .toList(); + } + + /** + * Create a RexFieldCollation that mirrors the given RelFieldCollation by referencing + * the corresponding field through the provided RelBuilder. + * + * @param relCollation the source RelFieldCollation whose field index, direction, + * and null ordering will be translated + * @param builder RelBuilder used to produce a RexNode referencing the field + * @return a RexFieldCollation with equivalent sort direction and null-ordering flags + */ + private static RexFieldCollation relCollationToRexCollation( + RelFieldCollation relCollation, RelBuilder builder) { + RexNode fieldRef = builder.field(relCollation.getFieldIndex()); + + // Convert direction flags to SqlKind set + Set flags = new HashSet<>(); + if (relCollation.direction == RelFieldCollation.Direction.DESCENDING + || relCollation.direction == RelFieldCollation.Direction.STRICTLY_DESCENDING) { + flags.add(SqlKind.DESCENDING); + } + if (relCollation.nullDirection == RelFieldCollation.NullDirection.FIRST) { + flags.add(SqlKind.NULLS_FIRST); + } else if (relCollation.nullDirection == RelFieldCollation.NullDirection.LAST) { + flags.add(SqlKind.NULLS_LAST); + } + + return new RexFieldCollation(fieldRef, flags); + } + + /** + * Wraps each window expression in a CASE that yields NULL when the provided group-not-null + * predicate is false, preserving any explicit alias on the original expression. + * + *

Each expression is transformed to: CASE groupNotNull WHEN TRUE THEN ELSE NULL END. + * + * @param overExpressions list of window expressions to wrap; if an expression is of the form + * `expr AS alias` the alias is preserved on the wrapped expression + * @param groupNotNull predicate used as the CASE condition to determine non-null groups + * @param context plan context providing RexBuilder/RelBuilder utilities + * @return a new list of `RexNode` where each input expression has been wrapped with the CASE + * nulling logic and retains its original alias when present + */ private List wrapWindowFunctionsWithGroupNotNull( List overExpressions, RexNode groupNotNull, CalcitePlanContext context) { List wrappedOverExprs = new ArrayList<>(overExpressions.size()); @@ -3499,4 +3613,4 @@ private RexNode createOptimizedTransliteration( throw new RuntimeException("Failed to optimize sed expression: " + sedExpression, e); } } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/calcite/CalciteRexNodeVisitor.java b/core/src/main/java/org/opensearch/sql/calcite/CalciteRexNodeVisitor.java index 632b67df9c5..4b91eefa41b 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/CalciteRexNodeVisitor.java +++ b/core/src/main/java/org/opensearch/sql/calcite/CalciteRexNodeVisitor.java @@ -462,6 +462,14 @@ public RexNode visitFunction(Function node, CalcitePlanContext context) { throw new IllegalArgumentException("Unsupported operator: " + node.getFuncName()); } + /** + * Translates a window function expression into a Calcite RexNode representing an OVER() window call. + * + * @param node the window function expression to translate + * @param context the planning context used for resolving arguments, partitions, and frame + * @return a RexNode for the windowed function call (an OVER expression) + * @throws UnsupportedOperationException if the function name is not a supported window function + */ @Override public RexNode visitWindowFunction(WindowFunction node, CalcitePlanContext context) { Function windowFunction = (Function) node.getFunction(); @@ -480,26 +488,8 @@ public RexNode visitWindowFunction(WindowFunction node, CalcitePlanContext conte (arguments.isEmpty() || arguments.size() == 1) ? Collections.emptyList() : arguments.subList(1, arguments.size()); - List nodes = - PPLFuncImpTable.INSTANCE.validateAggFunctionSignature( - functionName, field, args, context.rexBuilder); - return nodes != null - ? PlanUtils.makeOver( - context, - functionName, - nodes.getFirst(), - nodes.size() <= 1 ? Collections.emptyList() : nodes.subList(1, nodes.size()), - partitions, - List.of(), - node.getWindowFrame()) - : PlanUtils.makeOver( - context, - functionName, - field, - args, - partitions, - List.of(), - node.getWindowFrame()); + return PlanUtils.makeOver( + context, functionName, field, args, partitions, List.of(), node.getWindowFrame()); }) .orElseThrow( () -> @@ -680,4 +670,4 @@ public RexNode visitUnresolvedArgument(UnresolvedArgument node, CalcitePlanConte context.rexBuilder.makeLiteral(node.getArgName()), value); } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/calcite/ExtendedRexBuilder.java b/core/src/main/java/org/opensearch/sql/calcite/ExtendedRexBuilder.java index 9b8ac7dfc97..64d3eb3af9b 100644 --- a/core/src/main/java/org/opensearch/sql/calcite/ExtendedRexBuilder.java +++ b/core/src/main/java/org/opensearch/sql/calcite/ExtendedRexBuilder.java @@ -14,14 +14,19 @@ import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlCallBinding; import org.apache.calcite.sql.SqlIntervalQualifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeUtil; +import org.apache.calcite.sql.validate.implicit.TypeCoercionImpl; import org.opensearch.sql.ast.expression.SpanUnit; import org.opensearch.sql.calcite.type.AbstractExprRelDataType; 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.exception.ExpressionEvaluationException; import org.opensearch.sql.exception.SemanticCheckException; @@ -117,6 +122,27 @@ public SqlIntervalQualifier createIntervalUntil(SpanUnit unit) { return new SqlIntervalQualifier(timeUnit, timeUnit, SqlParserPos.ZERO); } + /** + * Casts the given expression to the requested SQL type, applying framework-specific conversions + * and special-case rules for booleans, user-defined types, and number-to-character casts. + * + *

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 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 types) { + // Handle UDTs separately, otherwise the least restrictive type will become VARCHAR + if (types.stream().anyMatch(OpenSearchTypeUtil::isUserDefinedType)) { + int nullCount = 0; + int anyCount = 0; + int nullableCount = 0; + int dateCount = 0; + int timeCount = 0; + int ipCount = 0; + int binaryCount = 0; + int otherCount = 0; + for (RelDataType t : types) { + if (t.isNullable()) { + nullableCount++; + } + if (t.getSqlTypeName() == SqlTypeName.NULL) { + nullCount++; + } else if (t.getSqlTypeName() == SqlTypeName.ANY) { + anyCount++; + } + if (t.getSqlTypeName() == SqlTypeName.OTHER) { + otherCount++; + } + if (OpenSearchTypeUtil.isDate(t)) { + dateCount++; + } else if (OpenSearchTypeUtil.isTime(t)) { + timeCount++; + } else if (OpenSearchTypeUtil.isIp(t)) { + ipCount++; + } else if (OpenSearchTypeUtil.isBinary(t)) { + binaryCount++; + } + } + if (nullCount == 0 && anyCount == 0) { + RelDataType udt; + if (dateCount == types.size()) { + udt = createUDT(ExprUDT.EXPR_DATE, nullableCount > 0); + } else if (timeCount == types.size()) { + udt = createUDT(ExprUDT.EXPR_TIME, nullableCount > 0); + } + // There are cases where UDT IP interleaves with its intermediate SQL type for validation + // OTHER, we check otherCount to patch such cases + else if (ipCount == types.size() || otherCount == types.size()) { + udt = createUDT(ExprUDT.EXPR_IP, nullableCount > 0); + } else if (binaryCount == types.size()) { + udt = createUDT(ExprUDT.EXPR_BINARY, nullableCount > 0); + } else if (binaryCount == 0 && ipCount == 0) { + udt = createUDT(ExprUDT.EXPR_TIMESTAMP, nullableCount > 0); + } else { + udt = createSqlType(SqlTypeName.VARCHAR, nullableCount > 0); + } + return udt; + } } - - // Check for OpenSearch UDT types (EXPR_TIMESTAMP mapped to VARCHAR) - if (isUserDefinedType(fieldType)) { - AbstractExprRelDataType exprType = (AbstractExprRelDataType) fieldType; - ExprType udtType = exprType.getExprType(); - return udtType == ExprCoreType.TIMESTAMP - || udtType == ExprCoreType.DATE - || udtType == ExprCoreType.TIME; + RelDataType type = leastRestrictive(types, PplTypeCoercionRule.assignmentInstance()); + // Convert CHAR(precision) to VARCHAR so that results won't be padded + if (type != null && SqlTypeName.CHAR.equals(type.getSqlTypeName())) { + return createSqlType(SqlTypeName.VARCHAR, type.isNullable()); } - - // Fallback check if type string contains EXPR_TIMESTAMP - return fieldType.toString().contains("EXPR_TIMESTAMP"); + return type; } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/calcite/utils/OpenSearchTypeUtil.java b/core/src/main/java/org/opensearch/sql/calcite/utils/OpenSearchTypeUtil.java new file mode 100644 index 00000000000..1df80270c1f --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/utils/OpenSearchTypeUtil.java @@ -0,0 +1,180 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.utils; + +import lombok.experimental.UtilityClass; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.sql.type.SqlTypeUtil; +import org.opensearch.sql.calcite.type.AbstractExprRelDataType; +import org.opensearch.sql.data.type.ExprCoreType; +import org.opensearch.sql.data.type.ExprType; + +/** + * Utility methods for to derive types, containing special handling logics for user-defined-types. + * + * @see SqlTypeUtil utilities used during SQL validation or type derivation. + */ +@UtilityClass +public class OpenSearchTypeUtil { + /** + * Determines if the given RelDataType is a user-defined type (UDT). + * + * @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 isNumericOrCharacter(RelDataType fieldType) { + // Check standard SQL numeric types & string types (VARCHAR, CHAR) + if (SqlTypeUtil.isNumeric(fieldType) || SqlTypeUtil.isCharacter(fieldType)) { + 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; + } + + /** + * Determines whether the given RelDataType represents a time-based field (timestamp, date, or time), + * including standard SQL datetime types and OpenSearch user-defined time types. + * + * @param fieldType the RelDataType to check + * @return `true` if the type is a timestamp, date, or time type, `false` otherwise + */ + public static boolean isDatetime(RelDataType fieldType) { + // Check standard SQL time types + if (SqlTypeUtil.isDatetime(fieldType)) { + return true; + } + + // Check for OpenSearch UDT types (EXPR_TIMESTAMP mapped to VARCHAR) + if (isUserDefinedType(fieldType)) { + AbstractExprRelDataType exprType = (AbstractExprRelDataType) fieldType; + ExprType udtType = exprType.getExprType(); + return udtType == ExprCoreType.TIMESTAMP + || udtType == ExprCoreType.DATE + || udtType == ExprCoreType.TIME; + } + + // Fallback check if type string contains EXPR_TIMESTAMP + return fieldType.toString().contains("EXPR_TIMESTAMP"); + } + + /** + * Checks whether a {@link RelDataType} represents a date type. + * + *

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 CALCITE_TO_SPARK_MAPPING = + ImmutableMap.of( + "ARG_MIN", "MIN_BY", + "ARG_MAX", "MAX_BY", + "SAFE_CAST", "TRY_CAST"); + + private static final Map CALL_SEPARATOR = ImmutableMap.of("SAFE_CAST", "AS"); + + private OpenSearchSparkSqlDialect() { + super(DEFAULT_CONTEXT); + } + + /** + * Unparses a SQL call, translating selected Calcite function names to their Spark SQL equivalents before writing. + * + * @param writer the SqlWriter used to emit the unparsed SQL + * @param call the SqlCall representing the function or operator invocation to unparse + * @param leftPrec the left operator precedence to use while unparsing + * @param rightPrec the right operator precedence to use while unparsing + */ + @Override + public void unparseCall(SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) { + String operatorName = call.getOperator().getName(); + + // Replace Calcite specific functions with their Spark SQL equivalents + if (CALCITE_TO_SPARK_MAPPING.containsKey(operatorName)) { + unparseFunction( + writer, + call, + CALCITE_TO_SPARK_MAPPING.get(operatorName), + leftPrec, + rightPrec, + CALL_SEPARATOR.getOrDefault(operatorName, ",")); + } else { + super.unparseCall(writer, call, leftPrec, rightPrec); + } + } + + /** + * Return a SQL CAST target specification for the provided Calcite type, treating OpenSearch IP + * types as an alien system type named "IP" so they can be represented for Spark. + * + * @param type the Calcite type to produce a CAST specification for + * @return a {@code SqlDataTypeSpec} describing the CAST target for {@code type} (for IP types, + * a {@code SqlDataTypeSpec} wrapping an {@code SqlAlienSystemTypeNameSpec} with name + * "IP"); or {@code null} if no cast specification is available + */ + @Override + public @Nullable SqlNode getCastSpec(RelDataType type) { + // ExprIPType has sql type name OTHER, which can not be handled by spark dialect + if (OpenSearchTypeUtil.isIp(type)) { + return new SqlDataTypeSpec( + // It will use SqlTypeName.OTHER by type.getSqlTypeName() as OTHER is "borrowed" to + // represent IP type (see also: PplTypeCoercionRule.java) + new SqlAlienSystemTypeNameSpec("IP", type.getSqlTypeName(), SqlParserPos.ZERO), + SqlParserPos.ZERO); + } + return super.getCastSpec(type); + } + + /** + * Writes the given SQL call as a function invocation with the specified name and operand separator. + * + * Unparses `call` as `functionName(arg1{separator}arg2{separator}...)` using parentheses. Each operand is + * unparsed with the provided right precedence. + * + * @param writer the SqlWriter to write to + * @param call the SqlCall whose operands will be unparsed as function arguments + * @param functionName the name to use for the function + * @param leftPrec the left precedence (provided for callers; not used for operand unparsing) + * @param rightPrec the precedence to use when unparsing each operand + * @param separator the string placed between arguments (e.g., "," or " AS ") + */ + private void unparseFunction( + SqlWriter writer, + SqlCall call, + String functionName, + int leftPrec, + int rightPrec, + String separator) { + writer.print(functionName); + final SqlWriter.Frame frame = writer.startList("(", ")"); + for (int i = 0; i < call.operandCount(); i++) { + if (i > 0) { + writer.sep(separator); + } + call.operand(i).unparse(writer, 0, rightPrec); + } + writer.endList(frame); + } + + /** + * Provide a SqlConformance that delegates to the superclass conformance but enables liberal mode. + * + * @return a SqlConformance delegating to the superclass conformance with `isLiberal()` overridden to return `true` + */ + @Override + public SqlConformance getConformance() { + return new ConformanceDelegate(super.getConformance()); + } + + /** SqlConformance delegator that enables liberal mode for LEFT SEMI/ANTI JOIN support. */ + private static class ConformanceDelegate implements SqlConformance { + @Delegate private final SqlConformance delegate; + + /** + * Creates a ConformanceDelegate that forwards conformance checks to the given delegate. + * + * @param delegate the {@link SqlConformance} instance to which all method calls will be delegated + */ + ConformanceDelegate(SqlConformance delegate) { + this.delegate = delegate; + } + + /** + * Indicate that SQL conformance operates in liberal mode to permit additional join forms. + * + * @return `true` to enable liberal SQL features such as LEFT ANTI JOIN and LEFT SEMI JOIN. + */ + @Override + public boolean isLiberal() { + // This allows SQL feature LEFT ANTI JOIN & LEFT SEMI JOIN + return true; + } + } +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/calcite/validate/PplConvertletTable.java b/core/src/main/java/org/opensearch/sql/calcite/validate/PplConvertletTable.java new file mode 100644 index 00000000000..9dcd840d607 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/validate/PplConvertletTable.java @@ -0,0 +1,100 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.validate; + +import java.util.HashMap; +import java.util.Map; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlFunction; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql2rel.ReflectiveConvertletTable; +import org.apache.calcite.sql2rel.SqlRexConvertlet; +import org.apache.calcite.sql2rel.StandardConvertletTable; +import org.checkerframework.checker.initialization.qual.UnderInitialization; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.opensearch.sql.calcite.utils.OpenSearchTypeUtil; +import org.opensearch.sql.expression.function.PPLBuiltinOperators; + +public class PplConvertletTable extends ReflectiveConvertletTable { + public static PplConvertletTable INSTANCE = new PplConvertletTable(); + private final Map map = new HashMap<>(); + + /** + * Initializes the convertlet table and registers PPL-specific operator convertlets. + * + *

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`.

+ */ + protected boolean coerceOperandType( + @Nullable SqlValidatorScope scope, SqlCall call, int index, RelDataType targetType) { + // Transform the JavaType to SQL type because the SqlDataTypeSpec + // does not support deriving JavaType yet. + if (RelDataTypeFactoryImpl.isJavaType(targetType)) { + targetType = ((JavaTypeFactory) factory).toSql(targetType); + } + + SqlNode operand = call.getOperandList().get(index); + if (operand instanceof SqlDynamicParam) { + // Do not support implicit type coercion for dynamic param. + return false; + } + requireNonNull(scope, "scope"); + RelDataType operandType = validator.deriveType(scope, operand); + if (coerceStringToArray(call, operand, index, operandType, targetType)) { + return true; + } + + // Check it early. + if (!needToCast(scope, operand, targetType, PplTypeCoercionRule.lenientInstance())) { + return false; + } + // Fix up nullable attr. + RelDataType targetType1 = ValidationUtils.syncAttributes(factory, operandType, targetType); + SqlNode desired = castTo(operand, targetType1); + call.setOperand(index, desired); + // SAFE_CAST always results in nullable return type. See + // SqlCastFunction#createTypeWithNullabilityFromExpr + if (SqlKind.SAFE_CAST.equals(desired.getKind())) { + targetType1 = factory.createTypeWithNullability(targetType1, true); + } + updateInferredType(desired, targetType1); + return true; + } + + /** + * Create a SqlNode that casts the given expression to the specified target type. + * + * If the target type is a datetime or IP type, produces the corresponding PPL builtin operator + * call (DATE, TIMESTAMP, TIME, or IP). For all other types, produces a SAFE_CAST node to the + * specified type and preserves the target type's nullability to avoid exceptions on malformed + * values. + * + * @param node the expression to cast + * @param type the desired target relational type; datetime/IP types produce PPL operator calls + * @return a SqlNode that performs the cast (PPL builtin call for datetime/IP, or SAFE_CAST otherwise) + * @throws UnsupportedOperationException if the target datetime/IP ExprType is not supported + */ + private static SqlNode castTo(SqlNode node, RelDataType type) { + if (OpenSearchTypeUtil.isDatetime(type) || OpenSearchTypeUtil.isIp(type)) { + ExprType exprType = OpenSearchTypeFactory.convertRelDataTypeToExprType(type); + return switch (exprType) { + case ExprCoreType.DATE -> + PPLBuiltinOperators.DATE.createCall(node.getParserPosition(), node); + case ExprCoreType.TIMESTAMP -> + PPLBuiltinOperators.TIMESTAMP.createCall(node.getParserPosition(), node); + case ExprCoreType.TIME -> + PPLBuiltinOperators.TIME.createCall(node.getParserPosition(), node); + case ExprCoreType.IP -> PPLBuiltinOperators.IP.createCall(node.getParserPosition(), node); + default -> throw new UnsupportedOperationException("Unsupported type: " + exprType); + }; + } + // Use SAFE_CAST instead of CAST to avoid throwing errors when numbers are malformatted + return SqlLibraryOperators.SAFE_CAST.createCall( + node.getParserPosition(), + node, + SqlTypeUtil.convertTypeToSpec(type).withNullable(type.isNullable())); + } +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/calcite/validate/PplTypeCoercionRule.java b/core/src/main/java/org/opensearch/sql/calcite/validate/PplTypeCoercionRule.java new file mode 100644 index 00000000000..3a8b4d63c11 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/validate/PplTypeCoercionRule.java @@ -0,0 +1,131 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.validate; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import lombok.NonNull; +import org.apache.calcite.sql.type.SqlTypeAssignmentRule; +import org.apache.calcite.sql.type.SqlTypeCoercionRule; +import org.apache.calcite.sql.type.SqlTypeName; + +/** + * Provides PPL-specific type coercion rules that extend Calcite's default type coercion behavior. + * + *

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: + * + *

+ * + * @see SqlTypeCoercionRule + * @see PplTypeCoercion + */ +public class PplTypeCoercionRule { + /** + * PPL-specific additional type mapping rules + * + * + */ + private static final Map> additionalMapping = + Map.of( + SqlTypeName.OTHER, + ImmutableSet.of(SqlTypeName.OTHER, SqlTypeName.VARCHAR, SqlTypeName.CHAR), + SqlTypeName.VARCHAR, + ImmutableSet.builder() + .add(SqlTypeName.OTHER) + .addAll(SqlTypeName.NUMERIC_TYPES) + .build(), + SqlTypeName.CHAR, + ImmutableSet.of(SqlTypeName.OTHER)); + + private static final SqlTypeCoercionRule INSTANCE = + SqlTypeCoercionRule.instance( + mergeMapping(SqlTypeCoercionRule.instance().getTypeMapping(), additionalMapping)); + private static final SqlTypeCoercionRule LENIENT_INSTANCE = + SqlTypeCoercionRule.instance( + mergeMapping(SqlTypeCoercionRule.lenientInstance().getTypeMapping(), additionalMapping)); + private static final SqlTypeCoercionRule ASSIGNMENT_INSTANCE = + SqlTypeCoercionRule.instance( + mergeMapping(SqlTypeAssignmentRule.instance().getTypeMapping(), additionalMapping)); + + /** + * Provides the standard PPL type coercion rule merging Calcite's default mappings with PPL-specific mappings. + * + * @return the standard SqlTypeCoercionRule augmented with PPL-specific type mappings + */ + public static SqlTypeCoercionRule instance() { + return INSTANCE; + } + + /** + * Provides the PPL type coercion rule configured for lenient coercions. + * + * @return the lenient PPL {@link SqlTypeCoercionRule} that merges Calcite's lenient mappings with PPL-specific additional mappings + */ + public static SqlTypeCoercionRule lenientInstance() { + return LENIENT_INSTANCE; + } + + /** + * Provides the PPL-specific type assignment validation rule. + * + * @return the SqlTypeCoercionRule that validates whether a type can be assigned from another type + */ + public static SqlTypeCoercionRule assignmentInstance() { + return ASSIGNMENT_INSTANCE; + } + + /** + * Merges two mappings from keys to immutable sets, unioning value-sets for duplicate keys. + * + * Combines entries from `base` and `addition`; when a key exists in both maps, the resulting + * value is the union of both sets. The returned map and all value sets are immutable. + * + * @param the key and value element type + * @param base the primary mapping to include in the result + * @param addition the secondary mapping whose entries are merged into `base` + * @return an immutable map containing all keys from both inputs with immutable sets of merged values + */ + private static Map> mergeMapping( + Map> base, Map> addition) { + return Stream.concat(base.entrySet().stream(), addition.entrySet().stream()) + .collect( + Collectors.collectingAndThen( + Collectors.toMap( + Map.Entry::getKey, + Map.Entry::getValue, + (b, a) -> { + Set combined = new HashSet<>(b); + combined.addAll(a); + return ImmutableSet.copyOf(combined); + }), + ImmutableMap::copyOf)); + } +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/calcite/validate/PplValidator.java b/core/src/main/java/org/opensearch/sql/calcite/validate/PplValidator.java new file mode 100644 index 00000000000..e822893640b --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/validate/PplValidator.java @@ -0,0 +1,251 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.validate; + +import static org.opensearch.sql.calcite.validate.ValidationUtils.createUDTWithAttributes; + +import java.util.List; +import java.util.function.Function; +import org.apache.calcite.jdbc.CalcitePrepare; +import org.apache.calcite.jdbc.CalciteSchema; +import org.apache.calcite.prepare.CalciteCatalogReader; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rel.type.RelRecordType; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.server.CalciteServerStatement; +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlOperatorTable; +import org.apache.calcite.sql.type.ArraySqlType; +import org.apache.calcite.sql.type.MapSqlType; +import org.apache.calcite.sql.type.MultisetSqlType; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.sql.validate.SqlValidatorCatalogReader; +import org.apache.calcite.sql.validate.SqlValidatorImpl; +import org.apache.calcite.sql.validate.SqlValidatorScope; +import org.apache.calcite.tools.FrameworkConfig; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.opensearch.sql.calcite.type.AbstractExprRelDataType; +import org.opensearch.sql.calcite.utils.OpenSearchTypeFactory; +import org.opensearch.sql.calcite.utils.OpenSearchTypeFactory.ExprUDT; +import org.opensearch.sql.calcite.utils.OpenSearchTypeUtil; +import org.opensearch.sql.calcite.utils.UserDefinedFunctionUtils; +import org.opensearch.sql.data.type.ExprCoreType; +import org.opensearch.sql.data.type.ExprType; + +/** + * Custom SQL validator for PPL queries. + * + *

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 convert) { + if (type == null) return null; + + if (type instanceof RelRecordType recordType) { + List subTypes = + recordType.getFieldList().stream().map(RelDataTypeField::getType).map(convert).toList(); + return typeFactory.createTypeWithNullability( + typeFactory.createStructType(subTypes, recordType.getFieldNames()), + recordType.isNullable()); + } + if (type instanceof ArraySqlType arrayType) { + return typeFactory.createTypeWithNullability( + typeFactory.createArrayType(convert.apply(arrayType.getComponentType()), -1), + arrayType.isNullable()); + } + if (type instanceof MapSqlType mapType) { + return typeFactory.createTypeWithNullability( + typeFactory.createMapType( + convert.apply(mapType.getKeyType()), convert.apply(mapType.getValueType())), + mapType.isNullable()); + } + if (type instanceof MultisetSqlType multisetType) { + return typeFactory.createTypeWithNullability( + typeFactory.createMultisetType(convert.apply(multisetType.getComponentType()), -1), + multisetType.isNullable()); + } + + return convert.apply(type); + } +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/calcite/validate/SqlOperatorTableProvider.java b/core/src/main/java/org/opensearch/sql/calcite/validate/SqlOperatorTableProvider.java new file mode 100644 index 00000000000..3dac79431c1 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/validate/SqlOperatorTableProvider.java @@ -0,0 +1,25 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.validate; + +import org.apache.calcite.sql.SqlOperatorTable; + +/** + * Provider interface for obtaining SqlOperatorTable instances. + * + *

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 acceptableErrorMessages = + List.of( + "Aggregate expressions cannot be nested", + "Windowed aggregate expression is illegal in GROUP BY clause"); + return e.getMessage() != null + && acceptableErrorMessages.stream().anyMatch(e.getMessage()::contains); + } +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/calcite/validate/converters/PplRelToSqlNodeConverter.java b/core/src/main/java/org/opensearch/sql/calcite/validate/converters/PplRelToSqlNodeConverter.java new file mode 100644 index 00000000000..08d643f1190 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/validate/converters/PplRelToSqlNodeConverter.java @@ -0,0 +1,175 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.validate.converters; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.Correlate; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.rel.rel2sql.RelToSqlConverter; +import org.apache.calcite.sql.JoinConditionType; +import org.apache.calcite.sql.JoinType; +import org.apache.calcite.sql.SqlDialect; +import org.apache.calcite.sql.SqlHint; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlJoin; +import org.apache.calcite.sql.SqlLiteral; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.parser.SqlParserPos; + +/** + * An extension of {@link RelToSqlConverter} to convert a relation algebra tree, translated from a + * PPL query, into a SQL statement. + * + *

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 hints = + e.getHints().stream() + .map(relHint -> (SqlNode) toSqlHint(relHint, POS)) + .collect(Collectors.toCollection(ArrayList::new)); + r.asSelect().setHints(SqlNodeList.of(POS, hints)); + } + return r; + } + + /** + * Convert a Calcite RelHint into an equivalent SqlHint. + * + * Converts key-value options to a KV_LIST, literal options to a LITERAL_LIST, + * and produces an EMPTY hint when no options are present. + * + * @param hint the relational hint to convert + * @param pos parser position to attach to created SQL nodes + * @return a SqlHint representing the same hint and options as {@code hint} + */ + private static SqlHint toSqlHint(RelHint hint, SqlParserPos pos) { + if (hint.kvOptions != null) { + return new SqlHint( + pos, + new SqlIdentifier(hint.hintName, pos), + SqlNodeList.of( + pos, + hint.kvOptions.entrySet().stream() + .flatMap( + e -> + Stream.of( + new SqlIdentifier(e.getKey(), pos), + SqlLiteral.createCharString(e.getValue(), pos))) + .collect(Collectors.toList())), + SqlHint.HintOptionFormat.KV_LIST); + } else if (hint.listOptions != null) { + return new SqlHint( + pos, + new SqlIdentifier(hint.hintName, pos), + SqlNodeList.of( + pos, + hint.listOptions.stream() + .map(e -> SqlLiteral.createCharString(e, pos)) + .collect(Collectors.toList())), + SqlHint.HintOptionFormat.LITERAL_LIST); + } + return new SqlHint( + pos, + new SqlIdentifier(hint.hintName, pos), + SqlNodeList.EMPTY, + SqlHint.HintOptionFormat.EMPTY); + } +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/calcite/validate/converters/PplSqlToRelConverter.java b/core/src/main/java/org/opensearch/sql/calcite/validate/converters/PplSqlToRelConverter.java new file mode 100644 index 00000000000..af8da354c05 --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/validate/converters/PplSqlToRelConverter.java @@ -0,0 +1,110 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.validate.converters; + +import java.util.List; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.prepare.Prepare; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.calcite.sql.JoinType; +import org.apache.calcite.sql.SqlJoin; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.sql2rel.SqlRexConvertletTable; +import org.apache.calcite.sql2rel.SqlToRelConverter; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class PplSqlToRelConverter extends SqlToRelConverter { + /** + * Creates a PplSqlToRelConverter configured for SQL-to-relational conversion with + * support for Spark-specific join semantics. + * + * @param viewExpander expands views into relational expressions + * @param validator SQL validator used during conversion, or {@code null} + * @param catalogReader catalog reader for resolving tables and schemas + * @param cluster optimization cluster that provides planner context and type factory + * @param convertletTable convertlet table for SQL-to-Rex conversions + * @param config conversion configuration options + */ + public PplSqlToRelConverter( + RelOptTable.ViewExpander viewExpander, + @Nullable SqlValidator validator, + Prepare.CatalogReader catalogReader, + RelOptCluster cluster, + SqlRexConvertletTable convertletTable, + Config config) { + super(viewExpander, validator, catalogReader, cluster, convertletTable, config); + } + + /** + * Override to support Spark SQL's LEFT ANTI JOIN and LEFT SEMI JOIN conversion to RelNode. + * + *

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 fieldNames) { + JoinType originalJoinType = null; + if (from instanceof SqlJoin join) { + JoinType joinType = join.getJoinType(); + if (joinType == JoinType.LEFT_SEMI_JOIN || joinType == JoinType.LEFT_ANTI_JOIN) { + join.setOperand(2, JoinType.LEFT.symbol(from.getParserPosition())); + originalJoinType = joinType; + } + } + super.convertFrom(bb, from, fieldNames); + if (originalJoinType != null) { + RelNode root = bb.root(); + if (root != null) { + JoinRelType correctJoinType = + originalJoinType == JoinType.LEFT_SEMI_JOIN ? JoinRelType.SEMI : JoinRelType.ANTI; + RelNode fixedRoot = modifyJoinType(root, correctJoinType); + bb.setRoot(fixedRoot, false); + } + } + } + + /** + * Recursively replaces LEFT joins in the given relational subtree with the provided join type. + * + * @param root the root of the relational subtree to inspect and potentially modify + * @param correctJoinType the JoinRelType to apply where a LEFT join should be replaced (e.g., SEMI or ANTI) + * @return the possibly-modified relational subtree; a new RelNode when a replacement or propagated change occurs, otherwise the original `root` + */ + private RelNode modifyJoinType(RelNode root, JoinRelType correctJoinType) { + if (root instanceof LogicalProject project) { + RelNode input = project.getInput(); + RelNode fixedInput = modifyJoinType(input, correctJoinType); + if (fixedInput != input) { + return project.copy( + project.getTraitSet(), fixedInput, project.getProjects(), project.getRowType()); + } + } else if (root instanceof LogicalJoin join) { + if (join.getJoinType() == JoinRelType.LEFT) { + return join.copy( + join.getTraitSet(), + join.getCondition(), + join.getLeft(), + join.getRight(), + correctJoinType, + join.isSemiJoinDone()); + } + } + return root; + } +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/calcite/validate/shuttles/PplRelToSqlRelShuttle.java b/core/src/main/java/org/opensearch/sql/calcite/validate/shuttles/PplRelToSqlRelShuttle.java new file mode 100644 index 00000000000..1b58dab62ac --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/validate/shuttles/PplRelToSqlRelShuttle.java @@ -0,0 +1,101 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.validate.shuttles; + +import java.math.BigDecimal; +import java.util.List; +import org.apache.calcite.avatica.util.TimeUnit; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelShuttleImpl; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.sql.SqlIntervalQualifier; +import org.apache.calcite.sql.fun.SqlLibraryOperators; +import org.apache.calcite.sql.type.SqlTypeName; + +/** + * A RelShuttle that recursively visits all RelNodes and their RexNode expressions to fix interval + * literals and float literal before/after SQL conversion. + * + *

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: + * + *

    + *
  • Binning on datetime types, which is only executable after pushdown. + *
  • Aggregates with multiple complex CASE statements, which cause field reference issues during + * the SQL-to-Rel conversion. + *
  • LogicalValues is used to populate empty row values + *
+ * + * Group by multiple CASE statements + * + *

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: + * + *

{@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
+ * }
+ * + *

During validation, this PPL query is converted to SQL: + * + *

{@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
+ * }
+ * + *

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: + * + *

{@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. + * + *

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> SKIP_CALLS; + + /** Predicates about logical aggregates that should not be validated */ + public static final List> SKIP_AGGREGATES; + + /** Predicates about logical values that should not be validated */ + public static final List> SKIP_VALUES; + + static { + // TODO: Make incompatible operations like bin-on-timestamp a validatable UDFs so that they can + // be still be converted to SqlNode and back to RelNode + Predicate binOnTimestamp = + call -> { + if ("WIDTH_BUCKET".equalsIgnoreCase(call.getOperator().getName())) { + if (!call.getOperands().isEmpty()) { + RexNode firstOperand = call.getOperands().get(0); + return OpenSearchTypeUtil.isDatetime(firstOperand.getType()); + } + } + return false; + }; + Predicate groupByMultipleCases = + aggregate -> { + if (aggregate.getGroupCount() > 1 + && aggregate.getInput() instanceof LogicalProject project) { + long nGroupByCase = + project.getProjects().stream().filter(p -> p.isA(SqlKind.CASE)).count(); + return nGroupByCase > 1; + } + return false; + }; + Predicate createEmptyRow = values -> values.getTuples().isEmpty(); + SKIP_CALLS = List.of(binOnTimestamp); + SKIP_AGGREGATES = List.of(groupByMultipleCases); + SKIP_VALUES = List.of(createEmptyRow); + } + + /** + * Visits a LogicalAggregate and marks validation to be skipped when any configured predicate matches. + * + * @param aggregate the aggregate node to inspect + * @return the original {@code aggregate} if a skip predicate matched and skipping was enabled; otherwise the visited aggregate node returned by the superclass + */ + @Override + public RelNode visit(LogicalAggregate aggregate) { + for (Predicate skipAgg : SKIP_AGGREGATES) { + if (skipAgg.test(aggregate)) { + shouldSkip = true; + return aggregate; + } + } + return super.visit(aggregate); + } + + /** + * Visits a LogicalValues node and marks validation to be skipped when it matches any configured skip predicate. + * + * @param values LogicalValues node to inspect for skip conditions + * @return the original {@code values} node if a skip predicate matches (and skipping is enabled), otherwise the result of visiting the node via the superclass + */ + @Override + public RelNode visit(LogicalValues values) { + for (Predicate skipValues : SKIP_VALUES) { + if (skipValues.test(values)) { + shouldSkip = true; + return values; + } + } + return super.visit(values); + } + + /** + * Initializes the SkipRelValidationShuttle and installs a RexShuttle that detects skip conditions within RexCall nodes. + * + *

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 skipCall : SKIP_CALLS) { + if (skipCall.test(call)) { + shouldSkip = true; + return call; + } + } + return super.visitCall(call); + } + }; + } + + /** + * Indicates whether traversal detected conditions that should bypass validation. + * + * @return true if validation should be skipped, false otherwise. + */ + public boolean shouldSkipValidation() { + return shouldSkip; + } + + /** + * Visits the specified child relational node and applies the shuttle for expression-level checks. + * + * @param parent the parent relational node containing the child + * @param i the index of the child within the parent + * @param child the child relational node to visit + * @return the child node after visiting and applying the internal RexShuttle (may be unchanged) + */ + @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/SqlRewriteShuttle.java b/core/src/main/java/org/opensearch/sql/calcite/validate/shuttles/SqlRewriteShuttle.java new file mode 100644 index 00000000000..00190efad1f --- /dev/null +++ b/core/src/main/java/org/opensearch/sql/calcite/validate/shuttles/SqlRewriteShuttle.java @@ -0,0 +1,90 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.calcite.validate.shuttles; + +import java.util.Collections; +import java.util.List; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.fun.SqlCountAggFunction; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.util.SqlShuttle; +import org.opensearch.sql.calcite.OpenSearchSchema; + +public class SqlRewriteShuttle extends SqlShuttle { + /** + * Normalize an identifier by stripping the OpenSearch schema qualifier when present. + * + * @param id the identifier to normalize + * @return a {@code SqlIdentifier} containing only the table name if {@code id} has two parts + * and is prefixed with the OpenSearch schema name; otherwise the visit result for {@code id} + */ + @Override + public SqlNode visit(SqlIdentifier id) { + // Remove database qualifier, keeping only table name + if (id.names.size() == 2 && OpenSearchSchema.OPEN_SEARCH_SCHEMA_NAME.equals(id.names.get(0))) { + return new SqlIdentifier(Collections.singletonList(id.names.get(1)), id.getParserPosition()); + } + return super.visit(id); + } + + /** + * Rewrite SQL call nodes to normalize specific patterns before delegation. + * + *

Performs two targeted rewrites: + *

    + *
  • Converts a COUNT() aggregate with no operands into COUNT(*) so that `isCountStar()` and + * type derivation behave correctly.
  • + *
  • For IN and NOT IN predicates whose first operand is a bare `SqlNodeList`, wraps that list + * in a `ROW(...)` call so tuple predicates validate and convert to relational form correctly.
  • + *
+ * + * @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. + * + *

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 filledTypes, RelDataTypeFactory typeFactory) { - RexCall rexCall = (RexCall) rexLambda.getExpression(); - SqlReturnTypeInference returnInfer = rexCall.getOperator().getReturnTypeInference(); - List lambdaOperands = rexCall.getOperands(); - List filledOperands = new ArrayList<>(); - for (RexNode rexNode : lambdaOperands) { - if (rexNode instanceof RexLambdaRef rexLambdaRef) { - if (rexLambdaRef.getType().getSqlTypeName() == SqlTypeName.ANY) { - filledOperands.add( - new RexLambdaRef( - rexLambdaRef.getIndex(), - rexLambdaRef.getName(), - filledTypes.get(rexLambdaRef.getName()))); - } else { - filledOperands.add(rexNode); - } - } else if (rexNode instanceof RexCall) { - filledOperands.add( - reInferReturnTypeForRexCallInsideLambda((RexCall) rexNode, filledTypes, typeFactory)); - } else { - filledOperands.add(rexNode); - } - } - return returnInfer.inferReturnType( - new RexCallBinding(typeFactory, rexCall.getOperator(), filledOperands, List.of())); - } - - public static RexCall reInferReturnTypeForRexCallInsideLambda( - RexCall rexCall, Map argTypes, RelDataTypeFactory typeFactory) { - List filledOperands = new ArrayList<>(); - List rexCallOperands = rexCall.getOperands(); - for (RexNode rexNode : rexCallOperands) { - if (rexNode instanceof RexLambdaRef rexLambdaRef) { - filledOperands.add( - new RexLambdaRef( - rexLambdaRef.getIndex(), - rexLambdaRef.getName(), - argTypes.get(rexLambdaRef.getName()))); - } else if (rexNode instanceof RexCall) { - filledOperands.add( - reInferReturnTypeForRexCallInsideLambda((RexCall) rexNode, argTypes, typeFactory)); - } else { - filledOperands.add(rexNode); - } - } - RelDataType returnType = - rexCall - .getOperator() - .inferReturnType( - new RexCallBinding(typeFactory, rexCall.getOperator(), filledOperands, List.of())); - return rexCall.clone(returnType, filledOperands); - } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MVAppendFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MVAppendFunctionImpl.java index 107df5eea4e..7c87cf05e6a 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MVAppendFunctionImpl.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MVAppendFunctionImpl.java @@ -18,6 +18,7 @@ import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rex.RexCall; import org.apache.calcite.sql.SqlOperatorBinding; +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; @@ -48,11 +49,26 @@ public SqlReturnTypeInference getReturnTypeInference() { }; } + /** + * Indicates that this UDF accepts a variadic (variable number of) operands. + * + * @return UDFOperandMetadata describing that the function accepts a variable number of operands + */ @Override public UDFOperandMetadata getOperandMetadata() { - return null; + return UDFOperandMetadata.wrap(OperandTypes.VARIADIC); } + /** + * Determines the element type to use for the function result by inspecting all operand types. + * + *

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 mergeKeys( keys.addAll(secondMap.keySet()); return keys; } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MapRemoveFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MapRemoveFunctionImpl.java index 1f86fcbe636..76d1b7160dd 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MapRemoveFunctionImpl.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/MapRemoveFunctionImpl.java @@ -16,7 +16,9 @@ import org.apache.calcite.linq4j.tree.Types; import org.apache.calcite.rel.type.RelDataType; 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.opensearch.sql.expression.function.ImplementorUDF; import org.opensearch.sql.expression.function.UDFOperandMetadata; @@ -40,9 +42,14 @@ public SqlReturnTypeInference getReturnTypeInference() { }; } + /** + * Provide operand metadata that enforces the function's expected operand types. + * + * @return a UDFOperandMetadata describing two operands: first a MAP and second an ARRAY + */ @Override public UDFOperandMetadata getOperandMetadata() { - return null; + return UDFOperandMetadata.wrap(OperandTypes.family(SqlTypeFamily.MAP, SqlTypeFamily.ARRAY)); } public static class MapRemoveImplementor implements NotNullImplementor { @@ -98,4 +105,4 @@ private static Map mapRemove( return resultMap; } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ReduceFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ReduceFunctionImpl.java index d60a700e816..ea24514caa6 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ReduceFunctionImpl.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/CollectionUDF/ReduceFunctionImpl.java @@ -5,13 +5,10 @@ package org.opensearch.sql.expression.function.CollectionUDF; -import static org.opensearch.sql.expression.function.CollectionUDF.LambdaUtils.inferReturnTypeFromLambda; import static org.opensearch.sql.expression.function.CollectionUDF.LambdaUtils.transferLambdaOutputToTargetType; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; import org.apache.calcite.adapter.enumerable.NotNullImplementor; import org.apache.calcite.adapter.enumerable.NullPolicy; import org.apache.calcite.adapter.enumerable.RexToLixTranslator; @@ -19,14 +16,19 @@ import org.apache.calcite.linq4j.tree.Expressions; import org.apache.calcite.linq4j.tree.Types; 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.type.ArraySqlType; +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlCallBinding; +import org.apache.calcite.sql.SqlLambda; +import org.apache.calcite.sql.SqlNode; +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.sql.validate.SqlValidator; +import org.opensearch.sql.common.utils.StringUtils; import org.opensearch.sql.expression.function.ImplementorUDF; import org.opensearch.sql.expression.function.UDFOperandMetadata; @@ -40,38 +42,75 @@ public ReduceFunctionImpl() { super(new ReduceImplementor(), NullPolicy.ANY); } + /** + * Creates a SqlReturnTypeInference that derives the UDF's return type from the operator binding and any provided lambda operands. + * + *

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 rexNodes = rexCallBinding.operands(); - ArraySqlType listType = (ArraySqlType) rexNodes.get(0).getType(); - RelDataType elementType = listType.getComponentType(); - RelDataType baseType = rexNodes.get(1).getType(); - Map map = new HashMap<>(); - RexLambda mergeLambda = (RexLambda) rexNodes.get(2); - map.put(mergeLambda.getParameters().get(0).getName(), baseType); - map.put(mergeLambda.getParameters().get(1).getName(), elementType); - RelDataType mergedReturnType = - inferReturnTypeFromLambda((RexLambda) rexNodes.get(2), map, typeFactory); - if (mergedReturnType != baseType) { // For different acc, we need to recalculate - map.put(mergeLambda.getParameters().get(0).getName(), mergedReturnType); - mergedReturnType = inferReturnTypeFromLambda((RexLambda) rexNodes.get(2), map, typeFactory); - } - RelDataType finalReturnType; - if (rexNodes.size() > 3) { - finalReturnType = inferReturnTypeFromLambda((RexLambda) rexNodes.get(3), map, typeFactory); - } else { - finalReturnType = mergedReturnType; + if (sqlOperatorBinding instanceof RexCallBinding) { + return sqlOperatorBinding.getOperandType(sqlOperatorBinding.getOperandCount() - 1); + } else if (sqlOperatorBinding instanceof SqlCallBinding callBinding) { + RelDataType elementType = callBinding.getOperandType(0).getComponentType(); + RelDataType baseType = callBinding.getOperandType(1); + SqlLambda reduce1 = callBinding.getCall().operand(2); + SqlNode function1 = reduce1.getExpression(); + SqlValidator validator = callBinding.getValidator(); + // The saved types are ANY because the lambda function is defined as (ANY, ..) -> ANY + // Force it to derive types again by removing existing saved types + validator.removeValidatedNodeType(function1); + if (function1 instanceof SqlCall call) { + List operands = call.getOperandList(); + // The first argument is base (accumulator), while the second is from the array + if (!operands.isEmpty()) validator.setValidatedNodeType(operands.get(0), baseType); + if (operands.size() > 1 && elementType != null) + validator.setValidatedNodeType(operands.get(1), elementType); + } + RelDataType returnType = SqlTypeUtil.deriveType(callBinding, function1); + if (callBinding.getOperandCount() > 3) { + SqlLambda reduce2 = callBinding.getCall().operand(3); + SqlNode function2 = reduce2.getExpression(); + validator.removeValidatedNodeType(function2); + if (function2 instanceof SqlCall call) { + List operands = call.getOperandList(); + if (!operands.isEmpty()) validator.setValidatedNodeType(operands.get(0), returnType); + } + returnType = SqlTypeUtil.deriveType(callBinding, function2); + } + return returnType; } - return finalReturnType; + throw new IllegalStateException( + StringUtils.format( + "sqlOperatorBinding can only be either RexCallBinding or SqlCallBinding, but got %s", + sqlOperatorBinding.getClass())); }; } + /** + * Describes the expected operand signature for the reducer UDF. + * + *

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 operands = rexCallBinding.operands(); - RelDataType lambdaReturnType = ((RexLambda) operands.get(1)).getExpression().getType(); + RelDataType lambdaReturnType = sqlOperatorBinding.getOperandType(1); return createArrayType( typeFactory, typeFactory.createTypeWithNullability(lambdaReturnType, true), true); }; } + /** + * Provides operand metadata that validates only the first two operands (array and function) + * and permits any number of additional arguments. + * + *

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 families = + List.of(SqlTypeFamily.ARRAY, SqlTypeFamily.FUNCTION); + + /** + * Copied from {@link FamilyOperandTypeChecker#checkSingleOperandType(SqlCallBinding + * callBinding, SqlNode node, int iFormalOperand, boolean throwOnFailure)} + */ + @Override + public boolean checkSingleOperandType( + SqlCallBinding callBinding, + SqlNode operand, + int iFormalOperand, + boolean throwOnFailure) { + // Do not check types after the second operands + if (iFormalOperand > 1) { + return true; + } + SqlTypeFamily family = families.get(iFormalOperand); + switch (family) { + case ANY: + final RelDataType type = SqlTypeUtil.deriveType(callBinding, operand); + SqlTypeName typeName = type.getSqlTypeName(); + + if (typeName == SqlTypeName.CURSOR) { + // We do not allow CURSOR operands, even for ANY + if (throwOnFailure) { + throw callBinding.newValidationSignatureError(); + } + return false; + } + // fall through + case IGNORE: + // no need to check + return true; + default: + break; + } + if (SqlUtil.isNullLiteral(operand, false)) { + if (callBinding.isTypeCoercionEnabled()) { + return true; + } else if (throwOnFailure) { + throw callBinding + .getValidator() + .newValidationError(operand, RESOURCE.nullIllegal()); + } else { + return false; + } + } + RelDataType type = SqlTypeUtil.deriveType(callBinding, operand); + SqlTypeName typeName = type.getSqlTypeName(); + + // Pass type checking for operators if it's of type 'ANY'. + if (typeName.getFamily() == SqlTypeFamily.ANY) { + return true; + } + + if (!family.getTypeNames().contains(typeName)) { + if (throwOnFailure) { + throw callBinding.newValidationSignatureError(); + } + return false; + } + return true; + } + + @Override + public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) { + if (!getOperandCountRange().isValidCount(callBinding.getOperandCount())) { + return false; + } + return IntStream.range(0, 2) + .allMatch( + i -> checkSingleOperandType(callBinding, callBinding.operand(i), i, false)); + } + + @Override + public SqlOperandCountRange getOperandCountRange() { + return SqlOperandCountRanges.from(2); + } + + @Override + public String getAllowedSignatures(SqlOperator op, String opName) { + return ""; + } + }); } public static class TransformImplementor implements NotNullImplementor { @@ -126,4 +241,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/PPLBuiltinOperators.java b/core/src/main/java/org/opensearch/sql/expression/function/PPLBuiltinOperators.java index 2d769194924..a4514cd9feb 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/PPLBuiltinOperators.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/PPLBuiltinOperators.java @@ -23,11 +23,20 @@ import org.apache.calcite.linq4j.tree.Expression; import org.apache.calcite.rex.RexCall; import org.apache.calcite.sql.SqlAggFunction; +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlFunction; +import org.apache.calcite.sql.SqlFunctionCategory; import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.fun.SqlLibraryOperators; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.type.OperandTypes; import org.apache.calcite.sql.type.ReturnTypes; import org.apache.calcite.sql.type.SqlTypeTransforms; import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable; +import org.apache.calcite.sql.validate.SqlValidator; import org.apache.calcite.util.BuiltInMethod; import org.opensearch.sql.calcite.udf.udaf.FirstAggFunction; import org.opensearch.sql.calcite.udf.udaf.LastAggFunction; @@ -72,7 +81,6 @@ import org.opensearch.sql.expression.function.udf.ToNumberFunction; import org.opensearch.sql.expression.function.udf.ToStringFunction; import org.opensearch.sql.expression.function.udf.condition.EarliestFunction; -import org.opensearch.sql.expression.function.udf.condition.EnhancedCoalesceFunction; import org.opensearch.sql.expression.function.udf.condition.LatestFunction; import org.opensearch.sql.expression.function.udf.datetime.AddSubDateFunction; import org.opensearch.sql.expression.function.udf.datetime.CurrentFunction; @@ -113,66 +121,66 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { Suppliers.memoize(() -> (PPLBuiltinOperators) new PPLBuiltinOperators().init()); // Json Functions - public static final SqlOperator JSON = new JsonFunctionImpl().toUDF("JSON"); - public static final SqlOperator JSON_ARRAY_LENGTH = + public static final SqlFunction JSON = new JsonFunctionImpl().toUDF("JSON"); + public static final SqlFunction JSON_ARRAY_LENGTH = new JsonArrayLengthFunctionImpl().toUDF("JSON_ARRAY_LENGTH"); - public static final SqlOperator JSON_EXTRACT = + public static final SqlFunction JSON_EXTRACT = new JsonExtractFunctionImpl().toUDF("JSON_EXTRACT"); - public static final SqlOperator JSON_EXTRACT_ALL = + public static final SqlFunction JSON_EXTRACT_ALL = new JsonExtractAllFunctionImpl().toUDF("JSON_EXTRACT_ALL"); - public static final SqlOperator JSON_KEYS = new JsonKeysFunctionImpl().toUDF("JSON_KEYS"); - public static final SqlOperator JSON_SET = new JsonSetFunctionImpl().toUDF("JSON_SET"); - public static final SqlOperator JSON_DELETE = new JsonDeleteFunctionImpl().toUDF("JSON_DELETE"); - public static final SqlOperator JSON_APPEND = new JsonAppendFunctionImpl().toUDF("JSON_APPEND"); - public static final SqlOperator JSON_EXTEND = new JsonExtendFunctionImpl().toUDF("JSON_EXTEND"); + public static final SqlFunction JSON_KEYS = new JsonKeysFunctionImpl().toUDF("JSON_KEYS"); + public static final SqlFunction JSON_SET = new JsonSetFunctionImpl().toUDF("JSON_SET"); + public static final SqlFunction JSON_DELETE = new JsonDeleteFunctionImpl().toUDF("JSON_DELETE"); + public static final SqlFunction JSON_APPEND = new JsonAppendFunctionImpl().toUDF("JSON_APPEND"); + public static final SqlFunction JSON_EXTEND = new JsonExtendFunctionImpl().toUDF("JSON_EXTEND"); // Math functions - public static final SqlOperator SPAN = new SpanFunction().toUDF("SPAN"); - public static final SqlOperator E = new EulerFunction().toUDF("E"); - public static final SqlOperator CONV = new ConvFunction().toUDF("CONVERT"); - public static final SqlOperator MOD = new ModFunction().toUDF("MOD"); - public static final SqlOperator DIVIDE = new DivideFunction().toUDF("DIVIDE"); - public static final SqlOperator SHA2 = CryptographicFunction.sha2().toUDF("SHA2"); - public static final SqlOperator CIDRMATCH = new CidrMatchFunction().toUDF("CIDRMATCH"); - public static final SqlOperator SCALAR_MAX = new ScalarMaxFunction().toUDF("SCALAR_MAX"); - public static final SqlOperator SCALAR_MIN = new ScalarMinFunction().toUDF("SCALAR_MIN"); + public static final SqlFunction SPAN = new SpanFunction().toUDF("SPAN"); + public static final SqlFunction E = new EulerFunction().toUDF("E"); + public static final SqlFunction CONV = new ConvFunction().toUDF("CONVERT"); + public static final SqlFunction MOD = new ModFunction().toUDF("MOD"); + public static final SqlFunction DIVIDE = new DivideFunction().toUDF("DIVIDE"); + public static final SqlFunction SHA2 = CryptographicFunction.sha2().toUDF("SHA2"); + public static final SqlFunction CIDRMATCH = new CidrMatchFunction().toUDF("CIDRMATCH"); + public static final SqlFunction SCALAR_MAX = new ScalarMaxFunction().toUDF("SCALAR_MAX"); + public static final SqlFunction SCALAR_MIN = new ScalarMinFunction().toUDF("SCALAR_MIN"); - public static final SqlOperator COSH = + public static final SqlFunction COSH = adaptMathFunctionToUDF( "cosh", ReturnTypes.DOUBLE_FORCE_NULLABLE, NullPolicy.ANY, PPLOperandTypes.NUMERIC) .toUDF("COSH"); - public static final SqlOperator SINH = + public static final SqlFunction SINH = adaptMathFunctionToUDF( "sinh", ReturnTypes.DOUBLE_FORCE_NULLABLE, NullPolicy.ANY, PPLOperandTypes.NUMERIC) .toUDF("SINH"); - public static final SqlOperator RINT = + public static final SqlFunction RINT = adaptMathFunctionToUDF( "rint", ReturnTypes.DOUBLE_FORCE_NULLABLE, NullPolicy.ANY, PPLOperandTypes.NUMERIC) .toUDF("RINT"); - public static final SqlOperator EXPM1 = + public static final SqlFunction EXPM1 = adaptMathFunctionToUDF( "expm1", ReturnTypes.DOUBLE_FORCE_NULLABLE, NullPolicy.ANY, PPLOperandTypes.NUMERIC) .toUDF("EXPM1"); // IP comparing functions - public static final SqlOperator NOT_EQUALS_IP = + public static final SqlFunction NOT_EQUALS_IP = CompareIpFunction.notEquals().toUDF("NOT_EQUALS_IP"); - public static final SqlOperator EQUALS_IP = CompareIpFunction.equals().toUDF("EQUALS_IP"); - public static final SqlOperator GREATER_IP = CompareIpFunction.greater().toUDF("GREATER_IP"); - public static final SqlOperator GTE_IP = CompareIpFunction.greaterOrEquals().toUDF("GTE_IP"); - public static final SqlOperator LESS_IP = CompareIpFunction.less().toUDF("LESS_IP"); - public static final SqlOperator LTE_IP = CompareIpFunction.lessOrEquals().toUDF("LTE_IP"); + public static final SqlFunction EQUALS_IP = CompareIpFunction.equals().toUDF("EQUALS_IP"); + public static final SqlFunction GREATER_IP = CompareIpFunction.greater().toUDF("GREATER_IP"); + public static final SqlFunction GTE_IP = CompareIpFunction.greaterOrEquals().toUDF("GTE_IP"); + public static final SqlFunction LESS_IP = CompareIpFunction.less().toUDF("LESS_IP"); + public static final SqlFunction LTE_IP = CompareIpFunction.lessOrEquals().toUDF("LTE_IP"); // Condition function - public static final SqlOperator EARLIEST = new EarliestFunction().toUDF("EARLIEST"); - public static final SqlOperator LATEST = new LatestFunction().toUDF("LATEST"); + public static final SqlFunction EARLIEST = new EarliestFunction().toUDF("EARLIEST"); + public static final SqlFunction LATEST = new LatestFunction().toUDF("LATEST"); // Datetime function - public static final SqlOperator TIMESTAMP = new TimestampFunction().toUDF("TIMESTAMP"); - public static final SqlOperator DATE = + public static final SqlFunction TIMESTAMP = new TimestampFunction().toUDF("TIMESTAMP"); + public static final SqlFunction DATE = adaptExprMethodToUDF( DateTimeFunctions.class, "exprDate", @@ -180,13 +188,13 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ARG0, PPLOperandTypes.DATE_OR_TIMESTAMP_OR_STRING) .toUDF("DATE"); - public static final SqlOperator YEARWEEK = new YearweekFunction().toUDF("YEARWEEK"); - public static final SqlOperator WEEKDAY = new WeekdayFunction().toUDF("WEEKDAY"); - public static final SqlOperator UNIX_TIMESTAMP = + public static final SqlFunction YEARWEEK = new YearweekFunction().toUDF("YEARWEEK"); + public static final SqlFunction WEEKDAY = new WeekdayFunction().toUDF("WEEKDAY"); + public static final SqlFunction UNIX_TIMESTAMP = new UnixTimestampFunction().toUDF("UNIX_TIMESTAMP"); - public static final SqlOperator STRFTIME = new StrftimeFunction().toUDF("STRFTIME"); - public static final SqlOperator TO_SECONDS = new ToSecondsFunction().toUDF("TO_SECONDS"); - public static final SqlOperator ADDTIME = + public static final SqlFunction STRFTIME = new StrftimeFunction().toUDF("STRFTIME"); + public static final SqlFunction TO_SECONDS = new ToSecondsFunction().toUDF("TO_SECONDS"); + public static final SqlFunction ADDTIME = adaptExprMethodWithPropertiesToUDF( DateTimeFunctions.class, "exprAddTime", @@ -194,7 +202,7 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.DATETIME_DATETIME) .toUDF("ADDTIME"); - public static final SqlOperator SUBTIME = + public static final SqlFunction SUBTIME = adaptExprMethodWithPropertiesToUDF( DateTimeFunctions.class, "exprSubTime", @@ -202,22 +210,22 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.DATETIME_DATETIME) .toUDF("SUBTIME"); - public static final SqlOperator ADDDATE = new AddSubDateFunction(true).toUDF("ADDDATE"); - public static final SqlOperator SUBDATE = new AddSubDateFunction(false).toUDF("SUBDATE"); - public static final SqlOperator DATE_ADD = new DateAddSubFunction(true).toUDF("DATE_ADD"); - public static final SqlOperator DATE_SUB = new DateAddSubFunction(false).toUDF("DATE_SUB"); - public static final SqlOperator EXTRACT = new ExtractFunction().toUDF("EXTRACT"); - public static final SqlOperator YEAR = new DatePartFunction(TimeUnit.YEAR).toUDF("YEAR"); - public static final SqlOperator QUARTER = new DatePartFunction(TimeUnit.QUARTER).toUDF("QUARTER"); - public static final SqlOperator MONTH = new DatePartFunction(TimeUnit.MONTH).toUDF("MONTH"); - public static final SqlOperator DAY = new DatePartFunction(TimeUnit.DAY).toUDF("DAY"); - public static final SqlOperator DAY_OF_WEEK = + public static final SqlFunction ADDDATE = new AddSubDateFunction(true).toUDF("ADDDATE"); + public static final SqlFunction SUBDATE = new AddSubDateFunction(false).toUDF("SUBDATE"); + public static final SqlFunction DATE_ADD = new DateAddSubFunction(true).toUDF("DATE_ADD"); + public static final SqlFunction DATE_SUB = new DateAddSubFunction(false).toUDF("DATE_SUB"); + public static final SqlFunction EXTRACT = new ExtractFunction().toUDF("EXTRACT"); + public static final SqlFunction YEAR = new DatePartFunction(TimeUnit.YEAR).toUDF("YEAR"); + public static final SqlFunction QUARTER = new DatePartFunction(TimeUnit.QUARTER).toUDF("QUARTER"); + public static final SqlFunction MONTH = new DatePartFunction(TimeUnit.MONTH).toUDF("MONTH"); + public static final SqlFunction DAY = new DatePartFunction(TimeUnit.DAY).toUDF("DAY"); + public static final SqlFunction DAY_OF_WEEK = new DatePartFunction(TimeUnit.DOW).toUDF("DAY_OF_WEEK"); - public static final SqlOperator DAY_OF_YEAR = + public static final SqlFunction DAY_OF_YEAR = new DatePartFunction(TimeUnit.DOY).toUDF("DAY_OF_YEAR"); - public static final SqlOperator HOUR = new DatePartFunction(TimeUnit.HOUR).toUDF("HOUR"); - public static final SqlOperator MINUTE = new DatePartFunction(TimeUnit.MINUTE).toUDF("MINUTE"); - public static final SqlOperator MINUTE_OF_DAY = + public static final SqlFunction HOUR = new DatePartFunction(TimeUnit.HOUR).toUDF("HOUR"); + public static final SqlFunction MINUTE = new DatePartFunction(TimeUnit.MINUTE).toUDF("MINUTE"); + public static final SqlFunction MINUTE_OF_DAY = adaptExprMethodToUDF( DateTimeFunctions.class, "exprMinuteOfDay", @@ -225,22 +233,22 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ARG0, PPLOperandTypes.TIME_OR_TIMESTAMP_OR_STRING) .toUDF("MINUTE_OF_DAY"); - public static final SqlOperator SECOND = new DatePartFunction(TimeUnit.SECOND).toUDF("SECOND"); - public static final SqlOperator MICROSECOND = + public static final SqlFunction SECOND = new DatePartFunction(TimeUnit.SECOND).toUDF("SECOND"); + public static final SqlFunction MICROSECOND = new DatePartFunction(TimeUnit.MICROSECOND).toUDF("MICROSECOND"); - public static final SqlOperator NOW = new CurrentFunction(ExprCoreType.TIMESTAMP).toUDF("NOW"); - public static final SqlOperator CURRENT_TIME = + public static final SqlFunction NOW = new CurrentFunction(ExprCoreType.TIMESTAMP).toUDF("NOW"); + public static final SqlFunction CURRENT_TIME = new CurrentFunction(ExprCoreType.TIME).toUDF("CURRENT_TIME"); - public static final SqlOperator CURRENT_DATE = + public static final SqlFunction CURRENT_DATE = new CurrentFunction(ExprCoreType.DATE).toUDF("CURRENT_DATE"); - public static final SqlOperator DATE_FORMAT = + public static final SqlFunction DATE_FORMAT = new FormatFunction(ExprCoreType.DATE).toUDF("DATE_FORMAT"); - public static final SqlOperator TIME_FORMAT = + public static final SqlFunction TIME_FORMAT = new FormatFunction(ExprCoreType.TIME).toUDF("TIME_FORMAT"); - public static final SqlOperator DAYNAME = new PeriodNameFunction(TimeUnit.DAY).toUDF("DAYNAME"); - public static final SqlOperator MONTHNAME = + public static final SqlFunction DAYNAME = new PeriodNameFunction(TimeUnit.DAY).toUDF("DAYNAME"); + public static final SqlFunction MONTHNAME = new PeriodNameFunction(TimeUnit.MONTH).toUDF("MONTHNAME"); - public static final SqlOperator CONVERT_TZ = + public static final SqlFunction CONVERT_TZ = adaptExprMethodToUDF( DateTimeFunctions.class, "exprConvertTZ", @@ -248,7 +256,7 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.TIMESTAMP_OR_STRING_STRING_STRING) .toUDF("CONVERT_TZ"); - public static final SqlOperator DATEDIFF = + public static final SqlFunction DATEDIFF = adaptExprMethodWithPropertiesToUDF( DateTimeFunctions.class, "exprDateDiff", @@ -256,10 +264,10 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.DATETIME_DATETIME) .toUDF("DATEDIFF"); - public static final SqlOperator TIMESTAMPDIFF = + public static final SqlFunction TIMESTAMPDIFF = new TimestampDiffFunction().toUDF("TIMESTAMPDIFF"); - public static final SqlOperator LAST_DAY = new LastDayFunction().toUDF("LAST_DAY"); - public static final SqlOperator FROM_DAYS = + public static final SqlFunction LAST_DAY = new LastDayFunction().toUDF("LAST_DAY"); + public static final SqlFunction FROM_DAYS = adaptExprMethodToUDF( DateTimeFunctions.class, "exprFromDays", @@ -267,8 +275,8 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.INTEGER) .toUDF("FROM_DAYS"); - public static final SqlOperator FROM_UNIXTIME = new FromUnixTimeFunction().toUDF("FROM_UNIXTIME"); - public static final SqlOperator GET_FORMAT = + public static final SqlFunction FROM_UNIXTIME = new FromUnixTimeFunction().toUDF("FROM_UNIXTIME"); + public static final SqlFunction GET_FORMAT = adaptExprMethodToUDF( DateTimeFunctions.class, "exprGetFormat", @@ -276,7 +284,7 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.STRING_STRING) .toUDF("GET_FORMAT"); - public static final SqlOperator MAKEDATE = + public static final SqlFunction MAKEDATE = adaptExprMethodToUDF( DateTimeFunctions.class, "exprMakeDate", @@ -284,7 +292,7 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.NUMERIC_NUMERIC) .toUDF("MAKEDATE"); - public static final SqlOperator MAKETIME = + public static final SqlFunction MAKETIME = adaptExprMethodToUDF( DateTimeFunctions.class, "exprMakeTime", @@ -292,7 +300,7 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.NUMERIC_NUMERIC_NUMERIC) .toUDF("MAKETIME"); - public static final SqlOperator PERIOD_DIFF = + public static final SqlFunction PERIOD_DIFF = adaptExprMethodToUDF( DateTimeFunctions.class, "exprPeriodDiff", @@ -300,7 +308,7 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.INTEGER_INTEGER) .toUDF("PERIOD_DIFF"); - public static final SqlOperator PERIOD_ADD = + public static final SqlFunction PERIOD_ADD = adaptExprMethodToUDF( DateTimeFunctions.class, "exprPeriodAdd", @@ -308,7 +316,7 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.INTEGER_INTEGER) .toUDF("PERIOD_ADD"); - public static final SqlOperator STR_TO_DATE = + public static final SqlFunction STR_TO_DATE = adaptExprMethodWithPropertiesToUDF( DateTimeFunctions.class, "exprStrToDate", @@ -316,9 +324,9 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.STRING_STRING) .toUDF("STR_TO_DATE"); - public static final SqlOperator SYSDATE = new SysdateFunction().toUDF("SYSDATE"); - public static final SqlOperator SEC_TO_TIME = new SecToTimeFunction().toUDF("SEC_TO_TIME"); - public static final SqlOperator TIME = + public static final SqlFunction SYSDATE = new SysdateFunction().toUDF("SYSDATE"); + public static final SqlFunction SEC_TO_TIME = new SecToTimeFunction().toUDF("SEC_TO_TIME"); + public static final SqlFunction TIME = adaptExprMethodToUDF( DateTimeFunctions.class, "exprTime", @@ -328,9 +336,9 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { .toUDF("TIME"); // IP cast function - public static final SqlOperator IP = + public static final SqlFunction IP = new IPFunction().toUDF(UserDefinedFunctionUtils.IP_FUNCTION_NAME); - public static final SqlOperator TIME_TO_SEC = + public static final SqlFunction TIME_TO_SEC = adaptExprMethodToUDF( DateTimeFunctions.class, "exprTimeToSec", @@ -338,7 +346,7 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ARG0, PPLOperandTypes.TIME_OR_TIMESTAMP_OR_STRING) .toUDF("TIME_TO_SEC"); - public static final SqlOperator TIMEDIFF = + public static final SqlFunction TIMEDIFF = UserDefinedFunctionUtils.adaptExprMethodToUDF( DateTimeFunctions.class, "exprTimeDiff", @@ -346,8 +354,8 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ANY, PPLOperandTypes.TIME_TIME) .toUDF("TIME_DIFF"); - public static final SqlOperator TIMESTAMPADD = new TimestampAddFunction().toUDF("TIMESTAMPADD"); - public static final SqlOperator TO_DAYS = + public static final SqlFunction TIMESTAMPADD = new TimestampAddFunction().toUDF("TIMESTAMPADD"); + public static final SqlFunction TO_DAYS = adaptExprMethodToUDF( DateTimeFunctions.class, "exprToDays", @@ -355,8 +363,8 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.ARG0, PPLOperandTypes.DATE_OR_TIMESTAMP_OR_STRING) .toUDF("TO_DAYS"); - public static final SqlOperator DATETIME = new DatetimeFunction().toUDF("DATETIME"); - public static final SqlOperator UTC_DATE = + public static final SqlFunction DATETIME = new DatetimeFunction().toUDF("DATETIME"); + public static final SqlFunction UTC_DATE = adaptExprMethodWithPropertiesToUDF( DateTimeFunctions.class, "exprUtcDate", @@ -364,7 +372,7 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.NONE, PPLOperandTypes.NONE) .toUDF("UTC_DATE"); - public static final SqlOperator UTC_TIME = + public static final SqlFunction UTC_TIME = adaptExprMethodWithPropertiesToUDF( DateTimeFunctions.class, "exprUtcTime", @@ -372,7 +380,7 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.NONE, PPLOperandTypes.NONE) .toUDF("UTC_TIME"); - public static final SqlOperator UTC_TIMESTAMP = + public static final SqlFunction UTC_TIMESTAMP = adaptExprMethodWithPropertiesToUDF( DateTimeFunctions.class, "exprUtcTimestamp", @@ -380,61 +388,61 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { NullPolicy.NONE, PPLOperandTypes.NONE) .toUDF("UTC_TIMESTAMP"); - public static final SqlOperator WEEK = new WeekFunction().toUDF("WEEK"); - public static final SqlOperator GROK = new ParseFunction().toUDF("GROK"); + public static final SqlFunction WEEK = new WeekFunction().toUDF("WEEK"); + public static final SqlFunction GROK = new ParseFunction().toUDF("GROK"); // TODO: Figure out if there is other option to perform multiple group match in Calcite // For now, keep V2's regexExpression logic to avoid breaking change - public static final SqlOperator PARSE = new ParseFunction().toUDF("PARSE"); - public static final SqlOperator PATTERN_PARSER = + public static final SqlFunction PARSE = new ParseFunction().toUDF("PARSE"); + public static final SqlFunction PATTERN_PARSER = new PatternParserFunctionImpl().toUDF("PATTERN_PARSER"); - public static final SqlOperator FORALL = new ForallFunctionImpl().toUDF("forall"); - public static final SqlOperator EXISTS = new ExistsFunctionImpl().toUDF("exists"); - public static final SqlOperator ARRAY = new ArrayFunctionImpl().toUDF("array"); - public static final SqlOperator MAP_APPEND = new MapAppendFunctionImpl().toUDF("map_append"); - public static final SqlOperator MAP_REMOVE = new MapRemoveFunctionImpl().toUDF("MAP_REMOVE"); - public static final SqlOperator MVAPPEND = new MVAppendFunctionImpl().toUDF("mvappend"); - public static final SqlOperator MVZIP = new MVZipFunctionImpl().toUDF("mvzip"); - public static final SqlOperator MVFIND = new MVFindFunctionImpl().toUDF("mvfind"); - public static final SqlOperator FILTER = new FilterFunctionImpl().toUDF("filter"); - public static final SqlOperator TRANSFORM = new TransformFunctionImpl().toUDF("transform"); - public static final SqlOperator REDUCE = new ReduceFunctionImpl().toUDF("reduce"); + public static final SqlFunction FORALL = new ForallFunctionImpl().toUDF("forall"); + public static final SqlFunction EXISTS = new ExistsFunctionImpl().toUDF("exists"); + public static final SqlFunction ARRAY = new ArrayFunctionImpl().toUDF("array"); + public static final SqlFunction MAP_APPEND = new MapAppendFunctionImpl().toUDF("map_append"); + public static final SqlFunction MAP_REMOVE = new MapRemoveFunctionImpl().toUDF("MAP_REMOVE"); + public static final SqlFunction MVAPPEND = new MVAppendFunctionImpl().toUDF("mvappend"); + public static final SqlFunction MVZIP = new MVZipFunctionImpl().toUDF("mvzip"); + public static final SqlFunction MVFIND = new MVFindFunctionImpl().toUDF("mvfind"); + public static final SqlFunction FILTER = new FilterFunctionImpl().toUDF("filter"); + public static final SqlFunction TRANSFORM = new TransformFunctionImpl().toUDF("transform"); + public static final SqlFunction REDUCE = new ReduceFunctionImpl().toUDF("reduce"); private static final RelevanceQueryFunction RELEVANCE_QUERY_FUNCTION_INSTANCE = new RelevanceQueryFunction(); - public static final SqlOperator MATCH = RELEVANCE_QUERY_FUNCTION_INSTANCE.toUDF("match"); - public static final SqlOperator MATCH_PHRASE = + public static final SqlFunction MATCH = RELEVANCE_QUERY_FUNCTION_INSTANCE.toUDF("match"); + public static final SqlFunction MATCH_PHRASE = RELEVANCE_QUERY_FUNCTION_INSTANCE.toUDF("match_phrase"); - public static final SqlOperator MATCH_BOOL_PREFIX = + public static final SqlFunction MATCH_BOOL_PREFIX = RELEVANCE_QUERY_FUNCTION_INSTANCE.toUDF("match_bool_prefix"); - public static final SqlOperator MATCH_PHRASE_PREFIX = + public static final SqlFunction MATCH_PHRASE_PREFIX = RELEVANCE_QUERY_FUNCTION_INSTANCE.toUDF("match_phrase_prefix"); - public static final SqlOperator SIMPLE_QUERY_STRING = + public static final SqlFunction SIMPLE_QUERY_STRING = RELEVANCE_QUERY_FUNCTION_INSTANCE.toUDF("simple_query_string", false); - public static final SqlOperator QUERY_STRING = + public static final SqlFunction QUERY_STRING = RELEVANCE_QUERY_FUNCTION_INSTANCE.toUDF("query_string", false); - public static final SqlOperator MULTI_MATCH = + public static final SqlFunction MULTI_MATCH = RELEVANCE_QUERY_FUNCTION_INSTANCE.toUDF("multi_match", false); - public static final SqlOperator NUMBER_TO_STRING = + public static final SqlFunction NUMBER_TO_STRING = new NumberToStringFunction().toUDF("NUMBER_TO_STRING"); - public static final SqlOperator TONUMBER = new ToNumberFunction().toUDF("TONUMBER"); - public static final SqlOperator TOSTRING = new ToStringFunction().toUDF("TOSTRING"); - public static final SqlOperator WIDTH_BUCKET = + public static final SqlFunction TONUMBER = new ToNumberFunction().toUDF("TONUMBER"); + public static final SqlFunction TOSTRING = new ToStringFunction().toUDF("TOSTRING"); + public static final SqlFunction WIDTH_BUCKET = new org.opensearch.sql.expression.function.udf.binning.WidthBucketFunction() .toUDF("WIDTH_BUCKET"); - public static final SqlOperator SPAN_BUCKET = + public static final SqlFunction SPAN_BUCKET = new org.opensearch.sql.expression.function.udf.binning.SpanBucketFunction() .toUDF("SPAN_BUCKET"); - public static final SqlOperator MINSPAN_BUCKET = + public static final SqlFunction MINSPAN_BUCKET = new org.opensearch.sql.expression.function.udf.binning.MinspanBucketFunction() .toUDF("MINSPAN_BUCKET"); - public static final SqlOperator RANGE_BUCKET = + public static final SqlFunction RANGE_BUCKET = new org.opensearch.sql.expression.function.udf.binning.RangeBucketFunction() .toUDF("RANGE_BUCKET"); - public static final SqlOperator REX_EXTRACT = new RexExtractFunction().toUDF("REX_EXTRACT"); - public static final SqlOperator REX_EXTRACT_MULTI = + public static final SqlFunction REX_EXTRACT = new RexExtractFunction().toUDF("REX_EXTRACT"); + public static final SqlFunction REX_EXTRACT_MULTI = new RexExtractMultiFunction().toUDF("REX_EXTRACT_MULTI"); - public static final SqlOperator REX_OFFSET = new RexOffsetFunction().toUDF("REX_OFFSET"); + public static final SqlFunction REX_OFFSET = new RexOffsetFunction().toUDF("REX_OFFSET"); // Aggregation functions public static final SqlAggFunction AVG_NULLABLE = new NullableSqlAvgAggFunction(SqlKind.AVG); @@ -463,13 +471,14 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { PercentileApproxFunction.class, "percentile_approx", ReturnTypes.ARG0_FORCE_NULLABLE, - PPLOperandTypes.NUMERIC_NUMERIC_OPTIONAL_NUMERIC); + PPLOperandTypes.NUMERIC_NUMERIC_OPTIONAL_NUMERIC_SYMBOL); public static final SqlAggFunction INTERNAL_PATTERN = createUserDefinedAggFunction( LogPatternAggFunction.class, "pattern", ReturnTypes.explicit(UserDefinedFunctionUtils.nullablePatternAggList), - null); + UDFOperandMetadata.wrap( + OperandTypes.VARIADIC)); // operand types of patterns are very flexible public static final SqlAggFunction LIST = createUserDefinedAggFunction( ListAggFunction.class, "LIST", PPLReturnTypes.STRING_ARRAY, PPLOperandTypes.ANY_SCALAR); @@ -480,8 +489,36 @@ public class PPLBuiltinOperators extends ReflectiveSqlOperatorTable { PPLReturnTypes.STRING_ARRAY, PPLOperandTypes.ANY_SCALAR_OPTIONAL_INTEGER); - public static final SqlOperator ENHANCED_COALESCE = - new EnhancedCoalesceFunction().toUDF("COALESCE"); + public static final SqlFunction ATAN = + new SqlFunction( + "ATAN", + SqlKind.OTHER_FUNCTION, + ReturnTypes.DOUBLE_NULLABLE, + null, + OperandTypes.NUMERIC_OPTIONAL_NUMERIC, + SqlFunctionCategory.NUMERIC) { + /** + * Rewrite an ATAN call to use the appropriate operator based on argument count. + * + *

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>> - functionRegistry; + private final ImmutableMap functionRegistry; /** * The external function registry. Functions whose implementations depend on a specific data * engine should be registered here. This reduces coupling between the core module and particular * storage backends. */ - private final Map>> - externalFunctionRegistry; + private final Map externalFunctionRegistry; /** * The registry for built-in agg functions. Agg Functions defined by the PPL specification, whose * implementations are independent of any specific data storage, should be registered here * internally. */ - private final ImmutableMap> - aggFunctionRegistry; + private final ImmutableMap aggFunctionRegistry; /** * The external agg function registry. Agg Functions whose implementations depend on a specific * data engine should be registered here. This reduces coupling between the core module and * particular storage backends. */ - private final Map> - aggExternalFunctionRegistry; + private final Map aggExternalFunctionRegistry; + /** + * Constructs the function implementation table by copying internal builder maps and + * initializing empty concurrent maps for external overrides. + * + * Copies the scalar and aggregate implementation mappings from the provided builders + * into immutable registries and prepares thread-safe maps for external function and + * aggregate overrides. + * + * @param builder builder containing registered scalar function implementations + * @param aggBuilder builder containing registered aggregate function implementations + */ private PPLFuncImpTable(Builder builder, AggBuilder aggBuilder) { - final ImmutableMap.Builder>> - mapBuilder = ImmutableMap.builder(); - builder.map.forEach((k, v) -> mapBuilder.put(k, List.copyOf(v))); + final ImmutableMap.Builder mapBuilder = + ImmutableMap.builder(); + mapBuilder.putAll(builder.map); this.functionRegistry = ImmutableMap.copyOf(mapBuilder.build()); this.externalFunctionRegistry = new ConcurrentHashMap<>(); - final ImmutableMap.Builder> - aggMapBuilder = ImmutableMap.builder(); - aggBuilder.map.forEach(aggMapBuilder::put); + final ImmutableMap.Builder aggMapBuilder = + ImmutableMap.builder(); + aggMapBuilder.putAll(aggBuilder.map); this.aggFunctionRegistry = ImmutableMap.copyOf(aggMapBuilder.build()); this.aggExternalFunctionRegistry = new ConcurrentHashMap<>(); } /** - * Register an operator from external services dynamically. + * Register an external scalar function implementation that overrides the built-in mapping. * - * @param functionName the name of the function, has to be defined in BuiltinFunctionName - * @param operator a SqlOperator representing an externally implemented function + * If the same function name is registered more than once, a warning is emitted and the latest + * registration replaces the previous one. The provided operator is used to build calls when the + * function is resolved. + * + * @param functionName the builtin function identifier to override + * @param operator a SqlOperator that will be used to construct calls for this function */ public void registerExternalOperator(BuiltinFunctionName functionName, SqlOperator operator) { - PPLTypeChecker typeChecker = - wrapSqlOperandTypeChecker( - operator.getOperandTypeChecker(), - functionName.name(), - operator instanceof SqlUserDefinedFunction); - CalciteFuncSignature signature = new CalciteFuncSignature(functionName.getName(), typeChecker); - externalFunctionRegistry.compute( - functionName, - (name, existingList) -> { - List> list = - existingList == null ? new ArrayList<>() : new ArrayList<>(existingList); - list.add(Pair.of(signature, (builder, args) -> builder.makeCall(operator, args))); - return list; - }); + if (externalFunctionRegistry.containsKey(functionName)) { + logger.warn( + String.format(Locale.ROOT, "Function %s is registered multiple times", functionName)); + } + externalFunctionRegistry.put(functionName, (builder, args) -> builder.makeCall(operator, args)); } /** - * Register an external aggregate operator dynamically. + * Registers an external aggregate implementation for the given built-in function name. * - * @param functionName the name of the function, has to be defined in BuiltinFunctionName - * @param aggFunction a SqlUserDefinedAggFunction representing the aggregate function - * implementation + *

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 validateAggFunctionSignature( - BuiltinFunctionName functionName, - RexNode field, - List argList, - RexBuilder rexBuilder) { - var implementation = getImplementation(functionName); - return validateFunctionArgs(implementation, functionName, field, argList, rexBuilder); + aggExternalFunctionRegistry.put(functionName, handler); } + /** + * Resolve an aggregate function call into a RelBuilder.AggCall. + * + * @param functionName the builtin aggregate function to resolve + * @param distinct whether the aggregation should consider distinct values + * @param field the primary field expression for the aggregation (may be null for count(*)) + * @param argList additional argument expressions for the aggregation + * @param context planner context used to build the aggregate call + * @return a RelBuilder.AggCall representing the resolved aggregate invocation + * @throws IllegalStateException if no implementation is available for the specified function + */ public RelBuilder.AggCall resolveAgg( BuiltinFunctionName functionName, boolean distinct, @@ -462,62 +464,17 @@ public RelBuilder.AggCall resolveAgg( List argList, CalcitePlanContext context) { var implementation = getImplementation(functionName); - - // Validation is done based on original argument types to generate error from user perspective. - List nodes = - validateFunctionArgs(implementation, functionName, field, argList, context.rexBuilder); - - var handler = implementation.getValue(); - return nodes != null - ? handler.apply(distinct, nodes.getFirst(), nodes.subList(1, nodes.size()), context) - : handler.apply(distinct, field, argList, context); - } - - static List validateFunctionArgs( - Pair implementation, - BuiltinFunctionName functionName, - RexNode field, - List argList, - RexBuilder rexBuilder) { - CalciteFuncSignature signature = implementation.getKey(); - - List argTypes = new ArrayList<>(); - if (field != null) { - argTypes.add(field.getType()); - } - - // Currently only PERCENTILE_APPROX, TAKE, EARLIEST, and LATEST have additional arguments. - // Their additional arguments will always come as a map of - List additionalArgTypes = - argList.stream().map(PlanUtils::derefMapCall).map(RexNode::getType).toList(); - argTypes.addAll(additionalArgTypes); - List coercionNodes = null; - if (!signature.match(functionName.getName(), argTypes)) { - List fields = new ArrayList<>(); - fields.add(field); - fields.addAll(argList); - if (CoercionUtils.hasString(fields)) { - coercionNodes = CoercionUtils.castArguments(rexBuilder, signature.typeChecker(), fields); - } - if (coercionNodes == null) { - String errorMessagePattern = - argTypes.size() <= 1 - ? "Aggregation function %s expects field type {%s}, but got %s" - : "Aggregation function %s expects field type and additional arguments {%s}, but" - + " got %s"; - throw new ExpressionEvaluationException( - String.format( - errorMessagePattern, - functionName, - signature.typeChecker().getAllowedSignatures(), - PlanUtils.getActualSignature(argTypes))); - } - } - return coercionNodes; + return implementation.apply(distinct, field, argList, context); } - private Pair getImplementation( - BuiltinFunctionName functionName) { + /** + * Resolve the aggregate implementation for the given builtin function name, preferring any externally registered override. + * + * @param functionName the builtin function name to resolve + * @return the AggHandler implementation for the specified function + * @throws IllegalStateException if no implementation (external or internal) is registered for the functionName + */ + private AggHandler getImplementation(BuiltinFunctionName functionName) { var implementation = aggExternalFunctionRegistry.get(functionName); if (implementation == null) { implementation = aggFunctionRegistry.get(functionName); @@ -536,17 +493,25 @@ public RexNode resolve(final RexBuilder builder, final String functionName, RexN return resolve(builder, funcNameOpt.get(), args); } + /** + * Resolve a built-in or externally-registered PPL function to a Calcite RexNode expression. + * + * @param builder the RexBuilder used to construct RexNode expressions + * @param functionName the name of the function to resolve + * @param args the function arguments as RexNode instances + * @return the resolved RexNode representing the function call + * @throws IllegalStateException if no implementation is registered for the given functionName + */ public RexNode resolve( final RexBuilder builder, final BuiltinFunctionName functionName, RexNode... args) { // Check the external function registry first. This allows the data-storage-dependent // function implementations to override the internal ones with the same name. - List> implementList = - externalFunctionRegistry.get(functionName); - // If the function is not part of the external registry, check the internal registry. - if (implementList == null) { - implementList = functionRegistry.get(functionName); - } - if (implementList == null || implementList.isEmpty()) { + // If the function is not part of the external registry, check the internal registry. + FunctionImp implementation = + externalFunctionRegistry.get(functionName) != null + ? externalFunctionRegistry.get(functionName) + : functionRegistry.get(functionName); + if (implementation == null) { throw new IllegalStateException(String.format("Cannot resolve function: %s", functionName)); } @@ -554,44 +519,19 @@ public RexNode resolve( // For example, the REDUCE function requires the second argument to be cast to the // return type of the lambda function. compulsoryCast(builder, functionName, args); - - List argTypes = Arrays.stream(args).map(RexNode::getType).toList(); - try { - for (Map.Entry implement : implementList) { - if (implement.getKey().match(functionName.getName(), argTypes)) { - return implement.getValue().resolve(builder, args); - } - } - - // If no implementation found with exact match, try to cast arguments to match the - // signatures. - RexNode coerced = resolveWithCoercion(builder, functionName, implementList, args); - if (coerced != null) { - return coerced; - } - } catch (Exception e) { - throw new ExpressionEvaluationException( - String.format( - "Cannot resolve function: %s, arguments: %s, caused by: %s", - functionName, PlanUtils.getActualSignature(argTypes), e.getMessage()), - e); - } - StringJoiner allowedSignatures = new StringJoiner(","); - for (var implement : implementList) { - String signature = implement.getKey().typeChecker().getAllowedSignatures(); - if (!signature.isEmpty()) { - allowedSignatures.add(signature); - } - } - throw new ExpressionEvaluationException( - String.format( - "%s function expects {%s}, but got %s", - functionName, allowedSignatures, PlanUtils.getActualSignature(argTypes))); + return implementation.resolve(builder, args); } /** - * Ad-hoc coercion for some functions that require specific casting of arguments. Now it only - * applies to the REDUCE function. + * Perform ad-hoc argument casting required by specific built-in functions. + * + *

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> implementList, - RexNode... args) { - if (BuiltinFunctionName.COMPARATORS.contains(functionName)) { - for (Map.Entry implement : implementList) { - var widenedArgs = CoercionUtils.widenArguments(builder, List.of(args)); - if (widenedArgs != null) { - boolean matchSignature = - implement - .getKey() - .typeChecker() - .checkOperandTypes(widenedArgs.stream().map(RexNode::getType).toList()); - if (matchSignature) { - return implement.getValue().resolve(builder, widenedArgs.toArray(new RexNode[0])); - } - } - } - } else { - for (Map.Entry implement : implementList) { - var signature = implement.getKey(); - var castedArgs = - CoercionUtils.castArguments(builder, signature.typeChecker(), List.of(args)); - if (castedArgs != null) { - // If compatible function is found, replace the original RexNode with cast node - // TODO: check - this is a return-once-found implementation, rest possible combinations - // will be skipped. - // Maybe can be improved to return the best match? E.g. convert to timestamp when date, - // time, and timestamp are all possible. - return implement.getValue().resolve(builder, castedArgs.toArray(new RexNode[0])); - } - } - } - return null; - } - @SuppressWarnings({"UnusedReturnValue", "SameParameterValue"}) private abstract static class AbstractBuilder { - /** Maps an operator to an implementation. */ + /** + * Register an implementation and its operand type checker for a built-in function name. + * + * @param functionName the built-in function identifier to register + * @param functionImp the implementation to invoke when resolving this function + * @param typeChecker the operand type checker used to validate this function's arguments + * @throws IllegalStateException if an implementation is already registered for {@code functionName} + */ abstract void register( - BuiltinFunctionName functionName, FunctionImp functionImp, PPLTypeChecker typeChecker); + BuiltinFunctionName functionName, + FunctionImp functionImp, + SqlOperandTypeChecker typeChecker); /** - * Register one or multiple operators under a single function name. This allows function - * overloading based on operand types. + * Registers an operator under the given function name using the operator's operand type checker. * - *

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>> map = - new HashMap<>(); + private final Map map = new HashMap<>(); + /** + * Registers a scalar function implementation for the given built-in function name. + * + * If an implementation is already registered for the same function name, this method + * throws an IllegalStateException to prevent duplicate registrations. + * + * @param functionName the built-in function identifier to register + * @param implement the function implementation to associate with the function name + * @param typeChecker the operand type checker for the function (currently accepted but not stored) + * @throws IllegalStateException if a registration already exists for {@code functionName} + */ @Override void register( - BuiltinFunctionName functionName, FunctionImp implement, PPLTypeChecker typeChecker) { - CalciteFuncSignature signature = - new CalciteFuncSignature(functionName.getName(), typeChecker); + BuiltinFunctionName functionName, + FunctionImp implement, + SqlOperandTypeChecker typeChecker) { if (map.containsKey(functionName)) { - map.get(functionName).add(Pair.of(signature, implement)); - } else { - map.put(functionName, new ArrayList<>(List.of(Pair.of(signature, implement)))); + throw new IllegalStateException( + String.format( + Locale.ROOT, + "Each function can only be registered with one operator: %s", + functionName)); } + map.put(functionName, implement); } } private static class AggBuilder { private static final double MEDIAN_PERCENTILE = 50.0; - private final Map> map = - new HashMap<>(); + private final Map map = new HashMap<>(); + /** + * Registers an aggregate function implementation for the specified built-in function name. + * + * @param functionName the built-in function identifier to register + * @param aggHandler handler that produces a RelBuilder.AggCall for this aggregate + * @param typeChecker operand type checker describing the function's accepted argument types + */ void register( - BuiltinFunctionName functionName, AggHandler aggHandler, PPLTypeChecker typeChecker) { - CalciteFuncSignature signature = - new CalciteFuncSignature(functionName.getName(), typeChecker); - map.put(functionName, Pair.of(signature, aggHandler)); + BuiltinFunctionName functionName, + AggHandler aggHandler, + SqlOperandTypeChecker typeChecker) { + map.put(functionName, aggHandler); } + /** + * Register an aggregate implementation for the given built-in function name. + * + * Creates and registers an AggHandler that constructs a RelBuilder.AggCall using the + * provided SqlAggFunction; argument expressions are converted via PlanUtils.derefMapCall + * before the aggregate call is created. The operand type checker is extracted from the + * provided aggFunction and used for registration. + * + * @param functionName the built-in function name to register the aggregate under + * @param aggFunction the SqlAggFunction used to produce the aggregate call + */ void registerOperator(BuiltinFunctionName functionName, SqlAggFunction aggFunction) { SqlOperandTypeChecker innerTypeChecker = extractTypeCheckerFromUDF(aggFunction); - PPLTypeChecker typeChecker = - wrapSqlOperandTypeChecker(innerTypeChecker, functionName.name(), true); AggHandler handler = (distinct, field, argList, ctx) -> { List newArgList = @@ -1326,9 +1232,24 @@ void registerOperator(BuiltinFunctionName functionName, SqlAggFunction aggFuncti return UserDefinedFunctionUtils.makeAggregateCall( aggFunction, List.of(field), newArgList, ctx.relBuilder); }; - register(functionName, handler, typeChecker); + register(functionName, handler, innerTypeChecker); } + /** + * Registers built-in aggregate function handlers into the builder's internal registry. + * + *

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: + *

    + *
  • MEDIAN is invoked with DISTINCT or with extra arguments, or when the input field type is null
  • + *
  • PERCENTILE_APPROX is invoked when the input field type is null
  • + *
  • Time-based aggregates (EARLIEST, LATEST) cannot resolve a default time field
  • + *
+ */ 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 resolveTimeField(List argList, CalcitePlanContext ctx) { if (argList.isEmpty()) { // Try to find @timestamp field @@ -1460,57 +1383,6 @@ static List resolveTimeField(List argList, CalcitePlanContext } } - /** - * Wraps a {@link SqlOperandTypeChecker} into a {@link PPLTypeChecker} for use in function - * signature validation. - * - * @param typeChecker the original SQL operand type checker - * @param functionName the name of the function for error reporting - * @param isUserDefinedFunction true if the function is user-defined, false otherwise - * @return a {@link PPLTypeChecker} that delegates to the provided {@code typeChecker} - */ - private static PPLTypeChecker wrapSqlOperandTypeChecker( - SqlOperandTypeChecker typeChecker, String functionName, boolean isUserDefinedFunction) { - PPLTypeChecker pplTypeChecker; - if (typeChecker instanceof ImplicitCastOperandTypeChecker implicitCastTypeChecker) { - pplTypeChecker = PPLTypeChecker.wrapFamily(implicitCastTypeChecker); - } else if (typeChecker instanceof CompositeOperandTypeChecker compositeTypeChecker) { - // UDFs implement their own composite type checkers, which always use OR logic for - // argument - // types. Verifying the composition type would require accessing a protected field in - // CompositeOperandTypeChecker. If access to this field is not allowed, type checking will - // be skipped, so we avoid checking the composition type here. - - // If compositeTypeChecker contains operand checkers other than family type checkers or - // other than OR compositions, the function with be registered with a null type checker, - // which means the function will not be type checked. - try { - pplTypeChecker = PPLTypeChecker.wrapComposite(compositeTypeChecker, !isUserDefinedFunction); - } catch (IllegalArgumentException | UnsupportedOperationException e) { - logger.debug( - String.format( - "Failed to create composite type checker for operator: %s. Will skip its type" - + " checking", - functionName), - e); - pplTypeChecker = null; - } - } else if (typeChecker instanceof SameOperandTypeChecker comparableTypeChecker) { - // Comparison operators like EQUAL, GREATER_THAN, LESS_THAN, etc. - // SameOperandTypeCheckers like COALESCE, IFNULL, etc. - pplTypeChecker = PPLTypeChecker.wrapComparable(comparableTypeChecker); - } else if (typeChecker instanceof UDFOperandMetadata.UDTOperandMetadata udtOperandMetadata) { - pplTypeChecker = PPLTypeChecker.wrapUDT(udtOperandMetadata.allowedParamTypes()); - } else if (typeChecker != null) { - pplTypeChecker = PPLTypeChecker.wrapDefault(typeChecker); - } else { - logger.info( - "Cannot create type checker for function: {}. Will skip its type checking", functionName); - pplTypeChecker = null; - } - return pplTypeChecker; - } - /** * Extracts the underlying {@link SqlOperandTypeChecker} from a {@link SqlOperator}. * @@ -1534,4 +1406,4 @@ private static SqlOperandTypeChecker extractTypeCheckerFromUDF(SqlOperator opera } return typeChecker; } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/UDFOperandMetadata.java b/core/src/main/java/org/opensearch/sql/expression/function/UDFOperandMetadata.java index dc4761b26e7..e963adc5eff 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/UDFOperandMetadata.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/UDFOperandMetadata.java @@ -12,8 +12,7 @@ import org.apache.calcite.sql.SqlCallBinding; import org.apache.calcite.sql.SqlOperandCountRange; import org.apache.calcite.sql.SqlOperator; -import org.apache.calcite.sql.type.CompositeOperandTypeChecker; -import org.apache.calcite.sql.type.FamilyOperandTypeChecker; +import org.apache.calcite.sql.type.SqlOperandCountRanges; import org.apache.calcite.sql.type.SqlOperandMetadata; import org.apache.calcite.sql.type.SqlOperandTypeChecker; import org.apache.calcite.sql.validate.SqlUserDefinedFunction; @@ -25,52 +24,23 @@ * SqlUserDefinedFunction} as a {@link SqlOperandMetadata}. */ public interface UDFOperandMetadata extends SqlOperandMetadata { - SqlOperandTypeChecker getInnerTypeChecker(); - - static UDFOperandMetadata wrap(FamilyOperandTypeChecker typeChecker) { - return new UDFOperandMetadata() { - @Override - public SqlOperandTypeChecker getInnerTypeChecker() { - return typeChecker; - } - - @Override - public List paramTypes(RelDataTypeFactory typeFactory) { - // This function is not used in the current context, so we return an empty list. - return Collections.emptyList(); - } - - @Override - public List paramNames() { - // This function is not used in the current context, so we return an empty list. - return Collections.emptyList(); - } - - @Override - public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) { - return typeChecker.checkOperandTypesWithoutTypeCoercion(callBinding, throwOnFailure); - } - - @Override - public SqlOperandCountRange getOperandCountRange() { - return typeChecker.getOperandCountRange(); - } - - @Override - public String getAllowedSignatures(SqlOperator op, String opName) { - return typeChecker.getAllowedSignatures(op, opName); - } - }; - } - - static UDFOperandMetadata wrap(CompositeOperandTypeChecker typeChecker) { - for (SqlOperandTypeChecker rule : typeChecker.getRules()) { - if (!(rule instanceof FamilyOperandTypeChecker)) { - throw new IllegalArgumentException( - "Currently only compositions of ImplicitCastOperandTypeChecker are supported"); - } - } - + /** + * Provides the underlying operand type checker used by this metadata. + * + * @return the {@link SqlOperandTypeChecker} responsible for validating operand types + */ +SqlOperandTypeChecker getInnerTypeChecker(); + + /** + * Wraps a SqlOperandTypeChecker as UDFOperandMetadata for use with SqlUserDefinedFunction. + * + *

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 paramTypes : allowedParamTypes) { + max = Math.max(max, paramTypes.size()); + min = Math.min(min, paramTypes.size()); + } + return SqlOperandCountRanges.between(min, max); } + /** + * Provide the allowed signatures string for the given operator. + * + *

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 pathStack) { } return builder.toString(); } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtractFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtractFunctionImpl.java index 76853706f64..7670531d2c4 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtractFunctionImpl.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonExtractFunctionImpl.java @@ -25,6 +25,7 @@ import org.apache.calcite.runtime.JsonFunctions; import org.apache.calcite.schema.impl.ScalarFunctionImpl; import org.apache.calcite.sql.SqlJsonValueEmptyOrErrorBehavior; +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; @@ -39,9 +40,14 @@ public SqlReturnTypeInference getReturnTypeInference() { return STRING_FORCE_NULLABLE; } + /** + * Provide operand metadata indicating the function accepts a variadic number of operands. + * + * @return the UDF operand metadata specifying variadic operands + */ @Override public UDFOperandMetadata getOperandMetadata() { - return null; + return UDFOperandMetadata.wrap(OperandTypes.VARIADIC); } public static class JsonExtractImplementor implements NotNullImplementor { @@ -101,4 +107,4 @@ private static String doJsonize(Object candidate) { return JsonFunctions.jsonize(candidate); } } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonFunctionImpl.java index 0379aeecb72..cd31deffd98 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonFunctionImpl.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonFunctionImpl.java @@ -15,6 +15,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.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_FORCE_NULLABLE; } + /** + * Specifies the operand metadata for this UDF, indicating it accepts a single operand of any SQL type. + * + * @return a UDFOperandMetadata instance that wraps OperandTypes.ANY + */ @Override public UDFOperandMetadata getOperandMetadata() { - return null; + return UDFOperandMetadata.wrap(OperandTypes.ANY); } public static class JsonImplementor implements NotNullImplementor { @@ -58,4 +64,4 @@ public static Object eval(Object... args) { } return null; } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonKeysFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonKeysFunctionImpl.java index 40214ca7556..26b4ebdb6ca 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonKeysFunctionImpl.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonKeysFunctionImpl.java @@ -17,6 +17,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; @@ -31,9 +32,14 @@ public SqlReturnTypeInference getReturnTypeInference() { return STRING_FORCE_NULLABLE; } + /** + * Specifies the operand metadata for this UDF as accepting any SQL type for all operands. + * + * @return a {@link UDFOperandMetadata} that wraps {@link org.apache.calcite.sql.type.OperandTypes#ANY} + */ @Override public UDFOperandMetadata getOperandMetadata() { - return null; + return UDFOperandMetadata.wrap(OperandTypes.ANY); } public static class JsonKeysImplementor implements NotNullImplementor { @@ -56,4 +62,4 @@ public static Object eval(Object... args) { } return value; } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonSetFunctionImpl.java b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonSetFunctionImpl.java index 27346b478e4..5c4a46accb4 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonSetFunctionImpl.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/jsonUDF/JsonSetFunctionImpl.java @@ -21,6 +21,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; @@ -35,9 +36,14 @@ public SqlReturnTypeInference getReturnTypeInference() { return STRING_FORCE_NULLABLE; } + /** + * Provide operand metadata enforcing that the function accepts one or more operands. + * + * @return UDFOperandMetadata describing the allowed operand types; `one or more` operands are required. + */ @Override public UDFOperandMetadata getOperandMetadata() { - return null; + return UDFOperandMetadata.wrap(OperandTypes.ONE_OR_MORE); } public static class JsonSetImplementor implements NotNullImplementor { @@ -66,4 +72,4 @@ public static Object eval(Object... args) { } return JsonFunctions.jsonSet(jsonStr, expands.toArray()); } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/udf/RelevanceQueryFunction.java b/core/src/main/java/org/opensearch/sql/expression/function/udf/RelevanceQueryFunction.java index d8e53704804..4ad13bc0bf9 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/udf/RelevanceQueryFunction.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/udf/RelevanceQueryFunction.java @@ -5,18 +5,16 @@ package org.opensearch.sql.expression.function.udf; -import com.google.common.collect.ImmutableList; import java.util.List; import org.apache.calcite.adapter.enumerable.NotNullImplementor; import org.apache.calcite.adapter.enumerable.NullPolicy; import org.apache.calcite.adapter.enumerable.RexToLixTranslator; import org.apache.calcite.linq4j.tree.Expression; import org.apache.calcite.rex.RexCall; -import org.apache.calcite.sql.type.CompositeOperandTypeChecker; import org.apache.calcite.sql.type.OperandTypes; import org.apache.calcite.sql.type.ReturnTypes; +import org.apache.calcite.sql.type.SqlOperandCountRanges; import org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.calcite.sql.type.SqlTypeFamily; import org.opensearch.sql.expression.function.ImplementorUDF; import org.opensearch.sql.expression.function.UDFOperandMetadata; @@ -31,63 +29,23 @@ public SqlReturnTypeInference getReturnTypeInference() { return ReturnTypes.BOOLEAN; } - /* - * The first parameter is always required (either fields or query). - * The second parameter is query when fields are present, otherwise it's the first parameter. - * Starting from the 3rd parameter (or 2nd when no fields), they are optional parameters for relevance queries. - * Different query has different parameter set, which will be validated in dedicated query builder. - * Query parameter is always required and cannot be null. + /** + * Provide operand metadata for the relevance query UDF. + * + *

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: + *

    + *
  • (CHARACTER, NUMERIC, CHARACTER)
  • + *
  • (DATETIME, NUMERIC, CHARACTER)
  • + *
  • (NUMERIC, NUMERIC, ANY)
  • + *
+ * + * @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. + * + *

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 translatedOperands) { @@ -86,7 +114,7 @@ public Expression implement( if (SqlTypeUtil.isDecimal(intervalType)) { interval = Expressions.call(interval, "doubleValue"); } - if (SqlTypeUtil.isNull(unitType)) { + if (SqlTypeUtil.isNull(unitType) || SqlTypeName.ANY.equals(unitType.getSqlTypeName())) { return switch (call.getType().getSqlTypeName()) { case BIGINT, INTEGER, SMALLINT, TINYINT -> Expressions.multiply(Expressions.divide(field, interval), interval); @@ -152,4 +180,4 @@ public static Object evalTimestamp( Rounding rounding = new TimestampRounding(exprInterval, unit); return rounding.round(exprValue).valueForCalcite(); } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/udf/binning/WidthBucketFunction.java b/core/src/main/java/org/opensearch/sql/expression/function/udf/binning/WidthBucketFunction.java index 08daf9c314b..c23973c5854 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/udf/binning/WidthBucketFunction.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/udf/binning/WidthBucketFunction.java @@ -16,8 +16,7 @@ import org.apache.calcite.rex.RexCall; import org.apache.calcite.sql.type.SqlReturnTypeInference; import org.apache.calcite.sql.type.SqlTypeName; -import org.opensearch.sql.calcite.type.ExprSqlType; -import org.opensearch.sql.calcite.utils.OpenSearchTypeFactory.ExprUDT; +import org.opensearch.sql.calcite.utils.OpenSearchTypeUtil; import org.opensearch.sql.calcite.utils.PPLOperandTypes; import org.opensearch.sql.calcite.utils.binning.BinConstants; import org.opensearch.sql.expression.function.ImplementorUDF; @@ -46,24 +45,29 @@ public WidthBucketFunction() { super(new WidthBucketImplementor(), NullPolicy.ANY); } + /** + * Determine the SQL return type for the WIDTH_BUCKET UDF based on the first operand. + * + * @return the function return type: the first operand's datetime type if that operand is a datetime; + * otherwise a nullable VARCHAR(2000) + */ @Override public SqlReturnTypeInference getReturnTypeInference() { return (opBinding) -> { RelDataTypeFactory typeFactory = opBinding.getTypeFactory(); RelDataType arg0Type = opBinding.getOperandType(0); - return dateRelatedType(arg0Type) + return OpenSearchTypeUtil.isDatetime(arg0Type) ? arg0Type : typeFactory.createTypeWithNullability( typeFactory.createSqlType(SqlTypeName.VARCHAR, 2000), true); }; } - public static boolean dateRelatedType(RelDataType type) { - return type instanceof ExprSqlType exprSqlType - && List.of(ExprUDT.EXPR_DATE, ExprUDT.EXPR_TIME, ExprUDT.EXPR_TIMESTAMP) - .contains(exprSqlType.getUdt()); - } - + /** + * Provide operand metadata describing the expected arguments for the WIDTH_BUCKET UDF. + * + * @return operand metadata defining the WIDTH_BUCKET function's expected operands + */ @Override public UDFOperandMetadata getOperandMetadata() { return PPLOperandTypes.WIDTH_BUCKET_OPERAND; @@ -191,4 +195,4 @@ private static int getAppropriateDecimalPlaces(double span) { } } } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/udf/math/DivideFunction.java b/core/src/main/java/org/opensearch/sql/expression/function/udf/math/DivideFunction.java index 1767a2fc69b..814023c6a0c 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/udf/math/DivideFunction.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/udf/math/DivideFunction.java @@ -15,6 +15,7 @@ import org.apache.calcite.linq4j.tree.Expressions; import org.apache.calcite.rel.type.RelDataTypeSystem; import org.apache.calcite.rex.RexCall; +import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.ReturnTypes; import org.apache.calcite.sql.type.SqlReturnTypeInference; import org.apache.calcite.sql.type.SqlTypeName; @@ -37,10 +38,30 @@ public class DivideFunction extends ImplementorUDF { */ public static final int MAX_NUMERIC_SCALE = 19; + /** + * Creates a DivideFunction configured with the divide implementor and a null policy that propagates nulls. + * + * Constructs a DivideFunction using DivideImplementor and NullPolicy.ANY so that any null operand yields a null result. + */ public DivideFunction() { super(new DivideImplementor(), NullPolicy.ANY); } + /** + * Indicates that this UDF represents the SQL DIVIDE operator. + * + * @return the SQL kind for the divide operator (SqlKind.DIVIDE) + */ + @Override + public SqlKind getKind() { + return SqlKind.DIVIDE; + } + + /** + * Infers the SQL return type as the numeric quotient and marks the result as nullable. + * + * @return the return-type inference representing a quotient with forced nullability + */ @Override public SqlReturnTypeInference getReturnTypeInference() { return ReturnTypes.QUOTIENT_FORCE_NULLABLE; @@ -88,4 +109,4 @@ public static Number divide(Number dividend, Number divisor) { return MathUtils.coerceToWidestFloatingType(dividend, divisor, result); } } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ModFunction.java b/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ModFunction.java index 7a8f8e75f92..152a5b868ab 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ModFunction.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ModFunction.java @@ -14,6 +14,7 @@ import org.apache.calcite.linq4j.tree.Expressions; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexCall; +import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.type.ReturnTypes; import org.apache.calcite.sql.type.SqlReturnTypeInference; import org.apache.calcite.sql.type.SqlTypeFamily; @@ -31,11 +32,31 @@ public ModFunction() { super(new ModImplementor(), NullPolicy.ANY); } + /** + * Provide return type inference that selects the least-restrictive operand type and forces the result to be nullable. + * + * @return a {@link SqlReturnTypeInference} that first determines the least-restrictive type among operands and then applies a transform to make the resulting type nullable + */ @Override public SqlReturnTypeInference getReturnTypeInference() { return ReturnTypes.LEAST_RESTRICTIVE.andThen(SqlTypeTransforms.FORCE_NULLABLE); } + /** + * Identifies the SQL operator kind as MOD. + * + * @return the SQL kind {@link SqlKind#MOD} + */ + @Override + public SqlKind getKind() { + return SqlKind.MOD; + } + + /** + * Specifies that this UDF accepts two numeric operands. + * + * @return operand metadata indicating both operands must be numeric + */ @Override public UDFOperandMetadata getOperandMetadata() { return PPLOperandTypes.NUMERIC_NUMERIC; @@ -100,4 +121,4 @@ public static Number floatingMod(Number dividend, Number divisor) { return MathUtils.coerceToWidestFloatingType(dividend, divisor, result.doubleValue()); } } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ScalarMaxFunction.java b/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ScalarMaxFunction.java index 9b4b0b48c73..bb4cde8e725 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ScalarMaxFunction.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ScalarMaxFunction.java @@ -14,6 +14,7 @@ import org.apache.calcite.linq4j.tree.Expression; import org.apache.calcite.linq4j.tree.Expressions; 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.SqlTypeName; import org.opensearch.sql.data.utils.MixedTypeComparator; @@ -35,9 +36,14 @@ public SqlReturnTypeInference getReturnTypeInference() { return opBinding -> opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY); } + /** + * Provide operand metadata indicating the function accepts a variadic number of arguments. + * + * @return a {@link UDFOperandMetadata} configured for variadic operands + */ @Override public UDFOperandMetadata getOperandMetadata() { - return null; + return UDFOperandMetadata.wrap(OperandTypes.VARIADIC); } public static class MaxImplementor implements NotNullImplementor { @@ -64,4 +70,4 @@ private static Object findMax(Object[] args) { .orElse(null); } } -} +} \ No newline at end of file diff --git a/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ScalarMinFunction.java b/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ScalarMinFunction.java index 441257a422e..678e1ef93c3 100644 --- a/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ScalarMinFunction.java +++ b/core/src/main/java/org/opensearch/sql/expression/function/udf/math/ScalarMinFunction.java @@ -14,6 +14,7 @@ import org.apache.calcite.linq4j.tree.Expression; import org.apache.calcite.linq4j.tree.Expressions; 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.SqlTypeName; import org.opensearch.sql.data.utils.MixedTypeComparator; @@ -35,9 +36,14 @@ public SqlReturnTypeInference getReturnTypeInference() { return opBinding -> opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY); } + /** + * Declares that this UDF accepts a variadic number of operands. + * + * @return metadata describing that the function accepts variadic operands (any types allowed per operand rules) + */ @Override public UDFOperandMetadata getOperandMetadata() { - return null; + return UDFOperandMetadata.wrap(OperandTypes.VARIADIC); } public static class MinImplementor implements NotNullImplementor { @@ -64,4 +70,4 @@ private static Object findMin(Object[] args) { .orElse(null); } } -} +} \ No newline at end of file diff --git a/core/src/test/java/org/opensearch/sql/expression/function/AggFunctionTestBase.java b/core/src/test/java/org/opensearch/sql/expression/function/AggFunctionTestBase.java index d20841a2cee..85097afeb0a 100644 --- a/core/src/test/java/org/opensearch/sql/expression/function/AggFunctionTestBase.java +++ b/core/src/test/java/org/opensearch/sql/expression/function/AggFunctionTestBase.java @@ -15,23 +15,31 @@ public abstract class AggFunctionTestBase { + /** + * Retrieve the aggregate function registry from the PPLFuncImpTable singleton. + * + * @return the map that associates each BuiltinFunctionName with its AggHandler + * @throws RuntimeException if the registry field cannot be accessed or read via reflection + */ @SuppressWarnings("unchecked") - protected Map> - getAggFunctionRegistry() { + protected Map getAggFunctionRegistry() { try { PPLFuncImpTable funcTable = PPLFuncImpTable.INSTANCE; Field field = PPLFuncImpTable.class.getDeclaredField("aggFunctionRegistry"); field.setAccessible(true); - return (Map>) - field.get(funcTable); + return (Map) field.get(funcTable); } catch (Exception e) { throw new RuntimeException("Failed to access aggFunctionRegistry", e); } } + /** + * Assert that the specified aggregate function is present in the registry and its handler is not null. + * + * @param functionName the builtin aggregate function enum to verify + */ protected void assertFunctionIsRegistered(BuiltinFunctionName functionName) { - Map> registry = - getAggFunctionRegistry(); + Map registry = getAggFunctionRegistry(); assertTrue( registry.containsKey(functionName), functionName.getName().getFunctionName() @@ -41,43 +49,50 @@ protected void assertFunctionIsRegistered(BuiltinFunctionName functionName) { functionName.getName().getFunctionName() + " function handler should not be null"); } + /** + * Assert that each provided aggregate function is registered and has a non-null handler. + * + * @param functionNames one or more BuiltinFunctionName values to validate in the registry + */ protected void assertFunctionsAreRegistered(BuiltinFunctionName... functionNames) { for (BuiltinFunctionName functionName : functionNames) { assertFunctionIsRegistered(functionName); } } + /** + * Asserts that each provided aggregate function has a registered, non-null handler. + * + * @param functionNames one or more BuiltinFunctionName values to verify; each must have a non-null handler in the registry + */ protected void assertFunctionHandlerTypes(BuiltinFunctionName... functionNames) { - Map> registry = - getAggFunctionRegistry(); + Map registry = getAggFunctionRegistry(); for (BuiltinFunctionName functionName : functionNames) { - org.apache.commons.lang3.tuple.Pair registryEntry = registry.get(functionName); - assertNotNull( - registryEntry, functionName.getName().getFunctionName() + " should be registered"); - - // Extract the AggHandler from the pair - AggHandler handler = registryEntry.getRight(); - + AggHandler handler = registry.get(functionName); assertNotNull( handler, functionName.getName().getFunctionName() + " handler should not be null"); - assertTrue( - handler instanceof AggHandler, - functionName.getName().getFunctionName() - + " handler should implement AggHandler interface"); } } + /** + * Verifies the aggregate function registry contains at least the specified number of entries. + * + * @param expectedMinimumSize the minimum number of aggregate functions expected in the registry + */ protected void assertRegistryMinimumSize(int expectedMinimumSize) { - Map> registry = - getAggFunctionRegistry(); + Map registry = getAggFunctionRegistry(); assertTrue( registry.size() >= expectedMinimumSize, "Registry should contain at least " + expectedMinimumSize + " aggregate functions"); } + /** + * Verifies that at least the provided built-in aggregate functions are present in the registry. + * + * @param knownFunctions the set of BuiltinFunctionName values expected to be present in the aggregate function registry + */ protected void assertKnownFunctionsPresent(Set knownFunctions) { - Map> registry = - getAggFunctionRegistry(); + Map registry = getAggFunctionRegistry(); long foundFunctions = registry.keySet().stream().filter(knownFunctions::contains).count(); assertTrue( @@ -111,4 +126,4 @@ protected void assertFunctionNameMapping(BuiltinFunctionName functionEnum, Strin functionEnum.getName().getFunctionName().equals(expectedName), "Function enum should map to expected name: " + expectedName); } -} +} \ No newline at end of file diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/clickbench/PPLClickBenchIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/clickbench/PPLClickBenchIT.java index 76b4d772e0a..57f1b203c81 100644 --- a/integ-test/src/test/java/org/opensearch/sql/calcite/clickbench/PPLClickBenchIT.java +++ b/integ-test/src/test/java/org/opensearch/sql/calcite/clickbench/PPLClickBenchIT.java @@ -13,6 +13,7 @@ import java.util.Map; import java.util.Set; import org.junit.AfterClass; +import org.junit.Assume; import org.junit.FixMethodOrder; import org.junit.Test; import org.junit.runners.MethodSorters; @@ -52,7 +53,17 @@ public static void reset() throws IOException { System.out.println(); } - /** Ignore queries that are not supported. */ + /** + * Determine which benchmark query indices to skip based on runtime capabilities. + * + * Adds indices for queries that are unsupported or require special runtime conditions: + * - 29 when Calcite is disabled (regexp_replace not supported in v2), + * - 30 when GCedMemoryUsage is not initialized (to avoid ResourceMonitor restrictions from script pushdown), + * - 41 when Calcite is enabled (requires special handling). + * + * @return a set of query indices to ignore + * @throws IOException if an I/O error occurs while checking runtime state + */ protected Set ignored() throws IOException { Set ignored = new HashSet(); if (!isCalciteEnabled()) { @@ -64,9 +75,22 @@ protected Set ignored() throws IOException { // because of too much script push down, which will cause ResourceMonitor restriction. ignored.add(30); } + if (isCalciteEnabled()) { + // Ignore q41 as it needs special handling + ignored.add(41); + } return ignored; } + /** + * Executes ClickBench PPL queries 1–43, validates explain plans when Calcite is enabled, and records timing. + * + * Skips indices returned by ignored(), loads and sanitizes each query from clickbench/queries/q{i}.ppl, + * compares the actual explain-plan YAML to clickbench/q{i}.yaml while ignoring generated IDs when Calcite is enabled, + * and records per-query timing in the shared summary. + * + * @throws IOException if a query or expected-plan file cannot be read + */ @Test public void test() throws IOException { for (int i = 1; i <= 43; i++) { @@ -83,4 +107,23 @@ public void test() throws IOException { timing(summary, "q" + i, ppl); } } -} + + /** + * Executes ClickBench query 41, verifies the produced explain-plan YAML matches either the primary + * or alternative expected plan, and records the query timing. + * + *

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 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= user_currency=USD", ImmutableMap.of("", ImmutableList.of("d664d7be-77d8-11f0-8880-0242f00b101d")))); } -} +} \ No newline at end of file diff --git a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteWhereCommandIT.java b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteWhereCommandIT.java index 582ce47000d..adfde9f45cf 100644 --- a/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteWhereCommandIT.java +++ b/integ-test/src/test/java/org/opensearch/sql/calcite/remote/CalciteWhereCommandIT.java @@ -5,18 +5,35 @@ package org.opensearch.sql.calcite.remote; +import static org.opensearch.sql.legacy.TestsConstants.TEST_INDEX_ACCOUNT; +import static org.opensearch.sql.util.MatcherUtils.rows; +import static org.opensearch.sql.util.MatcherUtils.verifyDataRows; + +import java.io.IOException; +import org.json.JSONObject; +import org.junit.jupiter.api.Test; import org.opensearch.sql.ppl.WhereCommandIT; public class CalciteWhereCommandIT extends WhereCommandIT { + /** + * Initialize the test environment and enable Calcite integration for the test class. + * + * @throws Exception if initialization or Calcite enabling fails + */ @Override public void init() throws Exception { super.init(); enableCalcite(); } - @Override - protected String getIncompatibleTypeErrMsg() { - return "In expression types are incompatible: fields type LONG, values type [INTEGER, INTEGER," - + " STRING]"; + @Test + public void testInWithMixedType() throws IOException { + // Mixed type coercion only work with Calcite enabled + JSONObject result = + executeQuery( + String.format( + "source=%s | where balance in (4180, 5686, '6077') | fields firstname", + TEST_INDEX_ACCOUNT)); + verifyDataRows(result, rows("Hattie"), rows("Dale"), rows("Hughes")); } -} +} \ No newline at end of file diff --git a/integ-test/src/test/java/org/opensearch/sql/ppl/DateTimeFunctionIT.java b/integ-test/src/test/java/org/opensearch/sql/ppl/DateTimeFunctionIT.java index cbfe3c84464..8fab2bcedb2 100644 --- a/integ-test/src/test/java/org/opensearch/sql/ppl/DateTimeFunctionIT.java +++ b/integ-test/src/test/java/org/opensearch/sql/ppl/DateTimeFunctionIT.java @@ -19,6 +19,7 @@ import java.time.LocalTime; import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; +import java.util.Locale; import java.util.TimeZone; import org.json.JSONObject; import org.junit.After; @@ -26,7 +27,6 @@ import org.junit.Ignore; import org.junit.jupiter.api.Test; import org.opensearch.client.Request; -import org.opensearch.sql.common.utils.StringUtils; @SuppressWarnings("unchecked") public class DateTimeFunctionIT extends PPLIntegTestCase { @@ -1136,12 +1136,27 @@ public void testToDays() throws IOException { verifySome(result.getJSONArray("datarows"), rows(738049)); } + /** + * Runs a PPL query that computes week(date, mode) for the given date and asserts the result. + * + * Executes a query selecting the week number for the provided date string using the specified + * mode, verifies the returned column has integer schema, and asserts the computed week equals + * expectedResult. + * + * @param date the date string to pass to the `date(...)` call in the query + * @param mode the week mode value used by the `week` function + * @param expectedResult the expected week number to verify in the query result + * @throws IOException if executing the query or fetching results fails + */ private void week(String date, int mode, int expectedResult) throws IOException { JSONObject result = executeQuery( - StringUtils.format( + String.format( + Locale.ROOT, "source=%s | eval f = week(date('%s'), %d) | fields f", - TEST_INDEX_DATE, date, mode)); + TEST_INDEX_DATE, + date, + mode)); verifySchema(result, schema("f", null, "int")); verifySome(result.getJSONArray("datarows"), rows(expectedResult)); } @@ -1589,4 +1604,4 @@ public void testCompareAgainstUTCDate() throws IOException { verifySchema(result, schema("birthdate", "timestamp")); verifyDataRows(result, rows(pplTimestamp)); } -} +} \ No newline at end of file diff --git a/integ-test/src/test/java/org/opensearch/sql/ppl/ExplainIT.java b/integ-test/src/test/java/org/opensearch/sql/ppl/ExplainIT.java index 62eadd7ef5e..4c7a26125e7 100644 --- a/integ-test/src/test/java/org/opensearch/sql/ppl/ExplainIT.java +++ b/integ-test/src/test/java/org/opensearch/sql/ppl/ExplainIT.java @@ -101,24 +101,33 @@ public void testFilterByCompareStringTimePushDownExplain() throws IOException { @Test public void testFilterByCompareIPCoercion() throws IOException { // Should automatically cast the string literal to IP and pushdown it as a range query - assertJsonEqualsIgnoreId( - loadExpectedPlan("explain_filter_compare_ip.json"), - explainQueryToString( + assertYamlEqualsIgnoreId( + loadExpectedPlan("explain_filter_compare_ip.yaml"), + explainQueryYaml( String.format( Locale.ROOT, "source=%s | where host > '1.1.1.1' | fields host", TEST_INDEX_WEBLOGS))); } + /** + * Verifies that an IPv6 string literal compared to an IP field with the literal on the left + * produces the expected explain plan showing range pushdown when Calcite is enabled. + * + * The test is skipped unless Calcite is enabled. It loads the expected YAML plan and asserts + * the actual explain YAML matches, ignoring plan IDs. + * + * @throws IOException if loading the expected plan file fails + */ @Test public void testFilterByCompareIpv6Swapped() throws IOException { // Ignored in v2: the serialized string is unstable because of function properties Assume.assumeTrue(isCalciteEnabled()); // Test swapping ip and string. In v2, this is pushed down as script; // with Calcite, it will still be pushed down as a range query - assertJsonEqualsIgnoreId( - loadExpectedPlan("explain_filter_compare_ipv6_swapped.json"), - explainQueryToString( + assertYamlEqualsIgnoreId( + loadExpectedPlan("explain_filter_compare_ipv6_swapped.yaml"), + explainQueryYaml( String.format( Locale.ROOT, "source=%s | where '::ffff:1234' <= host | fields host", @@ -127,12 +136,12 @@ public void testFilterByCompareIpv6Swapped() throws IOException { @Test public void testWeekArgumentCoercion() throws IOException { - String expected = loadExpectedPlan("explain_week_argument_coercion.json"); + String expected = loadExpectedPlan("explain_week_argument_coercion.yaml"); // Week accepts WEEK(timestamp/date/time, [optional int]), it should cast the string // argument to timestamp with Calcite. In v2, it accepts string, so there is no cast. - assertJsonEqualsIgnoreId( + assertYamlEqualsIgnoreId( expected, - explainQueryToString( + explainQueryYaml( String.format( Locale.ROOT, "source=%s | eval w = week('2024-12-10') | fields w", @@ -160,10 +169,10 @@ public void testCountAggPushDownExplain() throws IOException { @Test public void testSortPushDownExplain() throws IOException { - String expected = loadExpectedPlan("explain_sort_push.json"); - assertJsonEqualsIgnoreId( + String expected = loadExpectedPlan("explain_sort_push.yaml"); + assertYamlEqualsIgnoreId( expected, - explainQueryToString( + explainQueryYaml( "source=opensearch-sql_test_index_account" + "| sort age " + "| where age > 30" @@ -180,10 +189,10 @@ public void testSortWithCountPushDownExplain() throws IOException { @Test public void testSortWithDescPushDownExplain() throws IOException { - String expected = loadExpectedPlan("explain_sort_desc_push.json"); - assertJsonEqualsIgnoreId( + String expected = loadExpectedPlan("explain_sort_desc_push.yaml"); + assertYamlEqualsIgnoreId( expected, - explainQueryToString( + explainQueryYaml( "source=opensearch-sql_test_index_account | sort age desc, firstname | fields age," + " firstname")); } @@ -212,14 +221,23 @@ public void testSortWithAggregationExplain() throws IOException { // This test is covered in testExplain } + /** + * Verifies multi-stage sort pushdown behavior and key precedence in the explain plan. + * + * Loads the expected YAML plan and asserts the explain output matches it, ensuring + * sort keys from the later sort stage (e.g., `balance`, `gender`) take precedence + * over earlier sort keys. + * + * @throws IOException if reading the expected plan file fails + */ @Test public void testMultiSortPushDownExplain() throws IOException { // TODO: Fix the expected output in expectedOutput/ppl/explain_multi_sort_push.json (v2) // balance and gender should take precedence over account_number and firstname - String expected = loadExpectedPlan("explain_multi_sort_push.json"); - assertJsonEqualsIgnoreId( + String expected = loadExpectedPlan("explain_multi_sort_push.yaml"); + assertYamlEqualsIgnoreId( expected, - explainQueryToString( + explainQueryYaml( "source=opensearch-sql_test_index_account " + "| sort account_number, firstname, address, balance " + "| sort - balance, - gender, account_number " @@ -228,21 +246,30 @@ public void testMultiSortPushDownExplain() throws IOException { @Test public void testSortThenAggregatePushDownExplain() throws IOException { - String expected = loadExpectedPlan("explain_sort_then_agg_push.json"); - assertJsonEqualsIgnoreId( + String expected = loadExpectedPlan("explain_sort_then_agg_push.yaml"); + assertYamlEqualsIgnoreId( expected, - explainQueryToString( + explainQueryYaml( "source=opensearch-sql_test_index_account" + "| sort balance, age " + "| stats avg(balance) by state")); } + /** + * Verifies explain output when a field is renamed and an alias is used for sorting. + * + * Compares the generated explain YAML for a query that renames `firstname` to `name`, + * creates an `alias` from `name`, sorts by `alias`, and projects `alias` against the + * expected plan loaded from "explain_sort_rename_push.yaml". + * + * @throws IOException if loading the expected plan or producing the explain output fails + */ @Test public void testSortWithRenameExplain() throws IOException { - String expected = loadExpectedPlan("explain_sort_rename_push.json"); - assertJsonEqualsIgnoreId( + String expected = loadExpectedPlan("explain_sort_rename_push.yaml"); + assertYamlEqualsIgnoreId( expected, - explainQueryToString( + explainQueryYaml( "source=opensearch-sql_test_index_account " + "| rename firstname as name " + "| eval alias = name " @@ -368,10 +395,10 @@ public void testLimitWithMultipleOffsetPushdownExplain() throws IOException { @Test public void testFillNullPushDownExplain() throws IOException { - String expected = loadExpectedPlan("explain_fillnull_push.json"); - assertJsonEqualsIgnoreId( + String expected = loadExpectedPlan("explain_fillnull_push.yaml"); + assertYamlEqualsIgnoreId( expected, - explainQueryToString( + explainQueryYaml( "source=opensearch-sql_test_index_account" + " | fillnull with -1 in age,balance | fields age, balance")); } @@ -431,6 +458,16 @@ public void testPatternsSimplePatternMethodWithAggPushDownExplain() throws IOExc + " show_numbered_token=true")); } + /** + * Verifies the explain plan for the `brain` pattern method when aggregation pushdown is expected. + * + *

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