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

Jane Chan edited comment on FLINK-30282 at 6/9/23 8:04 AM:
-----------------------------------------------------------

I rechecked the discussion CALCITE-2464 and assumed the current behavior is by 
design, and I'd like to close this issue.

 
{code:scala}
  @Test
  def testNullability(): Unit = {
    // create temporary table t1 with r nullable
    tableEnv.executeSql(s"""
                           |create temporary table t1 (
                           |  r row<f0 int not null, f1 double, f2 string not 
null>
                           |) with (
                           | 'connector' = 'datagen'
                           |)
                           |""".stripMargin)

    // create temporary table t2 with r not null
    tableEnv.executeSql(s"""
                           |create temporary table t2 (
                           |  r row<f0 int not null, f1 double, f2 string not 
null> not null
                           |) with (
                           | 'connector' = 'datagen'
                           |)
                           |""".stripMargin)

    val catalogManager = 
tableEnv.asInstanceOf[StreamTableEnvironmentImpl].getCatalogManager
    val result1 =
      catalogManager.getTable(ObjectIdentifier.of("default_catalog", 
"default_database", "t1"))
    val result2 =
      catalogManager.getTable(ObjectIdentifier.of("default_catalog", 
"default_database", "t2"))
    println(result1.get().getResolvedSchema)
    println(result2.get().getResolvedSchema)
  }
{code}
{code:sql}
-- t1
(
  `r` ROW<`f0` INT, `f1` DOUBLE, `f2` STRING>
)

-- t2
(
  `r` ROW<`f0` INT NOT NULL, `f1` DOUBLE, `f2` STRING NOT NULL> NOT NULL
)
{code}


was (Author: qingyue):
I rechecked the discussion 
[CALCITE-2464|https://issues.apache.org/jira/browse/CALCITE-2464] and assume 
the current behavior is by design, and I'd like to close this issue.

> Logical type ROW lost inner field's nullability after convert to RelDataType
> ----------------------------------------------------------------------------
>
>                 Key: FLINK-30282
>                 URL: https://issues.apache.org/jira/browse/FLINK-30282
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table SQL / API
>    Affects Versions: 1.16.0, 1.16.1
>            Reporter: Jane Chan
>            Assignee: Jane Chan
>            Priority: Major
>              Labels: pull-request-available
>
> h3. Issue History
> This is not a new issue, FLINK-13604 has tracked it before, and FLINK-16344 
> spared efforts to fix it (but did not tweak the ut case mentioned in 
> FLINK-13604, i.e. 
> SqlToOperationConverterTest#testCreateTableWithFullDataTypes). Nevertheless, 
> the FunctionITCase added by FLINK-16344, which validates the fix, has been 
> removed in FLINK-16377. 
> h3. How to Reproduce
>  c.c2 lost nullability
> {code:java}
> Flink SQL> create table dummy (a array<int not null> not null, b array<string 
> not null>, c row<c1 int, c2 double not null>) with ('connector' = 'datagen');
> [INFO] Execute statement succeed.
> Flink SQL> desc dummy;
> +------+----------------------------+-------+-----+--------+-----------+
> | name |                       type |  null | key | extras | watermark |
> +------+----------------------------+-------+-----+--------+-----------+
> |    a |        ARRAY<INT NOT NULL> | FALSE |     |        |           |
> |    b |     ARRAY<STRING NOT NULL> |  TRUE |     |        |           |
> |    c | ROW<`c1` INT, `c2` DOUBLE> |  TRUE |     |        |           |
> +------+----------------------------+-------+-----+--------+-----------+
> 3 rows in set
> {code}
> h3. Root Cause
> Two places are causing this problem in ExtendedSqlRowTypeNameSpec.
> 1. dt.deriveType should also pass dt's nullability as well. See 
> [https://github.com/apache/flink/blob/fb27e6893506006b9a3b1ac3e9b878fb6cad061a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java#L159]
>  
> 2. StructKind should be PEEK_FIELDS_NO_EXPAND instead of FULLY_QUALIFIED(see 
> [https://github.com/apache/calcite/blob/main/core/src/main/java/org/apache/calcite/rel/type/StructKind.java]),
>  so that FlinkTypeFactory#createTypeWithNullability will not fall back to 
> super implement. See 
> [https://github.com/apache/flink/blob/fb27e6893506006b9a3b1ac3e9b878fb6cad061a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkTypeFactory.scala#L417]



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

Reply via email to