[
https://issues.apache.org/jira/browse/FLINK-29373?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
hk__lrzy updated FLINK-29373:
-----------------------------
Description:
When we try to transfrom datastream to table, *TypeInfoDataTypeConverter* will
try to convert *TypeInformation* to {*}DataType{*}, but if datastream's produce
types contains {*}BigDecimalTypeInfo{*}, *TypeInfoDataTypeConverter* will final
convert it to {*}RawDataType{*},then when we want tranform table to datastream
again, exception will hapend, and show the data type not match.
Blink planner also will has this exception.
!image-2022-09-21-15-12-11-082.png!
{code:java}
Caused by: org.apache.flink.table.codegen.CodeGenException: Incompatible types
of expression and result type.
Expression[GeneratedExpression(result$10,isNull$11,,Decimal(9,2),false)] type
is [Decimal(9,2)], result type is [GenericType<java.math.BigDecimal>] {code}
how to recurrent
{code:java}
// code placeholder
StreamExecutionEnvironment executionEnvironment =
StreamExecutionEnvironment.getExecutionEnvironment();
EnvironmentSettings.Builder envBuilder = EnvironmentSettings.newInstance()
.useOldPlanner()
.inStreamingMode();
StreamTableEnvironment streamTableEnvironment =
StreamTableEnvironment.create(executionEnvironment, envBuilder.build());
FromElementsFunction fromElementsFunction = new FromElementsFunction(new
BigDecimal(1.11D));
DataStreamSource dataStreamSource =
executionEnvironment.addSource(fromElementsFunction, new BigDecimalTypeInfo(10,
8));
streamTableEnvironment.createTemporaryView("tmp", dataStreamSource);
Table table = streamTableEnvironment.sqlQuery("select * from tmp");
streamTableEnvironment.toRetractStream(table, table.getSchema().toRowType());
{code}
was:
When we want to transfrom datastream to table, *TypeInfoDataTypeConverter* will
try to convert *TypeInformation* to {*}DataType{*}, but if datastream's produce
types contains {*}BigDecimalTypeInfo{*}, *TypeInfoDataTypeConverter* will final
convert it to {*}RawDataType{*},then when we want tranform table to datastream
again, exception will hapend, and show the data type not match.
Blink planner also will has this exception.
!image-2022-09-21-15-12-11-082.png!
{code:java}
Caused by: org.apache.flink.table.codegen.CodeGenException: Incompatible types
of expression and result type.
Expression[GeneratedExpression(result$10,isNull$11,,Decimal(9,2),false)] type
is [Decimal(9,2)], result type is [GenericType<java.math.BigDecimal>] {code}
how to recurrent
{code:java}
// code placeholder
StreamExecutionEnvironment executionEnvironment =
StreamExecutionEnvironment.getExecutionEnvironment();
EnvironmentSettings.Builder envBuilder = EnvironmentSettings.newInstance()
.useOldPlanner()
.inStreamingMode();
StreamTableEnvironment streamTableEnvironment =
StreamTableEnvironment.create(executionEnvironment, envBuilder.build());
FromElementsFunction fromElementsFunction = new FromElementsFunction(new
BigDecimal(1.11D));
DataStreamSource dataStreamSource =
executionEnvironment.addSource(fromElementsFunction, new BigDecimalTypeInfo(10,
8));
streamTableEnvironment.createTemporaryView("tmp", dataStreamSource);
Table table = streamTableEnvironment.sqlQuery("select * from tmp");
streamTableEnvironment.toRetractStream(table, table.getSchema().toRowType());
{code}
> DataStream to table not support BigDecimalTypeInfo
> --------------------------------------------------
>
> Key: FLINK-29373
> URL: https://issues.apache.org/jira/browse/FLINK-29373
> Project: Flink
> Issue Type: Bug
> Components: Table SQL / Planner
> Affects Versions: 1.17.0
> Reporter: hk__lrzy
> Priority: Major
> Attachments: image-2022-09-21-15-12-11-082.png
>
>
> When we try to transfrom datastream to table, *TypeInfoDataTypeConverter*
> will try to convert *TypeInformation* to {*}DataType{*}, but if datastream's
> produce types contains {*}BigDecimalTypeInfo{*}, *TypeInfoDataTypeConverter*
> will final convert it to {*}RawDataType{*},then when we want tranform table
> to datastream again, exception will hapend, and show the data type not match.
> Blink planner also will has this exception.
> !image-2022-09-21-15-12-11-082.png!
>
> {code:java}
> Caused by: org.apache.flink.table.codegen.CodeGenException: Incompatible
> types of expression and result type.
> Expression[GeneratedExpression(result$10,isNull$11,,Decimal(9,2),false)] type
> is [Decimal(9,2)], result type is [GenericType<java.math.BigDecimal>] {code}
> how to recurrent
> {code:java}
> // code placeholder
> StreamExecutionEnvironment executionEnvironment =
> StreamExecutionEnvironment.getExecutionEnvironment();
> EnvironmentSettings.Builder envBuilder = EnvironmentSettings.newInstance()
> .useOldPlanner()
> .inStreamingMode();
> StreamTableEnvironment streamTableEnvironment =
> StreamTableEnvironment.create(executionEnvironment, envBuilder.build());
> FromElementsFunction fromElementsFunction = new FromElementsFunction(new
> BigDecimal(1.11D));
> DataStreamSource dataStreamSource =
> executionEnvironment.addSource(fromElementsFunction, new
> BigDecimalTypeInfo(10, 8));
> streamTableEnvironment.createTemporaryView("tmp", dataStreamSource);
> Table table = streamTableEnvironment.sqlQuery("select * from tmp");
> streamTableEnvironment.toRetractStream(table, table.getSchema().toRowType());
> {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)