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 4b7210d2342 [FLINK-38768][table] Materialized table fails with
validation exception in case of `LATERAL TABLE`
4b7210d2342 is described below
commit 4b7210d23425c3bbbb0702f03276dee6983456e3
Author: Sergey Nuyanzin <[email protected]>
AuthorDate: Wed Dec 3 10:47:21 2025 +0100
[FLINK-38768][table] Materialized table fails with validation exception in
case of `LATERAL TABLE`
---
.../calcite/sql/validate/SqlValidatorImpl.java | 37 ++++++++++++----------
...erializedTableNodeToOperationConverterTest.java | 25 +++++++++++++++
2 files changed, 45 insertions(+), 17 deletions(-)
diff --git
a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index 89317e352f6..57dc5e740f1 100644
---
a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++
b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -169,22 +169,23 @@ import static org.apache.calcite.util.Util.first;
* Default implementation of {@link SqlValidator}, the class was copied over
because of
* CALCITE-4554.
*
- * <p>Lines 202 ~ 205, Flink improves error message for functions without
appropriate arguments in
+ * <p>Lines 207 ~ 210, Flink improves error message for functions without
appropriate arguments in
* handleUnresolvedFunction.
*
- * <p>Lines 1270 ~ 1272, CALCITE-7217, should be removed after upgrading
Calcite to 1.41.0.
+ * <p>Lines 1275 ~ 1277, CALCITE-7217, should be removed after upgrading
Calcite to 1.41.0.
*
- * <p>Lines 2031 ~ 2045, Flink improves error message for functions without
appropriate arguments in
+ * <p>Lines 2036 ~ 2050, Flink improves error message for functions without
appropriate arguments in
* handleUnresolvedFunction at {@link
SqlValidatorImpl#handleUnresolvedFunction}.
*
- * <p>Lines 2571 ~ 2588, CALCITE-7217, should be removed after upgrading
Calcite to 1.41.0.
+ * <p>Lines 2576 ~ 2595, CALCITE-7217, CALCITE-7312 should be removed after
upgrading Calcite to
+ * 1.42.0.
*
- * <p>Line 2618 ~2631, set the correct scope for VECTOR_SEARCH.
+ * <p>Line 2626 ~2644, set the correct scope for VECTOR_SEARCH.
*
- * <p>Lines 3920 ~ 3925, 6599 ~ 6606 Flink improves Optimize the retrieval of
sub-operands in
+ * <p>Lines 3923 ~ 3927, 6602 ~ 6608 Flink improves Optimize the retrieval of
sub-operands in
* SqlCall when using NamedParameters at {@link SqlValidatorImpl#checkRollUp}.
*
- * <p>Lines 5340 ~ 5347, FLINK-24352 Add null check for temporal table check
on SqlSnapshot.
+ * <p>Lines 5343 ~ 5349, FLINK-24352 Add null check for temporal table check
on SqlSnapshot.
*/
public class SqlValidatorImpl implements SqlValidatorWithHints {
// ~ Static fields/initializers
---------------------------------------------
@@ -2578,16 +2579,18 @@ public class SqlValidatorImpl implements
SqlValidatorWithHints {
// in order to make visible the left items
// of the JOIN tree.
scopes.put(node, usingScope);
- registerFrom(
- parentScope,
- usingScope,
- register,
- ((SqlCall) node).operand(0),
- enclosingNode,
- alias,
- extendList,
- forceNullable,
- true);
+ newOperand =
+ registerFrom(
+ parentScope,
+ usingScope,
+ register,
+ ((SqlCall) node).operand(0),
+ enclosingNode,
+ alias,
+ extendList,
+ forceNullable,
+ true);
+ sbc.setOperand(0, newOperand);
return sbc;
// ----- FLINK MODIFICATION END -----
diff --git
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java
index 63d1d759bfd..fdfbb8ab190 100644
---
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java
+++
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/operations/SqlMaterializedTableNodeToOperationConverterTest.java
@@ -311,6 +311,31 @@ class SqlMaterializedTableNodeToOperationConverterTest
+ "LATERAL
TABLE(`builtin`.`default`.`myFunc`(`b`)) AS `T` (`f1`, `f2`)");
}
+ @Test
+ void testCreateMaterializedTableWithUDTFQueryWithoutAlias() {
+ functionCatalog.registerCatalogFunction(
+ UnresolvedIdentifier.of(
+ ObjectIdentifier.of(
+ catalogManager.getCurrentCatalog(), "default",
"myFunc")),
+ FunctionDescriptor.forFunctionClass(TableFunc0.class).build(),
+ true);
+
+ final String sql =
+ "CREATE MATERIALIZED TABLE mtbl1 \n"
+ + "AS SELECT * FROM t1, LATERAL TABLE(myFunc(b))";
+ Operation operation = parse(sql);
+
assertThat(operation).isInstanceOf(CreateMaterializedTableOperation.class);
+
+ CreateMaterializedTableOperation createOperation =
+ (CreateMaterializedTableOperation) operation;
+
+
assertThat(createOperation.getCatalogMaterializedTable().getExpandedQuery())
+ .isEqualTo(
+ "SELECT `t1`.`a`, `t1`.`b`, `t1`.`c`, `t1`.`d`,
`EXPR$0`.`name`, `EXPR$0`.`age`\n"
+ + "FROM `builtin`.`default`.`t1` AS `t1`,\n"
+ + "LATERAL
TABLE(`builtin`.`default`.`myFunc`(`b`)) AS `EXPR$0`");
+ }
+
@Test
void testContinuousRefreshMode() {
// test continuous mode derived by specify freshness automatically