Skip to content

Commit

Permalink
Add explicit table
Browse files Browse the repository at this point in the history
  • Loading branch information
snuyanzin committed Oct 6, 2024
1 parent 7d091b1 commit ae980a7
Show file tree
Hide file tree
Showing 3 changed files with 13 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -419,7 +419,7 @@ protected void addToSelectList(
* A special {@link SqlSelect} to capture the origin of a {@link SqlKind#EXPLICIT_TABLE} within
* TVF operands.
*/
private static class ExplicitTableSqlSelect extends SqlSelect {
static class ExplicitTableSqlSelect extends SqlSelect {

private final List<SqlIdentifier> descriptors;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.calcite

import org.apache.flink.sql.parser.`type`.SqlMapTypeNameSpec
import org.apache.flink.table.api.ValidationException
import org.apache.flink.table.planner.calcite.FlinkCalciteSqlValidator.ExplicitTableSqlSelect
import org.apache.flink.table.planner.calcite.PreValidateReWriter.{newValidationError, notSupported}
import org.apache.flink.table.planner.calcite.SqlRewriterUtils.{rewriteSqlCall, rewriteSqlSelect, rewriteSqlValues}
import org.apache.flink.util.Preconditions.checkArgument
Expand Down Expand Up @@ -160,9 +161,14 @@ object SqlRewriterUtils {
operands.get(1).asInstanceOf[SqlNodeList],
operands.get(2),
operands.get(3))
case SqlKind.EXPLICIT_TABLE =>
val operands = call.getOperandList
val expTable = new ExplicitTableSqlSelect(
operands.get(0).asInstanceOf[SqlIdentifier],
Collections.emptyList())
rewriterUtils.rewriteSelect(expTable, targetRowType, assignedFields, targetPosition)
// Not support:
// case SqlKind.WITH =>
// case SqlKind.EXPLICIT_TABLE =>
case _ => throw new ValidationException(unsupportedErrorMessage())
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,9 @@ void testUpsertInto() {
"INSERT INTO t2 (a, b) SELECT * FROM t1",
"INSERT INTO t2 (a, b) SELECT *, 42 FROM t2_copy",
"INSERT INTO t2 (a, b) SELECT 42, * FROM t2_copy",
"INSERT INTO t2 (a, b) SELECT * FROM t_nested"
"INSERT INTO t2 (a, b) SELECT * FROM t_nested",
"INSERT INTO t2 (a, b) TABLE t_nested",
"INSERT INTO t2 (a, b) SELECT * FROM (TABLE t_nested)"
})
void testInvalidNumberOfColumnsWhileInsertInto(String sql) {
assertThatThrownBy(() -> plannerMocks.getParser().parse(sql))
Expand All @@ -95,7 +97,8 @@ void testInvalidNumberOfColumnsWhileInsertInto(String sql) {
"INSERT INTO t2 (a, b) SELECT * FROM t2_copy",
"INSERT INTO t2 (a, b) SELECT *, 42 FROM t1",
"INSERT INTO t2 (a, b) SELECT 42, * FROM t1",
"INSERT INTO t2 (a, b) SELECT f.* FROM t_nested"
"INSERT INTO t2 (a, b) SELECT f.* FROM t_nested",
"INSERT INTO t2 (a, b) TABLE t2_copy"
})
void validInsertIntoTest(final String sql) {
assertDoesNotThrow(
Expand Down

0 comments on commit ae980a7

Please sign in to comment.