[ 
https://issues.apache.org/jira/browse/FLINK-27247?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17522626#comment-17522626
 ] 

xuyang commented on FLINK-27247:
--------------------------------

Hi, [~matriv], I think currently this is not the problem with casting. Because 
this part of code gen doesn't not care of the nullable with the type and other 
part of code gen will avoid the nullable exists. For example, you can see the 
following code generated:
{code:java}
// the result rand(...) will be null only if the arg in rand is nullable, like 
"rand(cast (null as int))"
// so if the sql is "rand() + 1", the generated code is:
        
    isNull$3 = false;
    result$4 = random$2.nextDouble();

// and if the sql is "rand(cast(null as int))":

    isNull$3 = true;
    result$4 = -1.0d;
    if (!isNull$3) {
          result$4 = random$2.nextDouble();
    }{code}
This part that i fixed is only about the code : "result$4 = 
random$2.nextDouble();"  and this should just ignore the nullable between 
DOUBLE and DOUBLE NOT NULL. And actually the logic of the legacy code does this 
by pre-checking the same type not necessary to cast before casting this 
different types.

I strongly agree with you that the logic in casting should throw an exception 
if it meets casting a type from nullable to not nullable. But the problem of 
this issue is before casting logic.

By the way, I think converting casting logic to different rules is a good 
improvement but should not affect the base logic when change the code. You can 
see the code before and after rewriting casting rules:

before:
{code:java}
// no casting necessary
if (isInteroperable(operandType, resultType)) {
  operandTerm => s"$operandTerm"
}
// decimal to decimal, may have different precision/scale
else if (isDecimal(resultType) && isDecimal(operandType)) {
  val dt = resultType.asInstanceOf[DecimalType]
  operandTerm =>
    s"$DECIMAL_UTIL.castToDecimal($operandTerm, ${dt.getPrecision}, 
${dt.getScale})"
}
// non_decimal_numeric to decimal
else if ...{code}
after:
{code:java}
// All numeric rules are assumed to be instance of 
AbstractExpressionCodeGeneratorCastRule
val rule = CastRuleProvider.resolve(operandType, resultType)
rule match {
  case codeGeneratorCastRule: ExpressionCodeGeneratorCastRule[_, _] =>
    operandTerm =>
      codeGeneratorCastRule.generateExpression(
        toCodegenCastContext(ctx),
        operandTerm,
        operandType,
        resultType
      )
  case _ =>
    throw new CodeGenException(s"Unsupported casting from $operandType to 
$resultType.")
} {code}
Looking forward to your reply :)

 

 

> ScalarOperatorGens.numericCasting is not compatible with legacy behavior
> ------------------------------------------------------------------------
>
>                 Key: FLINK-27247
>                 URL: https://issues.apache.org/jira/browse/FLINK-27247
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>            Reporter: xuyang
>            Priority: Minor
>              Labels: pull-request-available
>
> Add the following test cases in ScalarFunctionsTest:
> {code:java}
> // code placeholder
> @Test
> def test(): Unit ={
>   testSqlApi("rand(1) + 1","")
> } {code}
> it will throw the following exception:
> {code:java}
> // code placeholder
> org.apache.flink.table.planner.codegen.CodeGenException: Unsupported casting 
> from DOUBLE to DOUBLE NOT NULL.
>     at 
> org.apache.flink.table.planner.codegen.calls.ScalarOperatorGens$.numericCasting(ScalarOperatorGens.scala:1734)
>     at 
> org.apache.flink.table.planner.codegen.calls.ScalarOperatorGens$.generateBinaryArithmeticOperator(ScalarOperatorGens.scala:85)
>     at 
> org.apache.flink.table.planner.codegen.ExprCodeGenerator.generateCallExpression(ExprCodeGenerator.scala:507)
>     at 
> org.apache.flink.table.planner.codegen.ExprCodeGenerator.visitCall(ExprCodeGenerator.scala:481)
>     at 
> org.apache.flink.table.planner.codegen.ExprCodeGenerator.visitCall(ExprCodeGenerator.scala:57)
>     at org.apache.calcite.rex.RexCall.accept(RexCall.java:174)
>     at 
> org.apache.flink.table.planner.codegen.ExprCodeGenerator.$anonfun$visitCall$1(ExprCodeGenerator.scala:478)
>     at 
> scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:233)
>     at 
> scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:58)
>     at 
> scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:51)
>     at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
>     at scala.collection.TraversableLike.map(TraversableLike.scala:233)
>     at scala.collection.TraversableLike.map$(TraversableLike.scala:226)
>     at scala.collection.AbstractTraversable.map(Traversable.scala:104)
>     at 
> org.apache.flink.table.planner.codegen.ExprCodeGenerator.visitCall(ExprCodeGenerator.scala:469)
> ... {code}
> This is because in ScalarOperatorGens#numericCasting,  FLINK-24779  lost the 
> logic that in some cases there is no need to casting the left and right type.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to