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

dalongliu edited comment on FLINK-29090 at 9/8/22 1:48 PM:
-----------------------------------------------------------

I didn't reproduce the exception through the following sql case in master 
branch:

```java
@Test
publicvoidtestMap() throws Exception

{ super.tableEnv() .executeSql( String.format( "create table parquet_source 
(\n" + "f1 int,\n" + " f_map map<varchar(20), int>\n" + ") with (\n" + " 
'connector' = 'filesystem',\n" + " 'path' = '%s',\n" + " 'format' = 
'parquet'\n" + ");", super.resultPath())); super.tableEnv().executeSql("insert 
into parquet_source select 1, map['k1', 1]").await(); List<Row> results = 
CollectionUtil.iteratorToList( super.tableEnv() .executeSql("select f1, 
f_map['k1'] from parquet_source") .collect()); 
assertThat(results.size()).isEqualTo(1); }

```

In the codegen code, the ColumnarMapData will be convert to BinaryMapData 
before get it, the codegen related code as following:

!image-2022-09-08-21-47-47-325.png!

 

Can you give me more context about reproducing the bug?


was (Author: lsy):
I didn't reproduce the exception through the following sql case:

```java
@Test
publicvoidtestMap() throws Exception {
super.tableEnv()
.executeSql(
String.format(
"create table parquet_source (\n"
+ "f1 int,\n"
+ " f_map map<varchar(20), int>\n"
+ ") with (\n"
+ " 'connector' = 'filesystem',\n"
+ " 'path' = '%s',\n"
+ " 'format' = 'parquet'\n"
+ ");",
super.resultPath()));

super.tableEnv().executeSql("insert into parquet_source select 1, map['k1', 
1]").await();

List<Row> results =
CollectionUtil.iteratorToList(
super.tableEnv()
.executeSql("select f1, f_map['k1'] from parquet_source")
.collect());
assertThat(results.size()).isEqualTo(1);
}
```

> Fix the code gen for ColumnarMapData and ColumnarArrayData
> ----------------------------------------------------------
>
>                 Key: FLINK-29090
>                 URL: https://issues.apache.org/jira/browse/FLINK-29090
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Runtime
>    Affects Versions: 1.16.0
>            Reporter: Danny Chen
>            Priority: Major
>             Fix For: 1.16.0
>
>         Attachments: image-2022-08-24-10-15-11-824.png, 
> image-2022-09-08-21-47-47-325.png
>
>
> !image-2022-08-24-10-15-11-824.png|width=589,height=284!
> Currently, the code generation for {{MapData}} assumes that it is the 
> {{{}GenericMapData{}}}, but the new introduced {{ColumnarMapData}} and 
> {{ColumnarArrayData}} can not be casted to {{{}GenericMapData{}}}.
> {{ColumnarMapData}} and {{ColumnarArrayData}} are introduced in
> FLINK-24614 
> [https://github.com/apache/flink/commit/5c731a37e1a8f71f9c9e813f6c741a1e203fa1a3]
> How to reproduce:
> {code:sql}
> create table parquet_source (
>   f_map map<varchar(20), int>
> ) with (
>   'connector' = 'filesystem',
>   'format' = 'parquet'
> );
> select f_map['k1'] from table parquet_source;
> {code}



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

Reply via email to