david radley created FLINK-38913:
------------------------------------

             Summary: ArrayIndexOutOfBoundsException when creating a table with 
nested objects 
                 Key: FLINK-38913
                 URL: https://issues.apache.org/jira/browse/FLINK-38913
             Project: Flink
          Issue Type: Technical Debt
          Components: Table SQL / Planner
    Affects Versions: 2.2.0
            Reporter: david radley


If you try to create a table in the SQL client with SQL like this :

_CREATE TABLE orders (_

         _`const_requestBody_arrayOfObjectsOfObjects` AS ARRAY[ROW(ROW('I''m 
happy', CAST(NULL AS BOOLEAN))), ROW(CAST(NULL AS ROW<`string` STRING, 
`boolean` BOOLEAN>))]_

  _) WITH (_

      _'connector' = 'filesystem',_

      _'path' = 
'file:///Users/davidradley/bashscripts/sqlscripts/json/test_avro.json',_

      _'format' = 'avro'_

  _);_

 

_We get an ArrayIndexOutOfBoundsException_ exception thrown. The problem is 
[here]([https://github.com/apache/flink/blob/0261d498cb4538aefc4bb2f14ef08d0dec6db812/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java#L124)]
 where we address an element in the comments array, but the element does not 
exist. The proposed fix is to change this code to:
_if (i < comments.size() && comments.get(i) != null) {_
 
There is one other place in the code base that has this if that I will also 
change. 
 
This does not occur at Flink 1.20. 
 
 

 

 

 
#18 1229.3 java.lang.ArrayIndexOutOfBoundsException
#18 1229.3  at 
java.base/java.util.ImmutableCollections$ListN.get(ImmutableCollections.java:449)
#18 1229.3  at 
org.apache.flink.sql.parser.type.ExtendedSqlRowTypeNameSpec.unparse(ExtendedSqlRowTypeNameSpec.java:124)
#18 1229.3  at 
org.apache.calcite.sql.SqlDataTypeSpec.unparse(SqlDataTypeSpec.java:190)
#18 1229.3  at 
org.apache.flink.sql.parser.type.ExtendedSqlRowTypeNameSpec.unparse(ExtendedSqlRowTypeNameSpec.java:120)
#18 1229.3  at 
org.apache.calcite.sql.SqlDataTypeSpec.unparse(SqlDataTypeSpec.java:190)
#18 1229.3  at 
org.apache.calcite.sql.fun.SqlCastFunction.unparse(SqlCastFunction.java:296)
#18 1229.3  at 
org.apache.calcite.sql.SqlDialect.unparseCall(SqlDialect.java:466)
#18 1229.3  at org.apache.calcite.sql.SqlCall.unparse(SqlCall.java:141)
#18 1229.3  at 
org.apache.calcite.sql.fun.SqlMultisetValueConstructor.unparse(SqlMultisetValueConstructor.java:124)
#18 1229.3  at 
org.apache.calcite.sql.SqlDialect.unparseCall(SqlDialect.java:466)
#18 1229.3  at org.apache.calcite.sql.SqlCall.unparse(SqlCall.java:141)
#18 1229.3  at org.apache.calcite.sql.SqlNode.toSqlString(SqlNode.java:156)
#18 1229.3  at org.apache.calcite.sql.SqlNode.toSqlString(SqlNode.java:178)
#18 1229.3  at org.apache.calcite.sql.SqlNode.toSqlString(SqlNode.java:187)
#18 1229.3  at 
org.apache.flink.table.planner.operations.SqlNodeConvertContext.toQuotedSqlString(SqlNodeConvertContext.java:110)
#18 1229.3  at 
org.apache.flink.table.planner.operations.converters.table.SchemaBuilderUtil.toUnresolvedComputedColumn(SchemaBuilderUtil.java:213)
#18 1229.3  at 
org.apache.flink.table.planner.operations.converters.table.MergeTableLikeUtil$SchemaBuilder.appendDerivedColumns(MergeTableLikeUtil.java:394)
#18 1229.3  at 
org.apache.flink.table.planner.operations.converters.table.MergeTableLikeUtil.mergeTables(MergeTableLikeUtil.java:157)
#18 1229.3  at 
org.apache.flink.table.planner.operations.converters.table.SqlCreateTableConverter$1.getMergedSchema(SqlCreateTableConverter.java:59)
#18 1229.3  at 
org.apache.flink.table.planner.operations.converters.table.AbstractCreateTableConverter.getResolvedCatalogTable(AbstractCreateTableConverter.java:82)
#18 1229.3  at 
org.apache.flink.table.planner.operations.converters.table.SqlCreateTableConverter.convertSqlNode(SqlCreateTableConverter.java:42)
#18 1229.3  at 
org.apache.flink.table.planner.operations.converters.table.SqlCreateTableConverter.convertSqlNode(SqlCreateTableConverter.java:37)
#18 1229.3  at 
org.apache.flink.table.planner.operations.converters.SqlNodeConverters.convertSqlNode(SqlNodeConverters.java:127)
#18 1229.3  at 
org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convertValidatedSqlNode(SqlNodeToOperationConversion.java:231)
#18 1229.3  at 
org.apache.flink.table.planner.operations.SqlNodeToOperationConversion.convert(SqlNodeToOperationConversion.java:221)
#18 1229.3  at 
org.apache.flink.table.planner.delegation.ParserImpl.parse(ParserImpl.java:106)
#18 1229.3  at 
org.apache.flink.table.api.internal.TableEnvironmentImpl.executeSql(TableEnvironmentImpl.java:937)
#18 1229.3  at 
com.ibm.ei.streamproc.model.jobgraph.TemporaryTableCreator.createAndValidateTemporaryTable(TemporaryTableCreator.java:62)



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

Reply via email to