diff --git a/flink-table/flink-sql-parser/pom.xml b/flink-table/flink-sql-parser/pom.xml index 336947c16e2c1..9097c23326e63 100644 --- a/flink-table/flink-sql-parser/pom.xml +++ b/flink-table/flink-sql-parser/pom.xml @@ -68,12 +68,12 @@ under the License. ${calcite.version} +<#list (parser.truncateStatementParserMethods!default.parser.truncateStatementParserMethods) as method> + truncate = ${method}(s) + <#sep>| + + ) + { + return truncate; + } +} + + /** * Parses a literal expression, allowing continued string literals. * Usually returns an SqlLiteral, but a continued string literal @@ -4738,7 +4838,7 @@ SqlNode MultisetConstructor() : /** Parses an ARRAY constructor */ SqlNode ArrayConstructor() : { - SqlNodeList args; + final SqlNodeList args; SqlNode e; final Span s; final String p; @@ -4746,14 +4846,22 @@ SqlNode ArrayConstructor() : { { s = span(); } ( - LOOKAHEAD(1) - - // by sub query "MULTISET(SELECT * FROM T)" - e = OrderedQueryOrExpr(ExprContext.ACCEPT_QUERY) - + ( + // nullary array function call: "array()" (Apache Spark) + LOOKAHEAD(2) + { args = SqlNodeList.EMPTY; } + | + args = ParenthesizedQueryOrCommaList(ExprContext.ACCEPT_ALL) + ) { - return SqlStdOperatorTable.ARRAY_QUERY.createCall( - s.end(this), e); + if (args.size() == 1 && args.get(0).isA(SqlKind.QUERY)) { + // Array query constructor, 'ARRAY (SELECT * FROM t)' + return SqlStdOperatorTable.ARRAY_QUERY.createCall(s.end(this), args.get(0)); + } else { + // Spark ARRAY function, 'ARRAY(1, 2)', + // equivalent to standard 'ARRAY [1, 2]' + return SqlLibraryOperators.ARRAY.createCall(s.end(this), args.getList()); + } } | // by enumeration "ARRAY[e0, e1, ..., eN]" @@ -5165,6 +5273,8 @@ SqlIntervalQualifier TimeUnit() : { | { return new SqlIntervalQualifier(TimeUnit.MINUTE, null, getPos()); } | { return new SqlIntervalQualifier(TimeUnit.HOUR, null, getPos()); } | { return new SqlIntervalQualifier(TimeUnit.DAY, null, getPos()); } +| { return new SqlIntervalQualifier(TimeUnit.DOW, null, getPos()); } +| { return new SqlIntervalQualifier(TimeUnit.DOY, null, getPos()); } | { return new SqlIntervalQualifier(TimeUnit.DOW, null, getPos()); } | { return new SqlIntervalQualifier(TimeUnit.DOY, null, getPos()); } | { return new SqlIntervalQualifier(TimeUnit.ISODOW, null, getPos()); } @@ -6032,11 +6142,17 @@ SqlNode BuiltinFunctionCall() : SqlDataTypeSpec dt; final SqlIntervalQualifier unit; final SqlNode node; + final SqlLiteral style; // mssql convert 'style' operand + final SqlFunction f; } { //~ FUNCTIONS WITH SPECIAL SYNTAX --------------------------------------- ( - { s = span(); } + ( { f = SqlStdOperatorTable.CAST; } + | { f = SqlLibraryOperators.SAFE_CAST; } + | { f = SqlLibraryOperators.TRY_CAST; } + ) + { s = span(); } AddExpression(args, ExprContext.ACCEPT_SUB_QUERY) ( @@ -6045,7 +6161,7 @@ SqlNode BuiltinFunctionCall() : e = IntervalQualifier() { args.add(e); } ) { - return SqlStdOperatorTable.CAST.createCall(s.end(this), args); + return f.createCall(s.end(this), args); } | { s = span(); } @@ -6073,11 +6189,47 @@ SqlNode BuiltinFunctionCall() : | { s = span(); } - AddExpression(args, ExprContext.ACCEPT_SUB_QUERY) - name = SimpleIdentifier() { args.add(name); } - { - return SqlStdOperatorTable.CONVERT.createCall(s.end(this), args); - } + ( + // CONVERT in the form of CONVERT(x USING y) + + // "AddExpression" matches INTERVAL, + // which can also be 1st token in args of MSSQL CONVERT + // So lookahead another token (to match vs. ) + LOOKAHEAD(2) + AddExpression(args, ExprContext.ACCEPT_SUB_QUERY) + ( + name = SimpleIdentifier() { args.add(name); } + { + return SqlStdOperatorTable.TRANSLATE.createCall(s.end(this), args); + } + | + e = SimpleIdentifier() { args.add(e); } + e = SimpleIdentifier() { args.add(e); } + { + return SqlStdOperatorTable.CONVERT.createCall(s.end(this), args); + } + ) + | + // MSSql CONVERT(type, val [,style]) + ( + dt = DataType() { args.add(dt); } + | + e = IntervalQualifier() { args.add(e); } + ) + + AddExpression(args, ExprContext.ACCEPT_SUB_QUERY) + [ + + ( + style = UnsignedNumericLiteral() { args.add(style); } + | + { args.add(SqlLiteral.createNull(getPos())); } + ) + ] + { + return SqlLibraryOperators.MSSQL_CONVERT.createCall(s.end(this), args); + } + ) | { s = span(); } @@ -6193,6 +6345,8 @@ SqlNode BuiltinFunctionCall() : node = DateDiffFunctionCall() { return node; } | node = DateTruncFunctionCall() { return node; } + | + node = DatetimeTruncFunctionCall() { return node; } | node = TimestampAddFunctionCall() { return node; } | @@ -6898,6 +7052,27 @@ SqlCall TimeDiffFunctionCall() : } } +/** + * Parses a call to DATETIME_TRUNC. + */ +SqlNode DatetimeTruncFunctionCall() : +{ + final List args = new ArrayList(); + final Span s; + final SqlIntervalQualifier unit; + final SqlNode literal; +} +{ + { s = span(); } + + AddExpression(args, ExprContext.ACCEPT_SUB_QUERY) + + unit = TimeUnitOrName() { args.add(unit); } + { + return SqlLibraryOperators.DATETIME_TRUNC.createCall(s.end(this), args); + } +} + /** * Parses a call to TIME_TRUNC. */ @@ -7132,6 +7307,9 @@ SqlNode NamedFunctionCall() : ( LOOKAHEAD(2) call = StringAggFunctionCall() + | + LOOKAHEAD(1) + call = PercentileFunctionCall() | call = NamedCall() ) @@ -7229,9 +7407,7 @@ SqlNode StandardFloorCeilOptions(Span s, boolean floorFlag) : } )? { - SqlOperator op = floorFlag - ? SqlStdOperatorTable.FLOOR - : SqlStdOperatorTable.CEIL; + SqlOperator op = SqlStdOperatorTable.floorCeil(floorFlag, this.conformance); function = op.createCall(s.end(this), args); } ( @@ -7335,8 +7511,6 @@ SqlIdentifier ReservedFunctionName() : | | | - | - | | | | @@ -7821,13 +7995,16 @@ SqlPostfixOperator PostfixRowOperator() : | < DATA: "DATA" > | < DATABASE: "DATABASE" > | < DATE: "DATE" > +| < DATE_DIFF: "DATE_DIFF" > | < DATE_TRUNC: "DATE_TRUNC" > | < DATETIME: "DATETIME" > | < DATETIME_DIFF: "DATETIME_DIFF" > | < DATETIME_INTERVAL_CODE: "DATETIME_INTERVAL_CODE" > | < DATETIME_INTERVAL_PRECISION: "DATETIME_INTERVAL_PRECISION" > -| < DATE_DIFF: "DATE_DIFF" > +| < DATETIME_TRUNC: "DATETIME_TRUNC" > | < DAY: "DAY" > +| < DAYOFWEEK: "DAYOFWEEK" > +| < DAYOFYEAR: "DAYOFYEAR" > | < DAYS: "DAYS" > | < DEALLOCATE: "DEALLOCATE" > | < DEC: "DEC" > @@ -8074,6 +8251,7 @@ SqlPostfixOperator PostfixRowOperator() : | < OR: "OR" > | < ORDER: "ORDER" > | < ORDERING: "ORDERING" > +| < ORDINAL: "ORDINAL" > | < ORDINALITY: "ORDINALITY" > | < OTHERS: "OTHERS" > | < OUT: "OUT" > @@ -8177,6 +8355,9 @@ SqlPostfixOperator PostfixRowOperator() : | < ROW_NUMBER: "ROW_NUMBER" > | < ROWS: "ROWS" > | < RUNNING: "RUNNING" > +| < SAFE_CAST: "SAFE_CAST" > +| < SAFE_OFFSET: "SAFE_OFFSET" > +| < SAFE_ORDINAL: "SAFE_ORDINAL" > | < SATURDAY: "SATURDAY" > | < SAVEPOINT: "SAVEPOINT" > | < SCALAR: "SCALAR" > @@ -8335,6 +8516,7 @@ SqlPostfixOperator PostfixRowOperator() : | < TRIM_ARRAY: "TRIM_ARRAY" > | < TRUE: "TRUE" > | < TRUNCATE: "TRUNCATE" > +| < TRY_CAST: "TRY_CAST" > | < TUESDAY: "TUESDAY" > | < TUMBLE: "TUMBLE" > | < TYPE: "TYPE" > @@ -8777,7 +8959,11 @@ MORE : TOKEN : { - < IDENTIFIER: (|)* > + <#if parser.customIdentifierToken?has_content> + ${parser.customIdentifierToken} + <#else> + < IDENTIFIER: (|)* > + } TOKEN : diff --git a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java index b1235d6a42c46..798770c618d36 100644 --- a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java +++ b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/validate/FlinkSqlConformance.java @@ -168,8 +168,8 @@ public SqlLibrary semantics() { } @Override - public boolean allowCoercionStringToArray() { - return SqlConformanceEnum.DEFAULT.allowCoercionStringToArray(); + public boolean allowLenientCoercion() { + return SqlConformanceEnum.DEFAULT.allowLenientCoercion(); } @Override diff --git a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java index a1f9c594ccd5e..d91c42eea0093 100644 --- a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java +++ b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/FlinkSqlParserImplTest.java @@ -50,6 +50,24 @@ public SqlParserFixture fixture() { return super.fixture().withConfig(c -> c.withParserFactory(FlinkSqlParserImpl.FACTORY)); } + @Test + void testArrayFunction() {} + + @Test + void testArrayQueryConstructor() {} + + @Test + void testPercentileCont() {} + + @Test + void testPercentileContBigQuery() {} + + @Test + void testPercentileDisc() {} + + @Test + void testPercentileDiscBigQuery() {} + @Test void testShowCatalogs() { sql("show catalogs").ok("SHOW CATALOGS"); @@ -2823,15 +2841,15 @@ void testSetReset() { @Test void testTryCast() { // Simple types - expr("try_cast(a as timestamp)").ok("TRY_CAST(`A` AS TIMESTAMP)"); - expr("try_cast('abc' as timestamp)").ok("TRY_CAST('abc' AS TIMESTAMP)"); + expr("try_cast(a as timestamp)").ok("(TRY_CAST(`A` AS TIMESTAMP))"); + expr("try_cast('abc' as timestamp)").ok("(TRY_CAST('abc' AS TIMESTAMP))"); // Complex types expr("try_cast(a as row(f0 int, f1 varchar))") - .ok("TRY_CAST(`A` AS ROW(`F0` INTEGER, `F1` VARCHAR))"); + .ok("(TRY_CAST(`A` AS ROW(`F0` INTEGER, `F1` VARCHAR)))"); expr("try_cast(a as row(f0 int array, f1 map, f2 STRING NOT NULL))") .ok( - "TRY_CAST(`A` AS ROW(`F0` INTEGER ARRAY, `F1` MAP< STRING, DECIMAL(10, 2) >, `F2` STRING NOT NULL))"); + "(TRY_CAST(`A` AS ROW(`F0` INTEGER ARRAY, `F1` MAP< STRING, DECIMAL(10, 2) >, `F2` STRING NOT NULL)))"); } @Test diff --git a/flink-table/flink-table-calcite-bridge/pom.xml b/flink-table/flink-table-calcite-bridge/pom.xml index 5b42ee8694d88..c686b8b949865 100644 --- a/flink-table/flink-table-calcite-bridge/pom.xml +++ b/flink-table/flink-table-calcite-bridge/pom.xml @@ -45,22 +45,22 @@ under the License. ${calcite.version} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java index 22c8f1e3b62f9..6327af906e443 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/metadata/RelMdPredicates.java @@ -685,6 +685,7 @@ public RelOptPredicateList inferPredicates(boolean includeEqualityInference) { case SEMI: case INNER: case LEFT: + case ANTI: infer( leftChildPredicates, allExprs, @@ -762,6 +763,7 @@ public RelOptPredicateList inferPredicates(boolean includeEqualityInference) { leftInferredPredicates, rightInferredPredicates); case LEFT: + case ANTI: return RelOptPredicateList.of( rexBuilder, RelOptUtil.conjunctions(leftChildPredicates), diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rex/RexUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rex/RexUtil.java index 430ead73b0cce..3f59e33aa8c2a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rex/RexUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rex/RexUtil.java @@ -76,7 +76,7 @@ * because of current Calcite way of inferring constants from IS NOT DISTINCT FROM clashes with * filter push down. * - *

Lines 397 ~ 399, Use Calcite 1.32.0 behavior for {@link RexUtil#gatherConstraints(Class, + *

Lines 399 ~ 401, Use Calcite 1.32.0 behavior for {@link RexUtil#gatherConstraints(Class, * RexNode, Map, Set, RexBuilder)}. */ public class RexUtil { @@ -870,8 +870,7 @@ public Void visitCall(RexCall call) { } /** - * Returns whether a given tree contains any input references (both {@link RexInputRef} or - * {@link RexTableArgCall}). + * Returns whether a given tree contains any {link RexInputRef} nodes. * * @param node a RexNode tree */ @@ -3000,7 +2999,8 @@ public RexNode visitCall(RexCall call) { if (simplifiedNode.getType().equals(call.getType())) { return simplifiedNode; } - return simplify.rexBuilder.makeCast(call.getType(), simplifiedNode, matchNullability); + return simplify.rexBuilder.makeCast( + call.getType(), simplifiedNode, matchNullability, false); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlFunction.java index 7b4d7003c9cd4..1e0a918fbb1cf 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlFunction.java @@ -277,6 +277,16 @@ private RelDataType deriveType( getKind(), validator.getCatalogReader().nameMatcher(), false); + + // If the call already has an operator and its syntax is SPECIAL, it must + // have been created intentionally by the parser. + if (function == null + && call.getOperator().getSyntax() == SqlSyntax.SPECIAL + && call.getOperator() instanceof SqlFunction + && validator.getOperatorTable().getOperatorList().contains(call.getOperator())) { + function = (SqlFunction) call.getOperator(); + } + try { // if we have a match on function name and parameter count, but // couldn't find a function with a COLUMN_LIST type, retry, but diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlUtil.java index 814cf606650c2..a5bd65a0cb85b 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/SqlUtil.java @@ -34,6 +34,7 @@ import org.apache.calcite.runtime.CalciteContextException; import org.apache.calcite.runtime.CalciteException; import org.apache.calcite.runtime.Resources; +import org.apache.calcite.sql.fun.SqlInOperator; import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlOperandMetadata; @@ -42,6 +43,7 @@ import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.type.SqlTypeUtil; import org.apache.calcite.sql.util.SqlBasicVisitor; +import org.apache.calcite.sql.util.SqlVisitor; import org.apache.calcite.sql.validate.SqlNameMatcher; import org.apache.calcite.sql.validate.SqlValidatorUtil; import org.apache.calcite.util.BarfingInvocationHandler; @@ -1197,6 +1199,48 @@ private static SqlNode createBalancedCall( return op.createCall(pos, leftNode, rightNode); } + /** + * Returns whether a given node contains a {@link SqlInOperator}. + * + * @param node AST tree + */ + public static boolean containsIn(SqlNode node) { + final Predicate callPredicate = + call -> call.getOperator() instanceof SqlInOperator; + return containsCall(node, callPredicate); + } + + /** + * Returns whether an AST tree contains a call to an aggregate function. + * + * @param node AST tree + */ + public static boolean containsAgg(SqlNode node) { + final Predicate callPredicate = call -> call.getOperator().isAggregator(); + return containsCall(node, callPredicate); + } + + /** Returns whether an AST tree contains a call that matches a given predicate. */ + private static boolean containsCall(SqlNode node, Predicate callPredicate) { + try { + SqlVisitor visitor = + new SqlBasicVisitor() { + @Override + public Void visit(SqlCall call) { + if (callPredicate.test(call)) { + throw new Util.FoundOne(call); + } + return super.visit(call); + } + }; + node.accept(visitor); + return false; + } catch (Util.FoundOne e) { + Util.swallow(e, null); + return true; + } + } + // ~ Inner Classes ---------------------------------------------------------- /** diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java index 636b71364df44..3d24f2fecb999 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableSetMultimap; import com.google.common.collect.SetMultimap; import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeFamily; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCallBinding; @@ -36,17 +37,26 @@ import org.apache.calcite.sql.SqlSyntax; import org.apache.calcite.sql.SqlUtil; import org.apache.calcite.sql.SqlWriter; +import org.apache.calcite.sql.type.FlinkSqlTypeMappingRule; import org.apache.calcite.sql.type.InferTypes; import org.apache.calcite.sql.type.SqlOperandCountRanges; +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.SqlMonotonicity; -import org.apache.calcite.sql.validate.SqlValidatorImpl; import java.text.Collator; +import java.util.ArrayList; +import java.util.List; import java.util.Objects; +import static com.google.common.base.Preconditions.checkArgument; +import static java.util.Objects.requireNonNull; +import static org.apache.calcite.sql.type.SqlTypeUtil.isArray; +import static org.apache.calcite.sql.type.SqlTypeUtil.isCollection; +import static org.apache.calcite.sql.type.SqlTypeUtil.isMap; +import static org.apache.calcite.sql.type.SqlTypeUtil.isRow; import static org.apache.calcite.util.Static.RESOURCE; /** @@ -88,29 +98,112 @@ public class SqlCastFunction extends SqlFunction { // ~ Constructors ----------------------------------------------------------- public SqlCastFunction() { - super("CAST", SqlKind.CAST, null, InferTypes.FIRST_KNOWN, null, SqlFunctionCategory.SYSTEM); + this(SqlKind.CAST.toString(), SqlKind.CAST); + } + + public SqlCastFunction(String name, SqlKind kind) { + super( + name, + kind, + returnTypeInference(kind == SqlKind.SAFE_CAST), + InferTypes.FIRST_KNOWN, + null, + SqlFunctionCategory.SYSTEM); + checkArgument(kind == SqlKind.CAST || kind == SqlKind.SAFE_CAST, kind); } // ~ Methods ---------------------------------------------------------------- - @Override - public RelDataType inferReturnType(SqlOperatorBinding opBinding) { - assert opBinding.getOperandCount() == 2; - RelDataType ret = opBinding.getOperandType(1); - RelDataType firstType = opBinding.getOperandType(0); - ret = opBinding.getTypeFactory().createTypeWithNullability(ret, firstType.isNullable()); - if (opBinding instanceof SqlCallBinding) { - SqlCallBinding callBinding = (SqlCallBinding) opBinding; - SqlNode operand0 = callBinding.operand(0); - - // dynamic parameters and null constants need their types assigned - // to them using the type they are casted to. - if (SqlUtil.isNullLiteral(operand0, false) || (operand0 instanceof SqlDynamicParam)) { - final SqlValidatorImpl validator = (SqlValidatorImpl) callBinding.getValidator(); - validator.setValidatedNodeType(operand0, ret); + static SqlReturnTypeInference returnTypeInference(boolean safe) { + return opBinding -> { + assert opBinding.getOperandCount() == 2; + final RelDataType ret = + deriveType( + opBinding.getTypeFactory(), + opBinding.getOperandType(0), + opBinding.getOperandType(1), + safe); + + if (opBinding instanceof SqlCallBinding) { + final SqlCallBinding callBinding = (SqlCallBinding) opBinding; + SqlNode operand0 = callBinding.operand(0); + + // dynamic parameters and null constants need their types assigned + // to them using the type they are casted to. + if (SqlUtil.isNullLiteral(operand0, false) || operand0 instanceof SqlDynamicParam) { + callBinding.getValidator().setValidatedNodeType(operand0, ret); + } } + return ret; + }; + } + + /** Derives the type of "CAST(expression AS targetType)". */ + public static RelDataType deriveType( + RelDataTypeFactory typeFactory, + RelDataType expressionType, + RelDataType targetType, + boolean safe) { + return typeFactory.createTypeWithNullability( + targetType, expressionType.isNullable() || safe); + } + + private static RelDataType createTypeWithNullabilityFromExpr( + RelDataTypeFactory typeFactory, + RelDataType expressionType, + RelDataType targetType, + boolean safe) { + boolean isNullable = expressionType.isNullable() || safe; + + if (isCollection(expressionType)) { + RelDataType expressionElementType = expressionType.getComponentType(); + RelDataType targetElementType = targetType.getComponentType(); + requireNonNull(expressionElementType, () -> "componentType of " + expressionType); + requireNonNull(targetElementType, () -> "componentType of " + targetType); + RelDataType newElementType = + createTypeWithNullabilityFromExpr( + typeFactory, expressionElementType, targetElementType, safe); + return isArray(targetType) + ? SqlTypeUtil.createArrayType(typeFactory, newElementType, isNullable) + : SqlTypeUtil.createMultisetType(typeFactory, newElementType, isNullable); + } + + if (isRow(expressionType)) { + final int fieldCount = expressionType.getFieldCount(); + final List typeList = new ArrayList<>(fieldCount); + for (int i = 0; i < fieldCount; ++i) { + RelDataType expressionElementType = expressionType.getFieldList().get(i).getType(); + RelDataType targetElementType = targetType.getFieldList().get(i).getType(); + typeList.add( + createTypeWithNullabilityFromExpr( + typeFactory, expressionElementType, targetElementType, safe)); + } + return typeFactory.createTypeWithNullability( + typeFactory.createStructType(typeList, targetType.getFieldNames()), isNullable); } - return ret; + + if (isMap(expressionType)) { + RelDataType expressionKeyType = + requireNonNull( + expressionType.getKeyType(), () -> "keyType of " + expressionType); + RelDataType expressionValueType = + requireNonNull( + expressionType.getValueType(), () -> "valueType of " + expressionType); + RelDataType targetKeyType = + requireNonNull(targetType.getKeyType(), () -> "keyType of " + targetType); + RelDataType targetValueType = + requireNonNull(targetType.getValueType(), () -> "valueType of " + targetType); + + RelDataType keyType = + createTypeWithNullabilityFromExpr( + typeFactory, expressionKeyType, targetKeyType, safe); + RelDataType valueType = + createTypeWithNullabilityFromExpr( + typeFactory, expressionValueType, targetValueType, safe); + SqlTypeUtil.createMapType(typeFactory, keyType, valueType, isNullable); + } + + return typeFactory.createTypeWithNullability(targetType, isNullable); } @Override @@ -175,7 +268,8 @@ private boolean canCastFrom(RelDataType toType, RelDataType fromType) { FlinkTypeFactory.toLogicalType(fromType), FlinkTypeFactory.toLogicalType(toType)); default: - return SqlTypeUtil.canCastFrom(toType, fromType, true); + return SqlTypeUtil.canCastFrom( + toType, fromType, FlinkSqlTypeMappingRule.instance()); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java deleted file mode 100644 index 97325c64c3a11..0000000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to you under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.calcite.sql.fun; - -import org.apache.calcite.avatica.util.TimeUnit; -import org.apache.calcite.rel.type.RelDataType; -import org.apache.calcite.rel.type.RelDataTypeFactory; -import org.apache.calcite.sql.SqlCall; -import org.apache.calcite.sql.SqlFunction; -import org.apache.calcite.sql.SqlFunctionCategory; -import org.apache.calcite.sql.SqlIntervalQualifier; -import org.apache.calcite.sql.SqlKind; -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.SqlTypeTransforms; -import org.apache.calcite.sql.validate.SqlValidator; -import org.apache.calcite.sql.validate.SqlValidatorScope; -import org.checkerframework.checker.nullness.qual.Nullable; - -import static org.apache.calcite.util.Util.first; - -/** - * The TIMESTAMPADD function, which adds an interval to a datetime (TIMESTAMP, TIME or - * DATE). - * - *

The SQL syntax is - * - *

- * - * TIMESTAMPADD(timestamp interval, quantity, - * datetime) - * - *
- * - *

The interval time unit can one of the following literals: - * - *

    - *
  • NANOSECOND (and synonym SQL_TSI_FRAC_SECOND) - *
  • MICROSECOND (and synonyms SQL_TSI_MICROSECOND, FRAC_SECOND) - *
  • SECOND (and synonym SQL_TSI_SECOND) - *
  • MINUTE (and synonym SQL_TSI_MINUTE) - *
  • HOUR (and synonym SQL_TSI_HOUR) - *
  • DAY (and synonym SQL_TSI_DAY) - *
  • WEEK (and synonym SQL_TSI_WEEK) - *
  • MONTH (and synonym SQL_TSI_MONTH) - *
  • QUARTER (and synonym SQL_TSI_QUARTER) - *
  • YEAR (and synonym SQL_TSI_YEAR) - *
- * - *

Returns modified datetime. - * - *

This class was copied over from Calcite to fix the return type deduction issue on timestamp - * with local time zone type (CALCITE-4698). - */ -public class SqlTimestampAddFunction extends SqlFunction { - - private static final int MILLISECOND_PRECISION = 3; - private static final int MICROSECOND_PRECISION = 6; - - private static final SqlReturnTypeInference RETURN_TYPE_INFERENCE = - opBinding -> - deduceType( - opBinding.getTypeFactory(), - opBinding.getOperandLiteralValue(0, TimeUnit.class), - opBinding.getOperandType(2)); - - // BEGIN FLINK MODIFICATION - // Reason: this method is changed to deduce return type on timestamp with local time zone - // correctly - // Whole class should be removed after CALCITE-4698 is fixed - public static RelDataType deduceType( - RelDataTypeFactory typeFactory, - @Nullable TimeUnit timeUnit, - RelDataType operandType1, - RelDataType operandType2) { - final RelDataType type = deduceType(typeFactory, timeUnit, operandType2); - return typeFactory.createTypeWithNullability( - type, operandType1.isNullable() || operandType2.isNullable()); - } - - static RelDataType deduceType( - RelDataTypeFactory typeFactory, @Nullable TimeUnit timeUnit, RelDataType datetimeType) { - final TimeUnit timeUnit2 = first(timeUnit, TimeUnit.EPOCH); - SqlTypeName typeName = datetimeType.getSqlTypeName(); - switch (timeUnit2) { - case MILLISECOND: - return typeFactory.createSqlType( - typeName, Math.max(MILLISECOND_PRECISION, datetimeType.getPrecision())); - case MICROSECOND: - return typeFactory.createSqlType( - typeName, Math.max(MICROSECOND_PRECISION, datetimeType.getPrecision())); - case HOUR: - case MINUTE: - case SECOND: - if (datetimeType.getFamily() == SqlTypeFamily.TIME) { - return datetimeType; - } else if (datetimeType.getFamily() == SqlTypeFamily.TIMESTAMP) { - return typeFactory.createSqlType(typeName, datetimeType.getPrecision()); - } else { - return typeFactory.createSqlType(SqlTypeName.TIMESTAMP); - } - default: - return datetimeType; - } - } - - @Override - public void validateCall( - SqlCall call, - SqlValidator validator, - SqlValidatorScope scope, - SqlValidatorScope operandScope) { - super.validateCall(call, validator, scope, operandScope); - - // This is either a time unit or a time frame: - // - // * In "TIMESTAMPADD(YEAR, 2, x)" operand 0 is a SqlIntervalQualifier - // with startUnit = YEAR and timeFrameName = null. - // - // * In "TIMESTAMPADD(MINUTE15, 2, x) operand 0 is a SqlIntervalQualifier - // with startUnit = EPOCH and timeFrameName = 'MINUTE15'. - // - // If the latter, check that timeFrameName is valid. - validator.validateTimeFrame((SqlIntervalQualifier) call.getOperandList().get(0)); - } - - // END FLINK MODIFICATION - - /** Creates a SqlTimestampAddFunction. */ - SqlTimestampAddFunction(String name) { - super( - name, - SqlKind.TIMESTAMP_ADD, - RETURN_TYPE_INFERENCE.andThen(SqlTypeTransforms.TO_NULLABLE), - null, - OperandTypes.family( - SqlTypeFamily.ANY, SqlTypeFamily.INTEGER, SqlTypeFamily.DATETIME), - SqlFunctionCategory.TIMEDATE); - } -} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/CompositeSingleOperandTypeChecker.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/CompositeSingleOperandTypeChecker.java deleted file mode 100644 index d7214ea876502..0000000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/CompositeSingleOperandTypeChecker.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to you under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.calcite.sql.type; - -import com.google.common.collect.ImmutableList; -import org.apache.calcite.sql.SqlCallBinding; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.util.Util; -import org.checkerframework.checker.nullness.qual.Nullable; - -/** - * Default implementation of {@link org.apache.calcite.sql.type.CompositeOperandTypeChecker}, the - * class was copied over because of current Calcite issue CALCITE-5380. - * - *

Lines 73 ~ 78, 101 ~ 107 - */ -public class CompositeSingleOperandTypeChecker extends CompositeOperandTypeChecker - implements SqlSingleOperandTypeChecker { - - // ~ Constructors ----------------------------------------------------------- - - /** - * Creates a CompositeSingleOperandTypeChecker. Outside this package, use {@link - * SqlSingleOperandTypeChecker#and(SqlSingleOperandTypeChecker)}, {@link OperandTypes#and}, - * {@link OperandTypes#or} and similar. - */ - CompositeSingleOperandTypeChecker( - CompositeOperandTypeChecker.Composition composition, - ImmutableList allowedRules, - @Nullable String allowedSignatures) { - super(composition, allowedRules, allowedSignatures, null, null); - } - - // ~ Methods ---------------------------------------------------------------- - - @SuppressWarnings("unchecked") - @Override - public ImmutableList getRules() { - return (ImmutableList) allowedRules; - } - - @Override - public boolean checkSingleOperandType( - SqlCallBinding callBinding, SqlNode node, int iFormalOperand, boolean throwOnFailure) { - assert allowedRules.size() >= 1; - - final ImmutableList rules = getRules(); - if (composition == Composition.SEQUENCE) { - return rules.get(iFormalOperand) - .checkSingleOperandType(callBinding, node, 0, throwOnFailure); - } - - int typeErrorCount = 0; - - boolean throwOnAndFailure = (composition == Composition.AND) && throwOnFailure; - - for (SqlSingleOperandTypeChecker rule : rules) { - if (!rule.checkSingleOperandType( - // FLINK MODIFICATION BEGIN - callBinding, - node, - rule.getClass() == FamilyOperandTypeChecker.class ? 0 : iFormalOperand, - throwOnAndFailure)) { - // FLINK MODIFICATION END - typeErrorCount++; - } - } - - boolean ret; - switch (composition) { - case AND: - ret = typeErrorCount == 0; - break; - case OR: - ret = typeErrorCount < allowedRules.size(); - break; - default: - // should never come here - throw Util.unexpected(composition); - } - - if (!ret && throwOnFailure) { - // In the case of a composite OR, we want to throw an error - // describing in more detail what the problem was, hence doing the - // loop again. - for (SqlSingleOperandTypeChecker rule : rules) { - // FLINK MODIFICATION BEGIN - rule.checkSingleOperandType( - callBinding, - node, - rule.getClass() == FamilyOperandTypeChecker.class ? 0 : iFormalOperand, - true); - // FLINK MODIFICATION END - } - - // If no exception thrown, just throw a generic validation signature - // error. - throw callBinding.newValidationSignatureError(); - } - - return ret; - } -} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/FlinkSqlTypeMappingRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/FlinkSqlTypeMappingRule.java new file mode 100644 index 0000000000000..1c51c2d8c5b43 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/FlinkSqlTypeMappingRule.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.calcite.sql.type; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** Rules that determine whether a type is castable from another type. */ +public class FlinkSqlTypeMappingRule implements SqlTypeMappingRule { + private static final FlinkSqlTypeMappingRule INSTANCE; + + private final Map> map; + + private FlinkSqlTypeMappingRule(Map> map) { + this.map = ImmutableMap.copyOf(map); + } + + public static FlinkSqlTypeMappingRule instance() { + return Objects.requireNonNull(FLINK_THREAD_PROVIDERS.get(), "flinkThreadProviders"); + } + + public static FlinkSqlTypeMappingRule instance( + Map> map) { + return new FlinkSqlTypeMappingRule(map); + } + + public Map> getTypeMapping() { + return this.map; + } + + static { + SqlTypeMappingRules.Builder coerceRules = SqlTypeMappingRules.builder(); + coerceRules.addAll(SqlTypeCoercionRule.lenientInstance().getTypeMapping()); + Map> map = + SqlTypeCoercionRule.lenientInstance().getTypeMapping(); + Set rule = new HashSet<>(); + rule.add(SqlTypeName.TINYINT); + rule.add(SqlTypeName.SMALLINT); + rule.add(SqlTypeName.INTEGER); + rule.add(SqlTypeName.BIGINT); + rule.add(SqlTypeName.DECIMAL); + rule.add(SqlTypeName.FLOAT); + rule.add(SqlTypeName.REAL); + rule.add(SqlTypeName.DOUBLE); + rule.add(SqlTypeName.CHAR); + rule.add(SqlTypeName.VARCHAR); + rule.add(SqlTypeName.BOOLEAN); + rule.add(SqlTypeName.TIMESTAMP); + rule.add(SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE); + coerceRules.add(SqlTypeName.FLOAT, rule); + coerceRules.add(SqlTypeName.DOUBLE, rule); + coerceRules.add(SqlTypeName.DECIMAL, rule); + INSTANCE = new FlinkSqlTypeMappingRule(coerceRules.map); + } + + public static final ThreadLocal<@Nullable FlinkSqlTypeMappingRule> FLINK_THREAD_PROVIDERS = + ThreadLocal.withInitial(() -> INSTANCE); +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java index a5f42aaeb1385..14ed16d2e9f77 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java @@ -39,8 +39,8 @@ * *

    *
  1. Should be removed after fixing CALCITE-6342: Lines 100-102 - *
  2. Should be removed after fixing CALCITE-6342: Lines 482-494 - *
  3. Should be removed after fix of FLINK-31350: Lines 561-573. + *
  4. Should be removed after fixing CALCITE-6342: Lines 484-496 + *
  5. Should be removed after fix of FLINK-31350: Lines 563-575. *
*/ public class SqlTypeFactoryImpl extends RelDataTypeFactoryImpl { @@ -452,7 +452,7 @@ private static void assertBasic(SqlTypeName typeName) { if (types.size() > (i + 1)) { RelDataType type1 = types.get(i + 1); if (SqlTypeUtil.isDatetime(type1)) { - resultType = type1; + resultType = leastRestrictiveIntervalDatetimeType(type1, type); return createTypeWithNullability( resultType, nullCount > 0 || nullableCount > 0); } @@ -472,8 +472,10 @@ private static void assertBasic(SqlTypeName typeName) { // datetime +/- interval (or integer) = datetime if (types.size() > (i + 1)) { RelDataType type1 = types.get(i + 1); - if (SqlTypeUtil.isInterval(type1) || SqlTypeUtil.isIntType(type1)) { - resultType = type; + final boolean isInterval1 = SqlTypeUtil.isInterval(type1); + final boolean isInt1 = SqlTypeUtil.isIntType(type1); + if (isInterval1 || isInt1) { + resultType = leastRestrictiveIntervalDatetimeType(type, type1); return createTypeWithNullability( resultType, nullCount > 0 || nullableCount > 0); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java index ccae916bc64e8..2716293cb03be 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java @@ -18,7 +18,6 @@ import org.apache.flink.table.planner.calcite.FlinkSqlCallBinding; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -44,6 +43,7 @@ import org.apache.calcite.runtime.CalciteContextException; import org.apache.calcite.runtime.CalciteException; import org.apache.calcite.runtime.Feature; +import org.apache.calcite.runtime.PairList; import org.apache.calcite.runtime.Resources; import org.apache.calcite.schema.ColumnStrategy; import org.apache.calcite.schema.Table; @@ -160,25 +160,22 @@ import static org.apache.calcite.sql.validate.SqlNonNullableAccessors.getCondition; import static org.apache.calcite.sql.validate.SqlNonNullableAccessors.getTable; import static org.apache.calcite.util.Static.RESOURCE; +import static org.apache.calcite.util.Util.first; /** * Default implementation of {@link SqlValidator}, the class was copied over because of * CALCITE-4554. * - *

Lines 200 ~ 203, Flink improves error message for functions without appropriate arguments in + *

Lines 197 ~ 200, Flink improves error message for functions without appropriate arguments in * handleUnresolvedFunction. * - *

Lines 2000 ~ 2020, Flink improves error message for functions without appropriate arguments in + *

Lines 2012 ~ 2032, Flink improves error message for functions without appropriate arguments in * handleUnresolvedFunction at {@link SqlValidatorImpl#handleUnresolvedFunction}. * - *

Lines 3814 ~ 3818, 6458 ~ 6464 Flink improves Optimize the retrieval of sub-operands in + *

Lines 3840 ~ 3844, 6511 ~ 6517 Flink improves Optimize the retrieval of sub-operands in * SqlCall when using NamedParameters at {@link SqlValidatorImpl#checkRollUp}. * - *

Lines 5196 ~ 5209, Flink enables TIMESTAMP and TIMESTAMP_LTZ for system time period - * specification type at {@link org.apache.calcite.sql.validate.SqlValidatorImpl#validateSnapshot}. - * - *

Lines 5553 ~ 5559, Flink enables TIMESTAMP and TIMESTAMP_LTZ for first orderBy column in - * matchRecognize at {@link SqlValidatorImpl#validateMatchRecognize}. + *

Lines 5246 ~ 5252, FLINK-24352 Add null check for temporal table check on SqlSnapshot. */ public class SqlValidatorImpl implements SqlValidatorWithHints { // ~ Static fields/initializers --------------------------------------------- @@ -284,7 +281,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints { new SqlValidatorImpl.ValidationErrorFunction(); // TypeCoercion instance used for implicit type coercion. - private TypeCoercion typeCoercion; + private final TypeCoercion typeCoercion; // ~ Constructors ----------------------------------------------------------- @@ -323,14 +320,15 @@ protected SqlValidatorImpl( TypeCoercion typeCoercion = config.typeCoercionFactory().create(typeFactory, this); this.typeCoercion = typeCoercion; - if (config.conformance().allowCoercionStringToArray()) { - SqlTypeCoercionRule rules = + if (config.conformance().allowLenientCoercion()) { + final SqlTypeCoercionRule rules = requireNonNull( config.typeCoercionRules() != null ? config.typeCoercionRules() - : SqlTypeCoercionRule.THREAD_PROVIDERS.get()); + : SqlTypeCoercionRule.THREAD_PROVIDERS.get(), + "rules"); - ImmutableSet arrayMapping = + final ImmutableSet arrayMapping = ImmutableSet.builder() .addAll( rules.getTypeMapping() @@ -340,11 +338,11 @@ protected SqlValidatorImpl( .build(); Map> mapping = - new HashMap(rules.getTypeMapping()); + new HashMap<>(rules.getTypeMapping()); mapping.replace(SqlTypeName.ARRAY, arrayMapping); - rules = SqlTypeCoercionRule.instance(mapping); + SqlTypeCoercionRule rules2 = SqlTypeCoercionRule.instance(mapping); - SqlTypeCoercionRule.THREAD_PROVIDERS.set(rules); + SqlTypeCoercionRule.THREAD_PROVIDERS.set(rules2); } else if (config.typeCoercionRules() != null) { SqlTypeCoercionRule.THREAD_PROVIDERS.set(config.typeCoercionRules()); } @@ -388,14 +386,13 @@ public TimeFrameSet getTimeFrameSet() { public SqlNodeList expandStar( SqlNodeList selectList, SqlSelect select, boolean includeSystemVars) { final List list = new ArrayList<>(); - final List> types = new ArrayList<>(); - for (int i = 0; i < selectList.size(); i++) { - final SqlNode selectItem = selectList.get(i); + final PairList types = PairList.of(); + for (SqlNode selectItem : selectList) { final RelDataType originalType = getValidatedNodeTypeIfKnown(selectItem); expandSelectItem( selectItem, select, - Util.first(originalType, unknownType), + first(originalType, unknownType), list, catalogReader.nameMatcher().createSet(), types, @@ -405,7 +402,6 @@ public SqlNodeList expandStar( return new SqlNodeList(list, SqlParserPos.ZERO); } - // implement SqlValidator @Override public void declareCursor(SqlSelect select, SqlValidatorScope parentScope) { cursorSet.add(select); @@ -414,33 +410,30 @@ public void declareCursor(SqlSelect select, SqlValidatorScope parentScope) { // the position of the cursor relative to other cursors in that call FunctionParamInfo funcParamInfo = requireNonNull(functionCallStack.peek(), "functionCall"); Map cursorMap = funcParamInfo.cursorPosToSelectMap; - int numCursors = cursorMap.size(); - cursorMap.put(numCursors, select); + final int cursorCount = cursorMap.size(); + cursorMap.put(cursorCount, select); // create a namespace associated with the result of the select // that is the argument to the cursor constructor; register it // with a scope corresponding to the cursor - SelectScope cursorScope = new SelectScope(parentScope, null, select); + SelectScope cursorScope = new SelectScope(parentScope, getEmptyScope(), select); clauseScopes.put(IdPair.of(select, Clause.CURSOR), cursorScope); final SelectNamespace selectNs = createSelectNamespace(select, select); final String alias = SqlValidatorUtil.alias(select, nextGeneratedId++); registerNamespace(cursorScope, alias, selectNs, false); } - // implement SqlValidator @Override public void pushFunctionCall() { FunctionParamInfo funcInfo = new FunctionParamInfo(); functionCallStack.push(funcInfo); } - // implement SqlValidator @Override public void popFunctionCall() { functionCallStack.pop(); } - // implement SqlValidator @Override public @Nullable String getParentCursor(String columnListParamName) { FunctionParamInfo funcParamInfo = requireNonNull(functionCallStack.peek(), "functionCall"); @@ -466,8 +459,8 @@ private boolean expandSelectItem( RelDataType targetType, List selectItems, Set aliases, - List> fields, - final boolean includeSystemVars) { + PairList fields, + boolean includeSystemVars) { final SelectScope scope = (SelectScope) getWhereScope(select); if (expandStar(selectItems, aliases, fields, includeSystemVars, scope, selectItem)) { return true; @@ -510,12 +503,12 @@ private boolean expandSelectItem( type = requireNonNull(selectScope.nullifyType(stripAs(expanded), type)); } setValidatedNodeType(expanded, type); - fields.add(Pair.of(alias, type)); + fields.add(alias, type); return false; } private static SqlNode expandExprFromJoin( - SqlJoin join, SqlIdentifier identifier, @Nullable SelectScope scope) { + SqlJoin join, SqlIdentifier identifier, SelectScope scope) { if (join.getConditionType() != JoinConditionType.USING) { return identifier; } @@ -534,15 +527,11 @@ private static SqlNode expandExprFromJoin( } assert qualifiedNode.size() == 2; - final SqlNode finalNode = - SqlStdOperatorTable.AS.createCall( - SqlParserPos.ZERO, - SqlStdOperatorTable.COALESCE.createCall( - SqlParserPos.ZERO, - qualifiedNode.get(0), - qualifiedNode.get(1)), - new SqlIdentifier(name, SqlParserPos.ZERO)); - return finalNode; + return SqlStdOperatorTable.AS.createCall( + SqlParserPos.ZERO, + SqlStdOperatorTable.COALESCE.createCall( + SqlParserPos.ZERO, qualifiedNode.get(0), qualifiedNode.get(1)), + new SqlIdentifier(name, SqlParserPos.ZERO)); } } @@ -588,7 +577,7 @@ private List deriveNaturalJoinColumnList(SqlJoin join) { private static SqlNode expandCommonColumn( SqlSelect sqlSelect, SqlNode selectItem, - @Nullable SelectScope scope, + SelectScope scope, SqlValidatorImpl validator) { if (!(selectItem instanceof SqlIdentifier)) { return selectItem; @@ -611,17 +600,13 @@ private static SqlNode expandCommonColumn( } private static void validateQualifiedCommonColumn( - SqlJoin join, - SqlIdentifier identifier, - @Nullable SelectScope scope, - SqlValidatorImpl validator) { + SqlJoin join, SqlIdentifier identifier, SelectScope scope, SqlValidatorImpl validator) { List names = validator.usingNames(join); if (names == null) { // Not USING or NATURAL. return; } - requireNonNull(scope, "scope"); // First we should make sure that the first component is the table name. // Then check whether the qualified identifier contains common column. for (ScopeChild child : scope.children) { @@ -645,7 +630,7 @@ private static void validateQualifiedCommonColumn( private boolean expandStar( List selectItems, Set aliases, - List> fields, + PairList fields, boolean includeSystemVars, SelectScope scope, SqlNode node) { @@ -659,6 +644,11 @@ private boolean expandStar( final SqlParserPos startPosition = identifier.getParserPosition(); switch (identifier.names.size()) { case 1: + SqlNode from = scope.getNode().getFrom(); + if (from == null) { + throw newValidationError(identifier, RESOURCE.selectStarRequiresFrom()); + } + boolean hasDynamicStruct = false; for (ScopeChild child : scope.children) { final int before = fields.size(); @@ -675,8 +665,8 @@ private boolean expandStar( addToSelectList( selectItems, aliases, fields, exp, scope, includeSystemVars); } else { - final SqlNode from = SqlNonNullableAccessors.getNode(child); - final SqlValidatorNamespace fromNs = getNamespaceOrThrow(from, scope); + final SqlNode from2 = SqlNonNullableAccessors.getNode(child); + final SqlValidatorNamespace fromNs = getNamespaceOrThrow(from2, scope); final RelDataType rowType = fromNs.getRowType(); for (RelDataTypeField field : rowType.getFieldList()) { String columnName = field.getName(); @@ -706,9 +696,8 @@ private boolean expandStar( if (!type.isNullable()) { fields.set( i, - Pair.of( - entry.getKey(), - typeFactory.createTypeWithNullability(type, true))); + entry.getKey(), + typeFactory.createTypeWithNullability(type, true)); } } } @@ -716,11 +705,10 @@ private boolean expandStar( // If NATURAL JOIN or USING is present, move key fields to the front of // the list, per standard SQL. Disabled if there are dynamic fields. if (!hasDynamicStruct || Bug.CALCITE_2400_FIXED) { - SqlNode from = - requireNonNull( - scope.getNode().getFrom(), - () -> "getFrom for " + scope.getNode()); - new Permute(from, 0).permute(selectItems, fields); + // If some fields before star identifier, + // we should move offset. + int offset = calculatePermuteOffset(selectItems); + new Permute(from, offset).permute(selectItems, fields); } return true; @@ -767,6 +755,17 @@ private boolean expandStar( } } + private static int calculatePermuteOffset(List selectItems) { + for (int i = 0; i < selectItems.size(); i++) { + SqlNode selectItem = selectItems.get(i); + SqlNode col = SqlUtil.stripAs(selectItem); + if (col.getKind() == SqlKind.IDENTIFIER && selectItem.getKind() != SqlKind.AS) { + return i; + } + } + return 0; + } + private SqlNode maybeCast(SqlNode node, RelDataType currentType, RelDataType desiredType) { return SqlTypeUtil.equalSansNullability(typeFactory, currentType, desiredType) ? node @@ -777,7 +776,7 @@ private SqlNode maybeCast(SqlNode node, RelDataType currentType, RelDataType des private boolean addOrExpandField( List selectItems, Set aliases, - List> fields, + PairList fields, boolean includeSystemVars, SelectScope scope, SqlIdentifier id, @@ -846,7 +845,7 @@ public List lookupHints(SqlNode topNode, SqlParserPos pos) { */ void lookupSelectHints(SqlSelect select, SqlParserPos pos, Collection hintList) { IdInfo info = idPositions.get(pos.toString()); - if ((info == null) || (info.scope == null)) { + if (info == null) { SqlNode fromNode = select.getFrom(); final SqlValidatorScope fromScope = getFromScope(select); lookupFromHints(fromNode, fromScope, pos, hintList); @@ -866,7 +865,7 @@ private void lookupSelectHints( private void lookupFromHints( @Nullable SqlNode node, - @Nullable SqlValidatorScope scope, + SqlValidatorScope scope, SqlParserPos pos, Collection hintList) { if (node == null) { @@ -903,7 +902,7 @@ private void lookupFromHints( private void lookupJoinHints( SqlJoin join, - @Nullable SqlValidatorScope scope, + SqlValidatorScope scope, SqlParserPos pos, Collection hintList) { SqlNode left = join.getLeft(); @@ -1062,8 +1061,7 @@ private SqlNode validateScopedExpression(SqlNode topNode, SqlValidatorScope scop } @Override - public void validateQuery( - SqlNode node, @Nullable SqlValidatorScope scope, RelDataType targetRowType) { + public void validateQuery(SqlNode node, SqlValidatorScope scope, RelDataType targetRowType) { final SqlValidatorNamespace ns = getNamespaceOrThrow(node, scope); if (node.getKind() == SqlKind.TABLESAMPLE) { List operands = ((SqlCall) node).getOperandList(); @@ -1108,7 +1106,7 @@ protected void validateNamespace( } } - @VisibleForTesting + @Override public SqlValidatorScope getEmptyScope() { return new EmptyScope(this); } @@ -1159,8 +1157,8 @@ public SqlValidatorScope getGroupScope(SqlSelect select) { } @Override - public @Nullable SqlValidatorScope getFromScope(SqlSelect select) { - return scopes.get(select); + public SqlValidatorScope getFromScope(SqlSelect select) { + return requireNonNull(scopes.get(select), () -> "no scope for " + select); } @Override @@ -1174,8 +1172,8 @@ public SqlValidatorScope getMatchRecognizeScope(SqlMatchRecognize node) { } @Override - public @Nullable SqlValidatorScope getJoinScope(SqlNode node) { - return scopes.get(stripAs(node)); + public SqlValidatorScope getJoinScope(SqlNode node) { + return requireNonNull(scopes.get(stripAs(node)), () -> "scope for " + node); } @Override @@ -1183,12 +1181,17 @@ public SqlValidatorScope getOverScope(SqlNode node) { return getScopeOrThrow(node); } + @Override + public SqlValidatorScope getWithScope(SqlNode withItem) { + assert withItem.getKind() == SqlKind.WITH_ITEM; + return getScopeOrThrow(withItem); + } + private SqlValidatorScope getScopeOrThrow(SqlNode node) { return requireNonNull(scopes.get(node), () -> "scope for " + node); } - private @Nullable SqlValidatorNamespace getNamespace( - SqlNode node, @Nullable SqlValidatorScope scope) { + private @Nullable SqlValidatorNamespace getNamespace(SqlNode node, SqlValidatorScope scope) { if (node instanceof SqlIdentifier && scope instanceof DelegatingScope) { final SqlIdentifier id = (SqlIdentifier) node; final DelegatingScope idScope = (DelegatingScope) ((DelegatingScope) scope).getParent(); @@ -1282,7 +1285,7 @@ SqlValidatorNamespace getNamespaceOrThrow(SqlNode node) { * @see #getNamespace(SqlNode) */ @API(since = "1.27", status = API.Status.INTERNAL) - SqlValidatorNamespace getNamespaceOrThrow(SqlNode node, @Nullable SqlValidatorScope scope) { + SqlValidatorNamespace getNamespaceOrThrow(SqlNode node, SqlValidatorScope scope) { return requireNonNull( getNamespace(node, scope), () -> "namespace for " + node + ", scope " + scope); } @@ -2151,7 +2154,7 @@ protected void addToSelectList( if (!Objects.equals(alias, uniqueAlias)) { exp = SqlValidatorUtil.addAlias(exp, uniqueAlias); } - fieldList.add(Pair.of(uniqueAlias, deriveType(scope, exp))); + ((PairList) fieldList).add(uniqueAlias, deriveType(scope, exp)); list.add(exp); } @@ -2285,7 +2288,7 @@ protected void registerNamespace( /** * Registers scopes and namespaces implied a relational expression in the FROM clause. * - *

{@code parentScope} and {@code usingScope} are often the same. They differ when the + *

{@code parentScope0} and {@code usingScope} are often the same. They differ when the * namespace are not visible within the parent. (Example needed.) * *

Likewise, {@code enclosingNode} and {@code node} are often the same. {@code enclosingNode} @@ -2293,7 +2296,7 @@ protected void registerNamespace( * AS alias) or a table sample clause are stripped away to get {@code node}. Both are * recorded in the namespace. * - * @param parentScope Parent scope which this scope turns to in order to resolve objects + * @param parentScope0 Parent scope that this scope turns to in order to resolve objects * @param usingScope Scope whose child list this scope should add itself to * @param register Whether to register this scope as a child of {@code usingScope} * @param node Node which namespace is based on @@ -2308,7 +2311,7 @@ protected void registerNamespace( * @return registered node, usually the same as {@code node} */ private SqlNode registerFrom( - SqlValidatorScope parentScope, + SqlValidatorScope parentScope0, SqlValidatorScope usingScope, boolean register, final SqlNode node, @@ -2365,19 +2368,22 @@ private SqlNode registerFrom( } } + final SqlValidatorScope parentScope; if (lateral) { SqlValidatorScope s = usingScope; while (s instanceof JoinScope) { s = ((JoinScope) s).getUsingScope(); } final SqlNode node2 = s != null ? s.getNode() : node; - final TableScope tableScope = new TableScope(parentScope, node2); + final TableScope tableScope = new TableScope(parentScope0, node2); if (usingScope instanceof ListScope) { for (ScopeChild child : ((ListScope) usingScope).children) { tableScope.addChild(child.namespace, child.name, child.nullable); } } parentScope = tableScope; + } else { + parentScope = parentScope0; } SqlCall call; @@ -2394,7 +2400,8 @@ private SqlNode registerFrom( final boolean needAliasNamespace = call.operandCount() > 2 || expr.getKind() == SqlKind.VALUES - || expr.getKind() == SqlKind.UNNEST; + || expr.getKind() == SqlKind.UNNEST + || expr.getKind() == SqlKind.COLLECTION_TABLE; newExpr = registerFrom( parentScope, @@ -2520,6 +2527,8 @@ private SqlNode registerFrom( if (newRight != right) { join.setRight(newRight); } + scopes.putIfAbsent(stripAs(join.getRight()), parentScope); + scopes.putIfAbsent(stripAs(join.getLeft()), parentScope); registerSubQueries(joinScope, join.getCondition()); final JoinNamespace joinNamespace = new JoinNamespace(this, join); registerNamespace(null, null, joinNamespace, forceNullable); @@ -2785,8 +2794,7 @@ private void registerQuery( final SqlSelect select = (SqlSelect) node; final SelectNamespace selectNs = createSelectNamespace(select, enclosingNode); registerNamespace(usingScope, alias, selectNs, forceNullable); - final SqlValidatorScope windowParentScope = - (usingScope != null) ? usingScope : parentScope; + final SqlValidatorScope windowParentScope = first(usingScope, parentScope); SelectScope selectScope = new SelectScope(parentScope, windowParentScope, select); scopes.put(select, selectScope); @@ -3046,6 +3054,7 @@ private void registerWith( boolean checkUpdate) { final WithNamespace withNamespace = new WithNamespace(this, with, enclosingNode); registerNamespace(usingScope, alias, withNamespace, forceNullable); + scopes.put(with, parentScope); SqlValidatorScope scope = parentScope; for (SqlNode withItem_ : with.withList) { @@ -3355,6 +3364,7 @@ public TimeFrame validateTimeFrame(SqlIntervalQualifier qualifier) { * @param scope Scope */ protected void validateFrom(SqlNode node, RelDataType targetRowType, SqlValidatorScope scope) { + requireNonNull(scope, "scope"); requireNonNull(targetRowType, "targetRowType"); switch (node.getKind()) { case AS: @@ -3688,8 +3698,7 @@ protected void validateSelect(SqlSelect select, RelDataType targetRowType) { } // Make sure that items in FROM clause have distinct aliases. - final SelectScope fromScope = - (SelectScope) requireNonNull(getFromScope(select), () -> "fromScope for " + select); + final SelectScope fromScope = (SelectScope) getFromScope(select); List<@Nullable String> names = fromScope.getChildNames(); if (!catalogReader.nameMatcher().isCaseSensitive()) { //noinspection RedundantTypeArguments @@ -3822,6 +3831,11 @@ private void checkRollUp( // we stripped the field access. Recurse to this method, the DOT's operand // can be another SqlCall, or an SqlIdentifier. checkRollUp(grandParent, parent, stripDot, scope, contextClause); + } else if (stripDot.getKind() == SqlKind.CONVERT + || stripDot.getKind() == SqlKind.TRANSLATE) { + // only need to check operand[0] for CONVERT or TRANSLATE + SqlNode child = ((SqlCall) stripDot).getOperandList().get(0); + checkRollUp(parent, current, child, scope, contextClause); } else { // ----- FLINK MODIFICATION BEGIN ----- SqlCall call = (SqlCall) stripDot; @@ -4122,8 +4136,7 @@ protected void validateWindowClause(SqlSelect select) { return; } - final SelectScope windowScope = - (SelectScope) requireNonNull(getFromScope(select), () -> "fromScope for " + select); + final SelectScope windowScope = (SelectScope) getFromScope(select); // 1. ensure window names are simple // 2. ensure they are unique within this scope @@ -4249,12 +4262,6 @@ public void validateSequenceValue(SqlValidatorScope scope, SqlIdentifier id) { throw newValidationError(id, RESOURCE.notASequence(id.toString())); } - @Override - public @Nullable SqlValidatorScope getWithScope(SqlNode withItem) { - assert withItem.getKind() == SqlKind.WITH_ITEM; - return scopes.get(withItem); - } - @Override public TypeCoercion getTypeCoercion() { assert config.typeCoercionEnabled(); @@ -4539,7 +4546,7 @@ protected RelDataType validateSelectList( final SqlValidatorScope selectScope = getSelectScope(select); final List expandedSelectItems = new ArrayList<>(); final Set aliases = new HashSet<>(); - final List> fieldList = new ArrayList<>(); + final PairList fieldList = PairList.of(); for (SqlNode selectItem : selectItems) { if (selectItem instanceof SqlSelect) { @@ -4635,7 +4642,7 @@ private void handleScalarSubQuery( SqlSelect selectItem, List expandedSelectItems, Set aliasList, - List> fieldList) { + PairList fieldList) { // A scalar sub-query only has one output column. if (1 != SqlNonNullableAccessors.getSelectList(selectItem).size()) { throw newValidationError(selectItem, RESOURCE.onlyScalarSubQueryAllowed()); @@ -4660,7 +4667,7 @@ private void handleScalarSubQuery( RelDataType nodeType = rec.getFieldList().get(0).getType(); nodeType = typeFactory.createTypeWithNullability(nodeType, true); - fieldList.add(Pair.of(alias, nodeType)); + fieldList.add(alias, nodeType); } /** @@ -4679,13 +4686,10 @@ protected RelDataType createTargetRowType( return baseRowType; } List targetFields = baseRowType.getFieldList(); - final List> fields = new ArrayList<>(); + final PairList fields = PairList.of(); if (append) { for (RelDataTypeField targetField : targetFields) { - fields.add( - Pair.of( - SqlUtil.deriveAliasFromOrdinal(fields.size()), - targetField.getType())); + fields.add(SqlUtil.deriveAliasFromOrdinal(fields.size()), targetField.getType()); } } final Set assignedFields = new HashSet<>(); @@ -4733,6 +4737,7 @@ public void validateInsert(SqlInsert insert) { validateSelect(sqlSelect, targetRowType); } else { final SqlValidatorScope scope = scopes.get(source); + requireNonNull(scope, "scope"); validateQuery(source, scope, targetRowType); } @@ -5232,14 +5237,13 @@ private void validateSnapshot( SqlSnapshot snapshot = (SqlSnapshot) node; SqlNode period = snapshot.getPeriod(); RelDataType dataType = deriveType(requireNonNull(scope, "scope"), period); - // ----- FLINK MODIFICATION BEGIN ----- - if (!(dataType.getSqlTypeName() == SqlTypeName.TIMESTAMP - || dataType.getSqlTypeName() == SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE)) { + if (!SqlTypeUtil.isTimestamp(dataType)) { throw newValidationError( period, Static.RESOURCE.illegalExpressionForTemporal( dataType.getSqlTypeName().getName())); } + // ----- FLINK MODIFICATION BEGIN ----- if (ns instanceof IdentifierNamespace && ns.resolve() instanceof WithItemNamespace) { // If the snapshot is used over a CTE, then we don't have a concrete underlying // table to operate on. This will be rechecked later in the planner rules. @@ -5587,13 +5591,9 @@ public void validateMatchRecognize(SqlCall call) { (SqlIdentifier) requireNonNull(firstOrderByColumn, "firstOrderByColumn"); } RelDataType firstOrderByColumnType = deriveType(scope, identifier); - // ----- FLINK MODIFICATION BEGIN ----- - if (!(firstOrderByColumnType.getSqlTypeName() == SqlTypeName.TIMESTAMP - || firstOrderByColumnType.getSqlTypeName() - == SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE)) { + if (!SqlTypeUtil.isTimestamp(firstOrderByColumnType)) { throw newValidationError(interval, RESOURCE.firstColumnOfOrderByMustBeTimestamp()); } - // ----- FLINK MODIFICATION END ----- SqlNode expand = expand(interval, scope); RelDataType type = deriveType(scope, expand); @@ -5632,13 +5632,14 @@ public void validateMatchRecognize(SqlCall call) { } } - List> measureColumns = + PairList measureColumns = validateMeasure(matchRecognize, scope, allRows); - for (Map.Entry c : measureColumns) { - if (!typeBuilder.nameExists(c.getKey())) { - typeBuilder.add(c.getKey(), c.getValue()); - } - } + measureColumns.forEach( + (name, type) -> { + if (!typeBuilder.nameExists(name)) { + typeBuilder.add(name, type); + } + }); final RelDataType rowType = typeBuilder.build(); if (matchRecognize.getMeasureList().size() == 0) { @@ -5648,12 +5649,12 @@ public void validateMatchRecognize(SqlCall call) { } } - private List> validateMeasure( + private PairList validateMeasure( SqlMatchRecognize mr, MatchRecognizeScope scope, boolean allRows) { final List aliases = new ArrayList<>(); final List sqlNodes = new ArrayList<>(); final SqlNodeList measures = mr.getMeasureList(); - final List> fields = new ArrayList<>(); + final PairList fields = PairList.of(); for (SqlNode measure : measures) { assert measure instanceof SqlCall; @@ -5668,7 +5669,7 @@ private List> validateMeasure( final RelDataType type = deriveType(scope, expand); setValidatedNodeType(measure, type); - fields.add(Pair.of(alias, type)); + fields.add(alias, type); sqlNodes.add( SqlStdOperatorTable.AS.createCall( SqlParserPos.ZERO, @@ -5757,8 +5758,7 @@ private static String alias(SqlNode item) { } public void validatePivot(SqlPivot pivot) { - final PivotScope scope = - requireNonNull((PivotScope) getJoinScope(pivot), () -> "joinScope for " + pivot); + final PivotScope scope = (PivotScope) getJoinScope(pivot); final PivotNamespace ns = getNamespaceOrThrow(pivot).unwrap(PivotNamespace.class); assert ns.rowType == null; @@ -5774,12 +5774,12 @@ public void validatePivot(SqlPivot pivot) { // an aggregate or as an axis. // Aggregates, e.g. "PIVOT (sum(x) AS sum_x, count(*) AS c)" - final List> aggNames = new ArrayList<>(); + final PairList<@Nullable String, RelDataType> aggNames = PairList.of(); pivot.forEachAgg( (alias, call) -> { call.validate(this, scope); final RelDataType type = deriveType(scope, call); - aggNames.add(Pair.of(alias, type)); + aggNames.add(alias, type); if (!(call instanceof SqlCall) || !(((SqlCall) call).getOperator() instanceof SqlAggFunction)) { throw newValidationError(call, RESOURCE.pivotAggMalformed()); @@ -5837,8 +5837,7 @@ public void validatePivot(SqlPivot pivot) { subNode)), true); }); - Pair.forEach( - aggNames, + aggNames.forEach( (aggAlias, aggType) -> typeBuilder.add( aggAlias == null ? alias : alias + "_" + aggAlias, @@ -5850,8 +5849,7 @@ public void validatePivot(SqlPivot pivot) { } public void validateUnpivot(SqlUnpivot unpivot) { - final UnpivotScope scope = - (UnpivotScope) requireNonNull(getJoinScope(unpivot), () -> "scope for " + unpivot); + final UnpivotScope scope = (UnpivotScope) getJoinScope(unpivot); final UnpivotNamespace ns = getNamespaceOrThrow(unpivot).unwrap(UnpivotNamespace.class); assert ns.rowType == null; @@ -5903,7 +5901,7 @@ public void validateUnpivot(SqlUnpivot unpivot) { columnNames.addAll(unusedColumnNames); // Gather the name and type of each measure. - final List> measureNameTypes = new ArrayList<>(); + final PairList measureNameTypes = PairList.of(); Ord.forEach( unpivot.measureList, (measure, i) -> { @@ -5928,7 +5926,7 @@ public void validateUnpivot(SqlUnpivot unpivot) { if (!columnNames.add(measureName)) { throw newValidationError(measure, RESOURCE.unpivotDuplicate(measureName)); } - measureNameTypes.add(Pair.of(measureName, type)); + measureNameTypes.add(measureName, type); }); // Gather the name and type of each axis. @@ -5942,7 +5940,7 @@ public void validateUnpivot(SqlUnpivot unpivot) { // The type of 'job' is derived as the least restrictive type of the values // ('CLERK', 'ANALYST'), namely VARCHAR(7). The derived type of 'deptno' is // the type of values (10, 20), namely INTEGER. - final List> axisNameTypes = new ArrayList<>(); + final PairList axisNameTypes = PairList.of(); Ord.forEach( unpivot.axisList, (axis, i) -> { @@ -5966,7 +5964,7 @@ public void validateUnpivot(SqlUnpivot unpivot) { if (!columnNames.add(axisName)) { throw newValidationError(axis, RESOURCE.unpivotDuplicate(axisName)); } - axisNameTypes.add(Pair.of(axisName, type)); + axisNameTypes.add(axisName, type); }); // Columns that have been seen as arguments to aggregates or as axes @@ -6081,26 +6079,39 @@ public void validateAggregateParams( throw new AssertionError(op); } + // Because there are two forms of the PERCENTILE_CONT/PERCENTILE_DISC functions, + // they are distinguished by their operand count and then validated accordingly. + // For example, the standard single operand form requires group order while the + // 2-operand form allows for null treatment and requires an OVER() clause. if (op.isPercentile()) { - assert op.requiresGroupOrder() == Optionality.MANDATORY; - assert orderList != null; - - // Validate that percentile function have a single ORDER BY expression - if (orderList.size() != 1) { - throw newValidationError(orderList, RESOURCE.orderByRequiresOneKey(op.getName())); - } - - // Validate that the ORDER BY field is of NUMERIC type - SqlNode node = orderList.get(0); - assert node != null; - - final RelDataType type = deriveType(scope, node); - final @Nullable SqlTypeFamily family = type.getSqlTypeName().getFamily(); - if (family == null || family.allowableDifferenceTypes().isEmpty()) { - throw newValidationError( - orderList, - RESOURCE.unsupportedTypeInOrderBy( - type.getSqlTypeName().getName(), op.getName())); + switch (aggCall.operandCount()) { + case 1: + assert op.requiresGroupOrder() == Optionality.MANDATORY; + assert orderList != null; + // Validate that percentile function have a single ORDER BY expression + if (orderList.size() != 1) { + throw newValidationError( + orderList, RESOURCE.orderByRequiresOneKey(op.getName())); + } + // Validate that the ORDER BY field is of NUMERIC type + SqlNode node = orderList.get(0); + assert node != null; + final RelDataType type = deriveType(scope, node); + final @Nullable SqlTypeFamily family = type.getSqlTypeName().getFamily(); + if (family == null || family.allowableDifferenceTypes().isEmpty()) { + throw newValidationError( + orderList, + RESOURCE.unsupportedTypeInOrderBy( + type.getSqlTypeName().getName(), op.getName())); + } + break; + case 2: + assert op.allowsNullTreatment(); + assert op.requiresOver(); + assert op.requiresGroupOrder() == Optionality.FORBIDDEN; + break; + default: + throw newValidationError(aggCall, RESOURCE.percentileFunctionsArgumentLimit()); } } } @@ -6834,7 +6845,7 @@ static class ExtendedExpander extends Expander { final boolean replaceAliases = clause.shouldReplaceAliases(validator.config); if (!replaceAliases) { - final SelectScope scope = validator.getRawSelectScope(select); + final SelectScope scope = validator.getRawSelectScopeNonNull(select); SqlNode node = expandCommonColumn(select, id, scope, validator); if (node != id) { return node; @@ -7291,8 +7302,10 @@ private class Permute { final List sources; final RelDataType rowType; final boolean trivial; + final int offset; Permute(SqlNode from, int offset) { + this.offset = offset; switch (from.getKind()) { case JOIN: final SqlJoin join = (SqlJoin) from; @@ -7357,16 +7370,17 @@ private RelDataTypeField field(String name) { } /** Moves fields according to the permutation. */ - public void permute( - List selectItems, List> fields) { + void permute(List selectItems, PairList fields) { if (trivial) { return; } final List oldSelectItems = ImmutableList.copyOf(selectItems); selectItems.clear(); - final List> oldFields = ImmutableList.copyOf(fields); + selectItems.addAll(oldSelectItems.subList(0, offset)); + final PairList oldFields = fields.immutable(); fields.clear(); + fields.addAll(oldFields.subList(0, offset)); for (ImmutableIntList source : sources) { final int p0 = source.get(0); Map.Entry field = oldFields.get(p0); @@ -7399,7 +7413,7 @@ public void permute( new SqlIdentifier(name, SqlParserPos.ZERO)); type = typeFactory.createTypeWithNullability(type2, nullable); } - fields.add(Pair.of(name, type)); + fields.add(name, type); selectItems.add(selectItem); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/AggConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/AggConverter.java new file mode 100644 index 0000000000000..849d222f6a845 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/AggConverter.java @@ -0,0 +1,623 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to you under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.calcite.sql2rel; + +import org.apache.flink.table.planner.calcite.FlinkSqlCallBinding; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.calcite.linq4j.Ord; +import org.apache.calcite.rel.RelCollation; +import org.apache.calcite.rel.RelCollations; +import org.apache.calcite.rel.RelFieldCollation; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.runtime.PairList; +import org.apache.calcite.sql.SqlAggFunction; +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlDataTypeSpec; +import org.apache.calcite.sql.SqlDynamicParam; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlIntervalQualifier; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlLiteral; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlSelectKeyword; +import org.apache.calcite.sql.SqlUtil; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.sql.util.SqlVisitor; +import org.apache.calcite.sql.validate.AggregatingSelectScope; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.sql.validate.SqlValidatorUtil; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.Litmus; +import org.apache.calcite.util.Util; +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.calcite.linq4j.Nullness.castNonNull; + +/** + * FLINK modifications are at lines + * + *

    + *
  1. Added in FLINK-34057, FLINK-34058, FLINK-34312: Lines 452 ~ 469 + *
+ */ +class AggConverter implements SqlVisitor { + private final SqlToRelConverter.Blackboard bb; + private final Map nameMap; + + /** The group-by expressions, in {@link SqlNode} format. */ + final SqlNodeList groupExprs = new SqlNodeList(SqlParserPos.ZERO); + + /** The auxiliary group-by expressions. */ + private final Map> auxiliaryGroupExprs = new HashMap<>(); + + /** Measure expressions, in {@link SqlNode} format. */ + private final SqlNodeList measureExprs = new SqlNodeList(SqlParserPos.ZERO); + + /** + * Input expressions for the group columns and aggregates, in {@link RexNode} format. The first + * elements of the list correspond to the elements in {@link #groupExprs}; the remaining + * elements are for aggregates. The right field of each pair is the name of the expression, + * where the expressions are simple mappings to input fields. + */ + final PairList convertedInputExprs = PairList.of(); + + /** + * Expressions to be evaluated as rows are being placed into the aggregate's hash table. This is + * when group functions such as TUMBLE cause rows to be expanded. + */ + final List aggCalls = new ArrayList<>(); + + private final Map aggMapping = new HashMap<>(); + private final Map aggCallMapping = new HashMap<>(); + private final SqlValidator validator; + private final AggregatingSelectScope scope; + + /** Whether we are directly inside a windowed aggregate. */ + boolean inOver = false; + + /** Creates an AggConverter. */ + private AggConverter(SqlToRelConverter.Blackboard bb, ImmutableMap nameMap) { + this(bb, nameMap, null, null); + } + + private AggConverter( + SqlToRelConverter.Blackboard bb, + ImmutableMap nameMap, + SqlValidator validator, + AggregatingSelectScope scope) { + this.bb = bb; + this.nameMap = nameMap; + this.validator = validator; + this.scope = scope; + } + + /** + * Creates an AggConverter for a pivot query. + * + * @param bb Blackboard + */ + static AggConverter create(SqlToRelConverter.Blackboard bb) { + return new AggConverter(bb, ImmutableMap.of()); + } + + /** + * Creates an AggConverter. + * + *

The {@code aggregatingSelectScope} parameter provides enough context to name aggregate + * calls which are top-level select list items. + * + * @param bb Blackboard + * @param scope Scope of a SELECT that has a GROUP BY + */ + static AggConverter create( + SqlToRelConverter.Blackboard bb, AggregatingSelectScope scope, SqlValidator validator) { + // Collect all expressions used in the select list so that aggregate + // calls can be named correctly. + final Map nameMap = new HashMap<>(); + Ord.forEach( + scope.getNode().getSelectList(), + (selectItem, i) -> { + final String name; + if (SqlUtil.isCallTo(selectItem, SqlStdOperatorTable.AS)) { + final SqlCall call = (SqlCall) selectItem; + selectItem = call.operand(0); + name = call.operand(1).toString(); + } else { + name = SqlValidatorUtil.alias(selectItem, i); + } + nameMap.put(selectItem.toString(), name); + }); + + final AggregatingSelectScope.Resolved resolved = scope.resolved.get(); + return new AggConverter(bb, ImmutableMap.copyOf(nameMap), validator, scope) { + @Override + AggregatingSelectScope.Resolved getResolved() { + return resolved; + } + }; + } + + int addGroupExpr(SqlNode expr) { + int ref = lookupGroupExpr(expr); + if (ref >= 0) { + return ref; + } + final int index = groupExprs.size(); + groupExprs.add(expr); + String name = nameMap.get(expr.toString()); + RexNode convExpr = bb.convertExpression(expr); + addExpr(convExpr, name); + + if (expr instanceof SqlCall) { + SqlCall call = (SqlCall) expr; + SqlStdOperatorTable.convertGroupToAuxiliaryCalls( + call, (node, converter) -> addAuxiliaryGroupExpr(node, index, converter)); + } + + return index; + } + + void addAuxiliaryGroupExpr(SqlNode node, int index, AuxiliaryConverter converter) { + for (SqlNode node2 : auxiliaryGroupExprs.keySet()) { + if (node2.equalsDeep(node, Litmus.IGNORE)) { + return; + } + } + auxiliaryGroupExprs.put(node, Ord.of(index, converter)); + } + + boolean addMeasureExpr(SqlNode expr) { + if (isMeasureExpr(expr)) { + return false; // already present + } + measureExprs.add(expr); + String name = nameMap.get(expr.toString()); + RexNode convExpr = bb.convertExpression(expr); + addExpr(convExpr, name); + return true; + } + + /** + * Adds an expression, deducing an appropriate name if possible. + * + * @param expr Expression + * @param name Suggested name + */ + private void addExpr(RexNode expr, @Nullable String name) { + if (name == null && expr instanceof RexInputRef) { + final int i = ((RexInputRef) expr).getIndex(); + name = bb.root().getRowType().getFieldList().get(i).getName(); + } + if (convertedInputExprs.rightList().contains(name)) { + // In case like 'SELECT ... GROUP BY x, y, x', don't add + // name 'x' twice. + name = null; + } + convertedInputExprs.add(expr, name); + } + + @Override + public Void visit(SqlIdentifier id) { + return null; + } + + @Override + public Void visit(SqlNodeList nodeList) { + nodeList.forEach(this::visitNode); + return null; + } + + @Override + public Void visit(SqlLiteral lit) { + return null; + } + + @Override + public Void visit(SqlDataTypeSpec type) { + return null; + } + + @Override + public Void visit(SqlDynamicParam param) { + return null; + } + + @Override + public Void visit(SqlIntervalQualifier intervalQualifier) { + return null; + } + + @Override + public Void visit(SqlCall call) { + switch (call.getKind()) { + case FILTER: + case IGNORE_NULLS: + case RESPECT_NULLS: + case WITHIN_DISTINCT: + case WITHIN_GROUP: + translateAgg(call); + return null; + case SELECT: + // rchen 2006-10-17: + // for now do not detect aggregates in sub-queries. + return null; + default: + break; + } + final boolean prevInOver = inOver; + // Ignore window aggregates and ranking functions (associated with OVER + // operator). However, do not ignore nested window aggregates. + if (call.getOperator().getKind() == SqlKind.OVER) { + // Track aggregate nesting levels only within an OVER operator. + List operandList = call.getOperandList(); + assert operandList.size() == 2; + + // Ignore the top level window aggregates and ranking functions + // positioned as the first operand of a OVER operator + inOver = true; + operandList.get(0).accept(this); + + // Normal translation for the second operand of a OVER operator + inOver = false; + operandList.get(1).accept(this); + return null; + } + + // Do not translate the top level window aggregate. Only do so for + // nested aggregates, if present + if (call.getOperator().isAggregator()) { + if (inOver) { + // Add the parent aggregate level before visiting its children + inOver = false; + } else { + // We're beyond the one ignored level + translateAgg(call); + return null; + } + } + for (SqlNode operand : call.getOperandList()) { + // Operands are occasionally null, e.g. switched CASE arg 0. + if (operand != null) { + operand.accept(this); + } + } + // Remove the parent aggregate level after visiting its children + inOver = prevInOver; + return null; + } + + private void translateAgg(SqlCall call) { + translateAgg(call, null, null, null, false, call); + } + + private void translateAgg( + SqlCall call, + @Nullable SqlNode filter, + @Nullable SqlNodeList distinctList, + @Nullable SqlNodeList orderList, + boolean ignoreNulls, + SqlCall outerCall) { + assert bb.agg == this; + final RexBuilder rexBuilder = bb.getRexBuilder(); + final List operands = call.getOperandList(); + final SqlParserPos pos = call.getParserPosition(); + final SqlCall call2; + final List operands2; + switch (call.getKind()) { + case FILTER: + assert filter == null; + translateAgg( + call.operand(0), + call.operand(1), + distinctList, + orderList, + ignoreNulls, + outerCall); + return; + case WITHIN_DISTINCT: + assert orderList == null; + translateAgg( + call.operand(0), + filter, + call.operand(1), + orderList, + ignoreNulls, + outerCall); + return; + case WITHIN_GROUP: + assert orderList == null; + translateAgg( + call.operand(0), + filter, + distinctList, + call.operand(1), + ignoreNulls, + outerCall); + return; + case IGNORE_NULLS: + ignoreNulls = true; + // fall through + case RESPECT_NULLS: + translateAgg( + call.operand(0), filter, distinctList, orderList, ignoreNulls, outerCall); + return; + + case COUNTIF: + // COUNTIF(b) ==> COUNT(*) FILTER (WHERE b) + // COUNTIF(b) FILTER (WHERE b2) ==> COUNT(*) FILTER (WHERE b2 AND b) + call2 = SqlStdOperatorTable.COUNT.createCall(pos, SqlIdentifier.star(pos)); + final SqlNode filter2 = SqlUtil.andExpressions(filter, call.operand(0)); + translateAgg(call2, filter2, distinctList, orderList, ignoreNulls, outerCall); + return; + + case STRING_AGG: + // Translate "STRING_AGG(s, sep ORDER BY x, y)" + // as if it were "LISTAGG(s, sep) WITHIN GROUP (ORDER BY x, y)"; + // and "STRING_AGG(s, sep)" as "LISTAGG(s, sep)". + if (!operands.isEmpty() && Util.last(operands) instanceof SqlNodeList) { + orderList = (SqlNodeList) Util.last(operands); + operands2 = Util.skipLast(operands); + } else { + operands2 = operands; + } + call2 = + SqlStdOperatorTable.LISTAGG.createCall( + call.getFunctionQuantifier(), pos, operands2); + translateAgg(call2, filter, distinctList, orderList, ignoreNulls, outerCall); + return; + + case GROUP_CONCAT: + // Translate "GROUP_CONCAT(s ORDER BY x, y SEPARATOR ',')" + // as if it were "LISTAGG(s, ',') WITHIN GROUP (ORDER BY x, y)". + // To do this, build a list of operands without ORDER BY with with sep. + operands2 = new ArrayList<>(operands); + final SqlNode separator; + if (!operands2.isEmpty() && Util.last(operands2).getKind() == SqlKind.SEPARATOR) { + final SqlCall sepCall = (SqlCall) operands2.remove(operands.size() - 1); + separator = sepCall.operand(0); + } else { + separator = null; + } + + if (!operands2.isEmpty() && Util.last(operands2) instanceof SqlNodeList) { + orderList = (SqlNodeList) operands2.remove(operands2.size() - 1); + } + + if (separator != null) { + operands2.add(separator); + } + + call2 = + SqlStdOperatorTable.LISTAGG.createCall( + call.getFunctionQuantifier(), pos, operands2); + translateAgg(call2, filter, distinctList, orderList, ignoreNulls, outerCall); + return; + + case ARRAY_AGG: + case ARRAY_CONCAT_AGG: + // Translate "ARRAY_AGG(s ORDER BY x, y)" + // as if it were "ARRAY_AGG(s) WITHIN GROUP (ORDER BY x, y)"; + // similarly "ARRAY_CONCAT_AGG". + if (!operands.isEmpty() && Util.last(operands) instanceof SqlNodeList) { + orderList = (SqlNodeList) Util.last(operands); + call2 = + call.getOperator() + .createCall( + call.getFunctionQuantifier(), + pos, + Util.skipLast(operands)); + translateAgg(call2, filter, distinctList, orderList, ignoreNulls, outerCall); + return; + } + // "ARRAY_AGG" and "ARRAY_CONCAT_AGG" without "ORDER BY" + // are handled normally; fall through. + + default: + break; + } + final List args = new ArrayList<>(); + int filterArg = -1; + final ImmutableBitSet distinctKeys; + try { + // switch out of agg mode + bb.agg = null; + // ----- FLINK MODIFICATION BEGIN ----- + FlinkSqlCallBinding binding = new FlinkSqlCallBinding(validator, scope, call); + List sqlNodes = binding.operands(); + for (int i = 0; i < sqlNodes.size(); i++) { + SqlNode operand = sqlNodes.get(i); + // special case for COUNT(*): delete the * + if (operand instanceof SqlIdentifier) { + SqlIdentifier id = (SqlIdentifier) operand; + if (id.isStar()) { + assert call.operandCount() == 1; + assert args.isEmpty(); + break; + } + } + RexNode convertedExpr = bb.convertExpression(operand); + args.add(lookupOrCreateGroupExpr(convertedExpr)); + } + // ----- FLINK MODIFICATION END ----- + + if (filter != null) { + RexNode convertedExpr = bb.convertExpression(filter); + if (convertedExpr.getType().isNullable()) { + convertedExpr = rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, convertedExpr); + } + filterArg = lookupOrCreateGroupExpr(convertedExpr); + } + + if (distinctList == null) { + distinctKeys = null; + } else { + final ImmutableBitSet.Builder distinctBuilder = ImmutableBitSet.builder(); + for (SqlNode distinct : distinctList) { + RexNode e = bb.convertExpression(distinct); + distinctBuilder.set(lookupOrCreateGroupExpr(e)); + } + distinctKeys = distinctBuilder.build(); + } + } finally { + // switch back into agg mode + bb.agg = this; + } + + SqlAggFunction aggFunction = (SqlAggFunction) call.getOperator(); + final RelDataType type = bb.getValidator().deriveType(bb.scope, call); + boolean distinct = false; + SqlLiteral quantifier = call.getFunctionQuantifier(); + if ((null != quantifier) && (quantifier.getValue() == SqlSelectKeyword.DISTINCT)) { + distinct = true; + } + boolean approximate = false; + if (aggFunction == SqlStdOperatorTable.APPROX_COUNT_DISTINCT) { + aggFunction = SqlStdOperatorTable.COUNT; + distinct = true; + approximate = true; + } + final RelCollation collation; + if (orderList == null || orderList.size() == 0) { + collation = RelCollations.EMPTY; + } else { + try { + // switch out of agg mode + bb.agg = null; + collation = + RelCollations.of( + orderList.stream() + .map( + order -> + bb.convertSortExpression( + order, + RelFieldCollation.Direction + .ASCENDING, + RelFieldCollation.NullDirection + .UNSPECIFIED, + this::sortToFieldCollation)) + .collect(Collectors.toList())); + } finally { + // switch back into agg mode + bb.agg = this; + } + } + final AggregateCall aggCall = + AggregateCall.create( + aggFunction, + distinct, + approximate, + ignoreNulls, + ImmutableList.of(), + args, + filterArg, + distinctKeys, + collation, + type, + nameMap.get(outerCall.toString())); + RexNode rex = + rexBuilder.addAggCall( + aggCall, + groupExprs.size(), + aggCalls, + aggCallMapping, + i -> convertedInputExprs.leftList().get(i).getType().isNullable()); + aggMapping.put(outerCall, rex); + } + + private RelFieldCollation sortToFieldCollation( + SqlNode expr, + RelFieldCollation.Direction direction, + RelFieldCollation.NullDirection nullDirection) { + final RexNode node = bb.convertExpression(expr); + final int fieldIndex = lookupOrCreateGroupExpr(node); + if (nullDirection == RelFieldCollation.NullDirection.UNSPECIFIED) { + nullDirection = direction.defaultNullDirection(); + } + return new RelFieldCollation(fieldIndex, direction, nullDirection); + } + + private int lookupOrCreateGroupExpr(RexNode expr) { + int index = 0; + for (RexNode convertedInputExpr : convertedInputExprs.leftList()) { + if (expr.equals(convertedInputExpr)) { + return index; + } + ++index; + } + + // not found -- add it + addExpr(expr, null); + return index; + } + + /** + * If an expression is structurally identical to one of the group-by expressions, returns a + * reference to the expression, otherwise returns null. + */ + int lookupGroupExpr(SqlNode expr) { + return SqlUtil.indexOfDeep(groupExprs, expr, Litmus.IGNORE); + } + + boolean isMeasureExpr(SqlNode expr) { + return SqlUtil.indexOfDeep(measureExprs, expr, Litmus.IGNORE) >= 0; + } + + @Nullable RexNode lookupMeasure(SqlNode expr) { + return aggMapping.get(expr); + } + + @Nullable RexNode lookupAggregates(SqlCall call) { + // assert call.getOperator().isAggregator(); + assert bb.agg == this; + + for (Map.Entry> e : auxiliaryGroupExprs.entrySet()) { + if (call.equalsDeep(e.getKey(), Litmus.IGNORE)) { + AuxiliaryConverter converter = e.getValue().e; + final RexBuilder rexBuilder = bb.getRexBuilder(); + final int groupOrdinal = e.getValue().i; + return converter.convert( + rexBuilder, + convertedInputExprs.leftList().get(groupOrdinal), + rexBuilder.makeInputRef(castNonNull(bb.root), groupOrdinal)); + } + } + + return aggMapping.get(call); + } + + /** + * Returns the resolved. Valid only if this AggConverter was created via {@link + * #create(SqlToRelConverter.Blackboard, AggregatingSelectScope)}. + */ + AggregatingSelectScope.Resolved getResolved() { + throw new UnsupportedOperationException(); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java index 0cf6f30723ded..212daeadc7407 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java @@ -18,7 +18,6 @@ import org.apache.flink.table.planner.plan.rules.logical.FlinkFilterProjectTransposeRule; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSortedMap; @@ -83,6 +82,7 @@ import org.apache.calcite.rex.RexSubQuery; import org.apache.calcite.rex.RexUtil; import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.runtime.PairList; import org.apache.calcite.sql.SqlExplainFormat; import org.apache.calcite.sql.SqlExplainLevel; import org.apache.calcite.sql.SqlFunction; @@ -131,9 +131,9 @@ *

TODO: * *

    - *
  1. Was changed within FLINK-29280, FLINK-28682, FLINK-35804: Line 218 ~ 225, Line 273 ~ 288 - *
  2. Should be removed after fix of FLINK-29540: Line 293 ~ 299 - *
  3. Should be removed after fix of FLINK-29540: Line 311 ~ 317 + *
  4. Was changed within FLINK-29280, FLINK-28682, FLINK-35804: Line 222 ~ 229, Line 277 ~ 292 + *
  5. Should be removed after fix of FLINK-29540: Line 297 ~ 303 + *
  6. Should be removed after fix of FLINK-29540: Line 315 ~ 321 *
*/ public class RelDecorrelator implements ReflectiveVisitor { @@ -551,7 +551,7 @@ protected RexNode removeCorrelationExpr( // Project projects the original expressions, // plus any correlated variables the input wants to pass along. - final List> projects = new ArrayList<>(); + final PairList projects = PairList.of(); List newInputOutput = newInput.getRowType().getFieldList(); @@ -572,7 +572,7 @@ protected RexNode removeCorrelationExpr( // add mapping of group keys. outputMap.put(idx, newPos); int newInputPos = requireNonNull(frame.oldToNewOutputs.get(idx)); - projects.add(RexInputRef.of2(newInputPos, newInputOutput)); + RexInputRef.add2(projects, newInputPos, newInputOutput); mapNewInputToProjOutputs.put(newInputPos, newPos); newPos++; } @@ -585,7 +585,7 @@ protected RexNode removeCorrelationExpr( // Now add the corVars from the input, starting from // position oldGroupKeyCount. for (Map.Entry entry : frame.corDefOutputs.entrySet()) { - projects.add(RexInputRef.of2(entry.getValue(), newInputOutput)); + RexInputRef.add2(projects, entry.getValue(), newInputOutput); corDefOutputs.put(entry.getKey(), newPos); mapNewInputToProjOutputs.put(entry.getValue(), newPos); @@ -597,7 +597,7 @@ protected RexNode removeCorrelationExpr( final int newGroupKeyCount = newPos; for (int i = 0; i < newInputOutput.size(); i++) { if (!mapNewInputToProjOutputs.containsKey(i)) { - projects.add(RexInputRef.of2(i, newInputOutput)); + RexInputRef.add2(projects, i, newInputOutput); mapNewInputToProjOutputs.put(i, newPos); newPos++; } @@ -610,7 +610,7 @@ protected RexNode removeCorrelationExpr( RelNode newProject = relBuilder .push(newInput) - .projectNamed(Pair.left(projects), Pair.right(projects), true) + .projectNamed(projects.leftList(), projects.rightList(), true) .build(); // update mappings: @@ -788,7 +788,7 @@ private static void shiftMapping(Map mapping, int startIndex, // Project projects the original expressions, // plus any correlated variables the input wants to pass along. - final List> projects = new ArrayList<>(); + final PairList projects = PairList.of(); // If this Project has correlated reference, create value generator // and produce the correlated variables in the new output. @@ -802,20 +802,19 @@ private static void shiftMapping(Map mapping, int startIndex, for (newPos = 0; newPos < oldProjects.size(); newPos++) { projects.add( newPos, - Pair.of( - decorrelateExpr( - requireNonNull(currentRel, "currentRel"), - map, - cm, - oldProjects.get(newPos)), - relOutput.get(newPos).getName())); + decorrelateExpr( + requireNonNull(currentRel, "currentRel"), + map, + cm, + oldProjects.get(newPos)), + relOutput.get(newPos).getName()); mapOldToNewOutputs.put(newPos, newPos); } // Project any correlated variables the input wants to pass along. final NavigableMap corDefOutputs = new TreeMap<>(); for (Map.Entry entry : frame.corDefOutputs.entrySet()) { - projects.add(RexInputRef.of2(entry.getValue(), frame.r.getRowType().getFieldList())); + RexInputRef.add2(projects, entry.getValue(), frame.r.getRowType().getFieldList()); corDefOutputs.put(entry.getKey(), newPos); newPos++; } @@ -823,7 +822,7 @@ private static void shiftMapping(Map mapping, int startIndex, RelNode newProject = relBuilder .push(frame.r) - .projectNamed(Pair.left(projects), Pair.right(projects), true) + .projectNamed(projects.leftList(), projects.rightList(), true) .build(); return register(rel, newProject, mapOldToNewOutputs, corDefOutputs); @@ -1456,14 +1455,14 @@ private RelNode projectJoinOutputWithNullability( true)); // now create the new project - List> newProjExprs = new ArrayList<>(); + final PairList newProjExprs = PairList.of(); // project everything from the LHS and then those from the original // projRel List leftInputFields = left.getRowType().getFieldList(); for (int i = 0; i < leftInputFields.size(); i++) { - newProjExprs.add(RexInputRef.of2(i, leftInputFields)); + RexInputRef.add2(newProjExprs, i, leftInputFields); } // Marked where the projected expr is coming from so that the types will @@ -1476,12 +1475,12 @@ private RelNode projectJoinOutputWithNullability( removeCorrelationExpr( pair.left, projectPulledAboveLeftCorrelator, nullIndicator); - newProjExprs.add(Pair.of(newProjExpr, pair.right)); + newProjExprs.add(newProjExpr, pair.right); } return relBuilder .push(join) - .projectNamed(Pair.left(newProjExprs), Pair.right(newProjExprs), true) + .projectNamed(newProjExprs.leftList(), newProjExprs.rightList(), true) .build(); } @@ -1500,14 +1499,14 @@ private RelNode aggregateCorrelatorOutput( final JoinRelType joinType = correlate.getJoinType(); // now create the new project - final List> newProjects = new ArrayList<>(); + final PairList newProjects = PairList.of(); // Project everything from the LHS and then those from the original // project final List leftInputFields = left.getRowType().getFieldList(); for (int i = 0; i < leftInputFields.size(); i++) { - newProjects.add(RexInputRef.of2(i, leftInputFields)); + RexInputRef.add2(newProjects, i, leftInputFields); } // Marked where the projected expr is coming from so that the types will @@ -1518,12 +1517,12 @@ private RelNode aggregateCorrelatorOutput( for (Pair pair : project.getNamedProjects()) { RexNode newProjExpr = removeCorrelationExpr(pair.left, projectPulledAboveLeftCorrelator, isCount); - newProjects.add(Pair.of(newProjExpr, pair.right)); + newProjects.add(newProjExpr, pair.right); } return relBuilder .push(correlate) - .projectNamed(Pair.left(newProjects), Pair.right(newProjects), true) + .projectNamed(newProjects.leftList(), newProjects.rightList(), true) .build(); } @@ -1599,20 +1598,19 @@ private void removeCorVarFromTree(Correlate correlate) { * @return the new Project */ private RelNode createProjectWithAdditionalExprs( - RelNode input, List> additionalExprs) { + RelNode input, PairList additionalExprs) { final List fieldList = input.getRowType().getFieldList(); - List> projects = new ArrayList<>(); + PairList projects = PairList.of(); Ord.forEach( fieldList, (field, i) -> projects.add( - Pair.of( - relBuilder.getRexBuilder().makeInputRef(field.getType(), i), - field.getName()))); + relBuilder.getRexBuilder().makeInputRef(field.getType(), i), + field.getName())); projects.addAll(additionalExprs); return relBuilder .push(input) - .projectNamed(Pair.left(projects), Pair.right(projects), true) + .projectNamed(projects.leftList(), projects.rightList(), true) .build(); } @@ -1650,11 +1648,7 @@ static boolean allLessThan(Collection integers, int limit, Litmus ret) } private static RelNode stripHep(RelNode rel) { - if (rel instanceof HepRelVertex) { - HepRelVertex hepRelVertex = (HepRelVertex) rel; - rel = hepRelVertex.getCurrentRel(); - } - return rel; + return rel instanceof HepRelVertex ? rel.stripped() : rel; } // ~ Inner Classes ---------------------------------------------------------- @@ -1884,20 +1878,32 @@ public RexNode visitCall(final RexCall call) { /** * Rule to remove an Aggregate with SINGLE_VALUE. For cases like: * - *

Aggregate(SINGLE_VALUE) Project(single expression) Aggregate + *

{@code
+     * Aggregate(SINGLE_VALUE)
+     *   Project(single expression)
+     *     Aggregate
+     * }
* - *

For instance (subtree taken from TPCH query 17): + *

For instance, the following subtree from TPCH query 17: * - *

LogicalAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)]) - * LogicalProject(EXPR$0=[*(0.2:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], - * agg#0=[AVG($0)]) LogicalProject(L_QUANTITY=[$4]) LogicalFilter(condition=[=($1, - * $cor0.P_PARTKEY)]) LogicalTableScan(table=[[TPCH_01, LINEITEM]]) + *

{@code
+     * LogicalAggregate(group=[{}], agg#0=[SINGLE_VALUE($0)])
+     *   LogicalProject(EXPR$0=[*(0.2:DECIMAL(2, 1), $0)])
+     *     LogicalAggregate(group=[{}], agg#0=[AVG($0)])
+     *       LogicalProject(L_QUANTITY=[$4])
+     *         LogicalFilter(condition=[=($1, $cor0.P_PARTKEY)])
+     *           LogicalTableScan(table=[[TPCH_01, LINEITEM]])
+     * }
* - *

Will be converted into: + *

will be converted into: * - *

LogicalProject($f0=[*(0.2:DECIMAL(2, 1), $0)]) LogicalAggregate(group=[{}], - * agg#0=[AVG($0)]) LogicalProject(L_QUANTITY=[$4]) LogicalFilter(condition=[=($1, - * $cor0.P_PARTKEY)]) LogicalTableScan(table=[[TPCH_01, LINEITEM]]) + *

{@code
+     * LogicalProject($f0=[*(0.2:DECIMAL(2, 1), $0)])
+     *   LogicalAggregate(group=[{}], agg#0=[AVG($0)])
+     *     LogicalProject(L_QUANTITY=[$4])
+     *       LogicalFilter(condition=[=($1, $cor0.P_PARTKEY)])
+     *         LogicalTableScan(table=[[TPCH_01, LINEITEM]])
+     * }
*/ public static final class RemoveSingleAggregateRule extends RelRule { @@ -1951,14 +1957,10 @@ public void onMatch(RelOptRuleCall call) { // ensure we keep the same type after removing the SINGLE_VALUE Aggregate final RelBuilder relBuilder = call.builder(); - final RelDataType singleAggType = - singleAggregate.getRowType().getFieldList().get(0).getType(); - final RexNode oldProjectExp = projExprs.get(0); - final RexNode newProjectExp = - singleAggType.equals(oldProjectExp.getType()) - ? oldProjectExp - : relBuilder.getRexBuilder().makeCast(singleAggType, oldProjectExp); - relBuilder.push(aggregate).project(newProjectExp); + relBuilder + .push(aggregate) + .project(project.getAliasedProjects(relBuilder)) + .convert(singleAggregate.getRowType(), false); call.transformTo(relBuilder.build()); } @@ -2071,7 +2073,7 @@ public void onMatch(RelOptRuleCall call) { right = filter.getInput(); assert right instanceof HepRelVertex; - right = ((HepRelVertex) right).getCurrentRel(); + right = right.stripped(); // check filter input contains no correlation if (RelOptUtil.getVariablesUsed(right).size() > 0) { @@ -2156,9 +2158,7 @@ public void onMatch(RelOptRuleCall call) { // make the new Project to provide a null indicator right = d.createProjectWithAdditionalExprs( - right, - ImmutableList.of( - Pair.of(d.relBuilder.literal(true), "nullIndicator"))); + right, PairList.of(d.relBuilder.literal(true), "nullIndicator")); // make the new aggRel right = RelOptUtil.createSingleValueAggRel(cluster, right); @@ -2316,7 +2316,7 @@ public void onMatch(RelOptRuleCall call) { right = filter.getInput(); assert right instanceof HepRelVertex; - right = ((HepRelVertex) right).getCurrentRel(); + right = right.stripped(); // check filter input contains no correlation if (RelOptUtil.getVariablesUsed(right).size() > 0) { @@ -2485,9 +2485,7 @@ public void onMatch(RelOptRuleCall call) { right = d.createProjectWithAdditionalExprs( - right, - ImmutableList.of( - Pair.of(rexBuilder.makeLiteral(true), "nullIndicator"))); + right, PairList.of(rexBuilder.makeLiteral(true), "nullIndicator")); Join join = (Join) d.relBuilder.push(left).push(right).join(joinType, joinCond).build(); @@ -2656,15 +2654,15 @@ public void onMatch(RelOptRuleCall call) { aggregate = call.rel(2); // Create identity projection - final List> projects = new ArrayList<>(); + final PairList projects = PairList.of(); final List fields = aggregate.getRowType().getFieldList(); for (int i = 0; i < fields.size(); i++) { - projects.add(RexInputRef.of2(projects.size(), fields)); + RexInputRef.add2(projects, projects.size(), fields); } final RelBuilder relBuilder = call.builder(); relBuilder .push(aggregate) - .projectNamed(Pair.left(projects), Pair.right(projects), true); + .projectNamed(projects.leftList(), projects.rightList(), true); aggOutputProject = (Project) relBuilder.build(); } onMatch2(call, correlate, left, aggOutputProject, aggregate); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java index 1226f6c7030a2..06fcc071f51e3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java @@ -106,6 +106,7 @@ import org.apache.calcite.rex.RexUtil; import org.apache.calcite.rex.RexWindowBound; import org.apache.calcite.rex.RexWindowBounds; +import org.apache.calcite.runtime.PairList; import org.apache.calcite.schema.ColumnStrategy; import org.apache.calcite.schema.ModifiableTable; import org.apache.calcite.schema.ModifiableView; @@ -203,7 +204,6 @@ import java.lang.reflect.Type; import java.math.BigDecimal; import java.time.ZoneId; -import java.util.AbstractList; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.BitSet; @@ -218,6 +218,7 @@ import java.util.Objects; import java.util.Set; import java.util.TreeSet; +import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Supplier; import java.util.function.UnaryOperator; @@ -227,6 +228,7 @@ import static java.util.Objects.requireNonNull; import static org.apache.calcite.linq4j.Nullness.castNonNull; import static org.apache.calcite.runtime.FlatLists.append; +import static org.apache.calcite.sql.SqlUtil.containsIn; import static org.apache.calcite.sql.SqlUtil.stripAs; import static org.apache.calcite.util.Static.RESOURCE; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -240,19 +242,18 @@ *

FLINK modifications are at lines * *

    - *
  1. Added in FLINK-29081, FLINK-28682, FLINK-33395: Lines 670 ~ 687 - *
  2. Added in Flink-24024: Lines 1463 ~ 1469 - *
  3. Added in Flink-24024: Lines 1483 ~ 1522 - *
  4. Added in Flink-37269: Lines 2239 ~ 2261 - *
  5. Added in FLINK-28682: Lines 2372 ~ 2389 - *
  6. Added in FLINK-28682: Lines 2426 ~ 2454 - *
  7. Added in FLINK-32474: Lines 2507 ~ 2509 - *
  8. Added in FLINK-32474: Lines 2513 ~ 2515 - *
  9. Added in FLINK-32474: Lines 2526 ~ 2528 - *
  10. Added in FLINK-32474: Lines 2934 ~ 2945 - *
  11. Added in FLINK-32474: Lines 3046 ~ 3080 - *
  12. Added in FLINK-34312: Lines 5827 ~ 5838 - *
  13. Added in FLINK-34057, FLINK-34058, FLINK-34312: Lines 6285 ~ 6303 + *
  14. Added in FLINK-29081, FLINK-28682, FLINK-33395: Lines 673 ~ 690 + *
  15. Added in Flink-24024: Lines 1440 ~ 1446 + *
  16. Added in Flink-24024: Lines 1460 ~ 1499 + *
  17. Added in Flink-37269: Lines 2237 ~ 2259 + *
  18. Added in FLINK-28682: Lines 2370 ~ 2387 + *
  19. Added in FLINK-28682: Lines 2424 ~ 2452 + *
  20. Added in FLINK-32474: Lines 2504 ~ 2506 + *
  21. Added in FLINK-32474: Lines 2510 ~ 2512 + *
  22. Added in FLINK-32474: Lines 2523 ~ 2525 + *
  23. Added in FLINK-32474: Lines 2929 ~ 2941 + *
  24. Added in FLINK-32474: Lines 3042 ~ 3076 + *
  25. Added in FLINK-34312: Lines 5805 ~ 5816 *
* *

In official extension point (i.e. {@link #convertExtendedExpression(SqlNode, Blackboard)}): @@ -734,9 +735,7 @@ public RelNode convertSelect(SqlSelect select, boolean top) { /** Factory method for creating translation workspace. */ protected Blackboard createBlackboard( - @Nullable SqlValidatorScope scope, - @Nullable Map nameToNodeMap, - boolean top) { + SqlValidatorScope scope, @Nullable Map nameToNodeMap, boolean top) { return new Blackboard(scope, nameToNodeMap, top); } @@ -864,19 +863,19 @@ private void distinctify(Blackboard bb, boolean checkForDupExprs) { final Map squished = new HashMap<>(); final List fields = rel.getRowType().getFieldList(); - final List> newProjects = new ArrayList<>(); + final PairList newProjects = PairList.of(); for (int i = 0; i < fields.size(); i++) { if (origins.get(i) == i) { squished.put(i, newProjects.size()); - newProjects.add(RexInputRef.of2(i, fields)); + RexInputRef.add2(newProjects, i, fields); } } rel = LogicalProject.create( rel, ImmutableList.of(), - Pair.left(newProjects), - Pair.right(newProjects), + newProjects.leftList(), + newProjects.rightList(), project.getVariablesSet()); bb.root = rel; distinctify(bb, false); @@ -884,22 +883,21 @@ private void distinctify(Blackboard bb, boolean checkForDupExprs) { // Create the expressions to reverse the mapping. // Project($0, $1, $0, $2). - final List> undoProjects = new ArrayList<>(); + final PairList undoProjects = PairList.of(); for (int i = 0; i < fields.size(); i++) { final int origin = origins.get(i); RelDataTypeField field = fields.get(i); undoProjects.add( - Pair.of( - new RexInputRef(castNonNull(squished.get(origin)), field.getType()), - field.getName())); + new RexInputRef(castNonNull(squished.get(origin)), field.getType()), + field.getName()); } rel = LogicalProject.create( rel, ImmutableList.of(), - Pair.left(undoProjects), - Pair.right(undoProjects), + undoProjects.leftList(), + undoProjects.rightList(), ImmutableSet.of()); bb.setRoot(rel, false); @@ -990,31 +988,6 @@ private boolean removeSortInSubQuery(boolean top) { return config.isRemoveSortInSubQuery() && !top; } - /** - * Returns whether a given node contains a {@link SqlInOperator}. - * - * @param node a RexNode tree - */ - private static boolean containsInOperator(SqlNode node) { - try { - SqlVisitor visitor = - new SqlBasicVisitor() { - @Override - public Void visit(SqlCall call) { - if (call.getOperator() instanceof SqlInOperator) { - throw new Util.FoundOne(call); - } - return super.visit(call); - } - }; - node.accept(visitor); - return false; - } catch (Util.FoundOne e) { - Util.swallow(e, null); - return true; - } - } - /** * Push down all the NOT logical operators into any IN/NOT IN operators. * @@ -1023,7 +996,7 @@ public Void visit(SqlCall call) { * @return the transformed SqlNode representation with NOT pushed down. */ private static SqlNode pushDownNotForIn(SqlValidatorScope scope, SqlNode sqlNode) { - if (!(sqlNode instanceof SqlCall) || !containsInOperator(sqlNode)) { + if (!(sqlNode instanceof SqlCall) || !containsIn(sqlNode)) { return sqlNode; } final SqlCall sqlCall = (SqlCall) sqlNode; @@ -1147,7 +1120,7 @@ private void convertWhere(final Blackboard bb, final @Nullable SqlNode where) { if (where == null) { return; } - SqlNode newWhere = pushDownNotForIn(bb.scope(), where); + SqlNode newWhere = pushDownNotForIn(bb.scope, where); replaceSubQueries(bb, newWhere, RelOptUtil.Logic.UNKNOWN_AS_FALSE); final RexNode convertedWhere = bb.convertExpression(newWhere); final RexNode convertedWhere2 = RexUtil.removeNullabilityCast(typeFactory, convertedWhere); @@ -1325,6 +1298,7 @@ private void substituteSubQuery(Blackboard bb, SubQuery subQuery) { false, false, ImmutableList.of(), + ImmutableList.of(), -1, null, RelCollations.EMPTY, @@ -1335,6 +1309,7 @@ private void substituteSubQuery(Blackboard bb, SubQuery subQuery) { false, false, false, + ImmutableList.of(), args, -1, null, @@ -1392,7 +1367,7 @@ private void substituteSubQuery(Blackboard bb, SubQuery subQuery) { final SqlValidatorScope seekScope = (query instanceof SqlSelect) ? validator().getSelectScope((SqlSelect) query) - : null; + : validator().getEmptyScope(); final Blackboard seekBb = createBlackboard(seekScope, null, false); final RelNode seekRel = convertQueryOrInList(seekBb, query, null); requireNonNull(seekRel, () -> "seekRel is null for query " + query); @@ -1535,7 +1510,7 @@ private void substituteSubQueryOfSetSemanticsInputTable(Blackboard bb, SubQuery private ImmutableBitSet buildPartitionKeys(Blackboard bb, SqlNodeList partitionList) { final ImmutableBitSet.Builder partitionKeys = ImmutableBitSet.builder(); for (SqlNode partition : partitionList) { - validator().deriveType(bb.scope(), partition); + validator().deriveType(bb.scope, partition); RexNode e = bb.convertExpression(partition); partitionKeys.set(parseFieldIdx(e)); } @@ -1909,7 +1884,9 @@ private RelOptUtil.Exists convertExists( boolean notIn, @Nullable RelDataType targetDataType) { final SqlValidatorScope seekScope = - (seek instanceof SqlSelect) ? validator().getSelectScope((SqlSelect) seek) : null; + (seek instanceof SqlSelect) + ? validator().getSelectScope((SqlSelect) seek) + : validator().getEmptyScope(); final Blackboard seekBb = createBlackboard(seekScope, null, false); RelNode seekRel = convertQueryOrInList(seekBb, seek, targetDataType); requireNonNull(seekRel, () -> "seekRel is null for query " + seek); @@ -2181,6 +2158,26 @@ private void findSubQueries( default: break; } + if (node instanceof SqlBasicCall + && ((SqlCall) node).getOperator() instanceof SqlQuantifyOperator + && ((SqlQuantifyOperator) ((SqlCall) node).getOperator()) + .tryDeriveTypeForCollection( + bb.getValidator(), bb.scope, (SqlCall) node) + != null) { + findSubQueries( + bb, + ((SqlCall) node).operand(0), + logic, + registerOnlyScalarSubQueries, + clause); + findSubQueries( + bb, + ((SqlCall) node).operand(1), + logic, + registerOnlyScalarSubQueries, + clause); + break; + } bb.registerSubQuery(node, logic, clause); break; default: @@ -2278,7 +2275,7 @@ private RexNode convertOver(Blackboard bb, SqlNode node) { } SqlNode windowOrRef = call.operand(1); - final SqlWindow window = validator().resolveWindow(windowOrRef, bb.scope()); + final SqlWindow window = validator().resolveWindow(windowOrRef, bb.scope); SqlNode sqlLowerBound = window.getLowerBound(); SqlNode sqlUpperBound = window.getUpperBound(); @@ -2309,7 +2306,7 @@ private RexNode convertOver(Blackboard bb, SqlNode node) { final SqlNodeList partitionList = window.getPartitionList(); final ImmutableList.Builder partitionKeys = ImmutableList.builder(); for (SqlNode partition : partitionList) { - validator().deriveType(bb.scope(), partition); + validator().deriveType(bb.scope, partition); partitionKeys.add(bb.convertExpression(partition)); } final RexNode lowerBound = @@ -2320,7 +2317,7 @@ private RexNode convertOver(Blackboard bb, SqlNode node) { // A logical range requires an ORDER BY clause. Use the implicit // ordering of this relation. There must be one, otherwise it would // have failed validation. - orderList = bb.scope().getOrderList(); + orderList = bb.scope.getOrderList(); if (orderList == null) { throw new AssertionError("Relation should have sort key for implicit ORDER BY"); } @@ -2775,7 +2772,6 @@ public RexNode visit(SqlLiteral literal) { protected void convertPivot(Blackboard bb, SqlPivot pivot) { final SqlValidatorScope scope = validator().getJoinScope(pivot); - final Blackboard pivotBb = createBlackboard(scope, null, false); // Convert input @@ -2786,7 +2782,7 @@ protected void convertPivot(Blackboard bb, SqlPivot pivot) { relBuilder.push(input); // Gather fields. - final AggConverter aggConverter = new AggConverter(pivotBb, (AggregatingSelectScope) null); + final AggConverter aggConverter = AggConverter.create(pivotBb); final Set usedColumnNames = pivot.usedColumnNames(); // 1. Gather group keys. @@ -2814,7 +2810,8 @@ protected void convertPivot(Blackboard bb, SqlPivot pivot) { // Project the fields that we will need. relBuilder.project( - Pair.left(aggConverter.getPreExprs()), Pair.right(aggConverter.getPreExprs())); + aggConverter.convertedInputExprs.leftList(), + aggConverter.convertedInputExprs.rightList()); // Build expressions. @@ -2866,7 +2863,6 @@ protected void convertPivot(Blackboard bb, SqlPivot pivot) { protected void convertUnpivot(Blackboard bb, SqlUnpivot unpivot) { final SqlValidatorScope scope = validator().getJoinScope(unpivot); - final Blackboard unpivotBb = createBlackboard(scope, null, false); // Convert input @@ -2985,7 +2981,7 @@ protected void convertCollectionTable(Blackboard bb, SqlCall call) { // Expand table macro if possible. It's more efficient than // LogicalTableFunctionScan. final SqlCallBinding callBinding = - new SqlCallBinding(bb.scope().getValidator(), bb.scope, call); + new SqlCallBinding(bb.scope.getValidator(), bb.scope, call); if (operator instanceof SqlUserDefinedTableMacro) { final SqlUserDefinedTableMacro udf = (SqlUserDefinedTableMacro) operator; final TranslatableTable table = udf.getTable(callBinding); @@ -3187,21 +3183,19 @@ protected RelNode createJoin( mapCorrelToDeferred.get(correlName), () -> "correlation variable is not found: " + correlName); RexFieldAccess fieldAccess = lookup.getFieldAccess(correlName); - String originalRelName = lookup.getOriginalRelName(); String originalFieldName = fieldAccess.getField().getName(); final SqlNameMatcher nameMatcher = bb.getValidator().getCatalogReader().nameMatcher(); final SqlValidatorScope.ResolvedImpl resolved = new SqlValidatorScope.ResolvedImpl(); - lookup.bb - .scope() - .resolve(ImmutableList.of(originalRelName), nameMatcher, false, resolved); + lookup.bb.scope.resolve( + ImmutableList.of(lookup.originalRelName), nameMatcher, false, resolved); assert resolved.count() == 1; final SqlValidatorScope.Resolve resolve = resolved.only(); final SqlValidatorNamespace foundNs = resolve.namespace; final RelDataType rowType = resolve.rowType(); final int childNamespaceIndex = resolve.path.steps().get(0).i; final SqlValidatorScope ancestorScope = resolve.scope; - boolean correlInCurrentScope = bb.scope().isWithin(ancestorScope); + boolean correlInCurrentScope = bb.scope.isWithin(ancestorScope); if (!correlInCurrentScope) { continue; @@ -3258,7 +3252,7 @@ protected RelNode createJoin( // correl not grouped throw new AssertionError( "Identifier '" - + originalRelName + + lookup.originalRelName + "." + originalFieldName + "' is not a group expr"); @@ -3304,14 +3298,13 @@ private boolean isSubQueryNonCorrelated(RelNode subq, Blackboard bb) { requireNonNull( mapCorrelToDeferred.get(correlName), () -> "correlation variable is not found: " + correlName); - String originalRelName = lookup.getOriginalRelName(); + String originalRelName = lookup.originalRelName; final SqlNameMatcher nameMatcher = - lookup.bb.scope().getValidator().getCatalogReader().nameMatcher(); + lookup.bb.scope.getValidator().getCatalogReader().nameMatcher(); final SqlValidatorScope.ResolvedImpl resolved = new SqlValidatorScope.ResolvedImpl(); - lookup.bb - .scope() - .resolve(ImmutableList.of(originalRelName), nameMatcher, false, resolved); + lookup.bb.scope.resolve( + ImmutableList.of(originalRelName), nameMatcher, false, resolved); SqlValidatorScope ancestorScope = resolved.only().scope; @@ -3345,15 +3338,12 @@ private void convertJoin(Blackboard bb, SqlJoin join) { SqlValidator validator = validator(); final SqlValidatorScope scope = validator.getJoinScope(join); final Blackboard fromBlackboard = createBlackboard(scope, null, false); + SqlNode left = join.getLeft(); SqlNode right = join.getRight(); - final SqlValidatorScope leftScope = - Util.first( - validator.getJoinScope(left), ((DelegatingScope) bb.scope()).getParent()); + final SqlValidatorScope leftScope = validator.getJoinScope(left); final Blackboard leftBlackboard = createBlackboard(leftScope, null, false); - final SqlValidatorScope rightScope = - Util.first( - validator.getJoinScope(right), ((DelegatingScope) bb.scope()).getParent()); + final SqlValidatorScope rightScope = validator.getJoinScope(right); final Blackboard rightBlackboard = createBlackboard(rightScope, null, false); convertFrom(leftBlackboard, left); final RelNode leftRel = requireNonNull(leftBlackboard.root, "leftBlackboard.root"); @@ -3539,12 +3529,16 @@ private static JoinRelType convertJoinType(JoinType joinType) { * @param orderExprList Additional expressions needed to implement ORDER BY */ protected void convertAgg(Blackboard bb, SqlSelect select, List orderExprList) { - assert bb.root != null : "precondition: child != null"; + requireNonNull(bb.root, "bb.root"); SqlNodeList groupList = select.getGroup(); SqlNodeList selectList = select.getSelectList(); SqlNode having = select.getHaving(); - final AggConverter aggConverter = new AggConverter(bb, select); + final AggConverter aggConverter = + AggConverter.create( + bb, + (AggregatingSelectScope) validator().getSelectScope(select), + validator()); createAggImpl(bb, aggConverter, selectList, groupList, having, orderExprList); } @@ -3587,15 +3581,16 @@ protected final void createAggImpl( // Calcite allows expressions, not just column references in // group by list. This is not SQL 2003 compliant, but hey. - final AggregatingSelectScope scope = - requireNonNull(aggConverter.aggregatingSelectScope, "aggregatingSelectScope"); - final AggregatingSelectScope.Resolved r = scope.resolved.get(); - for (SqlNode groupExpr : r.groupExprList) { - aggConverter.addGroupExpr(groupExpr); + final AggregatingSelectScope.Resolved r = aggConverter.getResolved(); + for (SqlNode e : r.groupExprList) { + aggConverter.addGroupExpr(e); + } + for (SqlNode e : r.measureExprList) { + aggConverter.addMeasureExpr(e); } final RexNode havingExpr; - final List> projects = new ArrayList<>(); + final PairList projects = PairList.of(); try { checkArgument(bb.agg == null, "already in agg mode"); @@ -3617,14 +3612,15 @@ protected final void createAggImpl( } // compute inputs to the aggregator - List> preExprs = aggConverter.getPreExprs(); - - if (preExprs.size() == 0) { + final PairList preExprs; + if (aggConverter.convertedInputExprs.isEmpty()) { // Special case for COUNT(*), where we can end up with no inputs // at all. The rest of the system doesn't like 0-tuples, so we // select a dummy constant here. final RexNode zero = rexBuilder.makeExactLiteral(BigDecimal.ZERO); - preExprs = ImmutableList.of(Pair.of(zero, null)); + preExprs = PairList.of(zero, null); + } else { + preExprs = aggConverter.convertedInputExprs; } final RelNode inputRel = bb.root(); @@ -3633,7 +3629,7 @@ protected final void createAggImpl( bb.setRoot( relBuilder .push(inputRel) - .projectNamed(Pair.left(preExprs), Pair.right(preExprs), false) + .projectNamed(preExprs.leftList(), preExprs.rightList(), false) .build(), false); bb.mapRootRelToFieldProjection.put(bb.root(), r.groupExprProjection); @@ -3645,20 +3641,19 @@ protected final void createAggImpl( // Tell bb which of group columns are sorted. bb.columnMonotonicities.clear(); for (SqlNode groupItem : groupList) { - bb.columnMonotonicities.add(bb.scope().getMonotonicity(groupItem)); + bb.columnMonotonicities.add(bb.scope.getMonotonicity(groupItem)); } // Add the aggregator bb.setRoot( - createAggregate( - bb, r.groupSet, r.groupSets.asList(), aggConverter.getAggCalls()), + createAggregate(bb, r.groupSet, r.groupSets.asList(), aggConverter.aggCalls), false); bb.mapRootRelToFieldProjection.put(bb.root(), r.groupExprProjection); // Replace sub-queries in having here and modify having to use // the replaced expressions if (having != null) { - SqlNode newHaving = pushDownNotForIn(bb.scope(), having); + SqlNode newHaving = pushDownNotForIn(bb.scope, having); replaceSubQueries(bb, newHaving, RelOptUtil.Logic.UNKNOWN_AS_FALSE); havingExpr = bb.convertExpression(newHaving); } else { @@ -3694,16 +3689,14 @@ protected final void createAggImpl( int sysFieldCount = selectList.size() - names.size(); for (SqlNode expr : selectList) { projects.add( - Pair.of( - bb.convertExpression(expr), - k < sysFieldCount - ? SqlValidatorUtil.alias(expr, k++) - : names.get(k++ - sysFieldCount))); + bb.convertExpression(expr), + k < sysFieldCount + ? SqlValidatorUtil.alias(expr, k++) + : names.get(k++ - sysFieldCount)); } for (SqlNode expr : orderExprList) { - projects.add( - Pair.of(bb.convertExpression(expr), SqlValidatorUtil.alias(expr, k++))); + projects.add(bb.convertExpression(expr), SqlValidatorUtil.alias(expr, k++)); } } finally { bb.agg = null; @@ -3711,18 +3704,16 @@ protected final void createAggImpl( // implement HAVING (we have already checked that it is non-trivial) relBuilder.push(bb.root()); - if (havingExpr != null) { - relBuilder.filter(havingExpr); - } + relBuilder.filter(havingExpr); // implement the SELECT list - relBuilder.project(Pair.left(projects), Pair.right(projects)).rename(Pair.right(projects)); + relBuilder.project(projects.leftList(), projects.rightList()).rename(projects.rightList()); bb.setRoot(relBuilder.build(), false); // Tell bb which of group columns are sorted. bb.columnMonotonicities.clear(); for (SqlNode selectItem : selectList) { - bb.columnMonotonicities.add(bb.scope().getMonotonicity(selectItem)); + bb.columnMonotonicities.add(bb.scope.getMonotonicity(selectItem)); } } @@ -4066,19 +4057,18 @@ private RelNode createSource( // filter. final RexNode constraint = modifiableView.getConstraint(rexBuilder, delegateRowType); RelOptUtil.inferViewPredicates(projectMap, filters, constraint); - final List> projects = new ArrayList<>(); + final PairList projects = PairList.of(); for (RelDataTypeField field : delegateRowType.getFieldList()) { RexNode node = projectMap.get(field.getIndex()); if (node == null) { node = rexBuilder.makeNullLiteral(field.getType()); } - projects.add( - Pair.of(rexBuilder.ensureType(field.getType(), node, false), field.getName())); + projects.add(rexBuilder.ensureType(field.getType(), node, false), field.getName()); } return relBuilder .push(source) - .projectNamed(Pair.left(projects), Pair.right(projects), false) + .projectNamed(projects.leftList(), projects.rightList(), false) .filter(filters) .build(); } @@ -4237,7 +4227,7 @@ private Blackboard createInsertBlackboard( nameToNodeMap.put(targetColumnName, rexBuilder.makeFieldAccess(sourceRef, j++)); } } - return createBlackboard(null, nameToNodeMap, false); + return createBlackboard(validator().getEmptyScope(), nameToNodeMap, false); } private static InitializerExpressionFactory getInitializerFactory( @@ -4498,12 +4488,7 @@ private RexNode convertIdentifier(Blackboard bb, SqlIdentifier identifier) { pv = identifier.names.get(0); } - final SqlQualified qualified; - if (bb.scope != null) { - qualified = bb.scope.fullyQualify(identifier); - } else { - qualified = SqlQualified.create(null, 1, null, identifier); - } + final SqlQualified qualified = bb.scope.fullyQualify(identifier); final Pair> e0 = bb.lookupExp(qualified); RexNode e = e0.left; @@ -4606,7 +4591,7 @@ private RelNode convertMultisets(final List operands, Blackboard bb) { } else { usedBb = createBlackboard( - new ListScope(bb.scope()) { + new ListScope(bb.scope) { @Override public SqlNode getNode() { return call; @@ -4952,16 +4937,15 @@ private void convertValuesImpl( SqlCall rowConstructor = (SqlCall) rowConstructor1; Blackboard tmpBb = createBlackboard(bb.scope, null, false); replaceSubQueries(tmpBb, rowConstructor, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN); - final List> exps = new ArrayList<>(); + final PairList exps = PairList.of(); Ord.forEach( rowConstructor.getOperandList(), (operand, i) -> exps.add( - Pair.of( - tmpBb.convertExpression(operand), - SqlValidatorUtil.alias(operand, i)))); + tmpBb.convertExpression(operand), + SqlValidatorUtil.alias(operand, i))); RelNode in = (null == tmpBb.root) ? LogicalValues.createOneRow(cluster) : tmpBb.root; - relBuilder.push(in).project(Pair.left(exps), Pair.right(exps)); + relBuilder.push(in).project(exps.leftList(), exps.rightList()); } bb.setRoot(relBuilder.union(true, values.getOperandList().size()).build(), true); @@ -4999,7 +4983,7 @@ R convert( /** Workspace for translating an individual SELECT statement (or sub-SELECT). */ protected class Blackboard implements SqlRexContext, SqlVisitor, InitializerContext { /** Collection of {@link RelNode} objects which correspond to a SELECT statement. */ - public final @Nullable SqlValidatorScope scope; + public final SqlValidatorScope scope; private final @Nullable Map nameToNodeMap; public @Nullable RelNode root; @@ -5054,7 +5038,7 @@ protected Blackboard( @Nullable SqlValidatorScope scope, @Nullable Map nameToNodeMap, boolean top) { - this.scope = scope; + this.scope = requireNonNull(scope, "scope"); this.nameToNodeMap = nameToNodeMap; this.top = top; } @@ -5063,8 +5047,9 @@ public RelNode root() { return requireNonNull(root, "root"); } + @Deprecated // to be removed before 2.0 public SqlValidatorScope scope() { - return requireNonNull(scope, "scope"); + return scope; } public void setPatternVarRef(boolean isVarRef) { @@ -5149,23 +5134,12 @@ public RexNode register( assert leftKeyCount == rightFieldLength - 1; final int rexRangeRefLength = leftKeyCount + rightFieldLength; - RelDataType returnType = - typeFactory.createStructType( - new AbstractList>() { - @Override - public Map.Entry get(int index) { - return join.getRowType() - .getFieldList() - .get(origLeftInputCount + index); - } - - @Override - public int size() { - return rexRangeRefLength; - } - }); - - return rexBuilder.makeRangeReference(returnType, origLeftInputCount, false); + final RelDataTypeFactory.Builder builder = typeFactory.builder(); + for (int i = 0; i < rexRangeRefLength; i++) { + builder.add(join.getRowType().getFieldList().get(origLeftInputCount + i)); + } + + return rexBuilder.makeRangeReference(builder.build(), origLeftInputCount, false); } else { return rexBuilder.makeRangeReference( rel.getRowType(), leftFieldCount, joinType.generatesNullsOnRight()); @@ -5259,9 +5233,9 @@ void setRoot(List inputs) { return Pair.of(node, null); } final SqlNameMatcher nameMatcher = - scope().getValidator().getCatalogReader().nameMatcher(); + scope.getValidator().getCatalogReader().nameMatcher(); final SqlValidatorScope.ResolvedImpl resolved = new SqlValidatorScope.ResolvedImpl(); - scope().resolve(qualified.prefix(), nameMatcher, false, resolved); + scope.resolve(qualified.prefix(), nameMatcher, false, resolved); if (resolved.count() != 1) { throw new AssertionError( "no unique expression found for " @@ -5280,7 +5254,6 @@ void setRoot(List inputs) { if ((inputs != null) && !isParent) { final LookupContext rels = new LookupContext(this, inputs, systemFieldList.size()); final RexNode node = lookup(resolve.path.steps().get(0).i, rels); - assert node != null; return Pair.of( node, (e, fieldName) -> { @@ -5344,8 +5317,9 @@ void setRoot(List inputs) { * from-list is {@code offset}. */ RexNode lookup(int offset, LookupContext lookupContext) { - Pair pair = lookupContext.findRel(offset); - return rexBuilder.makeRangeReference(pair.left.getRowType(), pair.right, false); + Map.Entry pair = lookupContext.findRel(offset); + return rexBuilder.makeRangeReference( + pair.getKey().getRowType(), pair.getValue(), false); } @Nullable RelDataTypeField getRootField(RexInputRef inputRef) { @@ -5368,13 +5342,13 @@ public void flatten( List rels, int systemFieldCount, int[] start, - List> relOffsetList) { + BiConsumer relOffsetList) { for (RelNode rel : rels) { if (leaves.containsKey(rel)) { - relOffsetList.add(Pair.of(rel, start[0])); + relOffsetList.accept(rel, start[0]); start[0] += leaves.get(rel); } else if (rel instanceof LogicalMatch) { - relOffsetList.add(Pair.of(rel, start[0])); + relOffsetList.accept(rel, start[0]); start[0] += rel.getRowType().getFieldCount(); } else { if (rel instanceof LogicalJoin || rel instanceof LogicalAggregate) { @@ -5426,7 +5400,7 @@ public RexNode convertExpression(SqlNode expr) { // GROUP BY clause, return a reference to the field. AggConverter agg = this.agg; if (agg != null) { - final SqlNode expandedGroupExpr = validator().expand(expr, scope()); + final SqlNode expandedGroupExpr = validator().expand(expr, scope); final int ref = agg.lookupGroupExpr(expandedGroupExpr); if (ref >= 0) { return rexBuilder.makeInputRef(root(), ref); @@ -5563,10 +5537,14 @@ public RexNode convertExpression(SqlNode expr) { case CURSOR: case IN: case NOT_IN: - subQuery = requireNonNull(getSubQuery(expr, null)); + subQuery = getSubQuery(expr, null); + if (subQuery == null && (kind == SqlKind.SOME || kind == SqlKind.ALL)) { + break; + } + assert subQuery != null; rex = requireNonNull(subQuery.expr); return StandardConvertletTable.castToValidatedType( - expr, rex, validator(), rexBuilder); + expr, rex, validator(), rexBuilder, false); case SELECT: case EXISTS: @@ -5874,23 +5852,17 @@ private static SqlQuantifyOperator negate(SqlQuantifyOperator operator) { /** Deferred lookup. */ private static class DeferredLookup { - Blackboard bb; - String originalRelName; + final Blackboard bb; + final String originalRelName; DeferredLookup(Blackboard bb, String originalRelName) { this.bb = bb; this.originalRelName = originalRelName; } - public RexFieldAccess getFieldAccess(CorrelationId name) { - return (RexFieldAccess) - requireNonNull( - bb.mapCorrelateToRex.get(name), - () -> "Correlation " + name + " is not found"); - } - - public String getOriginalRelName() { - return originalRelName; + RexFieldAccess getFieldAccess(CorrelationId name) { + return requireNonNull( + bb.mapCorrelateToRex.get(name), () -> "Correlation " + name + " is not found"); } } @@ -5911,556 +5883,9 @@ public RexNode convertSubQuery( } } - /** - * Converts expressions to aggregates. - * - *

Consider the expression - * - *

- * - * {@code SELECT deptno, SUM(2 * sal) FROM emp GROUP BY deptno} - * - *
- * - *

Then: - * - *

    - *
  • groupExprs = {SqlIdentifier(deptno)} - *
  • convertedInputExprs = {RexInputRef(deptno), 2 * RefInputRef(sal)} - *
  • inputRefs = {RefInputRef(#0), RexInputRef(#1)} - *
  • aggCalls = {AggCall(SUM, {1})} - *
- */ - protected class AggConverter implements SqlVisitor { - private final Blackboard bb; - public final @Nullable AggregatingSelectScope aggregatingSelectScope; - - private final Map nameMap = new HashMap<>(); - - /** The group-by expressions, in {@link SqlNode} format. */ - private final SqlNodeList groupExprs = new SqlNodeList(SqlParserPos.ZERO); - - /** The auxiliary group-by expressions. */ - private final Map> auxiliaryGroupExprs = new HashMap<>(); - - /** - * Input expressions for the group columns and aggregates, in {@link RexNode} format. The - * first elements of the list correspond to the elements in {@link #groupExprs}; the - * remaining elements are for aggregates. The right field of each pair is the name of the - * expression, where the expressions are simple mappings to input fields. - */ - private final List> convertedInputExprs = new ArrayList<>(); - - /** - * Expressions to be evaluated as rows are being placed into the aggregate's hash table. - * This is when group functions such as TUMBLE cause rows to be expanded. - */ - private final List aggCalls = new ArrayList<>(); - - private final Map aggMapping = new HashMap<>(); - private final Map aggCallMapping = new HashMap<>(); - - /** Whether we are directly inside a windowed aggregate. */ - private boolean inOver = false; - - AggConverter(Blackboard bb, @Nullable AggregatingSelectScope aggregatingSelectScope) { - this.bb = bb; - this.aggregatingSelectScope = aggregatingSelectScope; - } - - /** - * Creates an AggConverter. - * - *

The select parameter provides enough context to name aggregate calls - * which are top-level select list items. - * - * @param bb Blackboard - * @param select Query being translated; provides context to give - */ - public AggConverter(Blackboard bb, SqlSelect select) { - this(bb, (AggregatingSelectScope) bb.getValidator().getSelectScope(select)); - - // Collect all expressions used in the select list so that aggregate - // calls can be named correctly. - final SqlNodeList selectList = select.getSelectList(); - for (int i = 0; i < selectList.size(); i++) { - SqlNode selectItem = selectList.get(i); - String name = null; - if (SqlUtil.isCallTo(selectItem, SqlStdOperatorTable.AS)) { - final SqlCall call = (SqlCall) selectItem; - selectItem = call.operand(0); - name = call.operand(1).toString(); - } - if (name == null) { - name = SqlValidatorUtil.alias(selectItem, i); - } - nameMap.put(selectItem.toString(), name); - } - } - - public int addGroupExpr(SqlNode expr) { - int ref = lookupGroupExpr(expr); - if (ref >= 0) { - return ref; - } - final int index = groupExprs.size(); - groupExprs.add(expr); - String name = nameMap.get(expr.toString()); - RexNode convExpr = bb.convertExpression(expr); - addExpr(convExpr, name); - - if (expr instanceof SqlCall) { - SqlCall call = (SqlCall) expr; - for (Pair p : - SqlStdOperatorTable.convertGroupToAuxiliaryCalls(call)) { - addAuxiliaryGroupExpr(p.left, index, p.right); - } - } - - return index; - } - - void addAuxiliaryGroupExpr(SqlNode node, int index, AuxiliaryConverter converter) { - for (SqlNode node2 : auxiliaryGroupExprs.keySet()) { - if (node2.equalsDeep(node, Litmus.IGNORE)) { - return; - } - } - auxiliaryGroupExprs.put(node, Ord.of(index, converter)); - } - - /** - * Adds an expression, deducing an appropriate name if possible. - * - * @param expr Expression - * @param name Suggested name - */ - private void addExpr(RexNode expr, @Nullable String name) { - if ((name == null) && (expr instanceof RexInputRef)) { - final int i = ((RexInputRef) expr).getIndex(); - name = bb.root().getRowType().getFieldList().get(i).getName(); - } - if (Pair.right(convertedInputExprs).contains(name)) { - // In case like 'SELECT ... GROUP BY x, y, x', don't add - // name 'x' twice. - name = null; - } - convertedInputExprs.add(Pair.of(expr, name)); - } - - @Override - public Void visit(SqlIdentifier id) { - return null; - } - - @Override - public Void visit(SqlNodeList nodeList) { - for (int i = 0; i < nodeList.size(); i++) { - nodeList.get(i).accept(this); - } - return null; - } - - @Override - public Void visit(SqlLiteral lit) { - return null; - } - - @Override - public Void visit(SqlDataTypeSpec type) { - return null; - } - - @Override - public Void visit(SqlDynamicParam param) { - return null; - } - - @Override - public Void visit(SqlIntervalQualifier intervalQualifier) { - return null; - } - - @Override - public Void visit(SqlCall call) { - switch (call.getKind()) { - case FILTER: - case IGNORE_NULLS: - case RESPECT_NULLS: - case WITHIN_DISTINCT: - case WITHIN_GROUP: - translateAgg(call); - return null; - case SELECT: - // rchen 2006-10-17: - // for now do not detect aggregates in sub-queries. - return null; - default: - break; - } - final boolean prevInOver = inOver; - // Ignore window aggregates and ranking functions (associated with OVER - // operator). However, do not ignore nested window aggregates. - if (call.getOperator().getKind() == SqlKind.OVER) { - // Track aggregate nesting levels only within an OVER operator. - List operandList = call.getOperandList(); - assert operandList.size() == 2; - - // Ignore the top level window aggregates and ranking functions - // positioned as the first operand of a OVER operator - inOver = true; - operandList.get(0).accept(this); - - // Normal translation for the second operand of a OVER operator - inOver = false; - operandList.get(1).accept(this); - return null; - } - - // Do not translate the top level window aggregate. Only do so for - // nested aggregates, if present - if (call.getOperator().isAggregator()) { - if (inOver) { - // Add the parent aggregate level before visiting its children - inOver = false; - } else { - // We're beyond the one ignored level - translateAgg(call); - return null; - } - } - for (SqlNode operand : call.getOperandList()) { - // Operands are occasionally null, e.g. switched CASE arg 0. - if (operand != null) { - operand.accept(this); - } - } - // Remove the parent aggregate level after visiting its children - inOver = prevInOver; - return null; - } - - private void translateAgg(SqlCall call) { - translateAgg(call, null, null, null, false, call); - } - - private void translateAgg( - SqlCall call, - @Nullable SqlNode filter, - @Nullable SqlNodeList distinctList, - @Nullable SqlNodeList orderList, - boolean ignoreNulls, - SqlCall outerCall) { - assert bb.agg == this; - assert outerCall != null; - final List operands = call.getOperandList(); - final SqlParserPos pos = call.getParserPosition(); - final SqlCall call2; - switch (call.getKind()) { - case FILTER: - assert filter == null; - translateAgg( - call.operand(0), - call.operand(1), - distinctList, - orderList, - ignoreNulls, - outerCall); - return; - case WITHIN_DISTINCT: - assert orderList == null; - translateAgg( - call.operand(0), - filter, - call.operand(1), - orderList, - ignoreNulls, - outerCall); - return; - case WITHIN_GROUP: - assert orderList == null; - translateAgg( - call.operand(0), - filter, - distinctList, - call.operand(1), - ignoreNulls, - outerCall); - return; - case IGNORE_NULLS: - ignoreNulls = true; - // fall through - case RESPECT_NULLS: - translateAgg( - call.operand(0), - filter, - distinctList, - orderList, - ignoreNulls, - outerCall); - return; - - case COUNTIF: - // COUNTIF(b) ==> COUNT(*) FILTER (WHERE b) - // COUNTIF(b) FILTER (WHERE b2) ==> COUNT(*) FILTER (WHERE b2 AND b) - call2 = SqlStdOperatorTable.COUNT.createCall(pos, SqlIdentifier.star(pos)); - final SqlNode filter2 = SqlUtil.andExpressions(filter, call.operand(0)); - translateAgg(call2, filter2, distinctList, orderList, ignoreNulls, outerCall); - return; - - case STRING_AGG: - // Translate "STRING_AGG(s, sep ORDER BY x, y)" - // as if it were "LISTAGG(s, sep) WITHIN GROUP (ORDER BY x, y)"; - // and "STRING_AGG(s, sep)" as "LISTAGG(s, sep)". - final List operands2; - if (!operands.isEmpty() && Util.last(operands) instanceof SqlNodeList) { - orderList = (SqlNodeList) Util.last(operands); - operands2 = Util.skipLast(operands); - } else { - operands2 = operands; - } - call2 = - SqlStdOperatorTable.LISTAGG.createCall( - call.getFunctionQuantifier(), pos, operands2); - translateAgg(call2, filter, distinctList, orderList, ignoreNulls, outerCall); - return; - - case GROUP_CONCAT: - // Translate "GROUP_CONCAT(s ORDER BY x, y SEPARATOR ',')" - // as if it were "LISTAGG(s, ',') WITHIN GROUP (ORDER BY x, y)". - // To do this, build a list of operands without ORDER BY with with sep. - operands2 = new ArrayList<>(operands); - final SqlNode separator; - if (!operands2.isEmpty() - && Util.last(operands2).getKind() == SqlKind.SEPARATOR) { - final SqlCall sepCall = (SqlCall) operands2.remove(operands.size() - 1); - separator = sepCall.operand(0); - } else { - separator = null; - } - - if (!operands2.isEmpty() && Util.last(operands2) instanceof SqlNodeList) { - orderList = (SqlNodeList) operands2.remove(operands2.size() - 1); - } - - if (separator != null) { - operands2.add(separator); - } - - call2 = - SqlStdOperatorTable.LISTAGG.createCall( - call.getFunctionQuantifier(), pos, operands2); - translateAgg(call2, filter, distinctList, orderList, ignoreNulls, outerCall); - return; - - case ARRAY_AGG: - case ARRAY_CONCAT_AGG: - // Translate "ARRAY_AGG(s ORDER BY x, y)" - // as if it were "ARRAY_AGG(s) WITHIN GROUP (ORDER BY x, y)"; - // similarly "ARRAY_CONCAT_AGG". - if (!operands.isEmpty() && Util.last(operands) instanceof SqlNodeList) { - orderList = (SqlNodeList) Util.last(operands); - call2 = - call.getOperator() - .createCall( - call.getFunctionQuantifier(), - pos, - Util.skipLast(operands)); - translateAgg( - call2, filter, distinctList, orderList, ignoreNulls, outerCall); - return; - } - // "ARRAY_AGG" and "ARRAY_CONCAT_AGG" without "ORDER BY" - // are handled normally; fall through. - - default: - break; - } - final List args = new ArrayList<>(); - int filterArg = -1; - final ImmutableBitSet distinctKeys; - try { - // switch out of agg mode - bb.agg = null; - // ----- FLINK MODIFICATION BEGIN ----- - FlinkSqlCallBinding binding = - new FlinkSqlCallBinding(validator(), aggregatingSelectScope, call); - List sqlNodes = binding.operands(); - for (int i = 0; i < sqlNodes.size(); i++) { - SqlNode operand = sqlNodes.get(i); - // special case for COUNT(*): delete the * - if (operand instanceof SqlIdentifier) { - SqlIdentifier id = (SqlIdentifier) operand; - if (id.isStar()) { - assert call.operandCount() == 1; - assert args.isEmpty(); - break; - } - } - RexNode convertedExpr = bb.convertExpression(operand); - args.add(lookupOrCreateGroupExpr(convertedExpr)); - } - // ----- FLINK MODIFICATION END ----- - - if (filter != null) { - RexNode convertedExpr = bb.convertExpression(filter); - if (convertedExpr.getType().isNullable()) { - convertedExpr = - rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, convertedExpr); - } - filterArg = lookupOrCreateGroupExpr(convertedExpr); - } - - if (distinctList == null) { - distinctKeys = null; - } else { - final ImmutableBitSet.Builder distinctBuilder = ImmutableBitSet.builder(); - for (SqlNode distinct : distinctList) { - RexNode e = bb.convertExpression(distinct); - assert e != null; - distinctBuilder.set(lookupOrCreateGroupExpr(e)); - } - distinctKeys = distinctBuilder.build(); - } - } finally { - // switch back into agg mode - bb.agg = this; - } - - SqlAggFunction aggFunction = (SqlAggFunction) call.getOperator(); - final RelDataType type = validator().deriveType(bb.scope(), call); - boolean distinct = false; - SqlLiteral quantifier = call.getFunctionQuantifier(); - if ((null != quantifier) && (quantifier.getValue() == SqlSelectKeyword.DISTINCT)) { - distinct = true; - } - boolean approximate = false; - if (aggFunction == SqlStdOperatorTable.APPROX_COUNT_DISTINCT) { - aggFunction = SqlStdOperatorTable.COUNT; - distinct = true; - approximate = true; - } - final RelCollation collation; - if (orderList == null || orderList.size() == 0) { - collation = RelCollations.EMPTY; - } else { - try { - // switch out of agg mode - bb.agg = null; - collation = - RelCollations.of( - orderList.stream() - .map( - order -> - bb.convertSortExpression( - order, - RelFieldCollation.Direction - .ASCENDING, - RelFieldCollation.NullDirection - .UNSPECIFIED, - this::sortToFieldCollation)) - .collect(Collectors.toList())); - } finally { - // switch back into agg mode - bb.agg = this; - } - } - final AggregateCall aggCall = - AggregateCall.create( - aggFunction, - distinct, - approximate, - ignoreNulls, - args, - filterArg, - distinctKeys, - collation, - type, - nameMap.get(outerCall.toString())); - RexNode rex = - rexBuilder.addAggCall( - aggCall, - groupExprs.size(), - aggCalls, - aggCallMapping, - i -> convertedInputExprs.get(i).left.getType().isNullable()); - aggMapping.put(outerCall, rex); - } - - private RelFieldCollation sortToFieldCollation( - SqlNode expr, - RelFieldCollation.Direction direction, - RelFieldCollation.NullDirection nullDirection) { - final RexNode node = bb.convertExpression(expr); - final int fieldIndex = lookupOrCreateGroupExpr(node); - if (nullDirection == RelFieldCollation.NullDirection.UNSPECIFIED) { - nullDirection = direction.defaultNullDirection(); - } - return new RelFieldCollation(fieldIndex, direction, nullDirection); - } - - private int lookupOrCreateGroupExpr(RexNode expr) { - int index = 0; - for (RexNode convertedInputExpr : Pair.left(convertedInputExprs)) { - if (expr.equals(convertedInputExpr)) { - return index; - } - ++index; - } - - // not found -- add it - addExpr(expr, null); - return index; - } - - /** - * If an expression is structurally identical to one of the group-by expressions, returns a - * reference to the expression, otherwise returns null. - */ - public int lookupGroupExpr(SqlNode expr) { - for (int i = 0; i < groupExprs.size(); i++) { - SqlNode groupExpr = groupExprs.get(i); - if (expr.equalsDeep(groupExpr, Litmus.IGNORE)) { - return i; - } - } - return -1; - } - - public @Nullable RexNode lookupAggregates(SqlCall call) { - // assert call.getOperator().isAggregator(); - assert bb.agg == this; - - for (Map.Entry> e : auxiliaryGroupExprs.entrySet()) { - if (call.equalsDeep(e.getKey(), Litmus.IGNORE)) { - AuxiliaryConverter converter = e.getValue().e; - final int groupOrdinal = e.getValue().i; - return converter.convert( - rexBuilder, - convertedInputExprs.get(groupOrdinal).left, - rexBuilder.makeInputRef(castNonNull(bb.root), groupOrdinal)); - } - } - - return aggMapping.get(call); - } - - public List> getPreExprs() { - return convertedInputExprs; - } - - public List getAggCalls() { - return aggCalls; - } - - public RelDataTypeFactory getTypeFactory() { - return typeFactory; - } - } - /** Context to find a relational expression to a field offset. */ private static class LookupContext { - private final List> relOffsetList = new ArrayList<>(); + private final PairList relOffsetList = PairList.of(); /** * Creates a LookupContext with multiple input relational expressions. @@ -6470,7 +5895,7 @@ private static class LookupContext { * @param systemFieldCount Number of system fields */ LookupContext(Blackboard bb, List rels, int systemFieldCount) { - bb.flatten(rels, systemFieldCount, new int[] {0}, relOffsetList); + bb.flatten(rels, systemFieldCount, new int[] {0}, relOffsetList::add); } /** @@ -6484,7 +5909,7 @@ private static class LookupContext { * @param offset Offset of relational expression in FROM clause * @return Relational expression and the ordinal of its first field */ - Pair findRel(int offset) { + Map.Entry findRel(int offset) { return relOffsetList.get(offset); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java index a7363bd5a99fa..29064715339c2 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java @@ -58,6 +58,7 @@ import org.apache.calcite.sql.fun.SqlArrayValueConstructor; import org.apache.calcite.sql.fun.SqlBetweenOperator; import org.apache.calcite.sql.fun.SqlCase; +import org.apache.calcite.sql.fun.SqlCastFunction; import org.apache.calcite.sql.fun.SqlDatetimeSubtractionOperator; import org.apache.calcite.sql.fun.SqlExtractFunction; import org.apache.calcite.sql.fun.SqlInternalOperators; @@ -81,7 +82,6 @@ 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.SqlValidatorImpl; import org.apache.calcite.util.Pair; import org.apache.calcite.util.Util; import org.checkerframework.checker.nullness.qual.Nullable; @@ -96,7 +96,9 @@ import java.util.function.UnaryOperator; import java.util.stream.Collectors; +import static com.google.common.base.Preconditions.checkArgument; import static java.util.Objects.requireNonNull; +import static org.apache.calcite.sql.fun.SqlStdOperatorTable.QUANTIFY_OPERATORS; import static org.apache.calcite.sql.type.NonNullableAccessors.getComponentTypeOrThrow; import static org.apache.calcite.util.Util.first; @@ -106,7 +108,7 @@ *

FLINK modifications are at lines * *

    - *
  1. Added in Flink-35216: Lines 731 ~ 776 + *
  2. Added in Flink-35216: Lines 832 ~ 878 *
*/ public class StandardConvertletTable extends ReflectiveConvertletTable { @@ -131,6 +133,8 @@ private StandardConvertletTable() { // Register convertlets for specific objects. registerOp(SqlStdOperatorTable.CAST, this::convertCast); + registerOp(SqlLibraryOperators.SAFE_CAST, this::convertCast); + registerOp(SqlLibraryOperators.TRY_CAST, this::convertCast); registerOp(SqlLibraryOperators.INFIX_CAST, this::convertCast); registerOp( SqlStdOperatorTable.IS_DISTINCT_FROM, @@ -168,6 +172,9 @@ private StandardConvertletTable() { return e; }); + registerOp(SqlLibraryOperators.DATETIME_TRUNC, new TruncConvertlet()); + registerOp(SqlLibraryOperators.TIMESTAMP_TRUNC, new TruncConvertlet()); + registerOp(SqlLibraryOperators.LTRIM, new TrimConvertlet(SqlTrimFunction.Flag.LEADING)); registerOp(SqlLibraryOperators.RTRIM, new TrimConvertlet(SqlTrimFunction.Flag.TRAILING)); @@ -193,6 +200,9 @@ private StandardConvertletTable() { registerOp(SqlLibraryOperators.TIMESTAMP_DIFF3, new TimestampDiffConvertlet()); registerOp(SqlLibraryOperators.TIMESTAMP_SUB, new TimestampSubConvertlet()); + QUANTIFY_OPERATORS.forEach( + operator -> registerOp(operator, StandardConvertletTable::convertQuantifyOperator)); + registerOp(SqlLibraryOperators.NVL, StandardConvertletTable::convertNvl); registerOp(SqlLibraryOperators.DECODE, StandardConvertletTable::convertDecode); registerOp(SqlLibraryOperators.IF, StandardConvertletTable::convertIf); @@ -255,6 +265,8 @@ private StandardConvertletTable() { registerOp(SqlStdOperatorTable.ITEM, this::convertItem); // "AS" has no effect, so expand "x AS id" into "x". registerOp(SqlStdOperatorTable.AS, (cx, call) -> cx.convertExpression(call.operand(0))); + registerOp(SqlStdOperatorTable.CONVERT, this::convertCharset); + registerOp(SqlStdOperatorTable.TRANSLATE, this::translateCharset); // "SQRT(x)" is equivalent to "POWER(x, .5)" registerOp( SqlStdOperatorTable.SQRT, @@ -265,6 +277,19 @@ private StandardConvertletTable() { call.operand(0), SqlLiteral.createExactNumeric("0.5", SqlParserPos.ZERO)))); + // "STRPOS(string, substring) is equivalent to + // "POSITION(substring IN string)" + registerOp( + SqlLibraryOperators.STRPOS, + (cx, call) -> + cx.convertExpression( + SqlStdOperatorTable.POSITION.createCall( + SqlParserPos.ZERO, call.operand(1), call.operand(0)))); + + // "INSTR(string, substring, position, occurrence) is equivalent to + // "POSITION(substring, string, position, occurrence)" + registerOp(SqlLibraryOperators.INSTR, StandardConvertletTable::convertInstr); + // REVIEW jvs 24-Apr-2006: This only seems to be working from within a // windowed agg. I have added an optimizer rule // org.apache.calcite.rel.rules.AggregateReduceFunctionsRule which handles @@ -338,6 +363,24 @@ private StandardConvertletTable() { } } + /** Converts ALL or SOME operators. */ + private static RexNode convertQuantifyOperator(SqlRexContext cx, SqlCall call) { + final RexBuilder rexBuilder = cx.getRexBuilder(); + final RexNode left = cx.convertExpression(call.getOperandList().get(0)); + assert call.getOperandList().get(1) instanceof SqlNodeList; + final RexNode right = + cx.convertExpression(((SqlNodeList) call.getOperandList().get(1)).get(0)); + final RelDataType rightComponentType = requireNonNull(right.getType().getComponentType()); + final RelDataType returnType = + cx.getTypeFactory() + .createTypeWithNullability( + cx.getTypeFactory().createSqlType(SqlTypeName.BOOLEAN), + right.getType().isNullable() + || left.getType().isNullable() + || rightComponentType.isNullable()); + return rexBuilder.makeCall(returnType, call.getOperator(), ImmutableList.of(left, right)); + } + /** Converts a call to the {@code NVL} function (and also its synonym, {@code IFNULL}). */ private static RexNode convertNvl(SqlRexContext cx, SqlCall call) { final RexBuilder rexBuilder = cx.getRexBuilder(); @@ -362,6 +405,40 @@ private static RexNode convertNvl(SqlRexContext cx, SqlCall call) { operand1))); } + /** + * Converts a call to the INSTR function. INSTR(string, substring, position, occurrence) is + * equivalent to POSITION(substring, string, position, occurrence) + */ + private static RexNode convertInstr(SqlRexContext cx, SqlCall call) { + final RexBuilder rexBuilder = cx.getRexBuilder(); + final List operands = + convertOperands(cx, call, SqlOperandTypeChecker.Consistency.NONE); + final RelDataType type = cx.getValidator().getValidatedNodeType(call); + final List exprs = new ArrayList<>(); + switch (call.operandCount()) { + // Must reverse order of first 2 operands. + case 2: + exprs.add(operands.get(1)); // Substring + exprs.add(operands.get(0)); // String + break; + case 3: + exprs.add(operands.get(1)); // Substring + exprs.add(operands.get(0)); // String + exprs.add(operands.get(2)); // Position + break; + case 4: + exprs.add(operands.get(1)); // Substring + exprs.add(operands.get(0)); // String + exprs.add(operands.get(2)); // Position + exprs.add(operands.get(3)); // Occurrence + break; + default: + throw new UnsupportedOperationException( + "Position does not accept " + call.operandCount() + " operands"); + } + return rexBuilder.makeCall(type, SqlStdOperatorTable.POSITION, exprs); + } + /** Converts a call to the DECODE function. */ private static RexNode convertDecode(SqlRexContext cx, SqlCall call) { final RexBuilder rexBuilder = cx.getRexBuilder(); @@ -552,39 +629,41 @@ public RexNode convertJdbc(SqlRexContext cx, SqlJdbcFunctionCall op, SqlCall cal protected RexNode convertCast(SqlRexContext cx, final SqlCall call) { RelDataTypeFactory typeFactory = cx.getTypeFactory(); - assert call.getKind() == SqlKind.CAST; + final SqlValidator validator = cx.getValidator(); + final SqlKind kind = call.getKind(); + checkArgument(kind == SqlKind.CAST || kind == SqlKind.SAFE_CAST, kind); + final boolean safe = kind == SqlKind.SAFE_CAST; final SqlNode left = call.operand(0); final SqlNode right = call.operand(1); + final RexBuilder rexBuilder = cx.getRexBuilder(); if (right instanceof SqlIntervalQualifier) { final SqlIntervalQualifier intervalQualifier = (SqlIntervalQualifier) right; if (left instanceof SqlIntervalLiteral) { RexLiteral sourceInterval = (RexLiteral) cx.convertExpression(left); BigDecimal sourceValue = (BigDecimal) sourceInterval.getValue(); RexLiteral castedInterval = - cx.getRexBuilder().makeIntervalLiteral(sourceValue, intervalQualifier); - return castToValidatedType(cx, call, castedInterval); + rexBuilder.makeIntervalLiteral(sourceValue, intervalQualifier); + return castToValidatedType(call, castedInterval, validator, rexBuilder, safe); } else if (left instanceof SqlNumericLiteral) { RexLiteral sourceInterval = (RexLiteral) cx.convertExpression(left); - BigDecimal sourceValue = (BigDecimal) sourceInterval.getValue(); + BigDecimal sourceValue = + requireNonNull(sourceInterval.getValueAs(BigDecimal.class), "sourceValue"); final BigDecimal multiplier = intervalQualifier.getUnit().multiplier; - sourceValue = SqlFunctions.multiply(sourceValue, multiplier); RexLiteral castedInterval = - cx.getRexBuilder().makeIntervalLiteral(sourceValue, intervalQualifier); - return castToValidatedType(cx, call, castedInterval); + rexBuilder.makeIntervalLiteral( + SqlFunctions.multiply(sourceValue, multiplier), intervalQualifier); + return castToValidatedType(call, castedInterval, validator, rexBuilder, safe); } - return castToValidatedType(cx, call, cx.convertExpression(left)); - } - SqlDataTypeSpec dataType = (SqlDataTypeSpec) right; - RelDataType type = dataType.deriveType(cx.getValidator()); - if (type == null) { - type = cx.getValidator().getValidatedNodeType(dataType.getTypeName()); - } - RexNode arg = cx.convertExpression(left); - if (arg.getType().isNullable()) { - type = typeFactory.createTypeWithNullability(type, true); + RexNode value = cx.convertExpression(left); + return castToValidatedType(call, value, validator, rexBuilder, safe); } + + final RexNode arg = cx.convertExpression(left); + final SqlDataTypeSpec dataType = (SqlDataTypeSpec) right; + RelDataType type = + SqlCastFunction.deriveType( + cx.getTypeFactory(), arg.getType(), dataType.deriveType(validator), safe); if (SqlUtil.isNullLiteral(left, false)) { - final SqlValidatorImpl validator = (SqlValidatorImpl) cx.getValidator(); validator.setValidatedNodeType(left, type); return cx.convertExpression(left); } @@ -593,7 +672,7 @@ protected RexNode convertCast(SqlRexContext cx, final SqlCall call) { // arg.getType() may be ANY if (argComponentType == null) { - argComponentType = dataType.getComponentTypeSpec().deriveType(cx.getValidator()); + argComponentType = dataType.getComponentTypeSpec().deriveType(validator); } requireNonNull(argComponentType, () -> "componentType of " + arg); @@ -615,7 +694,7 @@ protected RexNode convertCast(SqlRexContext cx, final SqlCall call) { type = typeFactory.createTypeWithNullability(type, isn); } } - return cx.getRexBuilder().makeCast(type, arg); + return rexBuilder.makeCast(type, arg, safe, safe); } protected RexNode convertFloorCeil(SqlRexContext cx, SqlCall call) { @@ -655,6 +734,28 @@ protected RexNode convertFloorCeil(SqlRexContext cx, SqlCall call) { return convertFunction(cx, (SqlFunction) call.getOperator(), call); } + protected RexNode convertCharset(SqlRexContext cx, SqlCall call) { + final SqlNode expr = call.operand(0); + final String srcCharset = call.operand(1).toString(); + final String destCharset = call.operand(2).toString(); + final RexBuilder rexBuilder = cx.getRexBuilder(); + return rexBuilder.makeCall( + SqlStdOperatorTable.CONVERT, + cx.convertExpression(expr), + rexBuilder.makeLiteral(srcCharset), + rexBuilder.makeLiteral(destCharset)); + } + + protected RexNode translateCharset(SqlRexContext cx, SqlCall call) { + final SqlNode expr = call.operand(0); + final String transcodingName = call.operand(1).toString(); + final RexBuilder rexBuilder = cx.getRexBuilder(); + return rexBuilder.makeCall( + SqlStdOperatorTable.TRANSLATE, + cx.convertExpression(expr), + rexBuilder.makeLiteral(transcodingName)); + } + /** * Converts a call to the {@code EXTRACT} function. * @@ -1248,12 +1349,15 @@ private static Pair convertOverlapsOperand( return Pair.of(r0, r1); } - /** - * Casts a RexNode value to the validated type of a SqlCall. If the value was already of the - * validated type, then the value is returned without an additional cast. - */ + @Deprecated // to be removed before 2.0 public RexNode castToValidatedType(SqlRexContext cx, SqlCall call, RexNode value) { - return castToValidatedType(call, value, cx.getValidator(), cx.getRexBuilder()); + return castToValidatedType(call, value, cx.getValidator(), cx.getRexBuilder(), false); + } + + @Deprecated // to be removed before 2.0 + public static RexNode castToValidatedType( + SqlNode node, RexNode e, SqlValidator validator, RexBuilder rexBuilder) { + return castToValidatedType(node, e, validator, rexBuilder, false); } /** @@ -1261,12 +1365,12 @@ public RexNode castToValidatedType(SqlRexContext cx, SqlCall call, RexNode value * validated type, then the value is returned without an additional cast. */ public static RexNode castToValidatedType( - SqlNode node, RexNode e, SqlValidator validator, RexBuilder rexBuilder) { + SqlNode node, RexNode e, SqlValidator validator, RexBuilder rexBuilder, boolean safe) { final RelDataType type = validator.getValidatedNodeType(node); if (e.getType() == type) { return e; } - return rexBuilder.makeCast(type, e); + return rexBuilder.makeCast(type, e, safe, safe); } /** @@ -1842,7 +1946,7 @@ public RexNode convertCall(SqlRexContext cx, SqlCall call) { // If the TIMESTAMPADD call has type TIMESTAMP and op2 has type DATE // (which can happen for sub-day time frames such as HOUR), cast op2 to // TIMESTAMP. - final RexNode op2b = rexBuilder.makeCast(type, op2, false); + final RexNode op2b = rexBuilder.makeCast(type, op2); return rexBuilder.makeCall( type, SqlStdOperatorTable.TIMESTAMP_ADD, @@ -1880,6 +1984,25 @@ public RexNode convertCall(SqlRexContext cx, SqlCall call) { } } + /** + * Convertlet that handles the BigQuery {@code DATETIME_TRUNC} and {@code TIMESTAMP_TRUNC} + * functions. Ensures that DATE operands are cast to TIMESTAMPs to match the expected return + * type for BigQuery. + */ + private static class TruncConvertlet implements SqlRexConvertlet { + @Override + public RexNode convertCall(SqlRexContext cx, SqlCall call) { + final RexBuilder rexBuilder = cx.getRexBuilder(); + RexNode op1 = cx.convertExpression(call.operand(0)); + RexNode op2 = cx.convertExpression(call.operand(1)); + if (op1.getType().getSqlTypeName() == SqlTypeName.DATE) { + RelDataType type = cx.getValidator().getValidatedNodeType(call); + op1 = cx.getRexBuilder().makeCast(type, op1); + } + return rexBuilder.makeCall(call.getOperator(), op1, op2); + } + } + /** Convertlet that handles the BigQuery {@code TIMESTAMP_SUB} function. */ private static class TimestampSubConvertlet implements SqlRexConvertlet { @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/tools/RelBuilder.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/tools/RelBuilder.java deleted file mode 100644 index a2df489a9b633..0000000000000 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/tools/RelBuilder.java +++ /dev/null @@ -1,5258 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to you under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.calcite.tools; - -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.ImmutableSortedMultiset; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Multiset; -import com.google.common.collect.Sets; -import org.apache.calcite.linq4j.Ord; -import org.apache.calcite.linq4j.function.Experimental; -import org.apache.calcite.plan.Context; -import org.apache.calcite.plan.Contexts; -import org.apache.calcite.plan.Convention; -import org.apache.calcite.plan.RelOptCluster; -import org.apache.calcite.plan.RelOptPredicateList; -import org.apache.calcite.plan.RelOptSchema; -import org.apache.calcite.plan.RelOptTable; -import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.plan.ViewExpanders; -import org.apache.calcite.prepare.RelOptTableImpl; -import org.apache.calcite.rel.RelCollation; -import org.apache.calcite.rel.RelCollations; -import org.apache.calcite.rel.RelDistribution; -import org.apache.calcite.rel.RelFieldCollation; -import org.apache.calcite.rel.RelHomogeneousShuttle; -import org.apache.calcite.rel.RelNode; -import org.apache.calcite.rel.core.Aggregate; -import org.apache.calcite.rel.core.AggregateCall; -import org.apache.calcite.rel.core.Correlate; -import org.apache.calcite.rel.core.CorrelationId; -import org.apache.calcite.rel.core.Filter; -import org.apache.calcite.rel.core.Intersect; -import org.apache.calcite.rel.core.Join; -import org.apache.calcite.rel.core.JoinRelType; -import org.apache.calcite.rel.core.Match; -import org.apache.calcite.rel.core.Minus; -import org.apache.calcite.rel.core.Project; -import org.apache.calcite.rel.core.RelFactories; -import org.apache.calcite.rel.core.RepeatUnion; -import org.apache.calcite.rel.core.Snapshot; -import org.apache.calcite.rel.core.Sort; -import org.apache.calcite.rel.core.Spool; -import org.apache.calcite.rel.core.TableFunctionScan; -import org.apache.calcite.rel.core.TableScan; -import org.apache.calcite.rel.core.TableSpool; -import org.apache.calcite.rel.core.Uncollect; -import org.apache.calcite.rel.core.Union; -import org.apache.calcite.rel.core.Values; -import org.apache.calcite.rel.hint.Hintable; -import org.apache.calcite.rel.hint.RelHint; -import org.apache.calcite.rel.metadata.RelColumnMapping; -import org.apache.calcite.rel.metadata.RelMetadataQuery; -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.RelDataTypeFieldImpl; -import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexCall; -import org.apache.calcite.rex.RexCallBinding; -import org.apache.calcite.rex.RexCorrelVariable; -import org.apache.calcite.rex.RexDynamicParam; -import org.apache.calcite.rex.RexExecutor; -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.RexShuttle; -import org.apache.calcite.rex.RexSimplify; -import org.apache.calcite.rex.RexSubQuery; -import org.apache.calcite.rex.RexUtil; -import org.apache.calcite.rex.RexWindowBound; -import org.apache.calcite.rex.RexWindowBounds; -import org.apache.calcite.runtime.Hook; -import org.apache.calcite.schema.TransientTable; -import org.apache.calcite.schema.impl.ListTransientTable; -import org.apache.calcite.sql.SqlAggFunction; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlOperator; -import org.apache.calcite.sql.SqlUtil; -import org.apache.calcite.sql.SqlWindow; -import org.apache.calcite.sql.fun.SqlCountAggFunction; -import org.apache.calcite.sql.fun.SqlLikeOperator; -import org.apache.calcite.sql.fun.SqlQuantifyOperator; -import org.apache.calcite.sql.fun.SqlStdOperatorTable; -import org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.calcite.sql.type.SqlTypeName; -import org.apache.calcite.sql.type.TableFunctionReturnTypeInference; -import org.apache.calcite.sql.validate.SqlValidatorUtil; -import org.apache.calcite.sql2rel.SqlToRelConverter; -import org.apache.calcite.util.DateString; -import org.apache.calcite.util.Holder; -import org.apache.calcite.util.ImmutableBitSet; -import org.apache.calcite.util.ImmutableIntList; -import org.apache.calcite.util.ImmutableNullableList; -import org.apache.calcite.util.Litmus; -import org.apache.calcite.util.NlsString; -import org.apache.calcite.util.Optionality; -import org.apache.calcite.util.Pair; -import org.apache.calcite.util.Util; -import org.apache.calcite.util.mapping.Mapping; -import org.apache.calcite.util.mapping.Mappings; -import org.checkerframework.checker.nullness.qual.MonotonicNonNull; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.immutables.value.Value; - -import java.math.BigDecimal; -import java.util.AbstractList; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.BitSet; -import java.util.Collections; -import java.util.Deque; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; -import java.util.function.BiFunction; -import java.util.function.Function; -import java.util.function.UnaryOperator; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; - -import static java.util.Objects.requireNonNull; -import static org.apache.calcite.linq4j.Nullness.castNonNull; -import static org.apache.calcite.sql.SqlKind.UNION; -import static org.apache.calcite.util.Static.RESOURCE; - -/** - * Copied from calcite to workaround CALCITE-4668 - * - *

FLINK modifications are at lines - * - *

    - *
  1. Should be removed after fix of FLINK-29804: Lines 3000 ~ 3003 - *
- */ -@Value.Enclosing -public class RelBuilder { - protected final RelOptCluster cluster; - protected final @Nullable RelOptSchema relOptSchema; - private final Deque stack = new ArrayDeque<>(); - private RexSimplify simplifier; - private final Config config; - private final RelOptTable.ViewExpander viewExpander; - private RelFactories.Struct struct; - - protected RelBuilder( - @Nullable Context context, RelOptCluster cluster, @Nullable RelOptSchema relOptSchema) { - this.cluster = cluster; - this.relOptSchema = relOptSchema; - if (context == null) { - context = Contexts.EMPTY_CONTEXT; - } - this.config = getConfig(context); - this.viewExpander = getViewExpander(cluster, context); - this.struct = requireNonNull(RelFactories.Struct.fromContext(context)); - final RexExecutor executor = - context.maybeUnwrap(RexExecutor.class) - .orElse(Util.first(cluster.getPlanner().getExecutor(), RexUtil.EXECUTOR)); - final RelOptPredicateList predicates = RelOptPredicateList.EMPTY; - this.simplifier = new RexSimplify(cluster.getRexBuilder(), predicates, executor); - } - - /** - * Derives the view expander {@link org.apache.calcite.plan.RelOptTable.ViewExpander} to be used - * for this RelBuilder. - * - *

The ViewExpander instance is used for expanding views in the default table scan factory - * {@code RelFactories.TableScanFactoryImpl}. You can also define a new table scan factory in - * the {@code struct} to override the whole table scan creation. - * - *

The default view expander does not support expanding views. - */ - private static RelOptTable.ViewExpander getViewExpander( - RelOptCluster cluster, Context context) { - return context.maybeUnwrap(RelOptTable.ViewExpander.class) - .orElseGet(() -> ViewExpanders.simpleContext(cluster)); - } - - /** - * Derives the Config to be used for this RelBuilder. - * - *

Overrides {@link RelBuilder.Config#simplify} if {@link Hook#REL_BUILDER_SIMPLIFY} is set. - */ - private static Config getConfig(Context context) { - final Config config = context.maybeUnwrap(Config.class).orElse(Config.DEFAULT); - boolean simplify = Hook.REL_BUILDER_SIMPLIFY.get(config.simplify()); - return config.withSimplify(simplify); - } - - /** Creates a RelBuilder. */ - public static RelBuilder create(FrameworkConfig config) { - return Frameworks.withPrepare( - config, - (cluster, relOptSchema, rootSchema, statement) -> - new RelBuilder(config.getContext(), cluster, relOptSchema)); - } - - /** - * Creates a copy of this RelBuilder, with the same state as this, applying a transform to the - * config. - */ - public RelBuilder transform(UnaryOperator transform) { - final Context context = Contexts.of(struct, transform.apply(config)); - return new RelBuilder(context, cluster, relOptSchema); - } - - /** - * Performs an action on this RelBuilder. - * - *

For example, consider the following code: - * - *

- * - *
-     *   RelNode filterAndRename(RelBuilder relBuilder, RelNode rel,
-     *       RexNode condition, List<String> fieldNames) {
-     *     relBuilder.push(rel)
-     *         .filter(condition);
-     *     if (fieldNames != null) {
-     *       relBuilder.rename(fieldNames);
-     *     }
-     *     return relBuilder
-     *         .build();
- * - *
- * - *

The pipeline is disrupted by the 'if'. The {@code let} method allows you to perform the - * flow as a single pipeline: - * - *

- * - *
-     *   RelNode filterAndRename(RelBuilder relBuilder, RelNode rel,
-     *       RexNode condition, List<String> fieldNames) {
-     *     return relBuilder.push(rel)
-     *         .filter(condition)
-     *         .let(r -> fieldNames == null ? r : r.rename(fieldNames))
-     *         .build();
- * - *
- * - *

In pipelined cases such as this one, the lambda must return this RelBuilder. But {@code - * let} return values of other types. - */ - public R let(Function consumer) { - return consumer.apply(this); - } - - /** - * Converts this RelBuilder to a string. The string is the string representation of all of the - * RelNodes on the stack. - */ - @Override - public String toString() { - return stack.stream() - .map(frame -> RelOptUtil.toString(frame.rel)) - .collect(Collectors.joining("")); - } - - /** Returns the type factory. */ - public RelDataTypeFactory getTypeFactory() { - return cluster.getTypeFactory(); - } - - /** - * Returns new RelBuilder that adopts the convention provided. RelNode will be created with such - * convention if corresponding factory is provided. - */ - public RelBuilder adoptConvention(Convention convention) { - this.struct = convention.getRelFactories(); - return this; - } - - /** Returns the builder for {@link RexNode} expressions. */ - public RexBuilder getRexBuilder() { - return cluster.getRexBuilder(); - } - - /** - * Creates a {@link RelBuilderFactory}, a partially-created RelBuilder. Just add a {@link - * RelOptCluster} and a {@link RelOptSchema} - */ - public static RelBuilderFactory proto(final Context context) { - return (cluster, schema) -> new RelBuilder(context, cluster, schema); - } - - /** Creates a {@link RelBuilderFactory} that uses a given set of factories. */ - public static RelBuilderFactory proto(Object... factories) { - return proto(Contexts.of(factories)); - } - - public RelOptCluster getCluster() { - return cluster; - } - - public @Nullable RelOptSchema getRelOptSchema() { - return relOptSchema; - } - - public RelFactories.TableScanFactory getScanFactory() { - return struct.scanFactory; - } - - // Methods for manipulating the stack - - /** - * Adds a relational expression to be the input to the next relational expression constructed. - * - *

This method is usual when you want to weave in relational expressions that are not - * supported by the builder. If, while creating such expressions, you need to use previously - * built expressions as inputs, call {@link #build()} to pop those inputs. - */ - public RelBuilder push(RelNode node) { - stack.push(new Frame(node)); - return this; - } - - /** Adds a rel node to the top of the stack while preserving the field names and aliases. */ - private void replaceTop(RelNode node) { - final Frame frame = stack.pop(); - stack.push(new Frame(node, frame.fields)); - } - - /** Pushes a collection of relational expressions. */ - public RelBuilder pushAll(Iterable nodes) { - for (RelNode node : nodes) { - push(node); - } - return this; - } - - /** Returns the size of the stack. */ - public int size() { - return stack.size(); - } - - /** - * Returns the final relational expression. - * - *

Throws if the stack is empty. - */ - public RelNode build() { - return stack.pop().rel; - } - - /** Returns the relational expression at the top of the stack, but does not remove it. */ - public RelNode peek() { - return castNonNull(peek_()).rel; - } - - private @Nullable Frame peek_() { - return stack.peek(); - } - - /** - * Returns the relational expression {@code n} positions from the top of the stack, but does not - * remove it. - */ - public RelNode peek(int n) { - return peek_(n).rel; - } - - private Frame peek_(int n) { - if (n == 0) { - // more efficient than starting an iterator - return Objects.requireNonNull(stack.peek(), "stack.peek"); - } - return Iterables.get(stack, n); - } - - /** - * Returns the relational expression {@code n} positions from the top of the stack, but does not - * remove it. - */ - public RelNode peek(int inputCount, int inputOrdinal) { - return peek_(inputCount, inputOrdinal).rel; - } - - private Frame peek_(int inputCount, int inputOrdinal) { - return peek_(inputCount - 1 - inputOrdinal); - } - - /** - * Returns the number of fields in all inputs before (to the left of) the given input. - * - * @param inputCount Number of inputs - * @param inputOrdinal Input ordinal - */ - private int inputOffset(int inputCount, int inputOrdinal) { - int offset = 0; - for (int i = 0; i < inputOrdinal; i++) { - offset += peek(inputCount, i).getRowType().getFieldCount(); - } - return offset; - } - - /** Evaluates an expression with a relational expression temporarily on the stack. */ - public E with(RelNode r, Function fn) { - try { - push(r); - return fn.apply(this); - } finally { - stack.pop(); - } - } - - /** Performs an action with a temporary simplifier. */ - public E withSimplifier( - BiFunction simplifierTransform, - Function fn) { - final RexSimplify previousSimplifier = this.simplifier; - try { - this.simplifier = simplifierTransform.apply(this, previousSimplifier); - return fn.apply(this); - } finally { - this.simplifier = previousSimplifier; - } - } - - /** Performs an action using predicates of the {@link #peek() current node} to simplify. */ - public E withPredicates(RelMetadataQuery mq, Function fn) { - final RelOptPredicateList predicates = mq.getPulledUpPredicates(peek()); - return withSimplifier((r, s) -> s.withPredicates(predicates), fn); - } - - // Methods that return scalar expressions - - /** Creates a literal (constant expression). */ - public RexLiteral literal(@Nullable Object value) { - final RexBuilder rexBuilder = cluster.getRexBuilder(); - if (value == null) { - final RelDataType type = getTypeFactory().createSqlType(SqlTypeName.NULL); - return rexBuilder.makeNullLiteral(type); - } else if (value instanceof Boolean) { - return rexBuilder.makeLiteral((Boolean) value); - } else if (value instanceof BigDecimal) { - return rexBuilder.makeExactLiteral((BigDecimal) value); - } else if (value instanceof Float || value instanceof Double) { - return rexBuilder.makeApproxLiteral(BigDecimal.valueOf(((Number) value).doubleValue())); - } else if (value instanceof Number) { - return rexBuilder.makeExactLiteral(BigDecimal.valueOf(((Number) value).longValue())); - } else if (value instanceof String) { - return rexBuilder.makeLiteral((String) value); - } else if (value instanceof Enum) { - return rexBuilder.makeLiteral( - value, getTypeFactory().createSqlType(SqlTypeName.SYMBOL)); - } else if (value instanceof DateString) { - return rexBuilder.makeDateLiteral((DateString) value); - } else { - throw new IllegalArgumentException( - "cannot convert " + value + " (" + value.getClass() + ") to a constant"); - } - } - - /** Creates a correlation variable for the current input, and writes it into a Holder. */ - public RelBuilder variable(Holder v) { - v.set( - (RexCorrelVariable) - getRexBuilder().makeCorrel(peek().getRowType(), cluster.createCorrel())); - return this; - } - - /** - * Creates a reference to a field by name. - * - *

Equivalent to {@code field(1, 0, fieldName)}. - * - * @param fieldName Field name - */ - public RexInputRef field(String fieldName) { - return field(1, 0, fieldName); - } - - /** - * Creates a reference to a field of given input relational expression by name. - * - * @param inputCount Number of inputs - * @param inputOrdinal Input ordinal - * @param fieldName Field name - */ - public RexInputRef field(int inputCount, int inputOrdinal, String fieldName) { - final Frame frame = peek_(inputCount, inputOrdinal); - final List fieldNames = Pair.left(frame.fields()); - int i = fieldNames.indexOf(fieldName); - if (i >= 0) { - return field(inputCount, inputOrdinal, i); - } else { - throw new IllegalArgumentException( - "field [" + fieldName + "] not found; input fields are: " + fieldNames); - } - } - - /** - * Creates a reference to an input field by ordinal. - * - *

Equivalent to {@code field(1, 0, ordinal)}. - * - * @param fieldOrdinal Field ordinal - */ - public RexInputRef field(int fieldOrdinal) { - return (RexInputRef) field(1, 0, fieldOrdinal, false); - } - - /** - * Creates a reference to a field of a given input relational expression by ordinal. - * - * @param inputCount Number of inputs - * @param inputOrdinal Input ordinal - * @param fieldOrdinal Field ordinal within input - */ - public RexInputRef field(int inputCount, int inputOrdinal, int fieldOrdinal) { - return (RexInputRef) field(inputCount, inputOrdinal, fieldOrdinal, false); - } - - /** - * As {@link #field(int, int, int)}, but if {@code alias} is true, the method may apply an alias - * to make sure that the field has the same name as in the input frame. If no alias is applied - * the expression is definitely a {@link RexInputRef}. - */ - private RexNode field(int inputCount, int inputOrdinal, int fieldOrdinal, boolean alias) { - final Frame frame = peek_(inputCount, inputOrdinal); - final RelNode input = frame.rel; - final RelDataType rowType = input.getRowType(); - if (fieldOrdinal < 0 || fieldOrdinal > rowType.getFieldCount()) { - throw new IllegalArgumentException( - "field ordinal [" - + fieldOrdinal - + "] out of range; input fields are: " - + rowType.getFieldNames()); - } - final RelDataTypeField field = rowType.getFieldList().get(fieldOrdinal); - final int offset = inputOffset(inputCount, inputOrdinal); - final RexInputRef ref = - cluster.getRexBuilder().makeInputRef(field.getType(), offset + fieldOrdinal); - final RelDataTypeField aliasField = frame.fields().get(fieldOrdinal); - if (!alias || field.getName().equals(aliasField.getName())) { - return ref; - } else { - return alias(ref, aliasField.getName()); - } - } - - /** - * Creates a reference to a field of the current record which originated in a relation with a - * given alias. - */ - public RexNode field(String alias, String fieldName) { - return field(1, alias, fieldName); - } - - /** - * Creates a reference to a field which originated in a relation with the given alias. Searches - * for the relation starting at the top of the stack. - */ - public RexNode field(int inputCount, String alias, String fieldName) { - requireNonNull(alias, "alias"); - requireNonNull(fieldName, "fieldName"); - final List fields = new ArrayList<>(); - for (int inputOrdinal = 0; inputOrdinal < inputCount; ++inputOrdinal) { - final Frame frame = peek_(inputOrdinal); - for (Ord p : Ord.zip(frame.fields)) { - // If alias and field name match, reference that field. - if (p.e.left.contains(alias) && p.e.right.getName().equals(fieldName)) { - return field(inputCount, inputCount - 1 - inputOrdinal, p.i); - } - fields.add( - String.format( - Locale.ROOT, - "{aliases=%s,fieldName=%s}", - p.e.left, - p.e.right.getName())); - } - } - throw new IllegalArgumentException( - "{alias=" - + alias - + ",fieldName=" - + fieldName - + "} " - + "field not found; fields are: " - + fields); - } - - /** Returns a reference to a given field of a record-valued expression. */ - public RexNode field(RexNode e, String name) { - return getRexBuilder().makeFieldAccess(e, name, false); - } - - /** Returns references to the fields of the top input. */ - public ImmutableList fields() { - return fields(1, 0); - } - - /** Returns references to the fields of a given input. */ - public ImmutableList fields(int inputCount, int inputOrdinal) { - final RelNode input = peek(inputCount, inputOrdinal); - final RelDataType rowType = input.getRowType(); - final ImmutableList.Builder nodes = ImmutableList.builder(); - for (int fieldOrdinal : Util.range(rowType.getFieldCount())) { - nodes.add(field(inputCount, inputOrdinal, fieldOrdinal)); - } - return nodes.build(); - } - - /** Returns references to fields for a given collation. */ - public ImmutableList fields(RelCollation collation) { - final ImmutableList.Builder nodes = ImmutableList.builder(); - for (RelFieldCollation fieldCollation : collation.getFieldCollations()) { - RexNode node = field(fieldCollation.getFieldIndex()); - switch (fieldCollation.direction) { - case DESCENDING: - node = desc(node); - break; - default: - break; - } - switch (fieldCollation.nullDirection) { - case FIRST: - node = nullsFirst(node); - break; - case LAST: - node = nullsLast(node); - break; - default: - break; - } - nodes.add(node); - } - return nodes.build(); - } - - /** Returns references to fields for a given list of input ordinals. */ - public ImmutableList fields(List ordinals) { - final ImmutableList.Builder nodes = ImmutableList.builder(); - for (Number ordinal : ordinals) { - RexNode node = field(1, 0, ordinal.intValue(), false); - nodes.add(node); - } - return nodes.build(); - } - - /** Returns references to fields for a given bit set of input ordinals. */ - public ImmutableList fields(ImmutableBitSet ordinals) { - return fields(ordinals.asList()); - } - - /** Returns references to fields identified by name. */ - public ImmutableList fields(Iterable fieldNames) { - final ImmutableList.Builder builder = ImmutableList.builder(); - for (String fieldName : fieldNames) { - builder.add(field(fieldName)); - } - return builder.build(); - } - - /** Returns references to fields identified by a mapping. */ - public ImmutableList fields(Mappings.TargetMapping mapping) { - return fields(Mappings.asListNonNull(mapping)); - } - - /** Creates an access to a field by name. */ - public RexNode dot(RexNode node, String fieldName) { - final RexBuilder builder = cluster.getRexBuilder(); - return builder.makeFieldAccess(node, fieldName, true); - } - - /** Creates an access to a field by ordinal. */ - public RexNode dot(RexNode node, int fieldOrdinal) { - final RexBuilder builder = cluster.getRexBuilder(); - return builder.makeFieldAccess(node, fieldOrdinal); - } - - /** Creates a call to a scalar operator. */ - public RexNode call(SqlOperator operator, RexNode... operands) { - return call(operator, ImmutableList.copyOf(operands)); - } - - /** Creates a call to a scalar operator. */ - private RexCall call(SqlOperator operator, List operandList) { - switch (operator.getKind()) { - case LIKE: - case SIMILAR: - final SqlLikeOperator likeOperator = (SqlLikeOperator) operator; - if (likeOperator.isNegated()) { - final SqlOperator notLikeOperator = likeOperator.not(); - return (RexCall) not(call(notLikeOperator, operandList)); - } - break; - case BETWEEN: - assert operandList.size() == 3; - return (RexCall) - between(operandList.get(0), operandList.get(1), operandList.get(2)); - default: - break; - } - final RexBuilder builder = cluster.getRexBuilder(); - final RelDataType type = builder.deriveReturnType(operator, operandList); - return (RexCall) builder.makeCall(type, operator, operandList); - } - - /** Creates a call to a scalar operator. */ - public RexNode call(SqlOperator operator, Iterable operands) { - return call(operator, ImmutableList.copyOf(operands)); - } - - /** - * Creates an IN predicate with a list of values. - * - *

For example, - * - *

{@code
-     * b.scan("Emp")
-     *     .filter(b.in(b.field("deptno"), b.literal(10), b.literal(20)))
-     * }
- * - * is equivalent to SQL - * - *
{@code
-     * SELECT *
-     * FROM Emp
-     * WHERE deptno IN (10, 20)
-     * }
- */ - public RexNode in(RexNode arg, RexNode... ranges) { - return in(arg, ImmutableList.copyOf(ranges)); - } - - /** - * Creates an IN predicate with a list of values. - * - *

For example, - * - *

{@code
-     * b.scan("Emps")
-     *     .filter(
-     *         b.in(b.field("deptno"),
-     *             Arrays.asList(b.literal(10), b.literal(20))))
-     * }
- * - *

is equivalent to the SQL - * - *

{@code
-     * SELECT *
-     * FROM Emps
-     * WHERE deptno IN (10, 20)
-     * }
- */ - public RexNode in(RexNode arg, Iterable ranges) { - return getRexBuilder().makeIn(arg, ImmutableList.copyOf(ranges)); - } - - /** Creates an IN predicate with a sub-query. */ - @Experimental - public RexSubQuery in(RelNode rel, Iterable nodes) { - return RexSubQuery.in(rel, ImmutableList.copyOf(nodes)); - } - - /** - * Creates an IN predicate with a sub-query. - * - *

For example, - * - *

{@code
-     * b.scan("Emps")
-     *     .filter(
-     *         b.in(b.field("deptno"),
-     *             b2 -> b2.scan("Depts")
-     *                 .filter(
-     *                     b2.eq(b2.field("location"), b2.literal("Boston")))
-     *                 .project(b.field("deptno"))
-     *                 .build()))
-     * }
- * - *

is equivalent to the SQL - * - *

{@code
-     * SELECT *
-     * FROM Emps
-     * WHERE deptno IN (SELECT deptno FROM Dept WHERE location = 'Boston')
-     * }
- */ - @Experimental - public RexNode in(RexNode arg, Function f) { - final RelNode rel = f.apply(this); - return RexSubQuery.in(rel, ImmutableList.of(arg)); - } - - /** - * Creates a SOME (or ANY) predicate. - * - *

For example, - * - *

{@code
-     * b.scan("Emps")
-     *     .filter(
-     *         b.some(b.field("commission"),
-     *             SqlStdOperatorTable.GREATER_THAN,
-     *             b2 -> b2.scan("Emps")
-     *                 .filter(
-     *                     b2.eq(b2.field("job"), b2.literal("Manager")))
-     *                 .project(b2.field("sal"))
-     *                 .build()))
-     * }
- * - *

is equivalent to the SQL - * - *

{@code
-     * SELECT *
-     * FROM Emps
-     * WHERE commission > SOME (SELECT sal FROM Emps WHERE job = 'Manager')
-     * }
- * - *

or (since {@code SOME} and {@code ANY} are synonyms) the SQL - * - *

{@code
-     * SELECT *
-     * FROM Emps
-     * WHERE commission > ANY (SELECT sal FROM Emps WHERE job = 'Manager')
-     * }
- */ - @Experimental - public RexSubQuery some(RexNode node, SqlOperator op, Function f) { - return some_(node, op.kind, f); - } - - private RexSubQuery some_(RexNode node, SqlKind kind, Function f) { - final RelNode rel = f.apply(this); - final SqlQuantifyOperator quantifyOperator = SqlStdOperatorTable.some(kind); - return RexSubQuery.some(rel, ImmutableList.of(node), quantifyOperator); - } - - /** - * Creates an ALL predicate. - * - *

For example, - * - *

{@code
-     * b.scan("Emps")
-     *     .filter(
-     *         b.all(b.field("commission"),
-     *             SqlStdOperatorTable.GREATER_THAN,
-     *             b2 -> b2.scan("Emps")
-     *                 .filter(
-     *                     b2.eq(b2.field("job"), b2.literal("Manager")))
-     *                 .project(b2.field("sal"))
-     *                 .build()))
-     * }
- * - *

is equivalent to the SQL - * - *

{@code
-     * SELECT *
-     * FROM Emps
-     * WHERE commission > ALL (SELECT sal FROM Emps WHERE job = 'Manager')
-     * }
- * - *

Calcite translates {@code ALL} predicates to {@code NOT SOME}. The following SQL is - * equivalent to the previous: - * - *

{@code
-     * SELECT *
-     * FROM Emps
-     * WHERE NOT (commission <= SOME (SELECT sal FROM Emps WHERE job = 'Manager'))
-     * }
- */ - @Experimental - public RexNode all(RexNode node, SqlOperator op, Function f) { - return not(some_(node, op.kind.negateNullSafe(), f)); - } - - /** - * Creates an EXISTS predicate. - * - *

For example, - * - *

{@code
-     * b.scan("Depts")
-     *     .filter(
-     *         b.exists(b2 ->
-     *             b2.scan("Emps")
-     *                 .filter(
-     *                     b2.eq(b2.field("job"), b2.literal("Manager")))
-     *                 .build()))
-     * }
- * - *

is equivalent to the SQL - * - *

{@code
-     * SELECT *
-     * FROM Depts
-     * WHERE EXISTS (SELECT 1 FROM Emps WHERE job = 'Manager')
-     * }
- */ - @Experimental - public RexSubQuery exists(Function f) { - final RelNode rel = f.apply(this); - return RexSubQuery.exists(rel); - } - - /** - * Creates a UNIQUE predicate. - * - *

For example, - * - *

{@code
-     * b.scan("Depts")
-     *     .filter(
-     *         b.exists(b2 ->
-     *             b2.scan("Emps")
-     *                 .filter(
-     *                     b2.eq(b2.field("job"), b2.literal("Manager")))
-     *                 .project(b2.field("deptno")
-     *                 .build()))
-     * }
- * - *

is equivalent to the SQL - * - *

{@code
-     * SELECT *
-     * FROM Depts
-     * WHERE UNIQUE (SELECT deptno FROM Emps WHERE job = 'Manager')
-     * }
- */ - @Experimental - public RexSubQuery unique(Function f) { - final RelNode rel = f.apply(this); - return RexSubQuery.unique(rel); - } - - /** - * Creates a scalar sub-query. - * - *

For example, - * - *

{@code
-     * b.scan("Depts")
-     *     .project(
-     *         b.field("deptno")
-     *         b.scalarQuery(b2 ->
-     *             b2.scan("Emps")
-     *                 .aggregate(
-     *                     b2.eq(b2.field("job"), b2.literal("Manager")))
-     *                 .build()))
-     * }
- * - *

is equivalent to the SQL - * - *

{@code
-     * SELECT deptno, (SELECT MAX(sal) FROM Emps)
-     * FROM Depts
-     * }
- */ - @Experimental - public RexSubQuery scalarQuery(Function f) { - return RexSubQuery.scalar(f.apply(this)); - } - - /** - * Creates an ARRAY sub-query. - * - *

For example, - * - *

{@code
-     * b.scan("Depts")
-     *     .project(
-     *         b.field("deptno")
-     *         b.arrayQuery(b2 ->
-     *             b2.scan("Emps")
-     *                 .build()))
-     * }
- * - *

is equivalent to the SQL - * - *

{@code
-     * SELECT deptno, ARRAY (SELECT * FROM Emps)
-     * FROM Depts
-     * }
- */ - @Experimental - public RexSubQuery arrayQuery(Function f) { - return RexSubQuery.array(f.apply(this)); - } - - /** - * Creates a MULTISET sub-query. - * - *

For example, - * - *

{@code
-     * b.scan("Depts")
-     *     .project(
-     *         b.field("deptno")
-     *         b.multisetQuery(b2 ->
-     *             b2.scan("Emps")
-     *                 .build()))
-     * }
- * - *

is equivalent to the SQL - * - *

{@code
-     * SELECT deptno, MULTISET (SELECT * FROM Emps)
-     * FROM Depts
-     * }
- */ - @Experimental - public RexSubQuery multisetQuery(Function f) { - return RexSubQuery.multiset(f.apply(this)); - } - - /** - * Creates a MAP sub-query. - * - *

For example, - * - *

{@code
-     * b.scan("Depts")
-     *     .project(
-     *         b.field("deptno")
-     *         b.multisetQuery(b2 ->
-     *             b2.scan("Emps")
-     *                 .project(b2.field("empno"), b2.field("job"))
-     *                 .build()))
-     * }
- * - *

is equivalent to the SQL - * - *

{@code
-     * SELECT deptno, MAP (SELECT empno, job FROM Emps)
-     * FROM Depts
-     * }
- */ - @Experimental - public RexSubQuery mapQuery(Function f) { - return RexSubQuery.map(f.apply(this)); - } - - /** Creates an AND. */ - public RexNode and(RexNode... operands) { - return and(ImmutableList.copyOf(operands)); - } - - /** - * Creates an AND. - * - *

Simplifies the expression a little: {@code e AND TRUE} becomes {@code e}; {@code e AND e2 - * AND NOT e} becomes {@code e2}. - */ - public RexNode and(Iterable operands) { - return RexUtil.composeConjunction(getRexBuilder(), operands); - } - - /** Creates an OR. */ - public RexNode or(RexNode... operands) { - return or(ImmutableList.copyOf(operands)); - } - - /** Creates an OR. */ - public RexNode or(Iterable operands) { - return RexUtil.composeDisjunction(cluster.getRexBuilder(), operands); - } - - /** Creates a NOT. */ - public RexNode not(RexNode operand) { - return call(SqlStdOperatorTable.NOT, operand); - } - - /** Creates an {@code =}. */ - public RexNode equals(RexNode operand0, RexNode operand1) { - return call(SqlStdOperatorTable.EQUALS, operand0, operand1); - } - - /** Creates a {@code >}. */ - public RexNode greaterThan(RexNode operand0, RexNode operand1) { - return call(SqlStdOperatorTable.GREATER_THAN, operand0, operand1); - } - - /** Creates a {@code >=}. */ - public RexNode greaterThanOrEqual(RexNode operand0, RexNode operand1) { - return call(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL, operand0, operand1); - } - - /** Creates a {@code <}. */ - public RexNode lessThan(RexNode operand0, RexNode operand1) { - return call(SqlStdOperatorTable.LESS_THAN, operand0, operand1); - } - - /** Creates a {@code <=}. */ - public RexNode lessThanOrEqual(RexNode operand0, RexNode operand1) { - return call(SqlStdOperatorTable.LESS_THAN_OR_EQUAL, operand0, operand1); - } - - /** Creates a {@code <>}. */ - public RexNode notEquals(RexNode operand0, RexNode operand1) { - return call(SqlStdOperatorTable.NOT_EQUALS, operand0, operand1); - } - - /** - * Creates an expression equivalent to "{@code o0 IS NOT DISTINCT FROM o1}". It is also - * equivalent to "{@code o0 = o1 OR (o0 IS NULL AND o1 IS NULL)}". - */ - public RexNode isNotDistinctFrom(RexNode operand0, RexNode operand1) { - return RelOptUtil.isDistinctFrom(getRexBuilder(), operand0, operand1, true); - } - - /** - * Creates an expression equivalent to {@code o0 IS DISTINCT FROM o1}. It is also equivalent to - * "{@code NOT (o0 = o1 OR (o0 IS NULL AND o1 IS NULL))}. - */ - public RexNode isDistinctFrom(RexNode operand0, RexNode operand1) { - return RelOptUtil.isDistinctFrom(getRexBuilder(), operand0, operand1, false); - } - - /** Creates a {@code BETWEEN}. */ - public RexNode between(RexNode arg, RexNode lower, RexNode upper) { - return getRexBuilder().makeBetween(arg, lower, upper); - } - - /** Creates ab {@code IS NULL}. */ - public RexNode isNull(RexNode operand) { - return call(SqlStdOperatorTable.IS_NULL, operand); - } - - /** Creates an {@code IS NOT NULL}. */ - public RexNode isNotNull(RexNode operand) { - return call(SqlStdOperatorTable.IS_NOT_NULL, operand); - } - - /** Creates an expression that casts an expression to a given type. */ - public RexNode cast(RexNode expr, SqlTypeName typeName) { - final RelDataType type = cluster.getTypeFactory().createSqlType(typeName); - return cluster.getRexBuilder().makeCast(type, expr); - } - - /** - * Creates an expression that casts an expression to a type with a given name and precision or - * length. - */ - public RexNode cast(RexNode expr, SqlTypeName typeName, int precision) { - final RelDataType type = cluster.getTypeFactory().createSqlType(typeName, precision); - return cluster.getRexBuilder().makeCast(type, expr); - } - - /** - * Creates an expression that casts an expression to a type with a given name, precision and - * scale. - */ - public RexNode cast(RexNode expr, SqlTypeName typeName, int precision, int scale) { - final RelDataType type = cluster.getTypeFactory().createSqlType(typeName, precision, scale); - return cluster.getRexBuilder().makeCast(type, expr); - } - - /** - * Returns an expression wrapped in an alias. - * - *

This method is idempotent: If the expression is already wrapped in the correct alias, does - * nothing; if wrapped in an incorrect alias, removes the incorrect alias and applies the - * correct alias. - * - * @see #project - */ - public RexNode alias(RexNode expr, String alias) { - final RexNode aliasLiteral = literal(alias); - switch (expr.getKind()) { - case AS: - final RexCall call = (RexCall) expr; - if (call.operands.get(1).equals(aliasLiteral)) { - // current alias is correct - return expr; - } - expr = call.operands.get(0); - // strip current (incorrect) alias, and fall through - default: - return call(SqlStdOperatorTable.AS, expr, aliasLiteral); - } - } - - /** Converts a sort expression to descending. */ - public RexNode desc(RexNode node) { - return call(SqlStdOperatorTable.DESC, node); - } - - /** Converts a sort expression to nulls last. */ - public RexNode nullsLast(RexNode node) { - return call(SqlStdOperatorTable.NULLS_LAST, node); - } - - /** Converts a sort expression to nulls first. */ - public RexNode nullsFirst(RexNode node) { - return call(SqlStdOperatorTable.NULLS_FIRST, node); - } - - // Methods that create window bounds - - /** - * Creates an {@code UNBOUNDED PRECEDING} window bound, for use in methods such as {@link - * OverCall#rowsFrom(RexWindowBound)} and {@link OverCall#rangeBetween(RexWindowBound, - * RexWindowBound)}. - */ - public RexWindowBound unboundedPreceding() { - return RexWindowBounds.UNBOUNDED_PRECEDING; - } - - /** - * Creates a {@code bound PRECEDING} window bound, for use in methods such as {@link - * OverCall#rowsFrom(RexWindowBound)} and {@link OverCall#rangeBetween(RexWindowBound, - * RexWindowBound)}. - */ - public RexWindowBound preceding(RexNode bound) { - return RexWindowBounds.preceding(bound); - } - - /** - * Creates a {@code CURRENT ROW} window bound, for use in methods such as {@link - * OverCall#rowsFrom(RexWindowBound)} and {@link OverCall#rangeBetween(RexWindowBound, - * RexWindowBound)}. - */ - public RexWindowBound currentRow() { - return RexWindowBounds.CURRENT_ROW; - } - - /** - * Creates a {@code bound FOLLOWING} window bound, for use in methods such as {@link - * OverCall#rowsFrom(RexWindowBound)} and {@link OverCall#rangeBetween(RexWindowBound, - * RexWindowBound)}. - */ - public RexWindowBound following(RexNode bound) { - return RexWindowBounds.following(bound); - } - - /** - * Creates an {@code UNBOUNDED FOLLOWING} window bound, for use in methods such as {@link - * OverCall#rowsFrom(RexWindowBound)} and {@link OverCall#rangeBetween(RexWindowBound, - * RexWindowBound)}. - */ - public RexWindowBound unboundedFollowing() { - return RexWindowBounds.UNBOUNDED_FOLLOWING; - } - - // Methods that create group keys and aggregate calls - - /** Creates an empty group key. */ - public GroupKey groupKey() { - return groupKey(ImmutableList.of()); - } - - /** Creates a group key. */ - public GroupKey groupKey(RexNode... nodes) { - return groupKey(ImmutableList.copyOf(nodes)); - } - - /** Creates a group key. */ - public GroupKey groupKey(Iterable nodes) { - return new GroupKeyImpl(ImmutableList.copyOf(nodes), null, null); - } - - /** Creates a group key with grouping sets. */ - public GroupKey groupKey( - Iterable nodes, - Iterable> nodeLists) { - return groupKey_(nodes, nodeLists); - } - - // CHECKSTYLE: IGNORE 1 - /** - * @deprecated Now that indicator is deprecated, use {@link #groupKey(Iterable, Iterable)}, - * which has the same behavior as calling this method with {@code indicator = false}. - */ - @Deprecated // to be removed before 2.0 - public GroupKey groupKey( - Iterable nodes, - boolean indicator, - Iterable> nodeLists) { - Aggregate.checkIndicator(indicator); - return groupKey_(nodes, nodeLists); - } - - private static GroupKey groupKey_( - Iterable nodes, - Iterable> nodeLists) { - final ImmutableList.Builder> builder = ImmutableList.builder(); - for (Iterable nodeList : nodeLists) { - builder.add(ImmutableList.copyOf(nodeList)); - } - return new GroupKeyImpl(ImmutableList.copyOf(nodes), builder.build(), null); - } - - /** Creates a group key of fields identified by ordinal. */ - public GroupKey groupKey(int... fieldOrdinals) { - return groupKey(fields(ImmutableIntList.of(fieldOrdinals))); - } - - /** Creates a group key of fields identified by name. */ - public GroupKey groupKey(String... fieldNames) { - return groupKey(fields(ImmutableList.copyOf(fieldNames))); - } - - /** - * Creates a group key, identified by field positions in the underlying relational expression. - * - *

This method of creating a group key does not allow you to group on new expressions, only - * column projections, but is efficient, especially when you are coming from an existing {@link - * Aggregate}. - */ - public GroupKey groupKey(ImmutableBitSet groupSet) { - return groupKey_(groupSet, ImmutableList.of(groupSet)); - } - - /** - * Creates a group key with grouping sets, both identified by field positions in the underlying - * relational expression. - * - *

This method of creating a group key does not allow you to group on new expressions, only - * column projections, but is efficient, especially when you are coming from an existing {@link - * Aggregate}. - * - *

It is possible for {@code groupSet} to be strict superset of all {@code groupSets}. For - * example, in the pseudo SQL - * - *

{@code
-     * GROUP BY 0, 1, 2
-     * GROUPING SETS ((0, 1), 0)
-     * }
- * - *

column 2 does not appear in either grouping set. This is not valid SQL. We can approximate - * in actual SQL by adding an extra grouping set and filtering out using {@code HAVING}, as - * follows: - * - *

{@code
-     * GROUP BY GROUPING SETS ((0, 1, 2), (0, 1), 0)
-     * HAVING GROUPING_ID(0, 1, 2) <> 0
-     * }
- */ - public GroupKey groupKey( - ImmutableBitSet groupSet, Iterable groupSets) { - return groupKey_(groupSet, ImmutableList.copyOf(groupSets)); - } - - // CHECKSTYLE: IGNORE 1 - /** - * @deprecated Use {@link #groupKey(ImmutableBitSet, Iterable)}. - */ - @Deprecated // to be removed before 2.0 - public GroupKey groupKey( - ImmutableBitSet groupSet, - boolean indicator, - @Nullable ImmutableList groupSets) { - Aggregate.checkIndicator(indicator); - return groupKey_( - groupSet, - groupSets == null ? ImmutableList.of(groupSet) : ImmutableList.copyOf(groupSets)); - } - - private GroupKey groupKey_(ImmutableBitSet groupSet, ImmutableList groupSets) { - if (groupSet.length() > peek().getRowType().getFieldCount()) { - throw new IllegalArgumentException("out of bounds: " + groupSet); - } - requireNonNull(groupSets, "groupSets"); - final ImmutableList nodes = fields(groupSet); - return groupKey_(nodes, Util.transform(groupSets, this::fields)); - } - - @Deprecated // to be removed before 2.0 - public AggCall aggregateCall( - SqlAggFunction aggFunction, - boolean distinct, - RexNode filter, - @Nullable String alias, - RexNode... operands) { - return aggregateCall( - aggFunction, - distinct, - false, - false, - filter, - null, - ImmutableList.of(), - alias, - ImmutableList.copyOf(operands)); - } - - @Deprecated // to be removed before 2.0 - public AggCall aggregateCall( - SqlAggFunction aggFunction, - boolean distinct, - boolean approximate, - RexNode filter, - @Nullable String alias, - RexNode... operands) { - return aggregateCall( - aggFunction, - distinct, - approximate, - false, - filter, - null, - ImmutableList.of(), - alias, - ImmutableList.copyOf(operands)); - } - - @Deprecated // to be removed before 2.0 - public AggCall aggregateCall( - SqlAggFunction aggFunction, - boolean distinct, - RexNode filter, - @Nullable String alias, - Iterable operands) { - return aggregateCall( - aggFunction, - distinct, - false, - false, - filter, - null, - ImmutableList.of(), - alias, - ImmutableList.copyOf(operands)); - } - - @Deprecated // to be removed before 2.0 - public AggCall aggregateCall( - SqlAggFunction aggFunction, - boolean distinct, - boolean approximate, - RexNode filter, - @Nullable String alias, - Iterable operands) { - return aggregateCall( - aggFunction, - distinct, - approximate, - false, - filter, - null, - ImmutableList.of(), - alias, - ImmutableList.copyOf(operands)); - } - - /** - * Creates a call to an aggregate function. - * - *

To add other operands, apply {@link AggCall#distinct()}, {@link - * AggCall#approximate(boolean)}, {@link AggCall#filter(RexNode...)}, {@link AggCall#sort}, - * {@link AggCall#as} to the result. - */ - public AggCall aggregateCall(SqlAggFunction aggFunction, Iterable operands) { - return aggregateCall( - aggFunction, - false, - false, - false, - null, - null, - ImmutableList.of(), - null, - ImmutableList.copyOf(operands)); - } - - /** - * Creates a call to an aggregate function. - * - *

To add other operands, apply {@link AggCall#distinct()}, {@link - * AggCall#approximate(boolean)}, {@link AggCall#filter(RexNode...)}, {@link AggCall#sort}, - * {@link AggCall#as} to the result. - */ - public AggCall aggregateCall(SqlAggFunction aggFunction, RexNode... operands) { - return aggregateCall( - aggFunction, - false, - false, - false, - null, - null, - ImmutableList.of(), - null, - ImmutableList.copyOf(operands)); - } - - /** Creates a call to an aggregate function as a copy of an {@link AggregateCall}. */ - public AggCall aggregateCall(AggregateCall a) { - return aggregateCall( - a.getAggregation(), - a.isDistinct(), - a.isApproximate(), - a.ignoreNulls(), - a.filterArg < 0 ? null : field(a.filterArg), - a.distinctKeys == null ? null : fields(a.distinctKeys), - fields(a.collation), - a.name, - fields(a.getArgList())); - } - - /** - * Creates a call to an aggregate function as a copy of an {@link AggregateCall}, applying a - * mapping. - */ - public AggCall aggregateCall(AggregateCall a, Mapping mapping) { - return aggregateCall( - a.getAggregation(), - a.isDistinct(), - a.isApproximate(), - a.ignoreNulls(), - a.filterArg < 0 ? null : field(Mappings.apply(mapping, a.filterArg)), - a.distinctKeys == null ? null : fields(Mappings.apply(mapping, a.distinctKeys)), - fields(RexUtil.apply(mapping, a.collation)), - a.name, - fields(Mappings.apply2(mapping, a.getArgList()))); - } - - /** Creates a call to an aggregate function with all applicable operands. */ - protected AggCall aggregateCall( - SqlAggFunction aggFunction, - boolean distinct, - boolean approximate, - boolean ignoreNulls, - @Nullable RexNode filter, - @Nullable ImmutableList distinctKeys, - ImmutableList orderKeys, - @Nullable String alias, - ImmutableList operands) { - return new AggCallImpl( - aggFunction, - distinct, - approximate, - ignoreNulls, - filter, - alias, - operands, - distinctKeys, - orderKeys); - } - - /** Creates a call to the {@code COUNT} aggregate function. */ - public AggCall count(RexNode... operands) { - return count(false, null, operands); - } - - /** Creates a call to the {@code COUNT} aggregate function. */ - public AggCall count(Iterable operands) { - return count(false, null, operands); - } - - /** - * Creates a call to the {@code COUNT} aggregate function, optionally distinct and with an - * alias. - */ - public AggCall count(boolean distinct, @Nullable String alias, RexNode... operands) { - return aggregateCall( - SqlStdOperatorTable.COUNT, - distinct, - false, - false, - null, - null, - ImmutableList.of(), - alias, - ImmutableList.copyOf(operands)); - } - - /** - * Creates a call to the {@code COUNT} aggregate function, optionally distinct and with an - * alias. - */ - public AggCall count( - boolean distinct, @Nullable String alias, Iterable operands) { - return aggregateCall( - SqlStdOperatorTable.COUNT, - distinct, - false, - false, - null, - null, - ImmutableList.of(), - alias, - ImmutableList.copyOf(operands)); - } - - /** Creates a call to the {@code COUNT(*)} aggregate function. */ - public AggCall countStar(@Nullable String alias) { - return count(false, alias); - } - - /** Creates a call to the {@code SUM} aggregate function. */ - public AggCall sum(RexNode operand) { - return sum(false, null, operand); - } - - /** - * Creates a call to the {@code SUM} aggregate function, optionally distinct and with an alias. - */ - public AggCall sum(boolean distinct, @Nullable String alias, RexNode operand) { - return aggregateCall( - SqlStdOperatorTable.SUM, - distinct, - false, - false, - null, - null, - ImmutableList.of(), - alias, - ImmutableList.of(operand)); - } - - /** Creates a call to the {@code AVG} aggregate function. */ - public AggCall avg(RexNode operand) { - return avg(false, null, operand); - } - - /** - * Creates a call to the {@code AVG} aggregate function, optionally distinct and with an alias. - */ - public AggCall avg(boolean distinct, @Nullable String alias, RexNode operand) { - return aggregateCall( - SqlStdOperatorTable.AVG, - distinct, - false, - false, - null, - null, - ImmutableList.of(), - alias, - ImmutableList.of(operand)); - } - - /** Creates a call to the {@code MIN} aggregate function. */ - public AggCall min(RexNode operand) { - return min(null, operand); - } - - /** Creates a call to the {@code MIN} aggregate function, optionally with an alias. */ - public AggCall min(@Nullable String alias, RexNode operand) { - return aggregateCall( - SqlStdOperatorTable.MIN, - false, - false, - false, - null, - null, - ImmutableList.of(), - alias, - ImmutableList.of(operand)); - } - - /** Creates a call to the {@code MAX} aggregate function, optionally with an alias. */ - public AggCall max(RexNode operand) { - return max(null, operand); - } - - /** Creates a call to the {@code MAX} aggregate function. */ - public AggCall max(@Nullable String alias, RexNode operand) { - return aggregateCall( - SqlStdOperatorTable.MAX, - false, - false, - false, - null, - null, - ImmutableList.of(), - alias, - ImmutableList.of(operand)); - } - - // Methods for patterns - - /** - * Creates a reference to a given field of the pattern. - * - * @param alpha the pattern name - * @param type Type of field - * @param i Ordinal of field - * @return Reference to field of pattern - */ - public RexNode patternField(String alpha, RelDataType type, int i) { - return getRexBuilder().makePatternFieldRef(alpha, type, i); - } - - /** Creates a call that concatenates patterns; for use in {@link #match}. */ - public RexNode patternConcat(Iterable nodes) { - final ImmutableList list = ImmutableList.copyOf(nodes); - if (list.size() > 2) { - // Convert into binary calls - return patternConcat(patternConcat(Util.skipLast(list)), Util.last(list)); - } - final RelDataType t = getTypeFactory().createSqlType(SqlTypeName.NULL); - return getRexBuilder().makeCall(t, SqlStdOperatorTable.PATTERN_CONCAT, list); - } - - /** Creates a call that concatenates patterns; for use in {@link #match}. */ - public RexNode patternConcat(RexNode... nodes) { - return patternConcat(ImmutableList.copyOf(nodes)); - } - - /** Creates a call that creates alternate patterns; for use in {@link #match}. */ - public RexNode patternAlter(Iterable nodes) { - final RelDataType t = getTypeFactory().createSqlType(SqlTypeName.NULL); - return getRexBuilder() - .makeCall(t, SqlStdOperatorTable.PATTERN_ALTER, ImmutableList.copyOf(nodes)); - } - - /** Creates a call that creates alternate patterns; for use in {@link #match}. */ - public RexNode patternAlter(RexNode... nodes) { - return patternAlter(ImmutableList.copyOf(nodes)); - } - - /** Creates a call that creates quantify patterns; for use in {@link #match}. */ - public RexNode patternQuantify(Iterable nodes) { - final RelDataType t = getTypeFactory().createSqlType(SqlTypeName.NULL); - return getRexBuilder() - .makeCall(t, SqlStdOperatorTable.PATTERN_QUANTIFIER, ImmutableList.copyOf(nodes)); - } - - /** Creates a call that creates quantify patterns; for use in {@link #match}. */ - public RexNode patternQuantify(RexNode... nodes) { - return patternQuantify(ImmutableList.copyOf(nodes)); - } - - /** Creates a call that creates permute patterns; for use in {@link #match}. */ - public RexNode patternPermute(Iterable nodes) { - final RelDataType t = getTypeFactory().createSqlType(SqlTypeName.NULL); - return getRexBuilder() - .makeCall(t, SqlStdOperatorTable.PATTERN_PERMUTE, ImmutableList.copyOf(nodes)); - } - - /** Creates a call that creates permute patterns; for use in {@link #match}. */ - public RexNode patternPermute(RexNode... nodes) { - return patternPermute(ImmutableList.copyOf(nodes)); - } - - /** Creates a call that creates an exclude pattern; for use in {@link #match}. */ - public RexNode patternExclude(RexNode node) { - final RelDataType t = getTypeFactory().createSqlType(SqlTypeName.NULL); - return getRexBuilder() - .makeCall(t, SqlStdOperatorTable.PATTERN_EXCLUDE, ImmutableList.of(node)); - } - - // Methods that create relational expressions - - /** - * Creates a {@link TableScan} of the table with a given name. - * - *

Throws if the table does not exist. - * - *

Returns this builder. - * - * @param tableNames Name of table (can optionally be qualified) - */ - public RelBuilder scan(Iterable tableNames) { - final List names = ImmutableList.copyOf(tableNames); - requireNonNull(relOptSchema, "relOptSchema"); - final RelOptTable relOptTable = relOptSchema.getTableForMember(names); - if (relOptTable == null) { - throw RESOURCE.tableNotFound(String.join(".", names)).ex(); - } - final RelNode scan = - struct.scanFactory.createScan( - ViewExpanders.toRelContext(viewExpander, cluster), relOptTable); - push(scan); - rename(relOptTable.getRowType().getFieldNames()); - - // When the node is not a TableScan but from expansion, - // we need to explicitly add the alias. - if (!(scan instanceof TableScan)) { - as(Util.last(ImmutableList.copyOf(tableNames))); - } - return this; - } - - /** - * Creates a {@link TableScan} of the table with a given name. - * - *

Throws if the table does not exist. - * - *

Returns this builder. - * - * @param tableNames Name of table (can optionally be qualified) - */ - public RelBuilder scan(String... tableNames) { - return scan(ImmutableList.copyOf(tableNames)); - } - - /** - * Creates a {@link Snapshot} of a given snapshot period. - * - *

Returns this builder. - * - * @param period Name of table (can optionally be qualified) - */ - public RelBuilder snapshot(RexNode period) { - final Frame frame = stack.pop(); - final RelNode snapshot = struct.snapshotFactory.createSnapshot(frame.rel, period); - stack.push(new Frame(snapshot, frame.fields)); - return this; - } - - /** - * Gets column mappings of the operator. - * - * @param op operator instance - * @return column mappings associated with this function - */ - private static @Nullable Set getColumnMappings(SqlOperator op) { - SqlReturnTypeInference inference = op.getReturnTypeInference(); - if (inference instanceof TableFunctionReturnTypeInference) { - return ((TableFunctionReturnTypeInference) inference).getColumnMappings(); - } else { - return null; - } - } - - /** - * Creates a RexCall to the {@code CURSOR} function by ordinal. - * - * @param inputCount Number of inputs - * @param ordinal The reference to the relational input - * @return RexCall to CURSOR function - */ - public RexNode cursor(int inputCount, int ordinal) { - if (inputCount <= ordinal || ordinal < 0) { - throw new IllegalArgumentException("bad input count or ordinal"); - } - // Refer to the "ordinal"th input as if it were a field - // (because that's how things are laid out inside a TableFunctionScan) - final RelNode input = peek(inputCount, ordinal); - return call( - SqlStdOperatorTable.CURSOR, - getRexBuilder().makeInputRef(input.getRowType(), ordinal)); - } - - /** Creates a {@link TableFunctionScan}. */ - public RelBuilder functionScan(SqlOperator operator, int inputCount, RexNode... operands) { - return functionScan(operator, inputCount, ImmutableList.copyOf(operands)); - } - - /** Creates a {@link TableFunctionScan}. */ - public RelBuilder functionScan( - SqlOperator operator, int inputCount, Iterable operands) { - if (inputCount < 0 || inputCount > stack.size()) { - throw new IllegalArgumentException("bad input count"); - } - - // Gets inputs. - final List inputs = new ArrayList<>(); - for (int i = 0; i < inputCount; i++) { - inputs.add(0, build()); - } - - final RexCall call = call(operator, ImmutableList.copyOf(operands)); - final RelNode functionScan = - struct.tableFunctionScanFactory.createTableFunctionScan( - cluster, inputs, call, null, getColumnMappings(operator)); - push(functionScan); - return this; - } - - /** - * Creates a {@link Filter} of an array of predicates. - * - *

The predicates are combined using AND, and optimized in a similar way to the {@link #and} - * method. If the result is TRUE no filter is created. - */ - public RelBuilder filter(RexNode... predicates) { - return filter(ImmutableSet.of(), ImmutableList.copyOf(predicates)); - } - - /** - * Creates a {@link Filter} of a list of predicates. - * - *

The predicates are combined using AND, and optimized in a similar way to the {@link #and} - * method. If the result is TRUE no filter is created. - */ - public RelBuilder filter(Iterable predicates) { - return filter(ImmutableSet.of(), predicates); - } - - /** - * Creates a {@link Filter} of a list of correlation variables and an array of predicates. - * - *

The predicates are combined using AND, and optimized in a similar way to the {@link #and} - * method. If the result is TRUE no filter is created. - */ - public RelBuilder filter(Iterable variablesSet, RexNode... predicates) { - return filter(variablesSet, ImmutableList.copyOf(predicates)); - } - - /** - * Creates a {@link Filter} of a list of correlation variables and a list of predicates. - * - *

The predicates are combined using AND, and optimized in a similar way to the {@link #and} - * method. If simplification is on and the result is TRUE, no filter is created. - */ - public RelBuilder filter( - Iterable variablesSet, Iterable predicates) { - final RexNode conjunctionPredicates; - if (config.simplify()) { - conjunctionPredicates = simplifier.simplifyFilterPredicates(predicates); - } else { - conjunctionPredicates = RexUtil.composeConjunction(simplifier.rexBuilder, predicates); - } - - if (conjunctionPredicates == null || conjunctionPredicates.isAlwaysFalse()) { - return empty(); - } - if (conjunctionPredicates.isAlwaysTrue()) { - return this; - } - - final Frame frame = stack.pop(); - final RelNode filter = - struct.filterFactory.createFilter( - frame.rel, conjunctionPredicates, ImmutableSet.copyOf(variablesSet)); - stack.push(new Frame(filter, frame.fields)); - return this; - } - - /** Creates a {@link Project} of the given expressions. */ - public RelBuilder project(RexNode... nodes) { - return project(ImmutableList.copyOf(nodes)); - } - - /** - * Creates a {@link Project} of the given list of expressions. - * - *

Infers names as would {@link #project(Iterable, Iterable)} if all suggested names were - * null. - * - * @param nodes Expressions - */ - public RelBuilder project(Iterable nodes) { - return project(nodes, ImmutableList.of()); - } - - /** - * Creates a {@link Project} of the given list of expressions and field names. - * - * @param nodes Expressions - * @param fieldNames field names for expressions - */ - public RelBuilder project( - Iterable nodes, Iterable fieldNames) { - return project(nodes, fieldNames, false); - } - - /** - * Creates a {@link Project} of the given list of expressions, using the given names. - * - *

Names are deduced as follows: - * - *

    - *
  • If the length of {@code fieldNames} is greater than the index of the current entry in - * {@code nodes}, and the entry in {@code fieldNames} is not null, uses it; otherwise - *
  • If an expression projects an input field, or is a cast an input field, uses the input - * field name; otherwise - *
  • If an expression is a call to {@link SqlStdOperatorTable#AS} (see {@link #alias}), - * removes the call but uses the intended alias. - *
- * - *

After the field names have been inferred, makes the field names unique by appending - * numeric suffixes. - * - * @param nodes Expressions - * @param fieldNames Suggested field names - * @param force create project even if it is identity - */ - public RelBuilder project( - Iterable nodes, - Iterable fieldNames, - boolean force) { - return project(nodes, fieldNames, force, ImmutableSet.of()); - } - - /** - * The same with {@link #project(Iterable, Iterable, boolean)}, with additional variablesSet - * param. - * - * @param nodes Expressions - * @param fieldNames Suggested field names - * @param force create project even if it is identity - * @param variablesSet Correlating variables that are set when reading a row from the input, and - * which may be referenced from the projection expressions - */ - public RelBuilder project( - Iterable nodes, - Iterable fieldNames, - boolean force, - Iterable variablesSet) { - return project_(nodes, fieldNames, ImmutableList.of(), force, variablesSet); - } - - /** Creates a {@link Project} of all original fields, plus the given expressions. */ - public RelBuilder projectPlus(RexNode... nodes) { - return projectPlus(ImmutableList.copyOf(nodes)); - } - - /** Creates a {@link Project} of all original fields, plus the given list of expressions. */ - public RelBuilder projectPlus(Iterable nodes) { - return project(Iterables.concat(fields(), nodes)); - } - - /** - * Creates a {@link Project} of all original fields, except the given expressions. - * - * @throws IllegalArgumentException if the given expressions contain duplicates or there is an - * expression that does not match an existing field - */ - public RelBuilder projectExcept(RexNode... expressions) { - return projectExcept(ImmutableList.copyOf(expressions)); - } - - /** - * Creates a {@link Project} of all original fields, except the given list of expressions. - * - * @throws IllegalArgumentException if the given expressions contain duplicates or there is an - * expression that does not match an existing field - */ - public RelBuilder projectExcept(Iterable expressions) { - List allExpressions = new ArrayList<>(fields()); - Set excludeExpressions = new HashSet<>(); - for (RexNode excludeExp : expressions) { - if (!excludeExpressions.add(excludeExp)) { - throw new IllegalArgumentException( - "Input list contains duplicates. Expression " - + excludeExp - + " exists multiple times."); - } - if (!allExpressions.remove(excludeExp)) { - throw new IllegalArgumentException( - "Expression " + excludeExp.toString() + " not found."); - } - } - return this.project(allExpressions); - } - - /** - * Creates a {@link Project} of the given list of expressions, using the given names. - * - *

Names are deduced as follows: - * - *

    - *
  • If the length of {@code fieldNames} is greater than the index of the current entry in - * {@code nodes}, and the entry in {@code fieldNames} is not null, uses it; otherwise - *
  • If an expression projects an input field, or is a cast an input field, uses the input - * field name; otherwise - *
  • If an expression is a call to {@link SqlStdOperatorTable#AS} (see {@link #alias}), - * removes the call but uses the intended alias. - *
- * - *

After the field names have been inferred, makes the field names unique by appending - * numeric suffixes. - * - * @param nodes Expressions - * @param fieldNames Suggested field names - * @param hints Hints - * @param force create project even if it is identity - */ - private RelBuilder project_( - Iterable nodes, - Iterable fieldNames, - Iterable hints, - boolean force, - Iterable variablesSet) { - final Frame frame = requireNonNull(peek_(), "frame stack is empty"); - final RelDataType inputRowType = frame.rel.getRowType(); - final List nodeList = Lists.newArrayList(nodes); - final Set variables = ImmutableSet.copyOf(variablesSet); - - // Perform a quick check for identity. We'll do a deeper check - // later when we've derived column names. - if (!force && Iterables.isEmpty(fieldNames) && RexUtil.isIdentity(nodeList, inputRowType)) { - return this; - } - - final List<@Nullable String> fieldNameList = Lists.newArrayList(fieldNames); - while (fieldNameList.size() < nodeList.size()) { - fieldNameList.add(null); - } - - // Do not merge projection when top projection has correlation variables - bloat: - if (frame.rel instanceof Project && config.bloat() >= 0 && variables.isEmpty()) { - final Project project = (Project) frame.rel; - // Populate field names. If the upper expression is an input ref and does - // not have a recommended name, use the name of the underlying field. - for (int i = 0; i < fieldNameList.size(); i++) { - if (fieldNameList.get(i) == null) { - final RexNode node = nodeList.get(i); - if (node instanceof RexInputRef) { - final RexInputRef ref = (RexInputRef) node; - fieldNameList.set( - i, project.getRowType().getFieldNames().get(ref.getIndex())); - } - } - } - final List newNodes = - RelOptUtil.pushPastProjectUnlessBloat(nodeList, project, config.bloat()); - if (newNodes == null) { - // The merged expression is more complex than the input expressions. - // Do not merge. - break bloat; - } - - // Carefully build a list of fields, so that table aliases from the input - // can be seen for fields that are based on a RexInputRef. - final Frame frame1 = stack.pop(); - final List fields = new ArrayList<>(); - for (RelDataTypeField f : project.getInput().getRowType().getFieldList()) { - fields.add(new Field(ImmutableSet.of(), f)); - } - for (Pair pair : Pair.zip(project.getProjects(), frame1.fields)) { - switch (pair.left.getKind()) { - case INPUT_REF: - final int i = ((RexInputRef) pair.left).getIndex(); - final Field field = fields.get(i); - final ImmutableSet aliases = pair.right.left; - fields.set(i, new Field(aliases, field.right)); - break; - default: - break; - } - } - stack.push(new Frame(project.getInput(), ImmutableList.copyOf(fields))); - final ImmutableSet.Builder mergedHints = ImmutableSet.builder(); - mergedHints.addAll(project.getHints()); - mergedHints.addAll(hints); - // Keep bottom projection's variablesSet. - return project_( - newNodes, - fieldNameList, - mergedHints.build(), - force, - ImmutableSet.copyOf(project.getVariablesSet())); - } - - // Simplify expressions. - if (config.simplify()) { - for (int i = 0; i < nodeList.size(); i++) { - nodeList.set(i, simplifier.simplifyPreservingType(nodeList.get(i))); - } - } - - // Replace null names with generated aliases. - for (int i = 0; i < fieldNameList.size(); i++) { - if (fieldNameList.get(i) == null) { - fieldNameList.set(i, inferAlias(nodeList, nodeList.get(i), i)); - } - } - - final ImmutableList.Builder fields = ImmutableList.builder(); - final Set uniqueNameList = - getTypeFactory().getTypeSystem().isSchemaCaseSensitive() - ? new HashSet<>() - : new TreeSet<>(String.CASE_INSENSITIVE_ORDER); - // calculate final names and build field list - for (int i = 0; i < fieldNameList.size(); ++i) { - final RexNode node = nodeList.get(i); - String name = fieldNameList.get(i); - String originalName = name; - Field field; - if (name == null || uniqueNameList.contains(name)) { - int j = 0; - if (name == null) { - j = i; - } - do { - name = SqlValidatorUtil.F_SUGGESTER.apply(originalName, j, j++); - } while (uniqueNameList.contains(name)); - fieldNameList.set(i, name); - } - RelDataTypeField fieldType = new RelDataTypeFieldImpl(name, i, node.getType()); - switch (node.getKind()) { - case INPUT_REF: - // preserve rel aliases for INPUT_REF fields - final int index = ((RexInputRef) node).getIndex(); - field = new Field(frame.fields.get(index).left, fieldType); - break; - default: - field = new Field(ImmutableSet.of(), fieldType); - break; - } - uniqueNameList.add(name); - fields.add(field); - } - if (!force && RexUtil.isIdentity(nodeList, inputRowType)) { - if (fieldNameList.equals(inputRowType.getFieldNames())) { - // Do not create an identity project if it does not rename any fields - return this; - } else { - // create "virtual" row type for project only rename fields - stack.pop(); - // Ignore the hints. - stack.push(new Frame(frame.rel, fields.build())); - } - return this; - } - - // If the expressions are all literals, and the input is a Values with N - // rows, replace with a Values with same tuple N times. - if (config.simplifyValues() - && frame.rel instanceof Values - && nodeList.stream().allMatch(e -> e instanceof RexLiteral)) { - final Values values = (Values) build(); - final RelDataTypeFactory.Builder typeBuilder = getTypeFactory().builder(); - Pair.forEach( - fieldNameList, - nodeList, - (name, expr) -> typeBuilder.add(requireNonNull(name, "name"), expr.getType())); - @SuppressWarnings({"unchecked", "rawtypes"}) - final List tuple = (List) (List) nodeList; - return values(Collections.nCopies(values.tuples.size(), tuple), typeBuilder.build()); - } - - final RelNode project = - struct.projectFactory.createProject( - frame.rel, - ImmutableList.copyOf(hints), - ImmutableList.copyOf(nodeList), - fieldNameList, - variables); - stack.pop(); - stack.push(new Frame(project, fields.build())); - return this; - } - - /** - * Creates a {@link Project} of the given expressions and field names, and optionally - * optimizing. - * - *

If {@code fieldNames} is null, or if a particular entry in {@code fieldNames} is null, - * derives field names from the input expressions. - * - *

If {@code force} is false, and the input is a {@code Project}, and the expressions make - * the trivial projection ($0, $1, ...), modifies the input. - * - * @param nodes Expressions - * @param fieldNames Suggested field names, or null to generate - * @param force Whether to create a renaming Project if the projections are trivial - */ - public RelBuilder projectNamed( - Iterable nodes, - @Nullable Iterable fieldNames, - boolean force) { - return projectNamed(nodes, fieldNames, force, ImmutableSet.of()); - } - - /** - * Creates a {@link Project} of the given expressions and field names, and optionally - * optimizing. - * - *

If {@code fieldNames} is null, or if a particular entry in {@code fieldNames} is null, - * derives field names from the input expressions. - * - *

If {@code force} is false, and the input is a {@code Project}, and the expressions make - * the trivial projection ($0, $1, ...), modifies the input. - * - * @param nodes Expressions - * @param fieldNames Suggested field names, or null to generate - * @param force Whether to create a renaming Project if the projections are trivial - * @param variablesSet Correlating variables that are set when reading a row from the input, and - * which may be referenced from the projection expressions - */ - public RelBuilder projectNamed( - Iterable nodes, - @Nullable Iterable fieldNames, - boolean force, - Iterable variablesSet) { - @SuppressWarnings({"unchecked", "rawtypes"}) - final List nodeList = - nodes instanceof List ? (List) nodes : ImmutableList.copyOf(nodes); - final List<@Nullable String> fieldNameList = - fieldNames == null - ? null - : fieldNames instanceof List - ? (List<@Nullable String>) fieldNames - : ImmutableNullableList.copyOf(fieldNames); - final RelNode input = peek(); - final RelDataType rowType = - RexUtil.createStructType( - cluster.getTypeFactory(), - nodeList, - fieldNameList, - SqlValidatorUtil.F_SUGGESTER); - if (!force && RexUtil.isIdentity(nodeList, input.getRowType())) { - if (input instanceof Project && fieldNames != null) { - // Rename columns of child projection if desired field names are given. - final Frame frame = stack.pop(); - final Project childProject = (Project) frame.rel; - final Project newInput = - childProject.copy( - childProject.getTraitSet(), - childProject.getInput(), - childProject.getProjects(), - rowType); - stack.push(new Frame(newInput.attachHints(childProject.getHints()), frame.fields)); - } - if (input instanceof Values && fieldNameList != null) { - // Rename columns of child values if desired field names are given. - final Frame frame = stack.pop(); - final Values values = (Values) frame.rel; - final RelDataTypeFactory.Builder typeBuilder = getTypeFactory().builder(); - Pair.forEach( - fieldNameList, - rowType.getFieldList(), - (name, field) -> - typeBuilder.add(requireNonNull(name, "name"), field.getType())); - final RelDataType newRowType = typeBuilder.build(); - final RelNode newValues = - struct.valuesFactory.createValues(cluster, newRowType, values.tuples); - stack.push(new Frame(newValues, frame.fields)); - } - } else { - project(nodeList, rowType.getFieldNames(), force, variablesSet); - } - return this; - } - - /** - * Creates an {@link Uncollect} with given item aliases. - * - * @param itemAliases Operand item aliases, never null - * @param withOrdinality If {@code withOrdinality}, the output contains an extra {@code - * ORDINALITY} column - */ - public RelBuilder uncollect(List itemAliases, boolean withOrdinality) { - Frame frame = stack.pop(); - stack.push( - new Frame( - new Uncollect( - cluster, - cluster.traitSetOf(Convention.NONE), - frame.rel, - withOrdinality, - requireNonNull(itemAliases, "itemAliases")))); - return this; - } - - /** - * Ensures that the field names match those given. - * - *

If all fields have the same name, adds nothing; if any fields do not have the same name, - * adds a {@link Project}. - * - *

Note that the names can be short-lived. Other {@code RelBuilder} operations make no - * guarantees about the field names of the rows they produce. - * - * @param fieldNames List of desired field names; may contain null values or have fewer fields - * than the current row type - */ - public RelBuilder rename(List fieldNames) { - final List oldFieldNames = peek().getRowType().getFieldNames(); - Preconditions.checkArgument( - fieldNames.size() <= oldFieldNames.size(), "More names than fields"); - final List newFieldNames = new ArrayList<>(oldFieldNames); - for (int i = 0; i < fieldNames.size(); i++) { - final String s = fieldNames.get(i); - if (s != null) { - newFieldNames.set(i, s); - } - } - if (oldFieldNames.equals(newFieldNames)) { - return this; - } - if (peek() instanceof Values) { - // Special treatment for VALUES. Re-build it rather than add a project. - final Values v = (Values) build(); - final RelDataTypeFactory.Builder b = getTypeFactory().builder(); - for (Pair p : - Pair.zip(newFieldNames, v.getRowType().getFieldList())) { - b.add(p.left, p.right.getType()); - } - return values(v.tuples, b.build()); - } - - return project(fields(), newFieldNames, true); - } - - /** - * Infers the alias of an expression. - * - *

If the expression was created by {@link #alias}, replaces the expression in the project - * list. - */ - private @Nullable String inferAlias(List exprList, RexNode expr, int i) { - switch (expr.getKind()) { - case INPUT_REF: - final RexInputRef ref = (RexInputRef) expr; - return requireNonNull(stack.peek(), "empty frame stack") - .fields - .get(ref.getIndex()) - .getValue() - .getName(); - case CAST: - return inferAlias(exprList, ((RexCall) expr).getOperands().get(0), -1); - case AS: - final RexCall call = (RexCall) expr; - if (i >= 0) { - exprList.set(i, call.getOperands().get(0)); - } - NlsString value = (NlsString) ((RexLiteral) call.getOperands().get(1)).getValue(); - return castNonNull(value).getValue(); - default: - return null; - } - } - - /** Creates an {@link Aggregate} that makes the relational expression distinct on all fields. */ - public RelBuilder distinct() { - return aggregate(groupKey(fields())); - } - - /** Creates an {@link Aggregate} with an array of calls. */ - public RelBuilder aggregate(GroupKey groupKey, AggCall... aggCalls) { - return aggregate(groupKey, ImmutableList.copyOf(aggCalls)); - } - - /** Creates an {@link Aggregate} with an array of {@link AggregateCall}s. */ - public RelBuilder aggregate(GroupKey groupKey, List aggregateCalls) { - return aggregate( - groupKey, - aggregateCalls.stream() - .map( - aggregateCall -> - new AggCallImpl2( - aggregateCall, - aggregateCall.getArgList().stream() - .map(this::field) - .collect(Util.toImmutableList()))) - .collect(Collectors.toList())); - } - - /** Creates an {@link Aggregate} with multiple calls. */ - public RelBuilder aggregate(GroupKey groupKey, Iterable aggCalls) { - final Registrar registrar = new Registrar(fields(), peek().getRowType().getFieldNames()); - final GroupKeyImpl groupKey_ = (GroupKeyImpl) groupKey; - ImmutableBitSet groupSet = - ImmutableBitSet.of(registrar.registerExpressions(groupKey_.nodes)); - label: - if (Iterables.isEmpty(aggCalls)) { - final RelMetadataQuery mq = peek().getCluster().getMetadataQuery(); - if (groupSet.isEmpty()) { - final Double minRowCount = mq.getMinRowCount(peek()); - if (minRowCount == null || minRowCount < 1D) { - // We can't remove "GROUP BY ()" if there's a chance the rel could be - // empty. - break label; - } - } - if (registrar.extraNodes.size() == fields().size()) { - final Boolean unique = mq.areColumnsUnique(peek(), groupSet); - if (unique != null && unique && !config.aggregateUnique() && groupKey_.isSimple()) { - // Rel is already unique. - return project(fields(groupSet)); - } - } - final Double maxRowCount = mq.getMaxRowCount(peek()); - if (maxRowCount != null - && maxRowCount <= 1D - && !config.aggregateUnique() - && groupKey_.isSimple()) { - // If there is at most one row, rel is already unique. - return project(fields(groupSet)); - } - } - - ImmutableList groupSets; - if (groupKey_.nodeLists != null) { - final int sizeBefore = registrar.extraNodes.size(); - final List groupSetList = new ArrayList<>(); - for (ImmutableList nodeList : groupKey_.nodeLists) { - final ImmutableBitSet groupSet2 = - ImmutableBitSet.of(registrar.registerExpressions(nodeList)); - if (!groupSet.contains(groupSet2)) { - throw new IllegalArgumentException( - "group set element " + nodeList + " must be a subset of group key"); - } - groupSetList.add(groupSet2); - } - final ImmutableSortedMultiset groupSetMultiset = - ImmutableSortedMultiset.copyOf(ImmutableBitSet.COMPARATOR, groupSetList); - if (Iterables.any(aggCalls, RelBuilder::isGroupId) - || !ImmutableBitSet.ORDERING.isStrictlyOrdered(groupSetMultiset)) { - return rewriteAggregateWithDuplicateGroupSets( - groupSet, groupSetMultiset, ImmutableList.copyOf(aggCalls)); - } - groupSets = ImmutableList.copyOf(groupSetMultiset.elementSet()); - if (registrar.extraNodes.size() > sizeBefore) { - throw new IllegalArgumentException( - "group sets contained expressions " - + "not in group key: " - + Util.skip(registrar.extraNodes, sizeBefore)); - } - } else { - groupSets = ImmutableList.of(groupSet); - } - - for (AggCall aggCall : aggCalls) { - ((AggCallPlus) aggCall).register(registrar); - } - project(registrar.extraNodes); - rename(registrar.names); - final Frame frame = stack.pop(); - RelNode r = frame.rel; - final List aggregateCalls = new ArrayList<>(); - for (AggCall aggCall : aggCalls) { - aggregateCalls.add(((AggCallPlus) aggCall).aggregateCall(registrar, groupSet, r)); - } - - assert ImmutableBitSet.ORDERING.isStrictlyOrdered(groupSets) : groupSets; - for (ImmutableBitSet set : groupSets) { - assert groupSet.contains(set); - } - - List inFields = frame.fields; - if (config.pruneInputOfAggregate() && r instanceof Project) { - final Set fieldsUsed = RelOptUtil.getAllFields2(groupSet, aggregateCalls); - // Some parts of the system can't handle rows with zero fields, so - // pretend that one field is used. - if (fieldsUsed.isEmpty()) { - r = ((Project) r).getInput(); - } else if (fieldsUsed.size() < r.getRowType().getFieldCount()) { - // Some fields are computed but not used. Prune them. - final Map map = new HashMap<>(); - for (int source : fieldsUsed) { - map.put(source, map.size()); - } - - groupSet = groupSet.permute(map); - groupSets = - ImmutableBitSet.ORDERING.immutableSortedCopy( - ImmutableBitSet.permute(groupSets, map)); - - final Mappings.TargetMapping targetMapping = - Mappings.target(map, r.getRowType().getFieldCount(), fieldsUsed.size()); - final List oldAggregateCalls = new ArrayList<>(aggregateCalls); - aggregateCalls.clear(); - for (AggregateCall aggregateCall : oldAggregateCalls) { - aggregateCalls.add(aggregateCall.transform(targetMapping)); - } - inFields = Mappings.permute(inFields, targetMapping.inverse()); - - final Project project = (Project) r; - final List newProjects = new ArrayList<>(); - final RelDataTypeFactory.Builder builder = cluster.getTypeFactory().builder(); - for (int i : fieldsUsed) { - newProjects.add(project.getProjects().get(i)); - builder.add(project.getRowType().getFieldList().get(i)); - } - r = - project.copy( - cluster.traitSet(), - project.getInput(), - newProjects, - builder.build()); - } - } - - if (!config.dedupAggregateCalls() || Util.isDistinct(aggregateCalls)) { - return aggregate_( - groupSet, groupSets, r, aggregateCalls, registrar.extraNodes, inFields); - } - - // There are duplicate aggregate calls. Rebuild the list to eliminate - // duplicates, then add a Project. - final Set callSet = new HashSet<>(); - final List> projects = new ArrayList<>(); - Util.range(groupSet.cardinality()).forEach(i -> projects.add(Pair.of(i, null))); - final List distinctAggregateCalls = new ArrayList<>(); - for (AggregateCall aggregateCall : aggregateCalls) { - final int i; - if (callSet.add(aggregateCall)) { - i = distinctAggregateCalls.size(); - distinctAggregateCalls.add(aggregateCall); - } else { - i = distinctAggregateCalls.indexOf(aggregateCall); - assert i >= 0; - } - projects.add(Pair.of(groupSet.cardinality() + i, aggregateCall.name)); - } - aggregate_(groupSet, groupSets, r, distinctAggregateCalls, registrar.extraNodes, inFields); - final List fields = - projects.stream() - .map(p -> p.right == null ? field(p.left) : alias(field(p.left), p.right)) - .collect(Collectors.toList()); - return project(fields); - } - - /** - * Finishes the implementation of {@link #aggregate} by creating an {@link Aggregate} and - * pushing it onto the stack. - */ - private RelBuilder aggregate_( - ImmutableBitSet groupSet, - ImmutableList groupSets, - RelNode input, - List aggregateCalls, - List extraNodes, - List inFields) { - final RelNode aggregate = - struct.aggregateFactory.createAggregate( - input, ImmutableList.of(), groupSet, groupSets, aggregateCalls); - - // build field list - final ImmutableList.Builder fields = ImmutableList.builder(); - final List aggregateFields = aggregate.getRowType().getFieldList(); - int i = 0; - // first, group fields - for (Integer groupField : groupSet.asList()) { - RexNode node = extraNodes.get(groupField); - final SqlKind kind = node.getKind(); - switch (kind) { - case INPUT_REF: - fields.add(inFields.get(((RexInputRef) node).getIndex())); - break; - default: - String name = aggregateFields.get(i).getName(); - RelDataTypeField fieldType = new RelDataTypeFieldImpl(name, i, node.getType()); - fields.add(new Field(ImmutableSet.of(), fieldType)); - break; - } - i++; - } - // second, aggregate fields. retain `i' as field index - for (int j = 0; j < aggregateCalls.size(); ++j) { - final AggregateCall call = aggregateCalls.get(j); - final RelDataTypeField fieldType = - new RelDataTypeFieldImpl( - aggregateFields.get(i + j).getName(), i + j, call.getType()); - fields.add(new Field(ImmutableSet.of(), fieldType)); - } - stack.push(new Frame(aggregate, fields.build())); - return this; - } - - /** - * The {@code GROUP_ID()} function is used to distinguish duplicate groups. However, as - * Aggregate normalizes group sets to canonical form (i.e., flatten, sorting, redundancy - * removal), this information is lost in RelNode. Therefore, it is impossible to implement the - * function in runtime. - * - *

To fill this gap, an aggregation query that contains duplicate group sets is rewritten - * into a Union of Aggregate operators whose group sets are distinct. The number of inputs to - * the Union is equal to the maximum number of duplicates. In the {@code N}th input to the - * Union, calls to the {@code GROUP_ID} aggregate function are replaced by the integer literal - * {@code N}. - * - *

This method also handles the case where group sets are distinct but there is a call to - * {@code GROUP_ID}. That call is replaced by the integer literal {@code 0}. - * - *

Also see the discussion in [CALCITE-1824] GROUP_ID returns - * wrong result and [CALCITE-4748] If there are - * duplicate GROUPING SETS, Calcite should return duplicate rows. - */ - private RelBuilder rewriteAggregateWithDuplicateGroupSets( - ImmutableBitSet groupSet, - ImmutableSortedMultiset groupSets, - List aggregateCalls) { - final List fieldNamesIfNoRewrite = - Aggregate.deriveRowType( - getTypeFactory(), - peek().getRowType(), - false, - groupSet, - groupSets.asList(), - aggregateCalls.stream() - .map(c -> ((AggCallPlus) c).aggregateCall()) - .collect(Util.toImmutableList())) - .getFieldNames(); - - // If n duplicates exist for a particular grouping, the {@code GROUP_ID()} - // function produces values in the range 0 to n-1. For each value, - // we need to figure out the corresponding group sets. - // - // For example, "... GROUPING SETS (a, a, b, c, c, c, c)" - // (i) The max value of the GROUP_ID() function returns is 3 - // (ii) GROUPING SETS (a, b, c) produces value 0, - // GROUPING SETS (a, c) produces value 1, - // GROUPING SETS (c) produces value 2 - // GROUPING SETS (c) produces value 3 - final Map> groupIdToGroupSets = new HashMap<>(); - int maxGroupId = 0; - for (Multiset.Entry entry : groupSets.entrySet()) { - int groupId = entry.getCount() - 1; - if (groupId > maxGroupId) { - maxGroupId = groupId; - } - for (int i = 0; i <= groupId; i++) { - groupIdToGroupSets - .computeIfAbsent(i, k -> Sets.newTreeSet(ImmutableBitSet.COMPARATOR)) - .add(entry.getElement()); - } - } - - // AggregateCall list without GROUP_ID function - final List aggregateCallsWithoutGroupId = new ArrayList<>(aggregateCalls); - aggregateCallsWithoutGroupId.removeIf(RelBuilder::isGroupId); - - // For each group id value, we first construct an Aggregate without - // GROUP_ID() function call, and then create a Project node on top of it. - // The Project adds literal value for group id in right position. - final Frame frame = stack.pop(); - for (int groupId = 0; groupId <= maxGroupId; groupId++) { - // Create the Aggregate node without GROUP_ID() call - stack.push(frame); - aggregate( - groupKey(groupSet, castNonNull(groupIdToGroupSets.get(groupId))), - aggregateCallsWithoutGroupId); - - final List selectList = new ArrayList<>(); - final int groupExprLength = groupSet.cardinality(); - // Project fields in group by expressions - for (int i = 0; i < groupExprLength; i++) { - selectList.add(field(i)); - } - // Project fields in aggregate calls - int groupIdCount = 0; - for (int i = 0; i < aggregateCalls.size(); i++) { - if (isGroupId(aggregateCalls.get(i))) { - selectList.add( - getRexBuilder() - .makeExactLiteral( - BigDecimal.valueOf(groupId), - getTypeFactory().createSqlType(SqlTypeName.BIGINT))); - groupIdCount++; - } else { - selectList.add(field(groupExprLength + i - groupIdCount)); - } - } - project(selectList, fieldNamesIfNoRewrite); - } - - return union(true, maxGroupId + 1); - } - - private static boolean isGroupId(AggCall c) { - return ((AggCallPlus) c).op().kind == SqlKind.GROUP_ID; - } - - private RelBuilder setOp(boolean all, SqlKind kind, int n) { - List inputs = new ArrayList<>(); - for (int i = 0; i < n; i++) { - inputs.add(0, build()); - } - switch (kind) { - case UNION: - case INTERSECT: - case EXCEPT: - if (n < 1) { - throw new IllegalArgumentException("bad INTERSECT/UNION/EXCEPT input count"); - } - break; - default: - throw new AssertionError("bad setOp " + kind); - } - - if (n == 1) { - return push(inputs.get(0)); - } - - if (config.simplifyValues() - && kind == UNION - && inputs.stream().allMatch(r -> r instanceof Values)) { - List inputTypes = Util.transform(inputs, RelNode::getRowType); - RelDataType rowType = getTypeFactory().leastRestrictive(inputTypes); - requireNonNull(rowType, () -> "leastRestrictive(" + inputTypes + ")"); - final List> tuples = new ArrayList<>(); - for (RelNode input : inputs) { - tuples.addAll(((Values) input).tuples); - } - final List> tuples2 = all ? tuples : Util.distinctList(tuples); - return values(tuples2, rowType); - } - - return push(struct.setOpFactory.createSetOp(kind, inputs, all)); - } - - /** - * Creates a {@link Union} of the two most recent relational expressions on the stack. - * - * @param all Whether to create UNION ALL - */ - public RelBuilder union(boolean all) { - return union(all, 2); - } - - /** - * Creates a {@link Union} of the {@code n} most recent relational expressions on the stack. - * - * @param all Whether to create UNION ALL - * @param n Number of inputs to the UNION operator - */ - public RelBuilder union(boolean all, int n) { - return setOp(all, UNION, n); - } - - /** - * Creates an {@link Intersect} of the two most recent relational expressions on the stack. - * - * @param all Whether to create INTERSECT ALL - */ - public RelBuilder intersect(boolean all) { - return intersect(all, 2); - } - - /** - * Creates an {@link Intersect} of the {@code n} most recent relational expressions on the - * stack. - * - * @param all Whether to create INTERSECT ALL - * @param n Number of inputs to the INTERSECT operator - */ - public RelBuilder intersect(boolean all, int n) { - return setOp(all, SqlKind.INTERSECT, n); - } - - /** - * Creates a {@link Minus} of the two most recent relational expressions on the stack. - * - * @param all Whether to create EXCEPT ALL - */ - public RelBuilder minus(boolean all) { - return minus(all, 2); - } - - /** - * Creates a {@link Minus} of the {@code n} most recent relational expressions on the stack. - * - * @param all Whether to create EXCEPT ALL - */ - public RelBuilder minus(boolean all, int n) { - return setOp(all, SqlKind.EXCEPT, n); - } - - /** - * Creates a {@link TableScan} on a {@link TransientTable} with the given name, using as type - * the top of the stack's type. - * - * @param tableName table name - */ - @Experimental - public RelBuilder transientScan(String tableName) { - return this.transientScan(tableName, this.peek().getRowType()); - } - - /** - * Creates a {@link TableScan} on a {@link TransientTable} with the given name and type. - * - * @param tableName table name - * @param rowType row type of the table - */ - @Experimental - public RelBuilder transientScan(String tableName, RelDataType rowType) { - TransientTable transientTable = new ListTransientTable(tableName, rowType); - requireNonNull(relOptSchema, "relOptSchema"); - RelOptTable relOptTable = - RelOptTableImpl.create( - relOptSchema, rowType, transientTable, ImmutableList.of(tableName)); - RelNode scan = - struct.scanFactory.createScan( - ViewExpanders.toRelContext(viewExpander, cluster), relOptTable); - push(scan); - rename(rowType.getFieldNames()); - return this; - } - - /** - * Creates a {@link TableSpool} for the most recent relational expression. - * - * @param readType Spool's read type (as described in {@link Spool.Type}) - * @param writeType Spool's write type (as described in {@link Spool.Type}) - * @param table Table to write into - */ - private RelBuilder tableSpool(Spool.Type readType, Spool.Type writeType, RelOptTable table) { - RelNode spool = struct.spoolFactory.createTableSpool(peek(), readType, writeType, table); - replaceTop(spool); - return this; - } - - /** - * Creates a {@link RepeatUnion} associated to a {@link TransientTable} without a maximum number - * of iterations, i.e. repeatUnion(tableName, all, -1). - * - * @param tableName name of the {@link TransientTable} associated to the {@link RepeatUnion} - * @param all whether duplicates will be considered or not - */ - @Experimental - public RelBuilder repeatUnion(String tableName, boolean all) { - return repeatUnion(tableName, all, -1); - } - - /** - * Creates a {@link RepeatUnion} associated to a {@link TransientTable} of the two most recent - * relational expressions on the stack. - * - *

Warning: if these relational expressions are not correctly defined, this operation might - * lead to an infinite loop. - * - *

The generated {@link RepeatUnion} operates as follows: - * - *

    - *
  • Evaluate its left term once, propagating the results into the {@link TransientTable}; - *
  • Evaluate its right term (which may contain a {@link TableScan} on the {@link - * TransientTable}) over and over until it produces no more results (or until an optional - * maximum number of iterations is reached). On each iteration, the results are propagated - * into the {@link TransientTable}, overwriting the results from the previous one. - *
- * - * @param tableName Name of the {@link TransientTable} associated to the {@link RepeatUnion} - * @param all Whether duplicates are considered - * @param iterationLimit Maximum number of iterations; negative value means no limit - */ - @Experimental - public RelBuilder repeatUnion(String tableName, boolean all, int iterationLimit) { - RelOptTableFinder finder = new RelOptTableFinder(tableName); - for (int i = 0; i < stack.size(); i++) { // search scan(tableName) in the stack - peek(i).accept(finder); - if (finder.relOptTable != null) { // found - break; - } - } - if (finder.relOptTable == null) { - throw RESOURCE.tableNotFound(tableName).ex(); - } - - RelNode iterative = - tableSpool(Spool.Type.LAZY, Spool.Type.LAZY, finder.relOptTable).build(); - RelNode seed = tableSpool(Spool.Type.LAZY, Spool.Type.LAZY, finder.relOptTable).build(); - RelNode repeatUnion = - struct.repeatUnionFactory.createRepeatUnion( - seed, iterative, all, iterationLimit, finder.relOptTable); - return push(repeatUnion); - } - - /** Auxiliary class to find a certain RelOptTable based on its name. */ - private static final class RelOptTableFinder extends RelHomogeneousShuttle { - private @MonotonicNonNull RelOptTable relOptTable = null; - private final String tableName; - - private RelOptTableFinder(String tableName) { - this.tableName = tableName; - } - - @Override - public RelNode visit(TableScan scan) { - final RelOptTable scanTable = scan.getTable(); - final List qualifiedName = scanTable.getQualifiedName(); - if (qualifiedName.get(qualifiedName.size() - 1).equals(tableName)) { - relOptTable = scanTable; - } - return super.visit(scan); - } - } - - /** Creates a {@link Join} with an array of conditions. */ - public RelBuilder join(JoinRelType joinType, RexNode condition0, RexNode... conditions) { - return join(joinType, Lists.asList(condition0, conditions)); - } - - /** Creates a {@link Join} with multiple conditions. */ - public RelBuilder join(JoinRelType joinType, Iterable conditions) { - return join(joinType, and(conditions), ImmutableSet.of()); - } - - /** Creates a {@link Join} with one condition. */ - public RelBuilder join(JoinRelType joinType, RexNode condition) { - return join(joinType, condition, ImmutableSet.of()); - } - - /** Creates a {@link Join} with correlating variables. */ - public RelBuilder join( - JoinRelType joinType, RexNode condition, Set variablesSet) { - Frame right = stack.pop(); - final Frame left = stack.pop(); - final RelNode join; - // FLINK BEGIN MODIFICATION - // keep behavior of Calcite 1.27.0 - final boolean correlate = variablesSet.size() == 1; - // FLINK END MODIFICATION - RexNode postCondition = literal(true); - if (config.simplify()) { - // Normalize expanded versions IS NOT DISTINCT FROM so that simplifier does not - // transform the expression to something unrecognizable - if (condition instanceof RexCall) { - condition = - RelOptUtil.collapseExpandedIsNotDistinctFromExpr( - (RexCall) condition, getRexBuilder()); - } - condition = simplifier.simplifyUnknownAsFalse(condition); - } - if (correlate) { - final CorrelationId id = Iterables.getOnlyElement(variablesSet); - // Correlate does not have an ON clause. - switch (joinType) { - case LEFT: - case SEMI: - case ANTI: - // For a LEFT/SEMI/ANTI, predicate must be evaluated first. - stack.push(right); - filter(condition.accept(new Shifter(left.rel, id, right.rel))); - right = stack.pop(); - break; - case INNER: - // For INNER, we can defer. - postCondition = condition; - break; - default: - throw new IllegalArgumentException( - "Correlated " + joinType + " join is not supported"); - } - final ImmutableBitSet requiredColumns = RelOptUtil.correlationColumns(id, right.rel); - join = - struct.correlateFactory.createCorrelate( - left.rel, right.rel, ImmutableList.of(), id, requiredColumns, joinType); - } else { - RelNode join0 = - struct.joinFactory.createJoin( - left.rel, - right.rel, - ImmutableList.of(), - condition, - variablesSet, - joinType, - false); - - if (join0 instanceof Join && config.pushJoinCondition()) { - join = RelOptUtil.pushDownJoinConditions((Join) join0, this); - } else { - join = join0; - } - } - final ImmutableList.Builder fields = ImmutableList.builder(); - fields.addAll(left.fields); - fields.addAll(right.fields); - stack.push(new Frame(join, fields.build())); - filter(postCondition); - return this; - } - - /** - * Creates a {@link Correlate} with a {@link CorrelationId} and an array of fields that are used - * by correlation. - */ - public RelBuilder correlate( - JoinRelType joinType, CorrelationId correlationId, RexNode... requiredFields) { - return correlate(joinType, correlationId, ImmutableList.copyOf(requiredFields)); - } - - /** - * Creates a {@link Correlate} with a {@link CorrelationId} and a list of fields that are used - * by correlation. - */ - public RelBuilder correlate( - JoinRelType joinType, - CorrelationId correlationId, - Iterable requiredFields) { - Frame right = stack.pop(); - - final Registrar registrar = new Registrar(fields(), peek().getRowType().getFieldNames()); - - List requiredOrdinals = - registrar.registerExpressions(ImmutableList.copyOf(requiredFields)); - - project(registrar.extraNodes); - rename(registrar.names); - Frame left = stack.pop(); - - final RelNode correlate = - struct.correlateFactory.createCorrelate( - left.rel, - right.rel, - ImmutableList.of(), - correlationId, - ImmutableBitSet.of(requiredOrdinals), - joinType); - - final ImmutableList.Builder fields = ImmutableList.builder(); - fields.addAll(left.fields); - fields.addAll(right.fields); - stack.push(new Frame(correlate, fields.build())); - - return this; - } - - /** - * Creates a {@link Join} using USING syntax. - * - *

For each of the field names, both left and right inputs must have a field of that name. - * Constructs a join condition that the left and right fields are equal. - * - * @param joinType Join type - * @param fieldNames Field names - */ - public RelBuilder join(JoinRelType joinType, String... fieldNames) { - final List conditions = new ArrayList<>(); - for (String fieldName : fieldNames) { - conditions.add(equals(field(2, 0, fieldName), field(2, 1, fieldName))); - } - return join(joinType, conditions); - } - - /** - * Creates a {@link Join} with {@link JoinRelType#SEMI}. - * - *

A semi-join is a form of join that combines two relational expressions according to some - * condition, and outputs only rows from the left input for which at least one row from the - * right input matches. It only outputs columns from the left input, and ignores duplicates on - * the right. - * - *

For example, {@code EMP semi-join DEPT} finds all {@code EMP} records that do not have a - * corresponding {@code DEPT} record, similar to the following SQL: - * - *

- * - *
-     * SELECT * FROM EMP
-     * WHERE EXISTS (SELECT 1 FROM DEPT
-     *     WHERE DEPT.DEPTNO = EMP.DEPTNO)
- * - *
- */ - public RelBuilder semiJoin(Iterable conditions) { - final Frame right = stack.pop(); - final RelNode semiJoin = - struct.joinFactory.createJoin( - peek(), - right.rel, - ImmutableList.of(), - and(conditions), - ImmutableSet.of(), - JoinRelType.SEMI, - false); - replaceTop(semiJoin); - return this; - } - - /** - * Creates a {@link Join} with {@link JoinRelType#SEMI}. - * - * @see #semiJoin(Iterable) - */ - public RelBuilder semiJoin(RexNode... conditions) { - return semiJoin(ImmutableList.copyOf(conditions)); - } - - /** - * Creates an anti-join. - * - *

An anti-join is a form of join that combines two relational expressions according to some - * condition, but outputs only rows from the left input for which no rows from the right input - * match. - * - *

For example, {@code EMP anti-join DEPT} finds all {@code EMP} records that do not have a - * corresponding {@code DEPT} record, similar to the following SQL: - * - *

- * - *
-     * SELECT * FROM EMP
-     * WHERE NOT EXISTS (SELECT 1 FROM DEPT
-     *     WHERE DEPT.DEPTNO = EMP.DEPTNO)
- * - *
- */ - public RelBuilder antiJoin(Iterable conditions) { - final Frame right = stack.pop(); - final RelNode antiJoin = - struct.joinFactory.createJoin( - peek(), - right.rel, - ImmutableList.of(), - and(conditions), - ImmutableSet.of(), - JoinRelType.ANTI, - false); - replaceTop(antiJoin); - return this; - } - - /** - * Creates an anti-join. - * - * @see #antiJoin(Iterable) - */ - public RelBuilder antiJoin(RexNode... conditions) { - return antiJoin(ImmutableList.copyOf(conditions)); - } - - /** Assigns a table alias to the top entry on the stack. */ - public RelBuilder as(final String alias) { - final Frame pair = stack.pop(); - List newFields = Util.transform(pair.fields, field -> field.addAlias(alias)); - stack.push(new Frame(pair.rel, ImmutableList.copyOf(newFields))); - return this; - } - - /** - * Creates a {@link Values}. - * - *

The {@code values} array must have the same number of entries as {@code fieldNames}, or an - * integer multiple if you wish to create multiple rows. - * - *

If there are zero rows, or if all values of a any column are null, this method cannot - * deduce the type of columns. For these cases, call {@link #values(Iterable, RelDataType)}. - * - * @param fieldNames Field names - * @param values Values - */ - public RelBuilder values(@Nullable String[] fieldNames, @Nullable Object... values) { - if (fieldNames == null - || fieldNames.length == 0 - || values.length % fieldNames.length != 0 - || values.length < fieldNames.length) { - throw new IllegalArgumentException( - "Value count must be a positive multiple of field count"); - } - final int rowCount = values.length / fieldNames.length; - for (Ord<@Nullable String> fieldName : Ord.zip(fieldNames)) { - if (allNull(values, fieldName.i, fieldNames.length)) { - throw new IllegalArgumentException( - "All values of field '" - + fieldName.e - + "' (field index " - + fieldName.i - + ")" - + " are null; cannot deduce type"); - } - } - final ImmutableList> tupleList = - tupleList(fieldNames.length, values); - assert tupleList.size() == rowCount; - final List fieldNameList = - Util.transformIndexed( - Arrays.asList(fieldNames), - (name, i) -> name != null ? name : SqlUtil.deriveAliasFromOrdinal(i)); - return values(tupleList, fieldNameList); - } - - private RelBuilder values(List> tupleList, List fieldNames) { - final RelDataTypeFactory typeFactory = cluster.getTypeFactory(); - final RelDataTypeFactory.Builder builder = typeFactory.builder(); - Ord.forEach( - fieldNames, - (fieldName, i) -> { - final RelDataType type = - typeFactory.leastRestrictive( - new AbstractList() { - @Override - public RelDataType get(int index) { - return tupleList.get(index).get(i).getType(); - } - - @Override - public int size() { - return tupleList.size(); - } - }); - assert type != null : "can't infer type for field " + i + ", " + fieldName; - builder.add(fieldName, type); - }); - final RelDataType rowType = builder.build(); - return values(tupleList, rowType); - } - - private ImmutableList> tupleList( - int columnCount, @Nullable Object[] values) { - final ImmutableList.Builder> listBuilder = - ImmutableList.builder(); - final List valueList = new ArrayList<>(); - for (int i = 0; i < values.length; i++) { - Object value = values[i]; - valueList.add(literal(value)); - if ((i + 1) % columnCount == 0) { - listBuilder.add(ImmutableList.copyOf(valueList)); - valueList.clear(); - } - } - return listBuilder.build(); - } - - /** Returns whether all values for a given column are null. */ - private static boolean allNull(@Nullable Object[] values, int column, int columnCount) { - for (int i = column; i < values.length; i += columnCount) { - if (values[i] != null) { - return false; - } - } - return true; - } - - /** - * Creates a relational expression that reads from an input and throws all of the rows away. - * - *

Note that this method always pops one relational expression from the stack. {@code - * values}, in contrast, does not pop any relational expressions, and always produces a leaf. - * - *

The default implementation creates a {@link Values} with the same specified row type and - * aliases as the input, and ignores the input entirely. But schema-on-query systems such as - * Drill might override this method to create a relation expression that retains the input, just - * to read its schema. - */ - public RelBuilder empty() { - final Frame frame = stack.pop(); - final RelNode values = - struct.valuesFactory.createValues( - cluster, frame.rel.getRowType(), ImmutableList.of()); - stack.push(new Frame(values, frame.fields)); - return this; - } - - /** - * Creates a {@link Values} with a specified row type. - * - *

This method can handle cases that {@link #values(String[], Object...)} cannot, such as all - * values of a column being null, or there being zero rows. - * - * @param rowType Row type - * @param columnValues Values - */ - public RelBuilder values(RelDataType rowType, Object... columnValues) { - final ImmutableList> tupleList = - tupleList(rowType.getFieldCount(), columnValues); - RelNode values = - struct.valuesFactory.createValues( - cluster, rowType, ImmutableList.copyOf(tupleList)); - push(values); - return this; - } - - /** - * Creates a {@link Values} with a specified row type. - * - *

This method can handle cases that {@link #values(String[], Object...)} cannot, such as all - * values of a column being null, or there being zero rows. - * - * @param tupleList Tuple list - * @param rowType Row type - */ - public RelBuilder values(Iterable> tupleList, RelDataType rowType) { - RelNode values = struct.valuesFactory.createValues(cluster, rowType, copy(tupleList)); - push(values); - return this; - } - - /** - * Creates a {@link Values} with a specified row type and zero rows. - * - * @param rowType Row type - */ - public RelBuilder values(RelDataType rowType) { - return values(ImmutableList.>of(), rowType); - } - - /** - * Converts an iterable of lists into an immutable list of immutable lists with the same - * contents. Returns the same object if possible. - */ - private static ImmutableList> copy(Iterable> tupleList) { - final ImmutableList.Builder> builder = ImmutableList.builder(); - int changeCount = 0; - for (List literals : tupleList) { - final ImmutableList literals2 = ImmutableList.copyOf(literals); - builder.add(literals2); - if (literals != literals2) { - ++changeCount; - } - } - if (changeCount == 0 && tupleList instanceof ImmutableList) { - // don't make a copy if we don't have to - //noinspection unchecked - return (ImmutableList>) tupleList; - } - return builder.build(); - } - - /** Creates a limit without a sort. */ - public RelBuilder limit(int offset, int fetch) { - return sortLimit(offset, fetch, ImmutableList.of()); - } - - /** Creates an Exchange by distribution. */ - public RelBuilder exchange(RelDistribution distribution) { - RelNode exchange = struct.exchangeFactory.createExchange(peek(), distribution); - replaceTop(exchange); - return this; - } - - /** Creates a SortExchange by distribution and collation. */ - public RelBuilder sortExchange(RelDistribution distribution, RelCollation collation) { - RelNode exchange = - struct.sortExchangeFactory.createSortExchange(peek(), distribution, collation); - replaceTop(exchange); - return this; - } - - /** - * Creates a {@link Sort} by field ordinals. - * - *

Negative fields mean descending: -1 means field(0) descending, -2 means field(1) - * descending, etc. - */ - public RelBuilder sort(int... fields) { - final ImmutableList.Builder builder = ImmutableList.builder(); - for (int field : fields) { - builder.add(field < 0 ? desc(field(-field - 1)) : field(field)); - } - return sortLimit(-1, -1, builder.build()); - } - - /** Creates a {@link Sort} by expressions. */ - public RelBuilder sort(RexNode... nodes) { - return sortLimit(-1, -1, ImmutableList.copyOf(nodes)); - } - - /** Creates a {@link Sort} by expressions. */ - public RelBuilder sort(Iterable nodes) { - return sortLimit(-1, -1, nodes); - } - - /** Creates a {@link Sort} by expressions, with limit and offset. */ - public RelBuilder sortLimit(int offset, int fetch, RexNode... nodes) { - return sortLimit(offset, fetch, ImmutableList.copyOf(nodes)); - } - - /** Creates a {@link Sort} by specifying collations. */ - public RelBuilder sort(RelCollation collation) { - final RelNode sort = struct.sortFactory.createSort(peek(), collation, null, null); - replaceTop(sort); - return this; - } - - /** - * Creates a {@link Sort} by a list of expressions, with limit and offset. - * - * @param offset Number of rows to skip; non-positive means don't skip any - * @param fetch Maximum number of rows to fetch; negative means no limit - * @param nodes Sort expressions - */ - public RelBuilder sortLimit(int offset, int fetch, Iterable nodes) { - final @Nullable RexNode offsetNode = offset <= 0 ? null : literal(offset); - final @Nullable RexNode fetchNode = fetch < 0 ? null : literal(fetch); - return sortLimit(offsetNode, fetchNode, nodes); - } - - /** - * Creates a {@link Sort} by a list of expressions, with limitNode and offsetNode. - * - * @param offsetNode RexLiteral means number of rows to skip is deterministic, RexDynamicParam - * means number of rows to skip is dynamic. - * @param fetchNode RexLiteral means maximum number of rows to fetch is deterministic, - * RexDynamicParam mean maximum number is dynamic. - * @param nodes Sort expressions - */ - public RelBuilder sortLimit( - @Nullable RexNode offsetNode, - @Nullable RexNode fetchNode, - Iterable nodes) { - if (offsetNode != null) { - if (!(offsetNode instanceof RexLiteral || offsetNode instanceof RexDynamicParam)) { - throw new IllegalArgumentException( - "OFFSET node must be RexLiteral or RexDynamicParam"); - } - } - if (fetchNode != null) { - if (!(fetchNode instanceof RexLiteral || fetchNode instanceof RexDynamicParam)) { - throw new IllegalArgumentException( - "FETCH node must be RexLiteral or RexDynamicParam"); - } - } - - final Registrar registrar = new Registrar(fields(), ImmutableList.of()); - final List fieldCollations = registrar.registerFieldCollations(nodes); - final int fetch = fetchNode instanceof RexLiteral ? RexLiteral.intValue(fetchNode) : -1; - if (offsetNode == null && fetch == 0 && config.simplifyLimit()) { - return empty(); - } - if (offsetNode == null && fetchNode == null && fieldCollations.isEmpty()) { - return this; // sort is trivial - } - - if (fieldCollations.isEmpty()) { - assert registrar.addedFieldCount() == 0; - RelNode top = peek(); - if (top instanceof Sort) { - final Sort sort2 = (Sort) top; - if (sort2.offset == null && sort2.fetch == null) { - replaceTop(sort2.getInput()); - final RelNode sort = - struct.sortFactory.createSort( - peek(), sort2.collation, offsetNode, fetchNode); - replaceTop(sort); - return this; - } - } - if (top instanceof Project) { - final Project project = (Project) top; - if (project.getInput() instanceof Sort) { - final Sort sort2 = (Sort) project.getInput(); - if (sort2.offset == null && sort2.fetch == null) { - final RelNode sort = - struct.sortFactory.createSort( - sort2.getInput(), sort2.collation, offsetNode, fetchNode); - replaceTop( - struct.projectFactory.createProject( - sort, - project.getHints(), - project.getProjects(), - Pair.right(project.getNamedProjects()), - project.getVariablesSet())); - return this; - } - } - } - } - if (registrar.addedFieldCount() > 0) { - project(registrar.extraNodes); - } - final RelNode sort = - struct.sortFactory.createSort( - peek(), RelCollations.of(fieldCollations), offsetNode, fetchNode); - replaceTop(sort); - if (registrar.addedFieldCount() > 0) { - project(registrar.originalExtraNodes); - } - return this; - } - - private static RelFieldCollation collation( - RexNode node, - RelFieldCollation.Direction direction, - RelFieldCollation.NullDirection nullDirection, - List extraNodes) { - switch (node.getKind()) { - case INPUT_REF: - return new RelFieldCollation( - ((RexInputRef) node).getIndex(), - direction, - Util.first(nullDirection, direction.defaultNullDirection())); - case DESCENDING: - return collation( - ((RexCall) node).getOperands().get(0), - RelFieldCollation.Direction.DESCENDING, - nullDirection, - extraNodes); - case NULLS_FIRST: - return collation( - ((RexCall) node).getOperands().get(0), - direction, - RelFieldCollation.NullDirection.FIRST, - extraNodes); - case NULLS_LAST: - return collation( - ((RexCall) node).getOperands().get(0), - direction, - RelFieldCollation.NullDirection.LAST, - extraNodes); - default: - final int fieldIndex = extraNodes.size(); - extraNodes.add(node); - return new RelFieldCollation( - fieldIndex, - direction, - Util.first(nullDirection, direction.defaultNullDirection())); - } - } - - private static RexFieldCollation rexCollation( - RexNode node, - RelFieldCollation.Direction direction, - RelFieldCollation.NullDirection nullDirection) { - switch (node.getKind()) { - case DESCENDING: - return rexCollation( - ((RexCall) node).operands.get(0), - RelFieldCollation.Direction.DESCENDING, - nullDirection); - case NULLS_LAST: - return rexCollation( - ((RexCall) node).operands.get(0), - direction, - RelFieldCollation.NullDirection.LAST); - case NULLS_FIRST: - return rexCollation( - ((RexCall) node).operands.get(0), - direction, - RelFieldCollation.NullDirection.FIRST); - default: - final Set flags = EnumSet.noneOf(SqlKind.class); - if (direction == RelFieldCollation.Direction.DESCENDING) { - flags.add(SqlKind.DESCENDING); - } - if (nullDirection == RelFieldCollation.NullDirection.FIRST) { - flags.add(SqlKind.NULLS_FIRST); - } - if (nullDirection == RelFieldCollation.NullDirection.LAST) { - flags.add(SqlKind.NULLS_LAST); - } - return new RexFieldCollation(node, flags); - } - } - - /** - * Creates a projection that converts the current relational expression's output to a desired - * row type. - * - *

The desired row type and the row type to be converted must have the same number of fields. - * - * @param castRowType row type after cast - * @param rename if true, use field names from castRowType; if false, preserve field names from - * rel - */ - public RelBuilder convert(RelDataType castRowType, boolean rename) { - final RelNode r = build(); - final RelNode r2 = RelOptUtil.createCastRel(r, castRowType, rename, struct.projectFactory); - push(r2); - return this; - } - - public RelBuilder permute(Mapping mapping) { - assert mapping.getMappingType().isSingleSource(); - assert mapping.getMappingType().isMandatorySource(); - if (mapping.isIdentity()) { - return this; - } - final List exprList = new ArrayList<>(); - for (int i = 0; i < mapping.getTargetCount(); i++) { - exprList.add(field(mapping.getSource(i))); - } - return project(exprList); - } - - /** Creates a {@link Match}. */ - public RelBuilder match( - RexNode pattern, - boolean strictStart, - boolean strictEnd, - Map patternDefinitions, - Iterable measureList, - RexNode after, - Map> subsets, - boolean allRows, - Iterable partitionKeys, - Iterable orderKeys, - RexNode interval) { - final Registrar registrar = new Registrar(fields(), peek().getRowType().getFieldNames()); - final List fieldCollations = - registrar.registerFieldCollations(orderKeys); - - final ImmutableBitSet partitionBitSet = - ImmutableBitSet.of(registrar.registerExpressions(partitionKeys)); - - final RelDataTypeFactory.Builder typeBuilder = cluster.getTypeFactory().builder(); - for (RexNode partitionKey : partitionKeys) { - typeBuilder.add(partitionKey.toString(), partitionKey.getType()); - } - if (allRows) { - for (RexNode orderKey : orderKeys) { - if (!typeBuilder.nameExists(orderKey.toString())) { - typeBuilder.add(orderKey.toString(), orderKey.getType()); - } - } - - final RelDataType inputRowType = peek().getRowType(); - for (RelDataTypeField fs : inputRowType.getFieldList()) { - if (!typeBuilder.nameExists(fs.getName())) { - typeBuilder.add(fs); - } - } - } - - final ImmutableMap.Builder measures = ImmutableMap.builder(); - for (RexNode measure : measureList) { - List operands = ((RexCall) measure).getOperands(); - String alias = operands.get(1).toString(); - typeBuilder.add(alias, operands.get(0).getType()); - measures.put(alias, operands.get(0)); - } - - final RelNode match = - struct.matchFactory.createMatch( - peek(), - pattern, - typeBuilder.build(), - strictStart, - strictEnd, - patternDefinitions, - measures.build(), - after, - subsets, - allRows, - partitionBitSet, - RelCollations.of(fieldCollations), - interval); - stack.push(new Frame(match)); - return this; - } - - /** - * Creates a Pivot. - * - *

To achieve the same effect as the SQL - * - *

- * - *
{@code
-     * SELECT *
-     * FROM (SELECT mgr, deptno, job, sal FROM emp)
-     * PIVOT (SUM(sal) AS ss, COUNT(*) AS c
-     *     FOR (job, deptno)
-     *     IN (('CLERK', 10) AS c10, ('MANAGER', 20) AS m20))
-     * }
- * - *
- * - *

use the builder as follows: - * - *

- * - *
{@code
-     * RelBuilder b;
-     * b.scan("EMP");
-     * final RelBuilder.GroupKey groupKey = b.groupKey("MGR");
-     * final List aggCalls =
-     *     Arrays.asList(b.sum(b.field("SAL")).as("SS"),
-     *         b.count().as("C"));
-     * final List axes =
-     *     Arrays.asList(b.field("JOB"),
-     *         b.field("DEPTNO"));
-     * final ImmutableMap.Builder> valueMap =
-     *     ImmutableMap.builder();
-     * valueMap.put("C10",
-     *     Arrays.asList(b.literal("CLERK"), b.literal(10)));
-     * valueMap.put("M20",
-     *     Arrays.asList(b.literal("MANAGER"), b.literal(20)));
-     * b.pivot(groupKey, aggCalls, axes, valueMap.build().entrySet());
-     * }
- * - *
- * - *

Note that the SQL uses a sub-query to project away columns (e.g. {@code HIREDATE}) that it - * does not reference, so that they do not appear in the {@code GROUP BY}. You do not need to do - * that in this API, because the {@code groupKey} parameter specifies the keys. - * - *

Pivot is implemented by desugaring. The above example becomes the following: - * - *

- * - *
{@code
-     * SELECT mgr,
-     *     SUM(sal) FILTER (WHERE job = 'CLERK' AND deptno = 10) AS c10_ss,
-     *     COUNT(*) FILTER (WHERE job = 'CLERK' AND deptno = 10) AS c10_c,
-     *     SUM(sal) FILTER (WHERE job = 'MANAGER' AND deptno = 20) AS m20_ss,
-     *      COUNT(*) FILTER (WHERE job = 'MANAGER' AND deptno = 20) AS m20_c
-     * FROM emp
-     * GROUP BY mgr
-     * }
- * - *
- * - * @param groupKey Key columns - * @param aggCalls Aggregate expressions to compute for each value - * @param axes Columns to pivot - * @param values Values to pivot, and the alias for each column group - * @return this RelBuilder - */ - public RelBuilder pivot( - GroupKey groupKey, - Iterable aggCalls, - Iterable axes, - Iterable>> values) { - final List axisList = ImmutableList.copyOf(axes); - final List multipliedAggCalls = new ArrayList<>(); - Pair.forEach( - values, - (alias, expressions) -> { - final List expressionList = ImmutableList.copyOf(expressions); - if (expressionList.size() != axisList.size()) { - throw new IllegalArgumentException( - "value count must match axis count [" - + expressionList - + "], [" - + axisList - + "]"); - } - aggCalls.forEach( - aggCall -> { - final String alias2 = alias + "_" + ((AggCallPlus) aggCall).alias(); - final List filters = new ArrayList<>(); - Pair.forEach( - axisList, - expressionList, - (axis, expression) -> - filters.add(equals(axis, expression))); - multipliedAggCalls.add(aggCall.filter(and(filters)).as(alias2)); - }); - }); - return aggregate(groupKey, multipliedAggCalls); - } - - /** - * Creates an Unpivot. - * - *

To achieve the same effect as the SQL - * - *

- * - *
{@code
-     * SELECT *
-     * FROM (SELECT deptno, job, sal, comm FROM emp)
-     *   UNPIVOT INCLUDE NULLS (remuneration
-     *     FOR remuneration_type IN (comm AS 'commission',
-     *                               sal AS 'salary'))
-     * }
- * - *
- * - *

use the builder as follows: - * - *

- * - *
{@code
-     * RelBuilder b;
-     * b.scan("EMP");
-     * final List measureNames = Arrays.asList("REMUNERATION");
-     * final List axisNames = Arrays.asList("REMUNERATION_TYPE");
-     * final Map, List> axisMap =
-     *     ImmutableMap., List>builder()
-     *         .put(Arrays.asList(b.literal("commission")),
-     *             Arrays.asList(b.field("COMM")))
-     *         .put(Arrays.asList(b.literal("salary")),
-     *             Arrays.asList(b.field("SAL")))
-     *         .build();
-     * b.unpivot(false, measureNames, axisNames, axisMap);
-     * }
- * - *
- * - *

The query generates two columns: {@code remuneration_type} (an axis column) and {@code - * remuneration} (a measure column). Axis columns contain values to indicate the source of the - * row (in this case, {@code 'salary'} if the row came from the {@code sal} column, and {@code - * 'commission'} if the row came from the {@code comm} column). - * - * @param includeNulls Whether to include NULL values in the output - * @param measureNames Names of columns to be generated to hold pivoted measures - * @param axisNames Names of columns to be generated to hold qualifying values - * @param axisMap Mapping from the columns that hold measures to the values that the axis - * columns will hold in the generated rows - * @return This RelBuilder - */ - public RelBuilder unpivot( - boolean includeNulls, - Iterable measureNames, - Iterable axisNames, - Iterable< - ? extends - Map.Entry< - ? extends List, - ? extends List>> - axisMap) { - // Make immutable copies of all arguments. - final List measureNameList = ImmutableList.copyOf(measureNames); - final List axisNameList = ImmutableList.copyOf(axisNames); - final List, List>> map = - StreamSupport.stream(axisMap.spliterator(), false) - .map( - pair -> - Pair., List>of( - ImmutableList.copyOf(pair.getKey()), - ImmutableList.copyOf(pair.getValue()))) - .collect(Util.toImmutableList()); - - // Check that counts match. - Pair.forEach( - map, - (valueList, inputMeasureList) -> { - if (inputMeasureList.size() != measureNameList.size()) { - throw new IllegalArgumentException( - "Number of measures (" - + inputMeasureList.size() - + ") must match number of measure names (" - + measureNameList.size() - + ")"); - } - if (valueList.size() != axisNameList.size()) { - throw new IllegalArgumentException( - "Number of axis values (" - + valueList.size() - + ") match match number of axis names (" - + axisNameList.size() - + ")"); - } - }); - - final RelDataType leftRowType = peek().getRowType(); - final BitSet usedFields = new BitSet(); - Pair.forEach( - map, - (aliases, nodes) -> - nodes.forEach( - node -> { - if (node instanceof RexInputRef) { - usedFields.set(((RexInputRef) node).getIndex()); - } - })); - - // Create "VALUES (('commission'), ('salary')) AS t (remuneration_type)" - values(ImmutableList.copyOf(Pair.left(map)), axisNameList); - - join(JoinRelType.INNER); - - final ImmutableBitSet unusedFields = - ImmutableBitSet.range(leftRowType.getFieldCount()) - .except(ImmutableBitSet.fromBitSet(usedFields)); - final List projects = new ArrayList<>(fields(unusedFields)); - Ord.forEach( - axisNameList, - (dimensionName, d) -> - projects.add(alias(field(leftRowType.getFieldCount() + d), dimensionName))); - - final List conditions = new ArrayList<>(); - Ord.forEach( - measureNameList, - (measureName, m) -> { - final List caseOperands = new ArrayList<>(); - Pair.forEach( - map, - (literals, nodes) -> { - Ord.forEach( - literals, - (literal, d) -> - conditions.add( - equals( - field( - leftRowType.getFieldCount() - + d), - literal))); - caseOperands.add(and(conditions)); - conditions.clear(); - caseOperands.add(nodes.get(m)); - }); - caseOperands.add(literal(null)); - projects.add(alias(call(SqlStdOperatorTable.CASE, caseOperands), measureName)); - }); - project(projects); - - if (!includeNulls) { - // Add 'WHERE m1 IS NOT NULL OR m2 IS NOT NULL' - final BitSet notNullFields = new BitSet(); - Ord.forEach( - measureNameList, - (measureName, m) -> { - final int f = unusedFields.cardinality() + axisNameList.size() + m; - conditions.add(isNotNull(field(f))); - notNullFields.set(f); - }); - filter(or(conditions)); - if (measureNameList.size() == 1) { - // If there is one field, EXCLUDE NULLS will have converted it to NOT - // NULL. - final RelDataTypeFactory.Builder builder = getTypeFactory().builder(); - peek().getRowType() - .getFieldList() - .forEach( - field -> { - final RelDataType type = field.getType(); - builder.add( - field.getName(), - notNullFields.get(field.getIndex()) - ? getTypeFactory() - .createTypeWithNullability(type, false) - : type); - }); - convert(builder.build(), false); - } - conditions.clear(); - } - - return this; - } - - /** - * Attaches an array of hints to the stack top relational expression. - * - *

The redundant hints would be eliminated. - * - * @param hints Hints - * @throws AssertionError if the top relational expression does not implement {@link - * org.apache.calcite.rel.hint.Hintable} - */ - public RelBuilder hints(RelHint... hints) { - return hints(ImmutableList.copyOf(hints)); - } - - /** - * Attaches multiple hints to the stack top relational expression. - * - *

The redundant hints would be eliminated. - * - * @param hints Hints - * @throws AssertionError if the top relational expression does not implement {@link - * org.apache.calcite.rel.hint.Hintable} - */ - public RelBuilder hints(Iterable hints) { - requireNonNull(hints, "hints"); - final List relHintList = - hints instanceof List ? (List) hints : Lists.newArrayList(hints); - if (relHintList.isEmpty()) { - return this; - } - final Frame frame = peek_(); - assert frame != null : "There is no relational expression to attach the hints"; - assert frame.rel instanceof Hintable : "The top relational expression is not a Hintable"; - Hintable hintable = (Hintable) frame.rel; - replaceTop(hintable.attachHints(relHintList)); - return this; - } - - /** - * Clears the stack. - * - *

The builder's state is now the same as when it was created. - */ - public void clear() { - stack.clear(); - } - - /** - * Information necessary to create a call to an aggregate function. - * - * @see RelBuilder#aggregateCall - */ - public interface AggCall { - /** Returns a copy of this AggCall that applies a filter before aggregating values. */ - AggCall filter(@Nullable RexNode condition); - - /** - * Returns a copy of this AggCall that sorts its input values by {@code orderKeys} before - * aggregating, as in SQL's {@code WITHIN GROUP} clause. - */ - AggCall sort(Iterable orderKeys); - - /** - * Returns a copy of this AggCall that sorts its input values by {@code orderKeys} before - * aggregating, as in SQL's {@code WITHIN GROUP} clause. - */ - default AggCall sort(RexNode... orderKeys) { - return sort(ImmutableList.copyOf(orderKeys)); - } - - /** - * Returns a copy of this AggCall that makes its input values unique by {@code distinctKeys} - * before aggregating, as in SQL's {@code WITHIN DISTINCT} clause. - */ - AggCall unique(@Nullable Iterable distinctKeys); - - /** - * Returns a copy of this AggCall that makes its input values unique by {@code distinctKeys} - * before aggregating, as in SQL's {@code WITHIN DISTINCT} clause. - */ - default AggCall unique(RexNode... distinctKeys) { - return unique(ImmutableList.copyOf(distinctKeys)); - } - - /** - * Returns a copy of this AggCall that may return approximate results if {@code approximate} - * is true. - */ - AggCall approximate(boolean approximate); - - /** Returns a copy of this AggCall that ignores nulls. */ - AggCall ignoreNulls(boolean ignoreNulls); - - /** Returns a copy of this AggCall with a given alias. */ - AggCall as(@Nullable String alias); - - /** Returns a copy of this AggCall that is optionally distinct. */ - AggCall distinct(boolean distinct); - - /** Returns a copy of this AggCall that is distinct. */ - default AggCall distinct() { - return distinct(true); - } - - /** Converts this aggregate call to a windowed aggregate call. */ - OverCall over(); - } - - /** Internal methods shared by all implementations of {@link AggCall}. */ - private interface AggCallPlus extends AggCall { - /** Returns the aggregate function. */ - SqlAggFunction op(); - - /** Returns the alias. */ - @Nullable String alias(); - - /** - * Returns an {@link AggregateCall} that is approximately equivalent to this {@code AggCall} - * and is good for certain things, such as deriving field names. - */ - AggregateCall aggregateCall(); - - /** Converts this {@code AggCall} to a good {@link AggregateCall}. */ - AggregateCall aggregateCall(Registrar registrar, ImmutableBitSet groupSet, RelNode r); - - /** Registers expressions in operands and filters. */ - void register(Registrar registrar); - } - - /** - * Information necessary to create the GROUP BY clause of an Aggregate. - * - * @see RelBuilder#groupKey - */ - public interface GroupKey { - /** - * Assigns an alias to this group key. - * - *

Used to assign field names in the {@code group} operation. - */ - GroupKey alias(@Nullable String alias); - - /** Returns the number of columns in the group key. */ - int groupKeyCount(); - } - - /** Implementation of {@link RelBuilder.GroupKey}. */ - static class GroupKeyImpl implements GroupKey { - final ImmutableList nodes; - final @Nullable ImmutableList> nodeLists; - final @Nullable String alias; - - GroupKeyImpl( - ImmutableList nodes, - @Nullable ImmutableList> nodeLists, - @Nullable String alias) { - this.nodes = requireNonNull(nodes, "nodes"); - this.nodeLists = nodeLists; - this.alias = alias; - } - - @Override - public String toString() { - return alias == null ? nodes.toString() : nodes + " as " + alias; - } - - @Override - public int groupKeyCount() { - return nodes.size(); - } - - @Override - public GroupKey alias(@Nullable String alias) { - return Objects.equals(this.alias, alias) - ? this - : new GroupKeyImpl(nodes, nodeLists, alias); - } - - boolean isSimple() { - return nodeLists == null || nodeLists.size() == 1; - } - } - - /** - * Checks for {@link CorrelationId}, then validates the id is not used on left, and finally - * checks if id is actually used on right. - * - * @return true if a correlate id is present and used - * @throws IllegalArgumentException if the {@link CorrelationId} is used by left side or if the - * a {@link CorrelationId} is present and the {@link JoinRelType} is FULL or RIGHT. - */ - private static boolean checkIfCorrelated( - Set variablesSet, - JoinRelType joinType, - RelNode leftNode, - RelNode rightRel) { - if (variablesSet.size() != 1) { - return false; - } - CorrelationId id = Iterables.getOnlyElement(variablesSet); - if (!RelOptUtil.notContainsCorrelation(leftNode, id, Litmus.IGNORE)) { - throw new IllegalArgumentException( - "variable " + id + " must not be used by left input to correlation"); - } - switch (joinType) { - case RIGHT: - case FULL: - throw new IllegalArgumentException( - "Correlated " + joinType + " join is not supported"); - default: - return !RelOptUtil.correlationColumns( - Iterables.getOnlyElement(variablesSet), rightRel) - .isEmpty(); - } - } - - /** Implementation of {@link AggCall}. */ - private class AggCallImpl implements AggCallPlus { - private final SqlAggFunction aggFunction; - private final boolean distinct; - private final boolean approximate; - private final boolean ignoreNulls; - private final @Nullable RexNode filter; - private final @Nullable String alias; - private final ImmutableList operands; // may be empty - private final @Nullable ImmutableList distinctKeys; // may be empty or null - private final ImmutableList orderKeys; // may be empty - - AggCallImpl( - SqlAggFunction aggFunction, - boolean distinct, - boolean approximate, - boolean ignoreNulls, - @Nullable RexNode filter, - @Nullable String alias, - ImmutableList operands, - @Nullable ImmutableList distinctKeys, - ImmutableList orderKeys) { - this.aggFunction = requireNonNull(aggFunction, "aggFunction"); - // If the aggregate function ignores DISTINCT, - // make the DISTINCT flag FALSE. - this.distinct = distinct && aggFunction.getDistinctOptionality() != Optionality.IGNORED; - this.approximate = approximate; - this.ignoreNulls = ignoreNulls; - this.alias = alias; - this.operands = requireNonNull(operands, "operands"); - this.distinctKeys = distinctKeys; - this.orderKeys = requireNonNull(orderKeys, "orderKeys"); - if (filter != null) { - if (filter.getType().getSqlTypeName() != SqlTypeName.BOOLEAN) { - throw RESOURCE.filterMustBeBoolean().ex(); - } - if (filter.getType().isNullable()) { - filter = call(SqlStdOperatorTable.IS_TRUE, filter); - } - } - this.filter = filter; - } - - @Override - public String toString() { - final StringBuilder b = new StringBuilder(); - b.append(aggFunction.getName()).append('('); - if (distinct) { - b.append("DISTINCT "); - } - if (operands.size() > 0) { - b.append(operands.get(0)); - for (int i = 1; i < operands.size(); i++) { - b.append(", "); - b.append(operands.get(i)); - } - } - b.append(')'); - if (filter != null) { - b.append(" FILTER (WHERE ").append(filter).append(')'); - } - if (distinctKeys != null) { - b.append(" WITHIN DISTINCT (").append(distinctKeys).append(')'); - } - return b.toString(); - } - - @Override - public SqlAggFunction op() { - return aggFunction; - } - - @Override - public @Nullable String alias() { - return alias; - } - - @Override - public AggregateCall aggregateCall() { - // Use dummy values for collation and type. This method only promises to - // return a call that is "approximately equivalent ... and is good for - // deriving field names", so dummy values are good enough. - final RelCollation collation = RelCollations.EMPTY; - final RelDataType type = getTypeFactory().createSqlType(SqlTypeName.BOOLEAN); - return AggregateCall.create( - aggFunction, - distinct, - approximate, - ignoreNulls, - ImmutableList.of(), - -1, - null, - collation, - type, - alias); - } - - @Override - public AggregateCall aggregateCall( - Registrar registrar, ImmutableBitSet groupSet, RelNode r) { - List args = registrar.registerExpressions(this.operands); - final int filterArg = - this.filter == null ? -1 : registrar.registerExpression(this.filter); - if (this.distinct && !this.aggFunction.isQuantifierAllowed()) { - throw new IllegalArgumentException("DISTINCT not allowed"); - } - if (this.filter != null && !this.aggFunction.allowsFilter()) { - throw new IllegalArgumentException("FILTER not allowed"); - } - final @Nullable ImmutableBitSet distinctKeys = - this.distinctKeys == null - ? null - : ImmutableBitSet.of(registrar.registerExpressions(this.distinctKeys)); - final RelCollation collation = - RelCollations.of( - this.orderKeys.stream() - .map( - orderKey -> - collation( - orderKey, - RelFieldCollation.Direction.ASCENDING, - null, - Collections.emptyList())) - .collect(Collectors.toList())); - if (aggFunction instanceof SqlCountAggFunction && !distinct) { - args = args.stream().filter(r::fieldIsNullable).collect(Util.toImmutableList()); - } - return AggregateCall.create( - aggFunction, - distinct, - approximate, - ignoreNulls, - args, - filterArg, - distinctKeys, - collation, - groupSet.cardinality(), - r, - null, - alias); - } - - @Override - public void register(Registrar registrar) { - registrar.registerExpressions(operands); - if (filter != null) { - registrar.registerExpression(filter); - } - if (distinctKeys != null) { - registrar.registerExpressions(distinctKeys); - } - registrar.registerExpressions(orderKeys); - } - - @Override - public OverCall over() { - return new OverCallImpl(aggFunction, distinct, operands, ignoreNulls, alias); - } - - @Override - public AggCall sort(Iterable orderKeys) { - final ImmutableList orderKeyList = ImmutableList.copyOf(orderKeys); - return orderKeyList.equals(this.orderKeys) - ? this - : new AggCallImpl( - aggFunction, - distinct, - approximate, - ignoreNulls, - filter, - alias, - operands, - distinctKeys, - orderKeyList); - } - - @Override - public AggCall sort(RexNode... orderKeys) { - return sort(ImmutableList.copyOf(orderKeys)); - } - - @Override - public AggCall unique(@Nullable Iterable distinctKeys) { - final @Nullable ImmutableList distinctKeyList = - distinctKeys == null ? null : ImmutableList.copyOf(distinctKeys); - return Objects.equals(distinctKeyList, this.distinctKeys) - ? this - : new AggCallImpl( - aggFunction, - distinct, - approximate, - ignoreNulls, - filter, - alias, - operands, - distinctKeyList, - orderKeys); - } - - @Override - public AggCall approximate(boolean approximate) { - return approximate == this.approximate - ? this - : new AggCallImpl( - aggFunction, - distinct, - approximate, - ignoreNulls, - filter, - alias, - operands, - distinctKeys, - orderKeys); - } - - @Override - public AggCall filter(@Nullable RexNode condition) { - return Objects.equals(condition, this.filter) - ? this - : new AggCallImpl( - aggFunction, - distinct, - approximate, - ignoreNulls, - condition, - alias, - operands, - distinctKeys, - orderKeys); - } - - @Override - public AggCall as(@Nullable String alias) { - return Objects.equals(alias, this.alias) - ? this - : new AggCallImpl( - aggFunction, - distinct, - approximate, - ignoreNulls, - filter, - alias, - operands, - distinctKeys, - orderKeys); - } - - @Override - public AggCall distinct(boolean distinct) { - return distinct == this.distinct - ? this - : new AggCallImpl( - aggFunction, - distinct, - approximate, - ignoreNulls, - filter, - alias, - operands, - distinctKeys, - orderKeys); - } - - @Override - public AggCall ignoreNulls(boolean ignoreNulls) { - return ignoreNulls == this.ignoreNulls - ? this - : new AggCallImpl( - aggFunction, - distinct, - approximate, - ignoreNulls, - filter, - alias, - operands, - distinctKeys, - orderKeys); - } - } - - /** Implementation of {@link AggCall} that wraps an {@link AggregateCall}. */ - private class AggCallImpl2 implements AggCallPlus { - private final AggregateCall aggregateCall; - private final ImmutableList operands; - - AggCallImpl2(AggregateCall aggregateCall, ImmutableList operands) { - this.aggregateCall = requireNonNull(aggregateCall, "aggregateCall"); - this.operands = requireNonNull(operands, "operands"); - } - - @Override - public OverCall over() { - return new OverCallImpl( - aggregateCall.getAggregation(), - aggregateCall.isDistinct(), - operands, - aggregateCall.ignoreNulls(), - aggregateCall.name); - } - - @Override - public String toString() { - return aggregateCall.toString(); - } - - @Override - public SqlAggFunction op() { - return aggregateCall.getAggregation(); - } - - @Override - public @Nullable String alias() { - return aggregateCall.name; - } - - @Override - public AggregateCall aggregateCall() { - return aggregateCall; - } - - @Override - public AggregateCall aggregateCall( - Registrar registrar, ImmutableBitSet groupSet, RelNode r) { - return aggregateCall; - } - - @Override - public void register(Registrar registrar) { - // nothing to do - } - - @Override - public AggCall sort(Iterable orderKeys) { - throw new UnsupportedOperationException(); - } - - @Override - public AggCall sort(RexNode... orderKeys) { - throw new UnsupportedOperationException(); - } - - @Override - public AggCall unique(@Nullable Iterable distinctKeys) { - throw new UnsupportedOperationException(); - } - - @Override - public AggCall approximate(boolean approximate) { - throw new UnsupportedOperationException(); - } - - @Override - public AggCall filter(@Nullable RexNode condition) { - throw new UnsupportedOperationException(); - } - - @Override - public AggCall as(@Nullable String alias) { - throw new UnsupportedOperationException(); - } - - @Override - public AggCall distinct(boolean distinct) { - throw new UnsupportedOperationException(); - } - - @Override - public AggCall ignoreNulls(boolean ignoreNulls) { - throw new UnsupportedOperationException(); - } - } - - /** - * Call to a windowed aggregate function. - * - *

To create an {@code OverCall}, start with an {@link AggCall} (created by a method such as - * {@link #aggregateCall}, {@link #sum} or {@link #count}) and call its {@link AggCall#over()} - * method. For example, - * - *

{@code
-     * b.scan("EMP")
-     *    .project(b.field("DEPTNO"),
-     *       b.aggregateCall(SqlStdOperatorTable.ROW_NUMBER)
-     *          .over()
-     *          .partitionBy()
-     *          .orderBy(b.field("EMPNO"))
-     *          .rowsUnbounded()
-     *          .allowPartial(true)
-     *          .nullWhenCountZero(false)
-     *          .as("x"))
-     * }
- * - *

Unlike an aggregate call, a windowed aggregate call is an expression that you can use in a - * {@link Project} or {@link Filter}. So, to finish, call {@link OverCall#toRex()} to convert - * the {@code OverCall} to a {@link RexNode}; the {@link OverCall#as} method (used in the above - * example) does the same but also assigns an column alias. - */ - public interface OverCall { - /** Performs an action on this OverCall. */ - default R let(Function consumer) { - return consumer.apply(this); - } - - /** Sets the PARTITION BY clause to an array of expressions. */ - OverCall partitionBy(RexNode... expressions); - - /** Sets the PARTITION BY clause to a list of expressions. */ - OverCall partitionBy(Iterable expressions); - - /** - * Sets the ORDER BY BY clause to an array of expressions. - * - *

Use {@link #desc(RexNode)}, {@link #nullsFirst(RexNode)}, {@link #nullsLast(RexNode)} - * to control the sort order. - */ - OverCall orderBy(RexNode... expressions); - - /** - * Sets the ORDER BY BY clause to a list of expressions. - * - *

Use {@link #desc(RexNode)}, {@link #nullsFirst(RexNode)}, {@link #nullsLast(RexNode)} - * to control the sort order. - */ - OverCall orderBy(Iterable expressions); - - /** - * Sets an unbounded ROWS window, equivalent to SQL {@code ROWS BETWEEN UNBOUNDED PRECEDING - * AND UNBOUNDED FOLLOWING}. - */ - default OverCall rowsUnbounded() { - return rowsBetween( - RexWindowBounds.UNBOUNDED_PRECEDING, RexWindowBounds.UNBOUNDED_FOLLOWING); - } - - /** - * Sets a ROWS window with a lower bound, equivalent to SQL {@code ROWS BETWEEN lower AND - * CURRENT ROW}. - */ - default OverCall rowsFrom(RexWindowBound lower) { - return rowsBetween(lower, RexWindowBounds.UNBOUNDED_FOLLOWING); - } - - /** - * Sets a ROWS window with an upper bound, equivalent to SQL {@code ROWS BETWEEN CURRENT ROW - * AND upper}. - */ - default OverCall rowsTo(RexWindowBound upper) { - return rowsBetween(RexWindowBounds.UNBOUNDED_PRECEDING, upper); - } - - /** - * Sets a RANGE window with lower and upper bounds, equivalent to SQL {@code ROWS BETWEEN - * lower ROW AND upper}. - */ - OverCall rowsBetween(RexWindowBound lower, RexWindowBound upper); - - /** - * Sets an unbounded RANGE window, equivalent to SQL {@code RANGE BETWEEN UNBOUNDED - * PRECEDING AND UNBOUNDED FOLLOWING}. - */ - default OverCall rangeUnbounded() { - return rangeBetween( - RexWindowBounds.UNBOUNDED_PRECEDING, RexWindowBounds.UNBOUNDED_FOLLOWING); - } - - /** - * Sets a RANGE window with a lower bound, equivalent to SQL {@code RANGE BETWEEN lower AND - * CURRENT ROW}. - */ - default OverCall rangeFrom(RexWindowBound lower) { - return rangeBetween(lower, RexWindowBounds.CURRENT_ROW); - } - - /** - * Sets a RANGE window with an upper bound, equivalent to SQL {@code RANGE BETWEEN CURRENT - * ROW AND upper}. - */ - default OverCall rangeTo(RexWindowBound upper) { - return rangeBetween(RexWindowBounds.UNBOUNDED_PRECEDING, upper); - } - - /** - * Sets a RANGE window with lower and upper bounds, equivalent to SQL {@code RANGE BETWEEN - * lower ROW AND upper}. - */ - OverCall rangeBetween(RexWindowBound lower, RexWindowBound upper); - - /** Sets whether to allow partial width windows; default true. */ - OverCall allowPartial(boolean allowPartial); - - /** - * Sets whether the aggregate function should evaluate to null if no rows are in the window; - * default false. - */ - OverCall nullWhenCountZero(boolean nullWhenCountZero); - - /** - * Sets the alias of this expression, and converts it to a {@link RexNode}; default is the - * alias that was set via {@link AggCall#as(String)}. - */ - RexNode as(String alias); - - /** Converts this expression to a {@link RexNode}. */ - RexNode toRex(); - } - - /** Implementation of {@link OverCall}. */ - private class OverCallImpl implements OverCall { - private final ImmutableList operands; - private final boolean ignoreNulls; - private final @Nullable String alias; - private final boolean nullWhenCountZero; - private final boolean allowPartial; - private final boolean rows; - private final RexWindowBound lowerBound; - private final RexWindowBound upperBound; - private final ImmutableList partitionKeys; - private final ImmutableList sortKeys; - private final SqlAggFunction op; - private final boolean distinct; - - private OverCallImpl( - SqlAggFunction op, - boolean distinct, - ImmutableList operands, - boolean ignoreNulls, - @Nullable String alias, - ImmutableList partitionKeys, - ImmutableList sortKeys, - boolean rows, - RexWindowBound lowerBound, - RexWindowBound upperBound, - boolean nullWhenCountZero, - boolean allowPartial) { - this.op = op; - this.distinct = distinct; - this.operands = operands; - this.ignoreNulls = ignoreNulls; - this.alias = alias; - this.partitionKeys = partitionKeys; - this.sortKeys = sortKeys; - this.nullWhenCountZero = nullWhenCountZero; - this.allowPartial = allowPartial; - this.rows = rows; - this.lowerBound = lowerBound; - this.upperBound = upperBound; - } - - /** Creates an OverCallImpl with default settings. */ - OverCallImpl( - SqlAggFunction op, - boolean distinct, - ImmutableList operands, - boolean ignoreNulls, - @Nullable String alias) { - this( - op, - distinct, - operands, - ignoreNulls, - alias, - ImmutableList.of(), - ImmutableList.of(), - true, - RexWindowBounds.UNBOUNDED_PRECEDING, - RexWindowBounds.UNBOUNDED_FOLLOWING, - false, - true); - } - - @Override - public OverCall partitionBy(Iterable expressions) { - return partitionBy_(ImmutableList.copyOf(expressions)); - } - - @Override - public OverCall partitionBy(RexNode... expressions) { - return partitionBy_(ImmutableList.copyOf(expressions)); - } - - private OverCall partitionBy_(ImmutableList partitionKeys) { - return new OverCallImpl( - op, - distinct, - operands, - ignoreNulls, - alias, - partitionKeys, - sortKeys, - rows, - lowerBound, - upperBound, - nullWhenCountZero, - allowPartial); - } - - private OverCall orderBy_(ImmutableList sortKeys) { - return new OverCallImpl( - op, - distinct, - operands, - ignoreNulls, - alias, - partitionKeys, - sortKeys, - rows, - lowerBound, - upperBound, - nullWhenCountZero, - allowPartial); - } - - @Override - public OverCall orderBy(Iterable sortKeys) { - ImmutableList.Builder fieldCollations = ImmutableList.builder(); - sortKeys.forEach( - sortKey -> - fieldCollations.add( - rexCollation( - sortKey, - RelFieldCollation.Direction.ASCENDING, - RelFieldCollation.NullDirection.UNSPECIFIED))); - return orderBy_(fieldCollations.build()); - } - - @Override - public OverCall orderBy(RexNode... sortKeys) { - return orderBy(Arrays.asList(sortKeys)); - } - - @Override - public OverCall rowsBetween(RexWindowBound lowerBound, RexWindowBound upperBound) { - return new OverCallImpl( - op, - distinct, - operands, - ignoreNulls, - alias, - partitionKeys, - sortKeys, - true, - lowerBound, - upperBound, - nullWhenCountZero, - allowPartial); - } - - @Override - public OverCall rangeBetween(RexWindowBound lowerBound, RexWindowBound upperBound) { - return new OverCallImpl( - op, - distinct, - operands, - ignoreNulls, - alias, - partitionKeys, - sortKeys, - false, - lowerBound, - upperBound, - nullWhenCountZero, - allowPartial); - } - - @Override - public OverCall allowPartial(boolean allowPartial) { - return new OverCallImpl( - op, - distinct, - operands, - ignoreNulls, - alias, - partitionKeys, - sortKeys, - rows, - lowerBound, - upperBound, - nullWhenCountZero, - allowPartial); - } - - @Override - public OverCall nullWhenCountZero(boolean nullWhenCountZero) { - return new OverCallImpl( - op, - distinct, - operands, - ignoreNulls, - alias, - partitionKeys, - sortKeys, - rows, - lowerBound, - upperBound, - nullWhenCountZero, - allowPartial); - } - - @Override - public RexNode as(String alias) { - return new OverCallImpl( - op, - distinct, - operands, - ignoreNulls, - alias, - partitionKeys, - sortKeys, - rows, - lowerBound, - upperBound, - nullWhenCountZero, - allowPartial) - .toRex(); - } - - @Override - public RexNode toRex() { - final RexCallBinding bind = - new RexCallBinding(getTypeFactory(), op, operands, ImmutableList.of()) { - @Override - public int getGroupCount() { - return SqlWindow.isAlwaysNonEmpty(lowerBound, upperBound) ? 1 : 0; - } - }; - final RelDataType type = op.inferReturnType(bind); - final RexNode over = - getRexBuilder() - .makeOver( - type, - op, - operands, - partitionKeys, - sortKeys, - lowerBound, - upperBound, - rows, - allowPartial, - nullWhenCountZero, - distinct, - ignoreNulls); - return alias == null ? over : alias(over, alias); - } - } - - /** - * Collects the extra expressions needed for {@link #aggregate}. - * - *

The extra expressions come from the group key and as arguments to aggregate calls, and - * later there will be a {@link #project} or a {@link #rename(List)} if necessary. - */ - private static class Registrar { - final List originalExtraNodes; - final List extraNodes; - final List<@Nullable String> names; - - Registrar(Iterable fields, List fieldNames) { - originalExtraNodes = ImmutableList.copyOf(fields); - extraNodes = new ArrayList<>(originalExtraNodes); - names = new ArrayList<>(fieldNames); - } - - int registerExpression(RexNode node) { - switch (node.getKind()) { - case AS: - final List operands = ((RexCall) node).operands; - final int i = registerExpression(operands.get(0)); - names.set(i, RexLiteral.stringValue(operands.get(1))); - return i; - case DESCENDING: - case NULLS_FIRST: - case NULLS_LAST: - return registerExpression(((RexCall) node).operands.get(0)); - default: - final int i2 = extraNodes.indexOf(node); - if (i2 >= 0) { - return i2; - } - extraNodes.add(node); - names.add(null); - return extraNodes.size() - 1; - } - } - - List registerExpressions(Iterable nodes) { - final List builder = new ArrayList<>(); - for (RexNode node : nodes) { - builder.add(registerExpression(node)); - } - return builder; - } - - List registerFieldCollations(Iterable orderKeys) { - final List fieldCollations = new ArrayList<>(); - for (RexNode orderKey : orderKeys) { - final RelFieldCollation collation = - collation( - orderKey, RelFieldCollation.Direction.ASCENDING, null, extraNodes); - if (!RelCollations.ordinals(fieldCollations).contains(collation.getFieldIndex())) { - fieldCollations.add(collation); - } - } - return ImmutableList.copyOf(fieldCollations); - } - - /** Returns the number of fields added. */ - int addedFieldCount() { - return extraNodes.size() - originalExtraNodes.size(); - } - } - - /** - * Builder stack frame. - * - *

Describes a previously created relational expression and information about how table - * aliases map into its row type. - */ - private static class Frame { - final RelNode rel; - final ImmutableList fields; - - private Frame(RelNode rel, ImmutableList fields) { - this.rel = rel; - this.fields = fields; - } - - private Frame(RelNode rel) { - String tableAlias = deriveAlias(rel); - ImmutableList.Builder builder = ImmutableList.builder(); - ImmutableSet aliases = - tableAlias == null ? ImmutableSet.of() : ImmutableSet.of(tableAlias); - for (RelDataTypeField field : rel.getRowType().getFieldList()) { - builder.add(new Field(aliases, field)); - } - this.rel = rel; - this.fields = builder.build(); - } - - @Override - public String toString() { - return rel + ": " + fields; - } - - private static @Nullable String deriveAlias(RelNode rel) { - if (rel instanceof TableScan) { - TableScan scan = (TableScan) rel; - final List names = scan.getTable().getQualifiedName(); - if (!names.isEmpty()) { - return Util.last(names); - } - } - return null; - } - - List fields() { - return Pair.right(fields); - } - } - - /** A field that belongs to a stack {@link Frame}. */ - private static class Field extends Pair, RelDataTypeField> { - Field(ImmutableSet left, RelDataTypeField right) { - super(left, right); - } - - Field addAlias(String alias) { - if (left.contains(alias)) { - return this; - } - final ImmutableSet aliasList = - ImmutableSet.builder().addAll(left).add(alias).build(); - return new Field(aliasList, right); - } - } - - /** - * Shuttle that shifts a predicate's inputs to the left, replacing early ones with references to - * a {@link RexCorrelVariable}. - */ - private class Shifter extends RexShuttle { - private final RelNode left; - private final CorrelationId id; - private final RelNode right; - - Shifter(RelNode left, CorrelationId id, RelNode right) { - this.left = left; - this.id = id; - this.right = right; - } - - @Override - public RexNode visitInputRef(RexInputRef inputRef) { - final RelDataType leftRowType = left.getRowType(); - final RexBuilder rexBuilder = getRexBuilder(); - final int leftCount = leftRowType.getFieldCount(); - if (inputRef.getIndex() < leftCount) { - final RexNode v = rexBuilder.makeCorrel(leftRowType, id); - return rexBuilder.makeFieldAccess(v, inputRef.getIndex()); - } else { - return rexBuilder.makeInputRef(right, inputRef.getIndex() - leftCount); - } - } - } - - /** - * Configuration of RelBuilder. - * - *

It is immutable, and all fields are public. - * - *

Start with the {@link #DEFAULT} instance, and call {@code withXxx} methods to set its - * properties. - */ - @Value.Immutable - public interface Config { - /** Default configuration. */ - Config DEFAULT = ImmutableRelBuilder.Config.of(); - - /** - * Controls whether to merge two {@link Project} operators when inlining expressions causes - * complexity to increase. - * - *

Usually merging projects is beneficial, but occasionally the result is more complex - * than the original projects. Consider: - * - *

-         * P: Project(a+b+c AS x, d+e+f AS y, g+h+i AS z)  # complexity 15
-         * Q: Project(x*y*z AS p, x-y-z AS q)              # complexity 10
-         * R: Project((a+b+c)*(d+e+f)*(g+h+i) AS s,
-         *            (a+b+c)-(d+e+f)-(g+h+i) AS t)        # complexity 34
-         * 
- * - * The complexity of an expression is the number of nodes (leaves and operators). For - * example, {@code a+b+c} has complexity 5 (3 field references and 2 calls): - * - *
-         *       +
-         *      /  \
-         *     +    c
-         *    / \
-         *   a   b
-         * 
- * - *

A negative value never allows merges. - * - *

A zero or positive value, {@code bloat}, allows a merge if complexity of the result is - * less than or equal to the sum of the complexity of the originals plus {@code bloat}. - * - *

The default value, 100, allows a moderate increase in complexity but prevents cases - * where complexity would run away into the millions and run out of memory. Moderate - * complexity is OK; the implementation, say via {@link - * org.apache.calcite.adapter.enumerable.EnumerableCalc}, will often gather common - * sub-expressions and compute them only once. - */ - @Value.Default - default int bloat() { - return 100; - } - - /** Sets {@link #bloat}. */ - Config withBloat(int bloat); - - /** - * Whether {@link RelBuilder#aggregate} should eliminate duplicate aggregate calls; default - * true. - */ - @Value.Default - default boolean dedupAggregateCalls() { - return true; - } - - /** Sets {@link #dedupAggregateCalls}. */ - Config withDedupAggregateCalls(boolean dedupAggregateCalls); - - /** Whether {@link RelBuilder#aggregate} should prune unused input columns; default true. */ - @Value.Default - default boolean pruneInputOfAggregate() { - return true; - } - - /** Sets {@link #pruneInputOfAggregate}. */ - Config withPruneInputOfAggregate(boolean pruneInputOfAggregate); - - /** - * Whether to push down join conditions; default false (but {@link - * SqlToRelConverter#config()} by default sets this to true). - */ - @Value.Default - default boolean pushJoinCondition() { - return false; - } - - /** Sets {@link #pushJoinCondition()}. */ - Config withPushJoinCondition(boolean pushJoinCondition); - - /** Whether to simplify expressions; default true. */ - @Value.Default - default boolean simplify() { - return true; - } - - /** Sets {@link #simplify}. */ - Config withSimplify(boolean simplify); - - /** Whether to simplify LIMIT 0 to an empty relation; default true. */ - @Value.Default - default boolean simplifyLimit() { - return true; - } - - /** Sets {@link #simplifyLimit()}. */ - Config withSimplifyLimit(boolean simplifyLimit); - - /** - * Whether to simplify {@code Union(Values, Values)} or {@code Union(Project(Values))} to - * {@code Values}; default true. - */ - @Value.Default - default boolean simplifyValues() { - return true; - } - - /** Sets {@link #simplifyValues()}. */ - Config withSimplifyValues(boolean simplifyValues); - - /** - * Whether to create an Aggregate even if we know that the input is already unique; default - * false. - */ - @Value.Default - default boolean aggregateUnique() { - return false; - } - - /** Sets {@link #aggregateUnique()}. */ - Config withAggregateUnique(boolean aggregateUnique); - } -} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java index b2b33388f800a..53107b32daaba 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java @@ -73,7 +73,7 @@ public final class FlinkRelBuilder extends RelBuilder { public static final RelBuilder.Config FLINK_REL_BUILDER_CONFIG = - Config.DEFAULT.withSimplifyValues(false); + Config.DEFAULT.withSimplifyValues(false).withConvertCorrelateToJoin(false); private final QueryOperationConverter toRelNodeConverter; private final ExpandFactory expandFactory; @@ -189,7 +189,7 @@ public RelBuilder rank( /** Build non-window aggregate for either aggregate or table aggregate. */ @Override public RelBuilder aggregate( - RelBuilder.GroupKey groupKey, Iterable aggCalls) { + RelBuilder.GroupKey groupKey, Iterable aggCalls) { // build a relNode, the build() may also return a project RelNode relNode = super.aggregate(groupKey, aggCalls).build(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java index 97a073ac999b9..9b73ff59b3894 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java @@ -451,6 +451,7 @@ private List convertAggregateCalls(Aggregate agg) { call.isDistinct(), false, false, + call.rexList, call.getArgList(), call.filterArg, null, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LiteralAggFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LiteralAggFunction.java new file mode 100644 index 0000000000000..be24ffcb16773 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LiteralAggFunction.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.functions.aggfunctions; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.UnresolvedReferenceExpression; +import org.apache.flink.table.functions.DeclarativeAggregateFunction; +import org.apache.flink.table.types.DataType; + +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.sql.fun.SqlLiteralAggFunction; + +import static org.apache.flink.table.expressions.ApiExpressionUtils.unresolvedRef; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.literal; +import static org.apache.flink.table.planner.expressions.ExpressionBuilder.nullOf; + +/** + * Built-in literal aggregate function. This function is used for internal optimizations. It accepts + * zero regular aggregate arguments and returns a constant value. For more details see CALCITE-4334 and {@link + * SqlLiteralAggFunction}. + */ +public abstract class LiteralAggFunction extends DeclarativeAggregateFunction { + + private final UnresolvedReferenceExpression literalAgg = unresolvedRef("literalAgg"); + private final RexLiteral rexLiteral; + + public LiteralAggFunction(RexLiteral rexLiteral) { + this.rexLiteral = rexLiteral; + } + + @Override + public int operandCount() { + return 0; + } + + @Override + public UnresolvedReferenceExpression[] aggBufferAttributes() { + return new UnresolvedReferenceExpression[] {literalAgg}; + } + + @Override + public DataType[] getAggBufferTypes() { + return new DataType[] {getResultType()}; + } + + @Override + public Expression[] initialValuesExpressions() { + return new Expression[] {nullOf(getResultType())}; + } + + @Override + public Expression[] accumulateExpressions() { + return new Expression[] {literal(rexLiteral.getValue(), getResultType())}; + } + + @Override + public Expression[] retractExpressions() { + return new Expression[] {literal(rexLiteral.getValue(), getResultType())}; + } + + @Override + public Expression[] mergeExpressions() { + return new Expression[] {literal(rexLiteral.getValue(), getResultType())}; + } + + @Override + public Expression getValueExpression() { + return literal(rexLiteral.getValue(), getResultType()); + } + + /** Built-in Boolean Literal aggregate function. */ + public static class BooleanLiteralAggFunction extends LiteralAggFunction { + + public BooleanLiteralAggFunction(RexLiteral rexLiteral) { + super(rexLiteral); + } + + @Override + public DataType getResultType() { + return DataTypes.BOOLEAN(); + } + } + + /** Built-in Byte Literal aggregate function. */ + public static class ByteLiteralAggFunction extends LiteralAggFunction { + + public ByteLiteralAggFunction(RexLiteral rexLiteral) { + super(rexLiteral); + } + + @Override + public DataType getResultType() { + return DataTypes.TINYINT(); + } + } + + /** Built-in Short Literal aggregate function. */ + public static class ShortLiteralAggFunction extends LiteralAggFunction { + + public ShortLiteralAggFunction(RexLiteral rexLiteral) { + super(rexLiteral); + } + + @Override + public DataType getResultType() { + return DataTypes.SMALLINT(); + } + } + + /** Built-in Integer Literal aggregate function. */ + public static class IntLiteralAggFunction extends LiteralAggFunction { + + public IntLiteralAggFunction(RexLiteral rexLiteral) { + super(rexLiteral); + } + + @Override + public DataType getResultType() { + return DataTypes.INT(); + } + } + + /** Built-in Long Literal aggregate function. */ + public static class LongLiteralAggFunction extends LiteralAggFunction { + + public LongLiteralAggFunction(RexLiteral rexLiteral) { + super(rexLiteral); + } + + @Override + public DataType getResultType() { + return DataTypes.BIGINT(); + } + } + + /** Built-in Float Literal aggregate function. */ + public static class FloatLiteralAggFunction extends LiteralAggFunction { + + public FloatLiteralAggFunction(RexLiteral rexLiteral) { + super(rexLiteral); + } + + @Override + public DataType getResultType() { + return DataTypes.FLOAT(); + } + } + + /** Built-in Double Literal aggregate function. */ + public static class DoubleLiteralAggFunction extends LiteralAggFunction { + + public DoubleLiteralAggFunction(RexLiteral rexLiteral) { + super(rexLiteral); + } + + @Override + public DataType getResultType() { + return DataTypes.DOUBLE(); + } + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java index 2ad8e19055057..534348e4029b3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java @@ -639,10 +639,7 @@ public void lookupOperatorOverloads( ReturnTypes.explicit(SqlTypeName.VARCHAR, 128), SqlTypeTransforms.TO_NULLABLE), null, - OperandTypes.sequence( - "'SHA2(DATA, HASH_LENGTH)'", - OperandTypes.STRING, - OperandTypes.NUMERIC_INTEGER), + OperandTypes.family(SqlTypeFamily.STRING, SqlTypeFamily.INTEGER), SqlFunctionCategory.STRING); public static final SqlFunction DATE_FORMAT = diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AggregateCallJsonDeserializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AggregateCallJsonDeserializer.java index cc61db407a177..343585a3f84f3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AggregateCallJsonDeserializer.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AggregateCallJsonDeserializer.java @@ -85,6 +85,7 @@ public AggregateCall deserialize(JsonParser jsonParser, DeserializationContext c distinct, approximate, ignoreNulls, + new ArrayList<>(), argList, filterArg, null, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRule.java index e4347459d4722..db8dc73c3f99e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/AggregateReduceGroupingRule.java @@ -114,6 +114,7 @@ public void onMatch(RelOptRuleCall call) { false, false, false, + ImmutableList.of(), ImmutableList.of(column), -1, null, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRule.java index 2f88585259840..aab5e7bd2ce96 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkAggregateExpandDistinctAggregatesRule.java @@ -819,6 +819,7 @@ private void doRewrite( false, aggCall.isApproximate(), false, + aggCall.rexList, newArgs, newFilterArg, null, @@ -913,6 +914,7 @@ private static void rewriteAggCalls( false, aggCall.isApproximate(), false, + aggCall.rexList, newArgs, -1, null, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRule.java index 3af9cfe7b13f7..c7bea8fb238a8 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRule.java @@ -18,6 +18,8 @@ package org.apache.flink.table.planner.plan.rules.logical; +import org.apache.flink.table.planner.calcite.FlinkRelBuilder; + import org.apache.calcite.plan.RelOptRuleCall; import org.apache.calcite.plan.RelRule; import org.apache.calcite.rel.RelNode; @@ -92,8 +94,10 @@ public void onMatch(RelOptRuleCall call) { List fields = Util.range(intersect.getRowType().getFieldCount()); + final FlinkRelBuilder flinkRelBuilder = FlinkRelBuilder.of(call.rel(0).getCluster(), null); // 1. add marker to left rel node - RelBuilder leftBuilder = call.builder(); + RelBuilder leftBuilder = + flinkRelBuilder.transform(u -> u.withConvertCorrelateToJoin(false)); RelDataType boolType = leftBuilder.getTypeFactory().createSqlType(SqlTypeName.BOOLEAN); RelNode leftWithMarker = leftBuilder @@ -114,7 +118,8 @@ public void onMatch(RelOptRuleCall call) { .build(); // 2. add marker to right rel node - RelBuilder rightBuilder = call.builder(); + RelBuilder rightBuilder = + flinkRelBuilder.transform(u -> u.withConvertCorrelateToJoin(false)); RelNode rightWithMarker = rightBuilder .push(right) @@ -134,7 +139,7 @@ public void onMatch(RelOptRuleCall call) { .build(); // 3. union and aggregate - RelBuilder builder = call.builder(); + RelBuilder builder = flinkRelBuilder.transform(u -> u.withConvertCorrelateToJoin(false)); builder.push(leftWithMarker) .push(rightWithMarker) .union(true) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRule.java index 2e3f0d93a8ea0..ad40ef001f3d0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRule.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.rules.logical; +import org.apache.flink.table.planner.calcite.FlinkRelBuilder; import org.apache.flink.table.planner.plan.utils.SetOpRewriteUtil; import org.apache.calcite.plan.RelOptRuleCall; @@ -73,8 +74,10 @@ public void onMatch(RelOptRuleCall call) { List fields = Util.range(minus.getRowType().getFieldCount()); - // 1. add vcol_marker to left rel node - RelBuilder leftBuilder = call.builder(); + final FlinkRelBuilder flinkRelBuilder = FlinkRelBuilder.of(call.rel(0).getCluster(), null); + // 1. add marker to left rel node + RelBuilder leftBuilder = + flinkRelBuilder.transform(u -> u.withConvertCorrelateToJoin(false)); RelNode leftWithAddedVirtualCols = leftBuilder .push(left) @@ -91,7 +94,8 @@ public void onMatch(RelOptRuleCall call) { .build(); // 2. add vcol_marker to right rel node - RelBuilder rightBuilder = call.builder(); + RelBuilder rightBuilder = + flinkRelBuilder.transform(u -> u.withConvertCorrelateToJoin(false)); RelNode rightWithAddedVirtualCols = rightBuilder .push(right) @@ -108,7 +112,7 @@ public void onMatch(RelOptRuleCall call) { .build(); // 3. add union all and aggregate - RelBuilder builder = call.builder(); + RelBuilder builder = flinkRelBuilder.transform(u -> u.withConvertCorrelateToJoin(false)); builder.push(leftWithAddedVirtualCols) .push(rightWithAddedVirtualCols) .union(true) diff --git a/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE b/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE index 546b3c15784ac..63d049d9eff82 100644 --- a/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE +++ b/flink-table/flink-table-planner/src/main/resources/META-INF/NOTICE @@ -8,8 +8,8 @@ This project bundles the following dependencies under the Apache Software Licens - com.google.guava:guava:33.4.0-jre - com.google.guava:failureaccess:1.0.2 -- org.apache.calcite:calcite-core:1.34.0 -- org.apache.calcite:calcite-linq4j:1.34.0 +- org.apache.calcite:calcite-core:1.35.0 +- org.apache.calcite:calcite-linq4j:1.35.0 - org.apache.calcite.avatica:avatica-core:1.23.0 - org.apache.commons:commons-math3:3.6.1 - commons-codec:commons-codec:1.15 diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala index 1eb26c179fc3d..74c32fe2afaa6 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala @@ -675,6 +675,7 @@ class MatchCodeGenerator( false, false, false, + new util.ArrayList[RexNode](), a.exprIndices, -1, null, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggFunctionFactory.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggFunctionFactory.scala index 32ae59c513f95..54f942a46d5f4 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggFunctionFactory.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggFunctionFactory.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.table.api.TableException import org.apache.flink.table.functions.{BuiltInFunctionDefinitions, DeclarativeAggregateFunction, UserDefinedFunction} import org.apache.flink.table.planner.functions.aggfunctions._ +import org.apache.flink.table.planner.functions.aggfunctions.LiteralAggFunction.{BooleanLiteralAggFunction, ByteLiteralAggFunction, DoubleLiteralAggFunction, FloatLiteralAggFunction, IntLiteralAggFunction, LongLiteralAggFunction, ShortLiteralAggFunction} import org.apache.flink.table.planner.functions.aggfunctions.SingleValueAggFunction._ import org.apache.flink.table.planner.functions.aggfunctions.SumWithRetractAggFunction._ import org.apache.flink.table.planner.functions.bridging.BridgingSqlAggFunction @@ -31,7 +32,10 @@ import org.apache.flink.table.runtime.functions.aggregate.PercentileAggFunction. import org.apache.flink.table.types.logical._ import org.apache.flink.table.types.logical.LogicalTypeRoot._ +import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall +import org.apache.calcite.rex.{RexLiteral, RexNode} +import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.calcite.sql.{SqlAggFunction, SqlJsonConstructorNullClause, SqlKind, SqlRankFunction} import org.apache.calcite.sql.fun._ @@ -158,6 +162,9 @@ class AggFunctionFactory( val onNull = fn.asInstanceOf[SqlJsonArrayAggAggFunction].getNullClause new JsonArrayAggFunction(argTypes, onNull == SqlJsonConstructorNullClause.ABSENT_ON_NULL) + case a: SqlAggFunction if a.getKind == SqlKind.LITERAL_AGG => + createLiteralAggFunction(call.getType, call.rexList.get(0)) + case udagg: AggSqlFunction => // Can not touch the literals, Calcite make them in previous RelNode. // In here, all inputs are input refs. @@ -278,6 +285,31 @@ class AggFunctionFactory( } } + private def createLiteralAggFunction( + relDataType: RelDataType, + rexNode: RexNode): UserDefinedFunction = { + relDataType.getSqlTypeName match { + case SqlTypeName.BOOLEAN => + new BooleanLiteralAggFunction(rexNode.asInstanceOf[RexLiteral]) + case SqlTypeName.TINYINT => + new ByteLiteralAggFunction(rexNode.asInstanceOf[RexLiteral]) + case SqlTypeName.SMALLINT => + new ShortLiteralAggFunction(rexNode.asInstanceOf[RexLiteral]) + case SqlTypeName.INTEGER => + new IntLiteralAggFunction(rexNode.asInstanceOf[RexLiteral]) + case SqlTypeName.BIGINT => + new LongLiteralAggFunction(rexNode.asInstanceOf[RexLiteral]) + case SqlTypeName.FLOAT => + new FloatLiteralAggFunction(rexNode.asInstanceOf[RexLiteral]) + case SqlTypeName.DOUBLE => + new DoubleLiteralAggFunction(rexNode.asInstanceOf[RexLiteral]) + case t => + throw new TableException( + s"Literal aggregate function does not support type: ''$t''.\n" + + s"Please re-check the data type.") + } + } + private def createMinAggFunction( argTypes: Array[LogicalType], index: Int): UserDefinedFunction = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala index 8c9404de7b972..68f3cc0801900 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/AggregateUtil.scala @@ -56,6 +56,7 @@ import org.apache.calcite.rel.`type`._ import org.apache.calcite.rel.RelCollations import org.apache.calcite.rel.core.{Aggregate, AggregateCall} import org.apache.calcite.rel.core.Aggregate.AggCallBinding +import org.apache.calcite.rex.RexNode import org.apache.calcite.sql.`type`.{SqlTypeName, SqlTypeUtil} import org.apache.calcite.sql.{SqlAggFunction, SqlKind, SqlRankFunction} import org.apache.calcite.sql.fun._ @@ -782,6 +783,7 @@ object AggregateUtil extends Enumeration { false, false, false, + new util.ArrayList[RexNode](), new util.ArrayList[Integer](), -1, null, @@ -860,12 +862,14 @@ object AggregateUtil extends Enumeration { false, false, call.ignoreNulls, + call.rexList, call.getArgList, -1, // remove filterArg null, RelCollations.EMPTY, call.getType, - call.getName) + call.getName + ) } else { call } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlNodeToCallOperationTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlNodeToCallOperationTest.java index 0a0369bc3dce9..f18f531b00305 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlNodeToCallOperationTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlNodeToCallOperationTest.java @@ -153,9 +153,14 @@ void testCallStatement() { // should throw exception when the expression argument can't be reduced // to literal - assertThatThrownBy(() -> parse("call `system`.row_result(cast((1.2 + 2.4) as decimal))")) + assertThatThrownBy( + () -> + parse( + "call `system`.row_result(cast((1.2 + nullif(3, 2.4)) as decimal))")) .hasMessageContaining( - "The argument at position 0 CAST(CAST(1.2 + 2.4 AS DECIMAL) AS DECIMAL(10, 2)) for calling procedure can't be converted to literal."); + "The argument at position 0 " + + "CAST(CAST(1.2 + CASE WHEN 3 = 2.4 THEN NULL ELSE 3 END AS DECIMAL) AS DECIMAL(10, 2)) " + + "for calling procedure can't be converted to literal."); } private void verifyCallOperation(String sql, String expectSummary) { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AggregateCallSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AggregateCallSerdeTest.java index c73eba4ef0571..a72aac038d34d 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AggregateCallSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/AggregateCallSerdeTest.java @@ -88,6 +88,7 @@ public static Stream aggregateCallSpecs() { false, false, ImmutableList.of(), + ImmutableList.of(), -1, null, RelCollations.EMPTY, @@ -99,6 +100,7 @@ public static Stream aggregateCallSpecs() { false, false, false, + ImmutableList.of(), ImmutableList.of(3), -1, null, @@ -112,6 +114,7 @@ public static Stream aggregateCallSpecs() { false, false, ImmutableList.of(), + ImmutableList.of(), -1, null, RelCollations.EMPTY, @@ -145,6 +148,7 @@ private static AggregateCall getLegacyAggCall() { false, false, false, + ImmutableList.of(), ImmutableList.of(3), -1, null, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml index e942d41a39cdf..5d23710c48bc3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml @@ -259,13 +259,11 @@ LogicalProject(cnt=[$0], cnt0=[$1]) :- LogicalProject(cnt=[$0]) : +- LogicalFilter(condition=[>($0, 3)]) : +- LogicalAggregate(group=[{}], cnt=[COUNT()]) - : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, x]]) +- LogicalProject(cnt=[$0]) +- LogicalFilter(condition=[<($0, 5)]) +- LogicalAggregate(group=[{}], cnt=[COUNT()]) - +- LogicalProject($f0=[0]) - +- LogicalTableScan(table=[[default_catalog, default_database, x]]) + +- LogicalTableScan(table=[[default_catalog, default_database, x]]) ]]> @@ -275,8 +273,7 @@ NestedLoopJoin(joinType=[FullOuterJoin], where=[(cnt <> cnt0)], select=[cnt, cnt : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS cnt])(reuse_id=[1]) : +- Exchange(distribution=[single]) : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +: +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c]) +- Exchange(distribution=[single], shuffle_mode=[BATCH]) +- Calc(select=[cnt], where=[(cnt < 5)]) +- Reused(reference_id=[1]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml index ecef3e634e08f..c71dcc659623c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml @@ -139,8 +139,7 @@ LogicalProject(a=[$0]) : +- LogicalJoin(condition=[>($0, $3)], joinType=[full]) : :- LogicalTableScan(table=[[default_catalog, default_database, t]]) : +- LogicalAggregate(group=[{}], cnt=[COUNT()]) - : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -159,8 +158,7 @@ Calc(select=[a]) : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS cnt]) : +- Exchange(distribution=[single]) : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]) - : +- Calc(select=[0 AS $f0]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) @@ -196,8 +194,7 @@ LogicalProject(a=[$0]) : +- LogicalJoin(condition=[>($0, $3)], joinType=[full]) : :- LogicalTableScan(table=[[default_catalog, default_database, t]]) : +- LogicalAggregate(group=[{}], cnt=[COUNT()]) - : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -216,8 +213,7 @@ Calc(select=[a]) : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS cnt]) : +- Exchange(distribution=[single]) : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]) - : +- Calc(select=[0 AS $f0]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) @@ -253,8 +249,7 @@ LogicalProject(a=[$0]) : +- LogicalJoin(condition=[>($0, $3)], joinType=[full]) : :- LogicalTableScan(table=[[default_catalog, default_database, t]]) : +- LogicalAggregate(group=[{}], cnt=[COUNT()]) - : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, z]]) +- LogicalTableScan(table=[[default_catalog, default_database, z]]) ]]> @@ -273,8 +268,7 @@ Calc(select=[a]) : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS cnt]) : +- Exchange(distribution=[single]) : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]) - : +- Calc(select=[0 AS $f0]) - : +- Reused(reference_id=[1]) + : +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[a]) : +- TableSourceScan(table=[[default_catalog, default_database, x]], fields=[a, b, c, nx]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RowLevelDeleteTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RowLevelDeleteTest.xml index b835e944ea3fd..45291ffb57a5b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RowLevelDeleteTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RowLevelDeleteTest.xml @@ -135,8 +135,7 @@ LogicalSink(table=[default_catalog.default_database.t], fields=[a, b]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[AND(=($1, _UTF-16LE'123'), =(CAST($0):BIGINT, $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalTableScan(table=[[default_catalog, default_database, t]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) })))]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) @@ -150,8 +149,7 @@ Sink(table=[default_catalog.default_database.t], fields=[a, b]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b]) == Optimized Execution Plan == Sink(table=[default_catalog.default_database.t], fields=[a, b]) @@ -164,8 +162,7 @@ Sink(table=[default_catalog.default_database.t], fields=[a, b]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- Reused(reference_id=[1]) + +- Reused(reference_id=[1]) == Physical Execution Plan == { @@ -186,17 +183,6 @@ Sink(table=[default_catalog.default_database.t], fields=[a, b]) "ship_strategy" : "FORWARD", "side" : "second" } ] - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[0 AS $f0])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] }, { "id" : , "type" : "HashAggregate[]", @@ -495,8 +481,7 @@ LogicalSink(table=[default_catalog.default_database.t], fields=[a, b]) +- LogicalProject(a=[$0], b=[$1]) +- LogicalFilter(condition=[NOT(AND(=($1, _UTF-16LE'123'), =(CAST($0):BIGINT, $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalTableScan(table=[[default_catalog, default_database, t]]) + LogicalTableScan(table=[[default_catalog, default_database, t]]) }))))]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) @@ -509,8 +494,7 @@ Sink(table=[default_catalog.default_database.t], fields=[a, b]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b]) + +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b]) == Optimized Execution Plan == Sink(table=[default_catalog.default_database.t], fields=[a, b]) @@ -522,8 +506,7 @@ Sink(table=[default_catalog.default_database.t], fields=[a, b]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- Reused(reference_id=[1]) + +- Reused(reference_id=[1]) == Physical Execution Plan == { @@ -533,17 +516,6 @@ Sink(table=[default_catalog.default_database.t], fields=[a, b]) "pact" : "Data Source", "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, t]], fields=[a, b])", "parallelism" : 1 - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[0 AS $f0])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] }, { "id" : , "type" : "HashAggregate[]", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RowLevelUpdateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RowLevelUpdateTest.xml index 7fe6d81d87097..19557da72d8f6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RowLevelUpdateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/RowLevelUpdateTest.xml @@ -655,8 +655,7 @@ LogicalSink(table=[default_catalog.default_database.t], targetColumns=[[1]], fie +- LogicalProject(a=[$0], b=[_UTF-16LE'v2']) +- LogicalFilter(condition=[=(CAST($0):BIGINT, $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalTableScan(table=[[default_catalog, default_database, t1]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) }))]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) @@ -670,8 +669,7 @@ Sink(table=[default_catalog.default_database.t], targetColumns=[[1]], fields=[a, +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b]) + +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b]) == Optimized Execution Plan == Sink(table=[default_catalog.default_database.t], targetColumns=[[1]], fields=[a, b]) @@ -683,8 +681,7 @@ Sink(table=[default_catalog.default_database.t], targetColumns=[[1]], fields=[a, +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b]) + +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b]) == Physical Execution Plan == { @@ -711,17 +708,6 @@ Sink(table=[default_catalog.default_database.t], targetColumns=[[1]], fields=[a, "pact" : "Data Source", "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b])", "parallelism" : 1 - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[0 AS $f0])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] }, { "id" : , "type" : "HashAggregate[]", @@ -853,8 +839,7 @@ Sink(table=[default_catalog.default_database.t], targetColumns=[[1],[0]], fields LogicalSink(table=[default_catalog.default_database.t], targetColumns=[[1]], fields=[a, b]) +- LogicalProject(a=[$0], b=[IF(=(CAST($0):BIGINT, $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalTableScan(table=[[default_catalog, default_database, t1]]) + LogicalTableScan(table=[[default_catalog, default_database, t1]]) })), _UTF-16LE'v2', $1)]) +- LogicalTableScan(table=[[default_catalog, default_database, t]]) @@ -867,8 +852,7 @@ Sink(table=[default_catalog.default_database.t], targetColumns=[[1]], fields=[a, +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b]) + +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b]) == Optimized Execution Plan == Sink(table=[default_catalog.default_database.t], targetColumns=[[1]], fields=[a, b]) @@ -879,8 +863,7 @@ Sink(table=[default_catalog.default_database.t], targetColumns=[[1]], fields=[a, +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b]) + +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b]) == Physical Execution Plan == { @@ -896,17 +879,6 @@ Sink(table=[default_catalog.default_database.t], targetColumns=[[1]], fields=[a, "pact" : "Data Source", "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[a, b])", "parallelism" : 1 - }, { - "id" : , - "type" : "Calc[]", - "pact" : "Operator", - "contents" : "[]:Calc(select=[0 AS $f0])", - "parallelism" : 1, - "predecessors" : [ { - "id" : , - "ship_strategy" : "FORWARD", - "side" : "second" - } ] }, { "id" : , "type" : "HashAggregate[]", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml index 1ba58f298ae78..72791b2a01e89 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml @@ -126,15 +126,14 @@ Calc(select=[id, deepNested_nested1 AS nested1, ((deepNested_nested1.value + dee diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml index be082cc37ebc8..238a36d537c22 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.xml @@ -524,8 +524,7 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUN @@ -533,8 +532,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]), rowType=[RecordType(BIGINT EXPR$ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType=[RecordType(BIGINT EXPR$0)] +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)] +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)] - +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)] - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -545,16 +543,14 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType @@ -565,8 +561,7 @@ HashAggregate(isMerge=[false], select=[COUNT(*) AS EXPR$0]), rowType=[RecordType @@ -574,8 +569,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]), rowType=[RecordType(BIGINT EXPR$ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType=[RecordType(BIGINT EXPR$0)] +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)] +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)] - +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)] - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -586,15 +580,14 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType @@ -605,16 +598,14 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType @@ -625,15 +616,14 @@ HashAggregate(isMerge=[false], select=[COUNT(*) AS EXPR$0]), rowType=[RecordType diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml index 9320e5fdb573b..2aca5048d7856 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.xml @@ -647,8 +647,7 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count$0) AS EXPR$0, Final_COUN @@ -656,8 +655,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]), rowType=[RecordType(BIGINT EXPR$ SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType=[RecordType(BIGINT EXPR$0)] +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)] +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)] - +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)] - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -668,16 +666,14 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType @@ -688,8 +684,7 @@ SortAggregate(isMerge=[false], select=[COUNT(*) AS EXPR$0]), rowType=[RecordType @@ -697,8 +692,7 @@ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]), rowType=[RecordType(BIGINT EXPR$ SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType=[RecordType(BIGINT EXPR$0)] +- Exchange(distribution=[single]), rowType=[RecordType(BIGINT count1$0)] +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0]), rowType=[RecordType(BIGINT count1$0)] - +- Calc(select=[0 AS $f0]), rowType=[RecordType(INTEGER $f0)] - +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] + +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[byte, short, int, long, float, double, boolean, string, date, time, timestamp, decimal3020, decimal105]), rowType=[RecordType(TINYINT byte, SMALLINT short, INTEGER int, BIGINT long, FLOAT float, DOUBLE double, BOOLEAN boolean, VARCHAR(2147483647) string, DATE date, TIME(0) time, TIMESTAMP(3) timestamp, DECIMAL(30, 20) decimal3020, DECIMAL(10, 5) decimal105)] ]]> @@ -709,15 +703,14 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType @@ -728,16 +721,14 @@ SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]), rowType @@ -748,15 +739,14 @@ SortAggregate(isMerge=[false], select=[COUNT(*) AS EXPR$0]), rowType=[RecordType diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml index 0f8b17670d4eb..feb33bb64fff0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml @@ -1826,29 +1826,27 @@ LogicalProject(d=[$0]) = c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- HashJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], isBroadcast=[true], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c, c0, ck, i0])\n: : : +- HashJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i0], isBroadcast=[true], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#5] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: : : : +- [#6] Exchange(distribution=[broadcast])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) ++- MultipleInput(readOrder=[0,0,0,0,1,0], members=[\nHashJoin(joinType=[LeftAntiJoin], where=[((d IS NULL OR ($f3 = d)) AND (c = f))], select=[b, c, $f3], isBroadcast=[true], build=[right])\n:- Calc(select=[b, c, CASE(((c0 = 0) OR (i_0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- HashJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i_0, c1, ck0, EXPR$0, i], isBroadcast=[true], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i_0, c1, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c, c0, ck, i_0])\n: : : +- HashJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i_0], isBroadcast=[true], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#5] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: : : : +- [#6] Exchange(distribution=[broadcast])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, f]) : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) :- Exchange(distribution=[broadcast]) - : +- Calc(select=[EXPR$0, true AS i]) - : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) - : +- Exchange(distribution=[hash[EXPR$0]]) - : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) - : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) + : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0, Final_LITERAL_AGG(literalAgg$0) AS i]) + : +- Exchange(distribution=[hash[EXPR$0]]) + : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0, Partial_LITERAL_AGG(*) AS literalAgg$0]) + : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) : +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) - : +- Calc(select=[i, true AS i0]) - : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) - : +- Exchange(distribution=[hash[i]]) - : +- LocalHashAggregate(groupBy=[i], select=[i]) - : +- Calc(select=[i])(reuse_id=[3]) - : +- Reused(reference_id=[2]) + : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i, Final_LITERAL_AGG(literalAgg$0) AS i_0]) + : +- Exchange(distribution=[hash[i]]) + : +- LocalHashAggregate(groupBy=[i], select=[i, Partial_LITERAL_AGG(*) AS literalAgg$0]) + : +- Calc(select=[i])(reuse_id=[3]) + : +- Reused(reference_id=[2]) :- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml index 094c0b1f4f2c3..89acad4accdbf 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.xml @@ -669,10 +669,10 @@ Calc(select=[b]) : +- Calc(select=[d, f]) : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) :- Exchange(distribution=[broadcast]) - : +- Calc(select=[true AS i]) - : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) + : +- Calc(select=[i]) + : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0, Final_LITERAL_AGG(literalAgg$0) AS i]) : +- Exchange(distribution=[hash[EXPR$0]]) - : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0, Partial_LITERAL_AGG(*) AS literalAgg$0]) : +- Calc(select=[1 AS EXPR$0]) : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) :- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) @@ -1190,10 +1190,10 @@ Calc(select=[b]) : +- Calc(select=[d]) : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) :- Exchange(distribution=[broadcast]) - : +- Calc(select=[true AS i]) - : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) + : +- Calc(select=[i]) + : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0, Final_LITERAL_AGG(literalAgg$0) AS i]) : +- Exchange(distribution=[hash[EXPR$0]]) - : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0, Partial_LITERAL_AGG(*) AS literalAgg$0]) : +- Calc(select=[1 AS EXPR$0]) : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) :- Calc(select=[a, b]) @@ -2212,29 +2212,27 @@ LogicalProject(d=[$0]) = c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c, c0, ck, i0])\n: : : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i0], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#5] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: : : : +- [#6] Exchange(distribution=[broadcast])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) ++- MultipleInput(readOrder=[0,0,0,0,1,0], members=[\nNestedLoopJoin(joinType=[LeftAntiJoin], where=[((d IS NULL OR ($f3 = d)) AND (c = f))], select=[b, c, $f3], build=[right])\n:- Calc(select=[b, c, CASE(((c0 = 0) OR (i_0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i_0, c1, ck0, EXPR$0, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i_0, c1, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c, c0, ck, i_0])\n: : : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i_0], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#5] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: : : : +- [#6] Exchange(distribution=[broadcast])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[d, f]) : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) :- Exchange(distribution=[broadcast]) - : +- Calc(select=[EXPR$0, true AS i]) - : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) - : +- Exchange(distribution=[hash[EXPR$0]]) - : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) - : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) + : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0, Final_LITERAL_AGG(literalAgg$0) AS i]) + : +- Exchange(distribution=[hash[EXPR$0]]) + : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0, Partial_LITERAL_AGG(*) AS literalAgg$0]) + : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) + : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) : +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) - : +- Calc(select=[i, true AS i0]) - : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) - : +- Exchange(distribution=[hash[i]]) - : +- LocalHashAggregate(groupBy=[i], select=[i]) - : +- Calc(select=[i])(reuse_id=[3]) - : +- Reused(reference_id=[2]) + : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i, Final_LITERAL_AGG(literalAgg$0) AS i_0]) + : +- Exchange(distribution=[hash[i]]) + : +- LocalHashAggregate(groupBy=[i], select=[i, Partial_LITERAL_AGG(*) AS literalAgg$0]) + : +- Calc(select=[i])(reuse_id=[3]) + : +- Reused(reference_id=[2]) :- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) @@ -2442,27 +2440,25 @@ LogicalProject(e=[$1], d=[$0]) = c0) AND a IS NOT NULL)), 1, ((c = 0) OR (i IS NULL AND (ck0 >= c) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = j)], select=[a, b, c0, ck, i0, c, ck0, j, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i0, c, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c AS c0, ck, i0])\n: : : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, ck, i, i0], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#5] Calc(select=[a, b])\n: : : : +- [#6] Exchange(distribution=[broadcast])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) ++- MultipleInput(readOrder=[0,0,0,0,1,0], members=[\nNestedLoopJoin(joinType=[LeftAntiJoin], where=[((b IS NULL OR e IS NULL OR (b = e)) AND (d IS NULL OR ($f3 = d)))], select=[b, $f3], build=[right])\n:- Calc(select=[b, CASE(((c0 = 0) OR (i_0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c = 0) OR (i IS NULL AND (ck0 >= c) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = j)], select=[a, b, c0, ck, i_0, c, ck0, j, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i_0, c, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c AS c0, ck, i_0])\n: : : +- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, ck, i, i_0], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#5] Calc(select=[a, b])\n: : : : +- [#6] Exchange(distribution=[broadcast])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- [#3] Exchange(distribution=[broadcast])\n: +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[e, d]) : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) :- Exchange(distribution=[broadcast]) - : +- Calc(select=[j, true AS i]) - : +- HashAggregate(isMerge=[true], groupBy=[j], select=[j]) - : +- Exchange(distribution=[hash[j]]) - : +- LocalHashAggregate(groupBy=[j], select=[j]) - : +- TableSourceScan(table=[[default_catalog, default_database, t2]], fields=[j])(reuse_id=[1]) + : +- HashAggregate(isMerge=[true], groupBy=[j], select=[j, Final_LITERAL_AGG(literalAgg$0) AS i]) + : +- Exchange(distribution=[hash[j]]) + : +- LocalHashAggregate(groupBy=[j], select=[j, Partial_LITERAL_AGG(*) AS literalAgg$0]) + : +- TableSourceScan(table=[[default_catalog, default_database, t2]], fields=[j])(reuse_id=[1]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(j) AS count$1]) : +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) - : +- Calc(select=[i, true AS i0]) - : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) - : +- Exchange(distribution=[hash[i]]) - : +- LocalHashAggregate(groupBy=[i], select=[i]) - : +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[i])(reuse_id=[2]) + : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i, Final_LITERAL_AGG(literalAgg$0) AS i_0]) + : +- Exchange(distribution=[hash[i]]) + : +- LocalHashAggregate(groupBy=[i], select=[i, Partial_LITERAL_AGG(*) AS literalAgg$0]) + : +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[i])(reuse_id=[2]) :- Calc(select=[a, b]) : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) +- Exchange(distribution=[broadcast]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml index ec20c141adffb..26040f5857d19 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SemiAntiJoinTest.xml @@ -688,10 +688,10 @@ Calc(select=[b]) : +- Calc(select=[b, c, CASE(((c0 <> 0) AND i IS NOT NULL AND a IS NOT NULL), 1, 2) AS $f3]) : +- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = 1)], select=[a, b, c, c0, i], build=[right])\n:- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0], build=[right], singleRowJoin=[true])\n: :- [#2] TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) : :- Exchange(distribution=[broadcast]) - : : +- Calc(select=[true AS i]) - : : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) + : : +- Calc(select=[i]) + : : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0, Final_LITERAL_AGG(literalAgg$0) AS i]) : : +- Exchange(distribution=[hash[EXPR$0]]) - : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0, Partial_LITERAL_AGG(*) AS literalAgg$0]) : : +- Calc(select=[1 AS EXPR$0]) : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) : :- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) @@ -1231,10 +1231,10 @@ Calc(select=[b]) : +- Calc(select=[b, CASE(((c <> 0) AND i IS NOT NULL AND a IS NOT NULL), 1, 2) AS $f3]) : +- MultipleInput(readOrder=[0,1,0], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = 1)], select=[a, b, c, i], build=[right])\n:- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c], build=[right], singleRowJoin=[true])\n: :- [#2] Calc(select=[a, b])\n: +- [#3] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n]) : :- Exchange(distribution=[broadcast]) - : : +- Calc(select=[true AS i]) - : : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) + : : +- Calc(select=[i]) + : : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0, Final_LITERAL_AGG(literalAgg$0) AS i]) : : +- Exchange(distribution=[hash[EXPR$0]]) - : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0, Partial_LITERAL_AGG(*) AS literalAgg$0]) : : +- Calc(select=[1 AS EXPR$0]) : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) : :- Calc(select=[a, b]) @@ -2296,8 +2296,8 @@ LogicalProject(d=[$0]) Calc(select=[b]) +- HashJoin(joinType=[LeftAntiJoin], where=[((d IS NULL OR ($f3 = d)) AND (c = f))], select=[b, c, $f3], build=[right]) :- Exchange(distribution=[hash[c]]) - : +- Calc(select=[b, c, CASE(((c0 = 0) OR (i0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3]) - : +- MultipleInput(readOrder=[0,0,1,0,0], members=[\nHashJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], build=[right])\n:- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true])\n: :- Calc(select=[a, b, c, c0, ck, i0])\n: : +- HashJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i0], build=[right])\n: : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : :- [#3] Exchange(distribution=[hash[a]])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- Calc(select=[i, true AS i0])\n: : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i])\n: : +- [#5] Exchange(distribution=[hash[i]])\n: +- [#1] Exchange(distribution=[broadcast])\n+- Calc(select=[EXPR$0, true AS i])\n +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])\n +- [#2] Exchange(distribution=[hash[EXPR$0]])\n]) + : +- Calc(select=[b, c, CASE(((c0 = 0) OR (i_0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3]) + : +- MultipleInput(readOrder=[0,0,1,0,0], members=[\nHashJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i_0, c1, ck0, EXPR$0, i], build=[right])\n:- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i_0, c1, ck0], build=[right], singleRowJoin=[true])\n: :- Calc(select=[a, b, c, c0, ck, i_0])\n: : +- HashJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i_0], build=[right])\n: : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : :- [#3] Exchange(distribution=[hash[a]])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i, Final_LITERAL_AGG(literalAgg$0) AS i_0])\n: : +- [#5] Exchange(distribution=[hash[i]])\n: +- [#1] Exchange(distribution=[broadcast])\n+- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0, Final_LITERAL_AGG(literalAgg$0) AS i])\n +- [#2] Exchange(distribution=[hash[EXPR$0]])\n]) : :- Exchange(distribution=[broadcast]) : : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : +- Exchange(distribution=[single]) @@ -2305,7 +2305,7 @@ Calc(select=[b]) : : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) : :- Exchange(distribution=[hash[EXPR$0]]) - : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0, Partial_LITERAL_AGG(*) AS literalAgg$0]) : : +- Reused(reference_id=[1]) : :- Exchange(distribution=[hash[a]]) : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) @@ -2316,7 +2316,7 @@ Calc(select=[b]) : : +- Calc(select=[i])(reuse_id=[3]) : : +- Reused(reference_id=[2]) : +- Exchange(distribution=[hash[i]]) - : +- LocalHashAggregate(groupBy=[i], select=[i]) + : +- LocalHashAggregate(groupBy=[i], select=[i, Partial_LITERAL_AGG(*) AS literalAgg$0]) : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) @@ -2525,7 +2525,7 @@ LogicalProject(e=[$1], d=[$0]) = c0) AND a IS NOT NULL)), 1, ((c = 0) OR (i IS NULL AND (ck0 >= c) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- HashJoin(joinType=[LeftOuterJoin], where=[(a = j)], select=[a, b, c0, ck, i0, c, ck0, j, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i0, c, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c AS c0, ck, i0])\n: : : +- HashJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, ck, i, i0], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#4] Exchange(distribution=[hash[a]])\n: : : : +- [#5] Exchange(distribution=[broadcast])\n: : : +- Calc(select=[i, true AS i0])\n: : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i])\n: : : +- [#6] Exchange(distribution=[hash[i]])\n: : +- [#2] Exchange(distribution=[broadcast])\n: +- Calc(select=[j, true AS i])\n: +- HashAggregate(isMerge=[true], groupBy=[j], select=[j])\n: +- [#3] Exchange(distribution=[hash[j]])\n+- [#1] Exchange(distribution=[broadcast])\n]) ++- MultipleInput(readOrder=[0,0,0,1,0,0], members=[\nNestedLoopJoin(joinType=[LeftAntiJoin], where=[((b IS NULL OR e IS NULL OR (b = e)) AND (d IS NULL OR ($f3 = d)))], select=[b, $f3], build=[right])\n:- Calc(select=[b, CASE(((c0 = 0) OR (i_0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c = 0) OR (i IS NULL AND (ck0 >= c) AND a IS NOT NULL)), 2, 3) AS $f3])\n: +- HashJoin(joinType=[LeftOuterJoin], where=[(a = j)], select=[a, b, c0, ck, i_0, c, ck0, j, i], build=[right])\n: :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i_0, c, ck0], build=[right], singleRowJoin=[true])\n: : :- Calc(select=[a, b, c AS c0, ck, i_0])\n: : : +- HashJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, ck, i, i_0], build=[right])\n: : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, ck], build=[right], singleRowJoin=[true])\n: : : : :- [#4] Exchange(distribution=[hash[a]])\n: : : : +- [#5] Exchange(distribution=[broadcast])\n: : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i, Final_LITERAL_AGG(literalAgg$0) AS i_0])\n: : : +- [#6] Exchange(distribution=[hash[i]])\n: : +- [#2] Exchange(distribution=[broadcast])\n: +- HashAggregate(isMerge=[true], groupBy=[j], select=[j, Final_LITERAL_AGG(literalAgg$0) AS i])\n: +- [#3] Exchange(distribution=[hash[j]])\n+- [#1] Exchange(distribution=[broadcast])\n]) :- Exchange(distribution=[broadcast]) : +- Calc(select=[e, d]) : +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) @@ -2535,7 +2535,7 @@ Calc(select=[b]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(j) AS count$1]) : +- TableSourceScan(table=[[default_catalog, default_database, t2]], fields=[j])(reuse_id=[1]) :- Exchange(distribution=[hash[j]]) - : +- LocalHashAggregate(groupBy=[j], select=[j]) + : +- LocalHashAggregate(groupBy=[j], select=[j, Partial_LITERAL_AGG(*) AS literalAgg$0]) : +- Reused(reference_id=[1]) :- Exchange(distribution=[hash[a]]) : +- Calc(select=[a, b]) @@ -2546,7 +2546,7 @@ Calc(select=[b]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) : +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[i])(reuse_id=[2]) +- Exchange(distribution=[hash[i]]) - +- LocalHashAggregate(groupBy=[i], select=[i]) + +- LocalHashAggregate(groupBy=[i], select=[i, Partial_LITERAL_AGG(*) AS literalAgg$0]) +- Reused(reference_id=[2]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml index 8f29cd63d0841..957a0b8c763e3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.xml @@ -1979,8 +1979,8 @@ LogicalProject(d=[$0]) Calc(select=[b]) +- HashJoin(joinType=[LeftAntiJoin], where=[((d IS NULL OR ($f3 = d)) AND (c = f))], select=[b, c, $f3], build=[right]) :- Exchange(distribution=[hash[c]]) - : +- Calc(select=[b, c, CASE(((c0 = 0) OR (i0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3]) - : +- MultipleInput(readOrder=[0,0,1,0,0], members=[\nHashJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], build=[right])\n:- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true])\n: :- Calc(select=[a, b, c, c0, ck, i0])\n: : +- HashJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i0], build=[right])\n: : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : :- [#3] Exchange(distribution=[hash[a]])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- Calc(select=[i, true AS i0])\n: : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i])\n: : +- [#5] Exchange(distribution=[hash[i]])\n: +- [#1] Exchange(distribution=[broadcast])\n+- Calc(select=[EXPR$0, true AS i])\n +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])\n +- [#2] Exchange(distribution=[hash[EXPR$0]])\n]) + : +- Calc(select=[b, c, CASE(((c0 = 0) OR (i_0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3]) + : +- MultipleInput(readOrder=[0,0,1,0,0], members=[\nHashJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i_0, c1, ck0, EXPR$0, i], build=[right])\n:- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i_0, c1, ck0], build=[right], singleRowJoin=[true])\n: :- Calc(select=[a, b, c, c0, ck, i_0])\n: : +- HashJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i_0], build=[right])\n: : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : :- [#3] Exchange(distribution=[hash[a]])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i, Final_LITERAL_AGG(literalAgg$0) AS i_0])\n: : +- [#5] Exchange(distribution=[hash[i]])\n: +- [#1] Exchange(distribution=[broadcast])\n+- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0, Final_LITERAL_AGG(literalAgg$0) AS i])\n +- [#2] Exchange(distribution=[hash[EXPR$0]])\n]) : :- Exchange(distribution=[broadcast]) : : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : +- Exchange(distribution=[single]) @@ -1988,7 +1988,7 @@ Calc(select=[b]) : : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) : :- Exchange(distribution=[hash[EXPR$0]]) - : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0, Partial_LITERAL_AGG(*) AS literalAgg$0]) : : +- Reused(reference_id=[1]) : :- Exchange(distribution=[hash[a]]) : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) @@ -1999,7 +1999,7 @@ Calc(select=[b]) : : +- Calc(select=[i])(reuse_id=[3]) : : +- Reused(reference_id=[2]) : +- Exchange(distribution=[hash[i]]) - : +- LocalHashAggregate(groupBy=[i], select=[i]) + : +- LocalHashAggregate(groupBy=[i], select=[i, Partial_LITERAL_AGG(*) AS literalAgg$0]) : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.xml index a63d51dc74b8d..2cc3c7b55e567 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SingleRowJoinTest.xml @@ -55,8 +55,7 @@ Calc(select=[a1, a2]) LogicalProject(a1=[$1]) +- LogicalJoin(condition=[=($0, $2)], joinType=[right]) :- LogicalAggregate(group=[{}], cnt=[COUNT()]) - : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) +- LogicalTableScan(table=[[default_catalog, default_database, A]]) ]]> @@ -68,8 +67,7 @@ Calc(select=[a1]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS cnt]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - : +- Calc(select=[0 AS $f0]) - : +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2]) + : +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2]) +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2]) ]]> @@ -83,8 +81,7 @@ Calc(select=[a1]) LogicalProject(a1=[$1]) +- LogicalJoin(condition=[<($0, $2)], joinType=[right]) :- LogicalAggregate(group=[{}], cnt=[COUNT()]) - : +- LogicalProject($f0=[0]) - : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + : +- LogicalTableScan(table=[[default_catalog, default_database, B]]) +- LogicalTableScan(table=[[default_catalog, default_database, A]]) ]]> @@ -96,8 +93,7 @@ Calc(select=[a1]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS cnt]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - : +- Calc(select=[0 AS $f0]) - : +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2]) + : +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2]) +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a1, a2]) ]]> @@ -112,8 +108,7 @@ LogicalProject(a2=[$1]) +- LogicalJoin(condition=[>($0, $2)], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalAggregate(group=[{}], cnt=[COUNT()]) - +- LogicalProject($f0=[0]) - +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -125,8 +120,7 @@ Calc(select=[a2]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS cnt]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2]) ]]> @@ -169,8 +163,7 @@ LogicalProject(a2=[$1]) +- LogicalJoin(condition=[=($0, $2)], joinType=[left]) :- LogicalTableScan(table=[[default_catalog, default_database, A]]) +- LogicalAggregate(group=[{}], cnt=[COUNT()]) - +- LogicalProject($f0=[0]) - +- LogicalTableScan(table=[[default_catalog, default_database, B]]) + +- LogicalTableScan(table=[[default_catalog, default_database, B]]) ]]> @@ -182,8 +175,7 @@ Calc(select=[a2]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS cnt]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2]) + +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[b1, b2]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml index fd85c5db67a15..004973a3cb463 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.xml @@ -1979,8 +1979,8 @@ LogicalProject(d=[$0]) Calc(select=[b]) +- SortMergeJoin(joinType=[LeftAntiJoin], where=[((d IS NULL OR ($f3 = d)) AND (c = f))], select=[b, c, $f3]) :- Exchange(distribution=[hash[c]]) - : +- Calc(select=[b, c, CASE(((c0 = 0) OR (i0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3]) - : +- MultipleInput(readOrder=[0,0,1,0,1], members=[\nSortMergeJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i])\n:- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true])\n: :- Calc(select=[a, b, c, c0, ck, i0])\n: : +- SortMergeJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i0])\n: : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : :- [#3] Exchange(distribution=[hash[a]])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- Calc(select=[i, true AS i0])\n: : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i])\n: : +- [#5] Exchange(distribution=[hash[i]])\n: +- [#1] Exchange(distribution=[broadcast])\n+- Calc(select=[EXPR$0, true AS i])\n +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0])\n +- [#2] Exchange(distribution=[hash[EXPR$0]])\n]) + : +- Calc(select=[b, c, CASE(((c0 = 0) OR (i_0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3]) + : +- MultipleInput(readOrder=[0,0,1,0,1], members=[\nSortMergeJoin(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i_0, c1, ck0, EXPR$0, i])\n:- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i_0, c1, ck0], build=[right], singleRowJoin=[true])\n: :- Calc(select=[a, b, c, c0, ck, i_0])\n: : +- SortMergeJoin(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i_0])\n: : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true])\n: : : :- [#3] Exchange(distribution=[hash[a]])\n: : : +- [#4] Exchange(distribution=[broadcast])\n: : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i, Final_LITERAL_AGG(literalAgg$0) AS i_0])\n: : +- [#5] Exchange(distribution=[hash[i]])\n: +- [#1] Exchange(distribution=[broadcast])\n+- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0, Final_LITERAL_AGG(literalAgg$0) AS i])\n +- [#2] Exchange(distribution=[hash[EXPR$0]])\n]) : :- Exchange(distribution=[broadcast]) : : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) : : +- Exchange(distribution=[single]) @@ -1988,7 +1988,7 @@ Calc(select=[b]) : : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[1]) : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) : :- Exchange(distribution=[hash[EXPR$0]]) - : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0, Partial_LITERAL_AGG(*) AS literalAgg$0]) : : +- Reused(reference_id=[1]) : :- Exchange(distribution=[hash[a]]) : : +- TableSourceScan(table=[[default_catalog, default_database, l]], fields=[a, b, c]) @@ -1999,7 +1999,7 @@ Calc(select=[b]) : : +- Calc(select=[i])(reuse_id=[3]) : : +- Reused(reference_id=[2]) : +- Exchange(distribution=[hash[i]]) - : +- LocalHashAggregate(groupBy=[i], select=[i]) + : +- LocalHashAggregate(groupBy=[i], select=[i, Partial_LITERAL_AGG(*) AS literalAgg$0]) : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.xml index 255ad29c35a3e..3e3c8bde99e1c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.xml @@ -1126,10 +1126,9 @@ Calc(select=[a1, b1]) @@ -1201,15 +1199,14 @@ Calc(select=[a1]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.xml index 85579fb05b4bd..7cac6862d4932 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.xml @@ -1123,10 +1123,9 @@ Calc(select=[a1, b1]) @@ -1198,15 +1196,14 @@ Calc(select=[a1]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.xml index 0ac94bf4c22de..beedac11333e2 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.xml @@ -1157,10 +1157,9 @@ Calc(select=[a1, b1]) @@ -1232,15 +1230,14 @@ Calc(select=[a1]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.xml index d04f66495a85c..989bc3aacf458 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.xml @@ -1157,10 +1157,9 @@ Calc(select=[a1, b1]) @@ -1232,15 +1230,14 @@ Calc(select=[a1]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml index 8311a1f449aa5..8ef3786d42698 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml @@ -625,10 +625,9 @@ LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483 @@ -53,43 +50,39 @@ MultipleInput(readOrder=[0,0,1], members=[\nNestedLoopJoin(joinType=[InnerJoin], : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c3]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[right]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk])\n]) -: : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) -: : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) -: : : +- Calc(select=[dim_date_sk], where=[(price < 400)])(reuse_id=[1]) -: : : +- TableSourceScan(table=[[testCatalog, test_database, dim, project=[price, dim_date_sk], metadata=[]]], fields=[price, dim_date_sk])(reuse_id=[2]) -: : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk]) -: +- Exchange(distribution=[hash[dim_date_sk]]) -: +- Reused(reference_id=[1]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[right]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk])\n]) +: : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) +: : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) +: : : +- Calc(select=[id, male, amount, price, dim_date_sk], where=[(price < 400)])(reuse_id=[1]) +: : : +- TableSourceScan(table=[[testCatalog, test_database, dim]], fields=[id, male, amount, price, dim_date_sk])(reuse_id=[2]) +: : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk]) +: +- Exchange(distribution=[hash[dim_date_sk]]) +: +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c1]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- Calc(select=[fact_date_sk], where=[(price < 100)]) -: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[], project=[price, fact_date_sk], metadata=[]]], fields=[price, fact_date_sk]) -: +- Exchange(distribution=[hash[dim_date_sk]]) -: +- Calc(select=[dim_date_sk]) -: +- Reused(reference_id=[2]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 100)]) +: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[]]], fields=[id, name, amount, price, fact_date_sk]) +: +- Exchange(distribution=[hash[dim_date_sk]]) +: +- Reused(reference_id=[2]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c2]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[right]) - :- Exchange(distribution=[hash[fact_date_sk]]) - : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk])\n]) - : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) - : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) - : : +- Calc(select=[dim_date_sk], where=[(price < 200)])(reuse_id=[3]) - : : +- Reused(reference_id=[2]) - : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk]) - +- Exchange(distribution=[hash[dim_date_sk]]) - +- Reused(reference_id=[3]) + +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[right]) + :- Exchange(distribution=[hash[fact_date_sk]]) + : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk])\n]) + : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) + : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) + : : +- Calc(select=[id, male, amount, price, dim_date_sk], where=[(price < 200)])(reuse_id=[3]) + : : +- Reused(reference_id=[2]) + : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk]) + +- Exchange(distribution=[hash[dim_date_sk]]) + +- Reused(reference_id=[3]) ]]> @@ -103,23 +96,20 @@ LogicalProject(c1=[$0], c2=[$1], c3=[$2]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalAggregate(group=[{}], c1=[COUNT()]) - : : +- LogicalProject($f0=[0]) - : : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 100))]) - : : +- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - : : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + : : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 100))]) + : : +- LogicalJoin(condition=[true], joinType=[inner]) + : : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + : : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) : +- LogicalAggregate(group=[{}], c2=[COUNT()]) - : +- LogicalProject($f0=[0]) - : +- LogicalFilter(condition=[AND(=($4, $9), <($8, 200))]) - : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + : +- LogicalFilter(condition=[AND(=($4, $9), <($8, 200))]) + : +- LogicalJoin(condition=[true], joinType=[inner]) + : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) +- LogicalAggregate(group=[{}], c3=[COUNT()]) - +- LogicalProject($f0=[0]) - +- LogicalFilter(condition=[AND(=($4, $9), <($8, 400))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + +- LogicalFilter(condition=[AND(=($4, $9), <($8, 400))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) ]]> @@ -129,44 +119,39 @@ MultipleInput(readOrder=[0,0,1], members=[\nNestedLoopJoin(joinType=[InnerJoin], : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c3]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[right]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk])\n]) -: : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) -: : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) -: : : +- Calc(select=[dim_date_sk], where=[(price < 400)])(reuse_id=[1]) -: : : +- TableSourceScan(table=[[testCatalog, test_database, dim, project=[price, dim_date_sk], metadata=[]]], fields=[price, dim_date_sk])(reuse_id=[2]) -: : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk]) -: +- Exchange(distribution=[hash[dim_date_sk]], shuffle_mode=[BATCH]) -: +- Reused(reference_id=[1]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[right]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk])\n]) +: : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) +: : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) +: : : +- Calc(select=[id, male, amount, price, dim_date_sk], where=[(price < 400)])(reuse_id=[1]) +: : : +- TableSourceScan(table=[[testCatalog, test_database, dim]], fields=[id, male, amount, price, dim_date_sk])(reuse_id=[2]) +: : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk]) +: +- Exchange(distribution=[hash[dim_date_sk]], shuffle_mode=[BATCH]) +: +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c1]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- Calc(select=[fact_date_sk], where=[(price < 100)]) -: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[], project=[price, fact_date_sk], metadata=[]]], fields=[price, fact_date_sk]) -: +- Exchange(distribution=[hash[dim_date_sk]]) -: +- Calc(select=[dim_date_sk]) -: +- Exchange(distribution=[any], shuffle_mode=[BATCH]) -: +- Reused(reference_id=[2]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 100)]) +: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[]]], fields=[id, name, amount, price, fact_date_sk]) +: +- Exchange(distribution=[hash[dim_date_sk]], shuffle_mode=[BATCH]) +: +- Reused(reference_id=[2]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c2]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[right]) - :- Exchange(distribution=[hash[fact_date_sk]]) - : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk])\n]) - : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) - : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) - : : +- Calc(select=[dim_date_sk], where=[(price < 200)])(reuse_id=[3]) - : : +- Reused(reference_id=[2]) - : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk]) - +- Exchange(distribution=[hash[dim_date_sk]], shuffle_mode=[BATCH]) - +- Reused(reference_id=[3]) + +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[right]) + :- Exchange(distribution=[hash[fact_date_sk]]) + : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk])\n]) + : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) + : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) + : : +- Calc(select=[id, male, amount, price, dim_date_sk], where=[(price < 200)])(reuse_id=[3]) + : : +- Reused(reference_id=[2]) + : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk]) + +- Exchange(distribution=[hash[dim_date_sk]], shuffle_mode=[BATCH]) + +- Reused(reference_id=[3]) ]]> @@ -180,23 +165,20 @@ LogicalProject(c1=[$0], c2=[$1], c3=[$2]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalAggregate(group=[{}], c1=[COUNT()]) - : : +- LogicalProject($f0=[0]) - : : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 100))]) - : : +- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - : : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + : : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 100))]) + : : +- LogicalJoin(condition=[true], joinType=[inner]) + : : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + : : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) : +- LogicalAggregate(group=[{}], c2=[COUNT()]) - : +- LogicalProject($f0=[0]) - : +- LogicalFilter(condition=[AND(=($4, $9), <($8, 200))]) - : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + : +- LogicalFilter(condition=[AND(=($4, $9), <($8, 200))]) + : +- LogicalJoin(condition=[true], joinType=[inner]) + : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) +- LogicalAggregate(group=[{}], c3=[COUNT()]) - +- LogicalProject($f0=[0]) - +- LogicalFilter(condition=[AND(=($4, $9), <($8, 400))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + +- LogicalFilter(condition=[AND(=($4, $9), <($8, 400))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) ]]> @@ -206,44 +188,39 @@ MultipleInput(readOrder=[0,0,1], members=[\nNestedLoopJoin(joinType=[InnerJoin], : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c3]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[right]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk])\n]) -: : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) -: : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) -: : : +- Calc(select=[dim_date_sk], where=[(price < 400)])(reuse_id=[1]) -: : : +- TableSourceScan(table=[[testCatalog, test_database, dim, project=[price, dim_date_sk], metadata=[]]], fields=[price, dim_date_sk])(reuse_id=[2]) -: : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk]) -: +- Exchange(distribution=[hash[dim_date_sk]], shuffle_mode=[BATCH]) -: +- Reused(reference_id=[1]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[right]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk])\n]) +: : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) +: : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) +: : : +- Calc(select=[id, male, amount, price, dim_date_sk], where=[(price < 400)])(reuse_id=[1]) +: : : +- TableSourceScan(table=[[testCatalog, test_database, dim]], fields=[id, male, amount, price, dim_date_sk])(reuse_id=[2]) +: : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk]) +: +- Exchange(distribution=[hash[dim_date_sk]], shuffle_mode=[BATCH]) +: +- Reused(reference_id=[1]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c1]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- Calc(select=[fact_date_sk], where=[(price < 100)]) -: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[], project=[price, fact_date_sk], metadata=[]]], fields=[price, fact_date_sk]) -: +- Exchange(distribution=[hash[dim_date_sk]]) -: +- Calc(select=[dim_date_sk]) -: +- Exchange(distribution=[any], shuffle_mode=[BATCH]) -: +- Reused(reference_id=[2]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 100)]) +: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[]]], fields=[id, name, amount, price, fact_date_sk]) +: +- Exchange(distribution=[hash[dim_date_sk]], shuffle_mode=[BATCH]) +: +- Reused(reference_id=[2]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c2]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[right]) - :- Exchange(distribution=[hash[fact_date_sk]]) - : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk])\n]) - : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) - : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) - : : +- Calc(select=[dim_date_sk], where=[(price < 200)])(reuse_id=[3]) - : : +- Reused(reference_id=[2]) - : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part, project=[fact_date_sk], metadata=[]]], fields=[fact_date_sk]) - +- Exchange(distribution=[hash[dim_date_sk]], shuffle_mode=[BATCH]) - +- Reused(reference_id=[3]) + +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[right]) + :- Exchange(distribution=[hash[fact_date_sk]]) + : +- MultipleInput(members=[\nOrderEnforcer\n:- [#1] Exchange(distribution=[broadcast], shuffle_mode=[BATCH])\n+- [#2] DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk])\n]) + : :- Exchange(distribution=[broadcast], shuffle_mode=[BATCH]) + : : +- DynamicFilteringDataCollector(fields=[dim_date_sk]) + : : +- Calc(select=[id, male, amount, price, dim_date_sk], where=[(price < 200)])(reuse_id=[3]) + : : +- Reused(reference_id=[2]) + : +- DynamicFilteringTableSourceScan(table=[[testCatalog, test_database, fact_part]], fields=[id, name, amount, price, fact_date_sk]) + +- Exchange(distribution=[hash[dim_date_sk]], shuffle_mode=[BATCH]) + +- Reused(reference_id=[3]) ]]> @@ -257,23 +234,20 @@ LogicalProject(c1=[$0], c2=[$1], c3=[$2]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalAggregate(group=[{}], c1=[COUNT()]) - : : +- LogicalProject($f0=[0]) - : : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 100))]) - : : +- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - : : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + : : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 100))]) + : : +- LogicalJoin(condition=[true], joinType=[inner]) + : : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + : : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) : +- LogicalAggregate(group=[{}], c2=[COUNT()]) - : +- LogicalProject($f0=[0]) - : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 200))]) - : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 200))]) + : +- LogicalJoin(condition=[true], joinType=[inner]) + : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) +- LogicalAggregate(group=[{}], c3=[COUNT()]) - +- LogicalProject($f0=[0]) - +- LogicalFilter(condition=[AND(=($4, $9), <($3, 400))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + +- LogicalFilter(condition=[AND(=($4, $9), <($3, 400))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) ]]> @@ -283,32 +257,29 @@ MultipleInput(readOrder=[0,0,1], members=[\nNestedLoopJoin(joinType=[InnerJoin], : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c3]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- Calc(select=[fact_date_sk], where=[(price < 400)]) -: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[], project=[price, fact_date_sk], metadata=[]]], fields=[price, fact_date_sk])(reuse_id=[1]) -: +- Exchange(distribution=[hash[dim_date_sk]])(reuse_id=[2]) -: +- TableSourceScan(table=[[testCatalog, test_database, dim, project=[dim_date_sk], metadata=[]]], fields=[dim_date_sk]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 400)]) +: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[]]], fields=[id, name, amount, price, fact_date_sk])(reuse_id=[1]) +: +- Exchange(distribution=[hash[dim_date_sk]])(reuse_id=[2]) +: +- TableSourceScan(table=[[testCatalog, test_database, dim]], fields=[id, male, amount, price, dim_date_sk]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c1]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- Calc(select=[fact_date_sk], where=[(price < 100)]) -: : +- Reused(reference_id=[1]) -: +- Reused(reference_id=[2]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 100)]) +: : +- Reused(reference_id=[1]) +: +- Reused(reference_id=[2]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c2]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) - :- Exchange(distribution=[hash[fact_date_sk]]) - : +- Calc(select=[fact_date_sk], where=[(price < 200)]) - : +- Reused(reference_id=[1]) - +- Reused(reference_id=[2]) + +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) + :- Exchange(distribution=[hash[fact_date_sk]]) + : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 200)]) + : +- Reused(reference_id=[1]) + +- Reused(reference_id=[2]) ]]> @@ -322,23 +293,20 @@ LogicalProject(c1=[$0], c2=[$1], c3=[$2]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalAggregate(group=[{}], c1=[COUNT()]) - : : +- LogicalProject($f0=[0]) - : : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 100))]) - : : +- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - : : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + : : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 100))]) + : : +- LogicalJoin(condition=[true], joinType=[inner]) + : : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + : : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) : +- LogicalAggregate(group=[{}], c2=[COUNT()]) - : +- LogicalProject($f0=[0]) - : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 200))]) - : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 200))]) + : +- LogicalJoin(condition=[true], joinType=[inner]) + : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) +- LogicalAggregate(group=[{}], c3=[COUNT()]) - +- LogicalProject($f0=[0]) - +- LogicalFilter(condition=[AND(=($4, $9), <($3, 400))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + +- LogicalFilter(condition=[AND(=($4, $9), <($3, 400))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) ]]> @@ -348,32 +316,29 @@ MultipleInput(readOrder=[0,0,1], members=[\nNestedLoopJoin(joinType=[InnerJoin], : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c3]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- Calc(select=[fact_date_sk], where=[(price < 400)]) -: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[], project=[price, fact_date_sk], metadata=[]]], fields=[price, fact_date_sk])(reuse_id=[1]) -: +- Exchange(distribution=[hash[dim_date_sk]])(reuse_id=[2]) -: +- TableSourceScan(table=[[testCatalog, test_database, dim, project=[dim_date_sk], metadata=[]]], fields=[dim_date_sk]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 400)]) +: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[]]], fields=[id, name, amount, price, fact_date_sk])(reuse_id=[1]) +: +- Exchange(distribution=[hash[dim_date_sk]])(reuse_id=[2]) +: +- TableSourceScan(table=[[testCatalog, test_database, dim]], fields=[id, male, amount, price, dim_date_sk]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c1]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- Calc(select=[fact_date_sk], where=[(price < 100)]) -: : +- Reused(reference_id=[1]) -: +- Reused(reference_id=[2]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 100)]) +: : +- Reused(reference_id=[1]) +: +- Reused(reference_id=[2]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c2]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) - :- Exchange(distribution=[hash[fact_date_sk]]) - : +- Calc(select=[fact_date_sk], where=[(price < 200)]) - : +- Reused(reference_id=[1]) - +- Reused(reference_id=[2]) + +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) + :- Exchange(distribution=[hash[fact_date_sk]]) + : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 200)]) + : +- Reused(reference_id=[1]) + +- Reused(reference_id=[2]) ]]> @@ -387,23 +352,20 @@ LogicalProject(c1=[$0], c2=[$1], c3=[$2]) +- LogicalJoin(condition=[true], joinType=[inner]) :- LogicalJoin(condition=[true], joinType=[inner]) : :- LogicalAggregate(group=[{}], c1=[COUNT()]) - : : +- LogicalProject($f0=[0]) - : : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 100))]) - : : +- LogicalJoin(condition=[true], joinType=[inner]) - : : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - : : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + : : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 100))]) + : : +- LogicalJoin(condition=[true], joinType=[inner]) + : : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + : : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) : +- LogicalAggregate(group=[{}], c2=[COUNT()]) - : +- LogicalProject($f0=[0]) - : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 200))]) - : +- LogicalJoin(condition=[true], joinType=[inner]) - : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + : +- LogicalFilter(condition=[AND(=($4, $9), <($3, 200))]) + : +- LogicalJoin(condition=[true], joinType=[inner]) + : :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + : +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) +- LogicalAggregate(group=[{}], c3=[COUNT()]) - +- LogicalProject($f0=[0]) - +- LogicalFilter(condition=[AND(=($4, $9), <($3, 400))]) - +- LogicalJoin(condition=[true], joinType=[inner]) - :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) - +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) + +- LogicalFilter(condition=[AND(=($4, $9), <($3, 400))]) + +- LogicalJoin(condition=[true], joinType=[inner]) + :- LogicalTableScan(table=[[testCatalog, test_database, fact_part]]) + +- LogicalTableScan(table=[[testCatalog, test_database, dim]]) ]]> @@ -413,32 +375,29 @@ MultipleInput(readOrder=[0,0,1], members=[\nNestedLoopJoin(joinType=[InnerJoin], : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c3]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- Calc(select=[fact_date_sk], where=[(price < 400)]) -: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[], project=[price, fact_date_sk], metadata=[]]], fields=[price, fact_date_sk])(reuse_id=[1]) -: +- Exchange(distribution=[hash[dim_date_sk]])(reuse_id=[2]) -: +- TableSourceScan(table=[[testCatalog, test_database, dim, project=[dim_date_sk], metadata=[]]], fields=[dim_date_sk]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 400)]) +: : +- TableSourceScan(table=[[testCatalog, test_database, fact_part, filter=[]]], fields=[id, name, amount, price, fact_date_sk])(reuse_id=[1]) +: +- Exchange(distribution=[hash[dim_date_sk]])(reuse_id=[2]) +: +- TableSourceScan(table=[[testCatalog, test_database, dim]], fields=[id, male, amount, price, dim_date_sk]) :- Exchange(distribution=[broadcast]) : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c1]) : +- Exchange(distribution=[single]) : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) -: +- Calc(select=[0 AS $f0]) -: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) -: :- Exchange(distribution=[hash[fact_date_sk]]) -: : +- Calc(select=[fact_date_sk], where=[(price < 100)]) -: : +- Reused(reference_id=[1]) -: +- Reused(reference_id=[2]) +: +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) +: :- Exchange(distribution=[hash[fact_date_sk]]) +: : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 100)]) +: : +- Reused(reference_id=[1]) +: +- Reused(reference_id=[2]) +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c2]) +- Exchange(distribution=[single]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) - +- Calc(select=[0 AS $f0]) - +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[fact_date_sk, dim_date_sk], build=[left]) - :- Exchange(distribution=[hash[fact_date_sk]]) - : +- Calc(select=[fact_date_sk], where=[(price < 200)]) - : +- Reused(reference_id=[1]) - +- Reused(reference_id=[2]) + +- HashJoin(joinType=[InnerJoin], where=[(fact_date_sk = dim_date_sk)], select=[id, name, amount, price, fact_date_sk, id0, male, amount0, price0, dim_date_sk], build=[left]) + :- Exchange(distribution=[hash[fact_date_sk]]) + : +- Calc(select=[id, name, amount, price, fact_date_sk], where=[(price < 200)]) + : +- Reused(reference_id=[1]) + +- Reused(reference_id=[2]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml index 6ee7786d08166..527c0c53f6dff 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/CalcPruneAggregateCallRuleTest.xml @@ -110,8 +110,7 @@ LogicalCalc(expr#0=[{inputs}], expr#1=[1], EXPR$0=[$t1]) LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) +- LogicalFilter(condition=[EXISTS({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalTableScan(table=[[default_catalog, default_database, T1]]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -131,9 +130,8 @@ LogicalTableScan(table=[[default_catalog, default_database, T2]]) LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) +- LogicalFilter(condition=[EXISTS({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalFilter(condition=[=(1, 2)]) - LogicalTableScan(table=[[default_catalog, default_database, T1]]) + LogicalFilter(condition=[=(1, 2)]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.xml index 6af67a2837a65..b1a0af22e1777 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/JoinConditionTypeCoerceRuleTest.xml @@ -209,8 +209,7 @@ LogicalProject(a=[$0]), rowType=[RecordType(INTEGER a)] :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], a0=[CAST($0):BIGINT]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, BIGINT a0)] : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalAggregate(group=[{}], cnt=[COUNT()]), rowType=[RecordType(BIGINT cnt)] - +- LogicalProject($f0=[0]), rowType=[RecordType(INTEGER $f0)] - +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> @@ -221,8 +220,7 @@ LogicalProject(a=[$0]), rowType=[RecordType(INTEGER a)] :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], a0=[CAST($0):BIGINT]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e, BIGINT a0)] : +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] +- LogicalAggregate(group=[{}], cnt=[COUNT()]), rowType=[RecordType(BIGINT cnt)] - +- LogicalProject($f0=[0]), rowType=[RecordType(INTEGER $f0)] - +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(INTEGER a, BIGINT b, FLOAT c, DOUBLE d, DECIMAL(38, 18) e)] ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml index c1a88eef41e5b..cf104d4a80cc3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ProjectPruneAggregateCallRuleTest.xml @@ -110,8 +110,7 @@ LogicalProject(EXPR$0=[1]) LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) +- LogicalFilter(condition=[EXISTS({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalTableScan(table=[[default_catalog, default_database, T1]]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> @@ -131,9 +130,8 @@ LogicalTableScan(table=[[default_catalog, default_database, T2]]) LogicalProject(a2=[$0], b2=[$1], c2=[$2], d2=[$3]) +- LogicalFilter(condition=[EXISTS({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalFilter(condition=[=(1, 2)]) - LogicalTableScan(table=[[default_catalog, default_database, T1]]) + LogicalFilter(condition=[=(1, 2)]) + LogicalTableScan(table=[[default_catalog, default_database, T1]]) })]) +- LogicalTableScan(table=[[default_catalog, default_database, T2]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml index 2550377ba6954..8f25a6622b311 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml @@ -375,15 +375,13 @@ LogicalProject(id=[$0], metadata_3=[$4], metadata_1=[$2]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.xml index a4f54e8e5b531..c32350b43b61b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyFilterConditionRuleTest.xml @@ -26,9 +26,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[OR(AND(=($0, 1), =($1, 2)), AND(NOT(<>($0, 1)), =($2, 3), true, EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), =(2, 2), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalFilter(condition=[AND(=($0, 5), =($1, 6))]) - LogicalTableScan(table=[[default_catalog, default_database, z]]) + LogicalFilter(condition=[AND(=($0, 5), =($1, 6))]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) LogicalTableScan(table=[[default_catalog, default_database, y]]) })))], variablesSet=[[$cor0]]) @@ -41,9 +40,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(=($0, 1), OR(=($1, 2), AND(=($2, 3), EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalFilter(condition=[AND(=($0, 5), =($1, 6))]) - LogicalTableScan(table=[[default_catalog, default_database, z]]) + LogicalFilter(condition=[AND(=($0, 5), =($1, 6))]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) LogicalTableScan(table=[[default_catalog, default_database, y]]) }))))], variablesSet=[[$cor0]]) @@ -61,9 +59,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[OR(AND(=($0, 1), =($1, 2)), AND(NOT(<>($0, 1)), =($2, 3), true, EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalFilter(condition=[OR(AND(=($0, 5), =($1, 6)), AND(NOT(<>($0, 5)), =($1, 7), true))]) - LogicalTableScan(table=[[default_catalog, default_database, z]]) + LogicalFilter(condition=[OR(AND(=($0, 5), =($1, 6)), AND(NOT(<>($0, 5)), =($1, 7), true))]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) LogicalTableScan(table=[[default_catalog, default_database, y]]) })))], variablesSet=[[$cor0]]) @@ -76,9 +73,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(=($0, 1), OR(=($1, 2), AND(=($2, 3), EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalFilter(condition=[AND(=($0, 5), SEARCH($1, Sarg[6L:BIGINT, 7L:BIGINT]:BIGINT))]) - LogicalTableScan(table=[[default_catalog, default_database, z]]) + LogicalFilter(condition=[AND(=($0, 5), SEARCH($1, Sarg[6L:BIGINT, 7L:BIGINT]:BIGINT))]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) LogicalTableScan(table=[[default_catalog, default_database, y]]) }))))], variablesSet=[[$cor0]]) @@ -96,9 +92,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[OR(AND(=($0, 1), =($1, 2)), AND(NOT(<>($0, 1)), =($2, 3), true, EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), =(2, 2), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalFilter(condition=[OR(AND(=($0, 5), =($1, 6)), AND(NOT(<>($0, 5)), =($1, 7), true))]) - LogicalTableScan(table=[[default_catalog, default_database, z]]) + LogicalFilter(condition=[OR(AND(=($0, 5), =($1, 6)), AND(NOT(<>($0, 5)), =($1, 7), true))]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) LogicalTableScan(table=[[default_catalog, default_database, y]]) })))], variablesSet=[[$cor0]]) @@ -111,9 +106,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) +- LogicalFilter(condition=[AND(=($0, 1), OR(=($1, 2), AND(=($2, 3), EXISTS({ LogicalFilter(condition=[AND(=($cor0.a, $0), >($SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalFilter(condition=[AND(=($0, 5), SEARCH($1, Sarg[6L:BIGINT, 7L:BIGINT]:BIGINT))]) - LogicalTableScan(table=[[default_catalog, default_database, z]]) + LogicalFilter(condition=[AND(=($0, 5), SEARCH($1, Sarg[6L:BIGINT, 7L:BIGINT]:BIGINT))]) + LogicalTableScan(table=[[default_catalog, default_database, z]]) }), 0))]) LogicalTableScan(table=[[default_catalog, default_database, y]]) }))))], variablesSet=[[$cor0]]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.xml index 29dad4846ef2b..6dfd8abf107dc 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/SimplifyJoinConditionRuleTest.xml @@ -28,9 +28,8 @@ SELECT a FROM MyTable1 WHERE b = ( LogicalProject(a=[$0]) +- LogicalFilter(condition=[=($1, $SCALAR_QUERY({ LogicalAggregate(group=[{}], EXPR$0=[COUNT()]) - LogicalProject($f0=[0]) - LogicalFilter(condition=[OR(AND(=($0, $cor0.a), <($0, 2)), AND(=($0, $cor0.a), =($cor0.b, 5)))]) - LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) + LogicalFilter(condition=[OR(AND(=($0, $cor0.a), <($0, 2)), AND(=($0, $cor0.a), =($cor0.b, 5)))]) + LogicalTableScan(table=[[default_catalog, default_database, MyTable2]]) }))], variablesSet=[[$cor0]]) +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml index 61395048aa082..59920a7574732 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryAntiJoinTest.xml @@ -1070,8 +1070,8 @@ LogicalProject(b=[$1]) : : +- LogicalProject(c=[$0], ck=[$0]) : : +- LogicalAggregate(group=[{}], c=[COUNT()]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(EXPR$0=[1], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(EXPR$0=[1]) : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) @@ -1109,8 +1109,8 @@ LogicalProject(b=[$1]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(i=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(i=[$0], i0=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(i=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) @@ -1153,14 +1153,14 @@ LogicalProject(b=[$1]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : : : +- LogicalAggregate(group=[{0, 1}]) - : : : +- LogicalProject(i=[$0], i0=[true]) + : : : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : : : +- LogicalProject(i=[$0]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(j=[$0], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(j=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) @@ -1203,14 +1203,14 @@ LogicalProject(b=[$1]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : : : +- LogicalAggregate(group=[{0, 1}]) - : : : +- LogicalProject(i=[$0], i0=[true]) + : : : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : : : +- LogicalProject(i=[$0]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(j=[$0], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(j=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(e=[$1], d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) @@ -1836,8 +1836,8 @@ LogicalProject(b=[$1]) : : +- LogicalProject(c=[$0], ck=[$0]) : : +- LogicalAggregate(group=[{}], c=[COUNT()]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(EXPR$0=[1], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(EXPR$0=[1]) : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, r]]) @@ -1873,8 +1873,8 @@ LogicalProject(b=[$1]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(i=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(i=[$0], i0=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(i=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, r]]) @@ -1915,14 +1915,14 @@ LogicalProject(b=[$1]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : : : +- LogicalAggregate(group=[{0, 1}]) - : : : +- LogicalProject(i=[$0], i0=[true]) + : : : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : : : +- LogicalProject(i=[$0]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(j=[$0], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(j=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, r]]) @@ -1963,14 +1963,14 @@ LogicalProject(b=[$1]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : : : +- LogicalAggregate(group=[{0, 1}]) - : : : +- LogicalProject(i=[$0], i0=[true]) + : : : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : : : +- LogicalProject(i=[$0]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(j=[$0], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(j=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(e=[$1], d=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, r]]) @@ -2011,14 +2011,14 @@ LogicalProject(c=[$2]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : : : +- LogicalAggregate(group=[{0, 1}]) - : : : +- LogicalProject(i=[$0], i0=[true]) + : : : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : : : +- LogicalProject(i=[$0]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(j=[$0], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(j=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0], e=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, r]]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml index 831d380b7af38..1a11db3d73bc9 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQuerySemiJoinTest.xml @@ -1920,8 +1920,8 @@ LogicalProject(b=[$1]) : : +- LogicalProject(c=[$0], ck=[$0]) : : +- LogicalAggregate(group=[{}], c=[COUNT()]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(EXPR$0=[1], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(EXPR$0=[1]) : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) @@ -1959,8 +1959,8 @@ LogicalProject(b=[$1]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(i=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(i=[$0], i0=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(i=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) @@ -2003,14 +2003,14 @@ LogicalProject(b=[$1]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : : : +- LogicalAggregate(group=[{0, 1}]) - : : : +- LogicalProject(i=[$0], i0=[true]) + : : : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : : : +- LogicalProject(i=[$0]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$1]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(j=[$1], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(j=[$1]) : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) @@ -2053,14 +2053,14 @@ LogicalProject(b=[$1]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : : : +- LogicalAggregate(group=[{0, 1}]) - : : : +- LogicalProject(i=[$0], i0=[true]) + : : : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : : : +- LogicalProject(i=[$0]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$1]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(j=[$1], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(j=[$1]) : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(e=[$1], d=[$0], f=[$2]) +- LogicalFilter(condition=[true]) @@ -3336,8 +3336,8 @@ LogicalProject(b=[$1]) : : +- LogicalProject(c=[$0], ck=[$0]) : : +- LogicalAggregate(group=[{}], c=[COUNT()]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(EXPR$0=[1], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(EXPR$0=[1]) : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, r]]) @@ -3373,8 +3373,8 @@ LogicalProject(b=[$1]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(i=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(i=[$0], i0=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(i=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) +- LogicalProject(d=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, r]]) @@ -3415,14 +3415,14 @@ LogicalProject(b=[$1]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : : : +- LogicalAggregate(group=[{0, 1}]) - : : : +- LogicalProject(i=[$0], i0=[true]) + : : : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : : : +- LogicalProject(i=[$0]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(j=[$0], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(j=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, r]]) @@ -3463,14 +3463,14 @@ LogicalProject(b=[$1]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : : : +- LogicalAggregate(group=[{0, 1}]) - : : : +- LogicalProject(i=[$0], i0=[true]) + : : : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : : : +- LogicalProject(i=[$0]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(j=[$0], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(j=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(e=[$1], d=[$0]) +- LogicalTableScan(table=[[default_catalog, default_database, r]]) @@ -3513,14 +3513,14 @@ LogicalProject(c=[$2]) : : : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : : : +- LogicalProject(i=[$0]) : : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) - : : : +- LogicalAggregate(group=[{0, 1}]) - : : : +- LogicalProject(i=[$0], i0=[true]) + : : : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : : : +- LogicalProject(i=[$0]) : : : +- LogicalTableScan(table=[[default_catalog, default_database, t1]]) : : +- LogicalAggregate(group=[{}], c=[COUNT()], ck=[COUNT($0)]) : : +- LogicalProject(j=[$0]) : : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) - : +- LogicalAggregate(group=[{0, 1}]) - : +- LogicalProject(j=[$0], i=[true]) + : +- LogicalAggregate(group=[{0}], i=[LITERAL_AGG(true)]) + : +- LogicalProject(j=[$0]) : +- LogicalTableScan(table=[[default_catalog, default_database, t2]]) +- LogicalProject(d=[$0], e=[$1]) +- LogicalTableScan(table=[[default_catalog, default_database, r]]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml index 9679640e8309c..00ec68199145d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml @@ -374,15 +374,14 @@ Calc(select=[EXPR$0, type, name]) @@ -393,15 +392,14 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]) @@ -412,15 +410,14 @@ HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml index 929b396d21a35..3b8442cbc28b0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml @@ -23,17 +23,16 @@ limitations under the License. ($1, 1)]) - +- LogicalTableScan(table=[[default_catalog, default_database, src]]) ++- LogicalFilter(condition=[>($1, 1)]) + +- LogicalTableScan(table=[[default_catalog, default_database, src]]) ]]> (a, 1)], changelogMode=[I,UB,UA]) - +- TableSourceScan(table=[[default_catalog, default_database, src, filter=[], project=[a], metadata=[]]], fields=[a], changelogMode=[I,UB,UA]) + +- Calc(select=[ts, a, b], where=[>(a, 1)], changelogMode=[I,UB,UA]) + +- TableSourceScan(table=[[default_catalog, default_database, src, filter=[]]], fields=[ts, a, b], changelogMode=[I,UB,UA]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml index d5ffd5f32bf1e..25e49f40e4d95 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml @@ -152,15 +152,14 @@ WatermarkAssigner(rowtime=[rtime], watermark=[rtime]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml index 47ccb1cd87451..bc33b9bfc7beb 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.xml @@ -256,16 +256,14 @@ GroupAggregate(groupBy=[b], select=[b, SUM(a) AS EXPR$1]) @@ -276,17 +274,15 @@ GroupAggregate(select=[COUNT(*) AS EXPR$0]) @@ -297,17 +293,15 @@ GroupAggregate(select=[COUNT(*) AS EXPR$0]) @@ -318,17 +312,15 @@ GroupAggregate(select=[COUNT(*) AS EXPR$0]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml index 93f8ca912f590..9ef822d3d3f8b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/SemiAntiJoinTest.xml @@ -694,8 +694,8 @@ Calc(select=[b]) : : +- Exchange(distribution=[single]) : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) : +- Exchange(distribution=[single]) - : +- Calc(select=[true AS i]) - : +- GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : +- Calc(select=[i]) + : +- GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0, LITERAL_AGG(*) AS i]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- Calc(select=[1 AS EXPR$0]) : +- Reused(reference_id=[1]) @@ -1236,8 +1236,8 @@ Calc(select=[b]) : : +- Exchange(distribution=[single]) : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[1]) : +- Exchange(distribution=[single]) - : +- Calc(select=[true AS i]) - : +- GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : +- Calc(select=[i]) + : +- GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0, LITERAL_AGG(*) AS i]) : +- Exchange(distribution=[hash[EXPR$0]]) : +- Calc(select=[1 AS EXPR$0]) : +- Reused(reference_id=[1]) @@ -2273,13 +2273,13 @@ LogicalProject(d=[$0]) Calc(select=[b]) +- Join(joinType=[LeftAntiJoin], where=[((d IS NULL OR ($f3 = d)) AND (c = f))], select=[b, c, $f3], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[hash[c]]) - : +- Calc(select=[b, c, CASE(((c0 = 0) OR (i0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3]) - : +- Join(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) + : +- Calc(select=[b, c, CASE(((c0 = 0) OR (i_0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c1 = 0) OR (i IS NULL AND (ck0 >= c1) AND a IS NOT NULL)), 2, 3) AS $f3]) + : +- Join(joinType=[LeftOuterJoin], where=[(a = EXPR$0)], select=[a, b, c, c0, ck, i_0, c1, ck0, EXPR$0, i], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : :- Exchange(distribution=[hash[a]]) - : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) + : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i_0, c1, ck0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : : :- Exchange(distribution=[single]) - : : : +- Calc(select=[a, b, c, c0, ck, i0]) - : : : +- Join(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) + : : : +- Calc(select=[a, b, c, c0, ck, i_0]) + : : : +- Join(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, c0, ck, i, i_0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : : : :- Exchange(distribution=[hash[a]]) : : : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : : : : :- Exchange(distribution=[single]) @@ -2290,20 +2290,18 @@ Calc(select=[b]) : : : : +- Calc(select=[i])(reuse_id=[1]) : : : : +- TableSourceScan(table=[[default_catalog, default_database, t]], fields=[i, j, k])(reuse_id=[2]) : : : +- Exchange(distribution=[hash[i]]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- GroupAggregate(groupBy=[i], select=[i]) - : : : +- Exchange(distribution=[hash[i]]) - : : : +- Reused(reference_id=[1]) + : : : +- GroupAggregate(groupBy=[i], select=[i, LITERAL_AGG(*) AS i_0]) + : : : +- Exchange(distribution=[hash[i]]) + : : : +- Reused(reference_id=[1]) : : +- Exchange(distribution=[single]) : : +- GroupAggregate(select=[COUNT(*) AS c, COUNT(EXPR$0) AS ck]) : : +- Exchange(distribution=[single]) : : +- Calc(select=[CAST(j AS INTEGER) AS EXPR$0])(reuse_id=[3]) : : +- Reused(reference_id=[2]) : +- Exchange(distribution=[hash[EXPR$0]]) - : +- Calc(select=[EXPR$0, true AS i]) - : +- GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Exchange(distribution=[hash[EXPR$0]]) - : +- Reused(reference_id=[3]) + : +- GroupAggregate(groupBy=[EXPR$0], select=[EXPR$0, LITERAL_AGG(*) AS i]) + : +- Exchange(distribution=[hash[EXPR$0]]) + : +- Reused(reference_id=[3]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) @@ -2517,13 +2515,13 @@ LogicalProject(e=[$1], d=[$0]) Calc(select=[b]) +- Join(joinType=[LeftAntiJoin], where=[((b IS NULL OR e IS NULL OR (b = e)) AND (d IS NULL OR ($f3 = d)))], select=[b, $f3], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey]) :- Exchange(distribution=[single]) - : +- Calc(select=[b, CASE(((c0 = 0) OR (i0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c = 0) OR (i IS NULL AND (ck0 >= c) AND a IS NOT NULL)), 2, 3) AS $f3]) - : +- Join(joinType=[LeftOuterJoin], where=[(a = j)], select=[a, b, c0, ck, i0, c, ck0, j, i], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) + : +- Calc(select=[b, CASE(((c0 = 0) OR (i_0 IS NULL AND (ck >= c0) AND a IS NOT NULL)), 1, ((c = 0) OR (i IS NULL AND (ck0 >= c) AND a IS NOT NULL)), 2, 3) AS $f3]) + : +- Join(joinType=[LeftOuterJoin], where=[(a = j)], select=[a, b, c0, ck, i_0, c, ck0, j, i], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : :- Exchange(distribution=[hash[a]]) - : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i0, c, ck0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) + : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i_0, c, ck0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : : :- Exchange(distribution=[single]) - : : : +- Calc(select=[a, b, c AS c0, ck, i0]) - : : : +- Join(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, ck, i, i0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) + : : : +- Calc(select=[a, b, c AS c0, ck, i_0]) + : : : +- Join(joinType=[LeftOuterJoin], where=[(a = i)], select=[a, b, c, ck, i, i_0], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : : : :- Exchange(distribution=[hash[a]]) : : : : +- Join(joinType=[InnerJoin], where=[true], select=[a, b, c, ck], leftInputSpec=[NoUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey]) : : : : :- Exchange(distribution=[single]) @@ -2534,19 +2532,17 @@ Calc(select=[b]) : : : : +- Exchange(distribution=[single]) : : : : +- TableSourceScan(table=[[default_catalog, default_database, t1]], fields=[i])(reuse_id=[1]) : : : +- Exchange(distribution=[hash[i]]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- GroupAggregate(groupBy=[i], select=[i]) - : : : +- Exchange(distribution=[hash[i]]) - : : : +- Reused(reference_id=[1]) + : : : +- GroupAggregate(groupBy=[i], select=[i, LITERAL_AGG(*) AS i_0]) + : : : +- Exchange(distribution=[hash[i]]) + : : : +- Reused(reference_id=[1]) : : +- Exchange(distribution=[single]) : : +- GroupAggregate(select=[COUNT(*) AS c, COUNT(j) AS ck]) : : +- Exchange(distribution=[single]) : : +- TableSourceScan(table=[[default_catalog, default_database, t2]], fields=[j])(reuse_id=[2]) : +- Exchange(distribution=[hash[j]]) - : +- Calc(select=[j, true AS i]) - : +- GroupAggregate(groupBy=[j], select=[j]) - : +- Exchange(distribution=[hash[j]]) - : +- Reused(reference_id=[2]) + : +- GroupAggregate(groupBy=[j], select=[j, LITERAL_AGG(*) AS i]) + : +- Exchange(distribution=[hash[j]]) + : +- Reused(reference_id=[2]) +- Exchange(distribution=[single]) +- Calc(select=[e, d]) +- TableSourceScan(table=[[default_catalog, default_database, r]], fields=[d, e, f]) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala index e27225c529b3b..3e8bf996630a1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala @@ -1007,6 +1007,7 @@ class FlinkRelMdHandlerTestBase { false, false, false, + Seq().toList, Seq(Integer.valueOf(3)).toList, -1, null, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala index 725464260ab29..eea6b0b433605 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdSelectivityTest.scala @@ -495,6 +495,7 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { false, false, false, + ImmutableList.of[RexNode](), ImmutableList.of[Integer](), -1, null, @@ -506,6 +507,7 @@ class FlinkRelMdSelectivityTest extends FlinkRelMdHandlerTestBase { false, false, false, + ImmutableList.of(), ImmutableList.of(Integer.valueOf(3)), -1, null, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala index f8e34d9ce2d2d..ea527aebe0285 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala @@ -579,9 +579,9 @@ class JoinTest extends TableTestBase { // TODO the error message should be improved after we support extracting alias from table func util.addTemporarySystemFunction("TableFunc1", new TableFunc1) util.verifyExpectdException( - "SELECT /*+ LOOKUP('table'='D') */ T.a FROM t AS T CROSS JOIN LATERAL TABLE(TableFunc1(c)) AS D(c1)", + "SELECT /*+ LOOKUP('table'='DD') */ T.a FROM t AS T CROSS JOIN LATERAL TABLE(TableFunc1(c)) AS D(c1)", "The options of following hints cannot match the name of input tables or views: \n" + - "`D` in `LOOKUP`" + "`DD` in `LOOKUP`" ) } diff --git a/flink-table/pom.xml b/flink-table/pom.xml index 4c15284a9a51e..6074853437c57 100644 --- a/flink-table/pom.xml +++ b/flink-table/pom.xml @@ -78,8 +78,8 @@ under the License. - 1.34.0 - 3.1.10