This is an automated email from the ASF dual-hosted git repository.

snuyanzin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 2426faf2cbf [FLINK-38913][table] `ArrayIndexOutOfBoundsException` 
while unparsing `ExtendedSqlRowTypeNameSpec`
2426faf2cbf is described below

commit 2426faf2cbf282dc945043e6119ac5f06a0e0cf1
Author: David Radley <[email protected]>
AuthorDate: Wed Jan 21 23:48:03 2026 +0000

    [FLINK-38913][table] `ArrayIndexOutOfBoundsException` while unparsing 
`ExtendedSqlRowTypeNameSpec`
    
    
    ---------
    
    Signed-off-by: davidradl <[email protected]>
    Co-authored-by: Sergey Nuyanzin <[email protected]>
---
 .../parser/type/ExtendedSqlRowTypeNameSpec.java    |   3 +-
 .../sql/parser/ExtendedSqlRowTypeNameSpecTest.java | 103 +++++++++++++++++++++
 .../table/planner/calcite/SqlTypeUtilTest.java     |  66 +++++++++++++
 3 files changed, 171 insertions(+), 1 deletion(-)

diff --git 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java
 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java
index 9c6ac8868a3..1ba03412c90 100644
--- 
a/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java
+++ 
b/flink-table/flink-sql-parser/src/main/java/org/apache/flink/sql/parser/type/ExtendedSqlRowTypeNameSpec.java
@@ -121,7 +121,8 @@ public class ExtendedSqlRowTypeNameSpec extends 
SqlTypeNameSpec {
                 if (p.right.getNullable() != null && !p.right.getNullable()) {
                     writer.keyword("NOT NULL");
                 }
-                if (comments.get(i) != null) {
+                // With bounds check - prevents IndexOutOfBoundsException
+                if (i < comments.size() && comments.get(i) != null) {
                     comments.get(i).unparse(writer, leftPrec, rightPrec);
                 }
                 i += 1;
diff --git 
a/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/ExtendedSqlRowTypeNameSpecTest.java
 
b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/ExtendedSqlRowTypeNameSpecTest.java
new file mode 100644
index 00000000000..d67aca371a8
--- /dev/null
+++ 
b/flink-table/flink-sql-parser/src/test/java/org/apache/flink/sql/parser/ExtendedSqlRowTypeNameSpecTest.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.sql.parser;
+
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl;
+import org.apache.flink.sql.parser.type.ExtendedSqlRowTypeNameSpec;
+
+import org.apache.calcite.avatica.util.Casing;
+import org.apache.calcite.avatica.util.Quoting;
+import org.apache.calcite.sql.SqlBasicTypeNameSpec;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.dialect.CalciteSqlDialect;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Map;
+
+/** Tests for {@link ExtendedSqlRowTypeNameSpec}. */
+class ExtendedSqlRowTypeNameSpecTest {
+    @Test
+    void testExtendedRowWithNoComments() {
+        final ExtendedSqlRowTypeNameSpec spec =
+                new ExtendedSqlRowTypeNameSpec(
+                        SqlParserPos.ZERO,
+                        List.of(
+                                new SqlIdentifier("t1", SqlParserPos.ZERO),
+                                new SqlIdentifier("t2", SqlParserPos.ZERO),
+                                new SqlIdentifier("t3", SqlParserPos.ZERO)),
+                        List.of(
+                                new SqlDataTypeSpec(
+                                        new SqlBasicTypeNameSpec(
+                                                SqlTypeName.INTEGER, 
SqlParserPos.ZERO),
+                                        SqlParserPos.ZERO),
+                                new SqlDataTypeSpec(
+                                        new SqlBasicTypeNameSpec(
+                                                SqlTypeName.DATE, 
SqlParserPos.ZERO),
+                                        SqlParserPos.ZERO),
+                                new SqlDataTypeSpec(
+                                        new SqlBasicTypeNameSpec(
+                                                SqlTypeName.TIME, 
SqlParserPos.ZERO),
+                                        SqlParserPos.ZERO)),
+                        List.of(),
+                        false);
+        SqlWriter writer = getSqlWriter();
+        spec.unparse(writer, 0, 0);
+    }
+
+    private SqlWriter getSqlWriter() {
+        final Map<String, ?> options =
+                Map.ofEntries(
+                        Map.entry("quoting", Quoting.BACK_TICK),
+                        Map.entry("quotedCasing", Casing.UNCHANGED),
+                        Map.entry("unquotedCasing", Casing.UNCHANGED),
+                        Map.entry("caseSensitive", true),
+                        Map.entry("enableTypeCoercion", false),
+                        Map.entry("conformance", SqlConformanceEnum.DEFAULT),
+                        Map.entry("operatorTable", 
SqlStdOperatorTable.instance()),
+                        Map.entry("parserFactory", 
FlinkSqlParserImpl.FACTORY));
+        final SqlParser.Config parserConfig =
+                SqlParser.config()
+                        .withQuoting((Quoting) options.get("quoting"))
+                        .withUnquotedCasing((Casing) 
options.get("unquotedCasing"))
+                        .withQuotedCasing((Casing) options.get("quotedCasing"))
+                        .withConformance((SqlConformance) 
options.get("conformance"))
+                        .withCaseSensitive((boolean) 
options.get("caseSensitive"))
+                        .withParserFactory((SqlParserImplFactory) 
options.get("parserFactory"));
+
+        return new SqlPrettyWriter(
+                new CalciteSqlDialect(
+                        SqlDialect.EMPTY_CONTEXT
+                                
.withQuotedCasing(parserConfig.unquotedCasing())
+                                .withConformance(parserConfig.conformance())
+                                
.withUnquotedCasing(parserConfig.unquotedCasing())
+                                
.withIdentifierQuoteString(parserConfig.quoting().string)),
+                false);
+    }
+}
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/calcite/SqlTypeUtilTest.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/calcite/SqlTypeUtilTest.java
new file mode 100644
index 00000000000..58560bdee9a
--- /dev/null
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/calcite/SqlTypeUtilTest.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.calcite;
+
+import org.apache.flink.table.planner.typeutils.LogicalRelDataTypeConverter;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link SqlTypeUtil}. */
+class SqlTypeUtilTest {
+    /**
+     * Test case for <a 
href="https://issues.apache.org/jira/browse/FLINK-38913";>[FLINK-38913]
+     * ArrayIndexOutOfBoundsException when creating a table with computed rows 
including casts to
+     * null</a>.
+     */
+    @Test
+    void testConvertRowTypeToSpecAndUnparse() {
+        FlinkTypeFactory typeFactory =
+                new FlinkTypeFactory(
+                        Thread.currentThread().getContextClassLoader(), 
FlinkTypeSystem.INSTANCE);
+        RowType rowType =
+                RowType.of(
+                        new LogicalType[] {new IntType(), new VarCharType(1)},
+                        new String[] {"a", "b"});
+        RelDataType relDataType = 
LogicalRelDataTypeConverter.toRelDataType(rowType, typeFactory);
+        SqlDataTypeSpec typeSpec = SqlTypeUtil.convertTypeToSpec(relDataType);
+        SqlWriter writer =
+                new SqlPrettyWriter(
+                        SqlPrettyWriter.config()
+                                .withAlwaysUseParentheses(false)
+                                .withSelectListItemsOnSeparateLines(false)
+                                .withIndentation(0));
+        // unparse that will end up passing no comments through
+        typeSpec.unparse(writer, 0, 0);
+        String result = writer.toSqlString().getSql();
+        assertThat(result)
+                .hasToString("ROW(\"a\" INTEGER, \"b\" VARCHAR(1) CHARACTER 
SET \"UTF-16LE\")");
+    }
+}

Reply via email to