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

Rion Williams commented on FLINK-39122:
---------------------------------------

[~twalthr] 

I was digging around and came across this issue and would be interested in 
working on it.

A bit of cursory digging points to a pretty clear culprit within the following 
snippet from `DecimalData.fromBigDecimal`:
{code:java}
public static @Nullable DecimalData fromBigDecimal(BigDecimal bd, int 
precision, int scale) {
    bd = bd.setScale(scale, RoundingMode.HALF_UP);
    if (bd.precision() > precision) {
        return null;  
      }
      // Omitted for brevity
} {code}
However, I suspect this is something that we'd likely want to capture during 
validation. It doesn't look like for these types of implicit casts (e.g. 
DECIMAL to DECIMAL) that we ever validate precision values.

I think one or two reasonable options might be:
 * Add an explicit check after performing our casting to validate decimal 
precision values (similar to what is done above but upstream) in the 
`TypeReferenceUtil.castArguments()` stage.
 * Ensure that our `DecimalToDecimalCastRule` actually overrides `canFail()` 
similar to other rules (obviously affects all decimal-decimal casts in terms of 
blast-radius but would only come up during actual errors)

Any thoughts?

 

> Decimals with incorrect precision lead to null value in PTF
> -----------------------------------------------------------
>
>                 Key: FLINK-39122
>                 URL: https://issues.apache.org/jira/browse/FLINK-39122
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table SQL / Planner
>            Reporter: Timo Walther
>            Priority: Major
>
> Instead of a type error the following call leads to a null value for the 
> decimal:
> {code}
>         // execute a Flink SQL job and print the result locally
>         tableEnv.fromCall(RulePtf.class, new 
> BigDecimal("123.456")).execute().print();
>     }
>     /** A PTF that filters data based on a dynamic allowlist. */
>     public static class RulePtf extends ProcessTableFunction<String> {
>         public void eval(@DataTypeHint("DECIMAL(2, 2)") BigDecimal decimal) 
> throws Exception {
>             System.out.println(decimal);
>         }
>     }
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to