You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Caizhi Weng (Jira)" <ji...@apache.org> on 2021/10/14 03:45:00 UTC

[jira] [Updated] (FLINK-24537) Array of rows containing constant strings of different length and an integer throws AssertionError

     [ https://issues.apache.org/jira/browse/FLINK-24537?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Caizhi Weng updated FLINK-24537:
--------------------------------
    Description: 
This issue is from [the mailing list|https://lists.apache.org/thread.html/r112786ef139a1029f13565852f66bf542ff9bd62394cedf5b471cac2%40%3Cuser-zh.flink.apache.org%3E].

Add the following test case to {{TableEnvironmentITCase}} to reproduce this issue.
{code:scala}
@Test
def myTest(): Unit = {
  tEnv.executeSql(
    """
      |CREATE TABLE T (
      |  aa INT,
      |  b INT
      |) WITH (
      |  'connector' = 'values'
      |)
      |""".stripMargin)
  tEnv.executeSql(
    """
      |SELECT array[row('aa', aa), row('b', b)] FROM T
      |""".stripMargin).print()
}
{code}
The exception stack is
{code:java}
java.lang.AssertionError: Conversion to relational algebra failed to preserve datatypes:
validated type:
RecordType(RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER EXPR$1) NOT NULL ARRAY NOT NULL EXPR$0) NOT NULL
converted type:
RecordType(RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER NOT NULL EXPR$1) NOT NULL ARRAY NOT NULL EXPR$0) NOT NULL
rel:
LogicalProject(EXPR$0=[ARRAY(CAST(ROW(_UTF-16LE'aa', $0)):RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER NOT NULL EXPR$1) NOT NULL, CAST(ROW(_UTF-16LE'b', $1)):RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER NOT NULL EXPR$1) NOT NULL)])
  LogicalTableScan(table=[[default_catalog, default_database, T]])


	at org.apache.calcite.sql2rel.SqlToRelConverter.checkConvertedType(SqlToRelConverter.java:467)
	at org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:582)
	at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:181)
	at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:173)
	at org.apache.flink.table.planner.operations.SqlToOperationConverter.toQueryOperation(SqlToOperationConverter.java:1074)
	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convertSqlQuery(SqlToOperationConverter.java:1043)
	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:305)
	at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:101)
	at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:739)
{code}
One interesting aspect is that the following two SQLs are OK.
{code:sql}
-- constant strings are of same length
SELECT array[row('a', aa), row('b', b)] FROM T

-- constant strings are casted to varchar
SELECT array[row(CAST('aa' AS VARCHAR), aa), row(CAST('b' AS VARCHAR), b)] FROM T
{code}

  was:
Add the following test case to {{TableEnvironmentITCase}} to reproduce this issue.

{code:scala}
@Test
def myTest(): Unit = {
  tEnv.executeSql(
    """
      |CREATE TABLE T (
      |  aa INT,
      |  b INT
      |) WITH (
      |  'connector' = 'values'
      |)
      |""".stripMargin)
  tEnv.executeSql(
    """
      |SELECT array[row('aa', aa), row('b', b)] FROM T
      |""".stripMargin).print()
}
{code}

The exception stack is

{code}
java.lang.AssertionError: Conversion to relational algebra failed to preserve datatypes:
validated type:
RecordType(RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER EXPR$1) NOT NULL ARRAY NOT NULL EXPR$0) NOT NULL
converted type:
RecordType(RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER NOT NULL EXPR$1) NOT NULL ARRAY NOT NULL EXPR$0) NOT NULL
rel:
LogicalProject(EXPR$0=[ARRAY(CAST(ROW(_UTF-16LE'aa', $0)):RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER NOT NULL EXPR$1) NOT NULL, CAST(ROW(_UTF-16LE'b', $1)):RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER NOT NULL EXPR$1) NOT NULL)])
  LogicalTableScan(table=[[default_catalog, default_database, T]])


	at org.apache.calcite.sql2rel.SqlToRelConverter.checkConvertedType(SqlToRelConverter.java:467)
	at org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:582)
	at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:181)
	at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:173)
	at org.apache.flink.table.planner.operations.SqlToOperationConverter.toQueryOperation(SqlToOperationConverter.java:1074)
	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convertSqlQuery(SqlToOperationConverter.java:1043)
	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:305)
	at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:101)
	at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:739)
{code}

