twalthr commented on code in PR #26547: URL: https://github.com/apache/flink/pull/26547#discussion_r2111591075
########## flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java: ########## @@ -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 Review Comment: do we need to document those function synonyms? ########## flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java: ########## @@ -240,19 +242,18 @@ * <p>FLINK modifications are at lines * * <ol> - * <li>Added in FLINK-29081, FLINK-28682, FLINK-33395: Lines 670 ~ 687 - * <li>Added in Flink-24024: Lines 1463 ~ 1469 - * <li>Added in Flink-24024: Lines 1483 ~ 1522 - * <li>Added in Flink-37269: Lines 2239 ~ 2261 - * <li>Added in FLINK-28682: Lines 2372 ~ 2389 - * <li>Added in FLINK-28682: Lines 2426 ~ 2454 - * <li>Added in FLINK-32474: Lines 2507 ~ 2509 - * <li>Added in FLINK-32474: Lines 2513 ~ 2515 - * <li>Added in FLINK-32474: Lines 2526 ~ 2528 - * <li>Added in FLINK-32474: Lines 2934 ~ 2945 - * <li>Added in FLINK-32474: Lines 3046 ~ 3080 - * <li>Added in FLINK-34312: Lines 5827 ~ 5838 - * <li>Added in FLINK-34057, FLINK-34058, FLINK-34312: Lines 6285 ~ 6303 + * <li>Added in FLINK-29081, FLINK-28682, FLINK-33395: Lines 673 ~ 690 Review Comment: Thank you for updating all these lines. Very helpful, but I'm sure also a lot of work! ########## flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/type/FlinkCalciteTableMappingRule.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 FlinkCalciteTableMappingRule implements SqlTypeMappingRule { Review Comment: `FlinkSqlTypeMappingRule ` ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LiteralAggFunction.java: ########## @@ -0,0 +1,166 @@ +/* + * 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 <a + * href="https://issues.apache.org/jira/browse/CALCITE-4334">CALCITE-4334</a> 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}; Review Comment: Do we actually need to use any state? Can't this be empty array? ########## flink-table/flink-sql-parser/src/main/codegen/templates/Parser.jj: ########## @@ -8001,6 +8003,8 @@ SqlPostfixOperator PostfixRowOperator() : | < DATETIME_INTERVAL_PRECISION: "DATETIME_INTERVAL_PRECISION" > | < DATETIME_TRUNC: "DATETIME_TRUNC" > | < DAY: "DAY" > +| < DAYOFWEEK: "DAYOFWEEK" > Review Comment: update documentation, do we also support those in runtime code? or at least throw a proper error message? ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/aggfunctions/LiteralAggFunction.java: ########## @@ -0,0 +1,166 @@ +/* + * 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 <a + * href="https://issues.apache.org/jira/browse/CALCITE-4334">CALCITE-4334</a> 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 Long Literal aggregate function. */ + public static class LongLiteralAggFunction extends LiteralAggFunction { Review Comment: No INT? Are byte and short ever called? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org