xuyangzhong commented on code in PR #25418: URL: https://github.com/apache/flink/pull/25418#discussion_r1810079375
########## flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java: ########## @@ -98,7 +103,21 @@ 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()); + + RelOptCluster relOptCluster = + ((FlinkCalciteSqlValidator) ((SqlCallBinding) opBinding).getValidator()) + .getRelOptCluster(); + TableConfig tableConfig = ShortcutUtils.unwrapContext(relOptCluster).getTableConfig(); + boolean legacyCastEnabled = + tableConfig + .get(ExecutionConfigOptions.TABLE_EXEC_LEGACY_CAST_BEHAVIOUR) + .isEnabled(); + ret = + opBinding + .getTypeFactory() + .createTypeWithNullability( + ret, firstType.isNullable() || legacyCastEnabled); Review Comment: Please include a note about this behavior in the class comment as well. ########## flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java: ########## @@ -98,7 +103,21 @@ 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()); + + RelOptCluster relOptCluster = + ((FlinkCalciteSqlValidator) ((SqlCallBinding) opBinding).getValidator()) Review Comment: Can you check if opBinding is an instance of SqlCallBinding just like the if block below? What about ? ``` boolean forceOutputTypeNullable = false; if (opBinding instanceof SqlCallBinding && ((SqlCallBinding) opBinding).getValidator() instanceof FlinkCalciteSqlValidator) { FlinkCalciteSqlValidator validator = (FlinkCalciteSqlValidator) ((SqlCallBinding) opBinding).getValidator(); TableConfig tableConfig = ShortcutUtils.unwrapContext(validator.getRelOptCluster()).getTableConfig(); forceOutputTypeNullable = tableConfig .get(ExecutionConfigOptions.TABLE_EXEC_LEGACY_CAST_BEHAVIOUR) .isEnabled(); } ret = opBinding .getTypeFactory() .createTypeWithNullability( ret, firstType.isNullable() || forceOutputTypeNullable); ``` ########## flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java: ########## @@ -99,6 +105,8 @@ /** * Standard implementation of {@link SqlRexConvertletTable}. * + * <p>Lines 555-564 fix incorrect cast behavior (FLINK-36399). + * * <p>Lines 691-736 implement supporting RETURNING clause in JSON_QUERY (CALCITE-6365). Review Comment: nit: update the line number. ########## flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java: ########## @@ -545,7 +553,16 @@ protected RexNode convertCast(SqlRexContext cx, final SqlCall call) { type = cx.getValidator().getValidatedNodeType(dataType.getTypeName()); } RexNode arg = cx.convertExpression(left); - if (arg.getType().isNullable()) { + + RelOptCluster relOptCluster = Review Comment: Add comments ``` // BEGIN FLINK MODIFICATION ... // END FLINK MODIFICATION ``` ########## flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscLegacyITCase.java: ########## @@ -51,11 +52,12 @@ Stream<TestSetSpec> getTestSetSpecs() { BuiltInFunctionDefinitions.CAST, "legacy cast failure returns null") .onFieldsWithData("invalid") .andDataTypes(STRING().notNull()) - .testSqlRuntimeError( - "CAST(f0 AS BIGINT)", - "Column 'EXPR$0' is NOT NULL, however, a null value is " - + "being written into it. You can set job configuration " - + "'table.exec.sink.not-null-enforcer'='DROP' to suppress " - + "this exception and drop such records silently.")); + .testSqlResult("CAST(f0 AS BIGINT)", null, BIGINT()), + TestSetSpec.forFunction( + BuiltInFunctionDefinitions.CAST, "legacy cast failure returns null") + .onFieldsWithData("100") + .andDataTypes(STRING().notNull()) + .testSqlResult( + "CAST(CAST(f0 AS DECIMAL(2, 0)) AS STRING)", null, STRING())); Review Comment: IIUC, you only tested runtime issues and did not test for constant folding issues, right? ########## flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java: ########## @@ -545,7 +553,16 @@ protected RexNode convertCast(SqlRexContext cx, final SqlCall call) { type = cx.getValidator().getValidatedNodeType(dataType.getTypeName()); } RexNode arg = cx.convertExpression(left); - if (arg.getType().isNullable()) { + + RelOptCluster relOptCluster = + ((FlinkCalciteSqlValidator) cx.getValidator()).getRelOptCluster(); Review Comment: ditto. Avoid forced type casts to prevent unexpected errors. -- 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