One interesting aspect is that the following SQL is OK.

{code:sql}
-- constant strings are of same length
SELECT array[row('a', aa), row('b', b)] FROM T
{code}


> Array of rows containing constant strings of different length and an integer throws AssertionError
> --------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-24537
>                 URL: https://issues.apache.org/jira/browse/FLINK-24537
>             Project: Flink
>          Issue Type: Bug
>          Components: Table SQL / Planner
>    Affects Versions: 1.15.0
>            Reporter: Caizhi Weng
>            Priority: Major
>
> This issue is from [the mailing list|https://lists.apache.org/thread.html/r112786ef139a1029f13565852f66bf542ff9bd62394cedf5b471cac2%40%3Cuser-zh.flink.apache.org%3E].
> Add the following test case to {{TableEnvironmentITCase}} to reproduce this issue.
> {code:scala}
> @Test
> def myTest(): Unit = {
>   tEnv.executeSql(
>     """
>       |CREATE TABLE T (
>       |  aa INT,
>       |  b INT
>       |) WITH (
>       |  'connector' = 'values'
>       |)
>       |""".stripMargin)
>   tEnv.executeSql(
>     """
>       |SELECT array[row('aa', aa), row('b', b)] FROM T
>       |""".stripMargin).print()
> }
> {code}
> The exception stack is
> {code:java}
> java.lang.AssertionError: Conversion to relational algebra failed to preserve datatypes:
> validated type:
> RecordType(RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER EXPR$1) NOT NULL ARRAY NOT NULL EXPR$0) NOT NULL
> converted type:
> RecordType(RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER NOT NULL EXPR$1) NOT NULL ARRAY NOT NULL EXPR$0) NOT NULL
> rel:
> LogicalProject(EXPR$0=[ARRAY(CAST(ROW(_UTF-16LE'aa', $0)):RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER NOT NULL EXPR$1) NOT NULL, CAST(ROW(_UTF-16LE'b', $1)):RecordType(VARCHAR(2) CHARACTER SET "UTF-16LE" NOT NULL EXPR$0, INTEGER NOT NULL EXPR$1) NOT NULL)])
>   LogicalTableScan(table=[[default_catalog, default_database, T]])
> 	at org.apache.calcite.sql2rel.SqlToRelConverter.checkConvertedType(SqlToRelConverter.java:467)
> 	at org.apache.calcite.sql2rel.SqlToRelConverter.convertQuery(SqlToRelConverter.java:582)
> 	at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.org$apache$flink$table$planner$calcite$FlinkPlannerImpl$$rel(FlinkPlannerImpl.scala:181)
> 	at org.apache.flink.table.planner.calcite.FlinkPlannerImpl.rel(FlinkPlannerImpl.scala:173)
> 	at org.apache.flink.table.planner.operations.SqlToOperationConverter.toQueryOperation(SqlToOperationConverter.java:1074)
> 	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convertSqlQuery(SqlToOperationConverter.java:1043)
> 	at org.apache.flink.table.planner.operations.SqlToOperationConverter.convert(SqlToOperationConverter.java:305)
> 	at org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:101)
> 	at org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:739)
> {code}
> One interesting aspect is that the following two SQLs are OK.
> {code:sql}
> -- constant strings are of same length
> SELECT array[row('a', aa), row('b', b)] FROM T
> -- constant strings are casted to varchar
> SELECT array[row(CAST('aa' AS VARCHAR), aa), row(CAST('b' AS VARCHAR), b)] FROM T
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)