Skip to content

Commit

Permalink
[FLINK-26495][table-planner] Prohibit hints(dynamic table options) on…
Browse files Browse the repository at this point in the history
… view

This closes apache#19007
  • Loading branch information
LadyForest committed Mar 22, 2022
1 parent ff33362 commit e32c04c
Show file tree
Hide file tree
Showing 5 changed files with 69 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -136,7 +136,7 @@ public void testSqlStatements() throws IOException {
testSqlStatements.stream().map(s -> s.sql).collect(Collectors.toList());
List<Result> actualResults = runSqlStatements(sqlStatements);
String out = transformOutput(testSqlStatements, actualResults);
assertThat(in).isEqualTo(out);
assertThat(out).isEqualTo(in);
}

/**
Expand Down
6 changes: 6 additions & 0 deletions flink-table/flink-sql-client/src/test/resources/sql/view.q
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,12 @@ create temporary view if not exists v1 as select * from orders;
[INFO] Execute statement succeed.
!info

# test query a view with hint
select * from v1 /*+ OPTIONS('number-of-rows' = '1') */;
[ERROR] Could not execute SQL statement. Reason:
org.apache.flink.table.api.ValidationException: View '`default_catalog`.`default_database`.`v1`' cannot be enriched with new options. Hints can only be applied to tables.
!error

# test create a view reference another view
create temporary view if not exists v2 as select * from v1;
[INFO] Execute statement succeed.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@

package org.apache.flink.table.planner.plan.schema;

import org.apache.flink.table.api.ValidationException;
import org.apache.flink.table.catalog.ObjectIdentifier;
import org.apache.flink.table.planner.plan.stats.FlinkStatistic;

import org.apache.calcite.plan.RelOptSchema;
Expand Down Expand Up @@ -56,6 +58,15 @@ public final RelNode toRel(RelOptTable.ToRelContext context) {
}

private RelNode expand(RelOptTable.ToRelContext context) {
// view with hints is prohibited
if (context.getTableHints().size() > 0) {
throw new ValidationException(
String.format(
"View '%s' cannot be enriched with new options. "
+ "Hints can only be applied to tables.",
ObjectIdentifier.of(
super.names.get(0), super.names.get(1), super.names.get(2))));
}
final RelNode rel = convertToRel(context);
// Expand any views
return rel.accept(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -344,6 +344,46 @@ class TableEnvironmentTest {
tableEnv.executeSql("alter table MyTable compact")
}

@Test
def testQueryViewWithHints(): Unit = {
val statement =
"""
|CREATE TABLE MyTable (
| a bigint,
| b int,
| c varchar
|) WITH (
| 'connector' = 'COLLECTION'
|)
""".stripMargin
tableEnv.executeSql(statement)
tableEnv.executeSql("CREATE TEMPORARY VIEW my_view AS SELECT a, c FROM MyTable")

assertThatThrownBy(
() => tableEnv.executeSql("SELECT c FROM my_view /*+ OPTIONS('is-bounded' = 'true') */"))
.hasMessageContaining("View '`default_catalog`.`default_database`.`my_view`' " +
"cannot be enriched with new options. Hints can only be applied to tables.")
.isInstanceOf(classOf[ValidationException])

assertThatThrownBy(
() => tableEnv.executeSql(
"CREATE TEMPORARY VIEW your_view AS " +
"SELECT c FROM my_view /*+ OPTIONS('is-bounded' = 'true') */"))
.hasMessageContaining("View '`default_catalog`.`default_database`.`my_view`' " +
"cannot be enriched with new options. Hints can only be applied to tables.")
.isInstanceOf(classOf[ValidationException])

tableEnv.executeSql(
"CREATE TEMPORARY VIEW your_view AS SELECT c FROM my_view ")

assertThatThrownBy(
() => tableEnv.executeSql("SELECT * FROM your_view /*+ OPTIONS('is-bounded' = 'true') */"))
.hasMessageContaining("View '`default_catalog`.`default_database`.`your_view`' " +
"cannot be enriched with new options. Hints can only be applied to tables.")
.isInstanceOf(classOf[ValidationException])

}

@Test
def testAlterTableCompactOnManagedTableUnderStreamingMode(): Unit = {
val statement =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import org.apache.flink.table.planner.plan.hint.OptionsHintTest.{IS_BOUNDED, Par
import org.apache.flink.table.planner.plan.nodes.calcite.LogicalLegacySink
import org.apache.flink.table.planner.utils.{OptionsTableSink, TableTestBase, TableTestUtil}

import org.assertj.core.api.Assertions.assertThatThrownBy
import org.hamcrest.Matchers._
import org.junit.Assert.{assertEquals, assertThat}
import org.junit.runner.RunWith
Expand Down Expand Up @@ -142,9 +143,12 @@ class OptionsHintTest(param: Param)
def testOptionsHintOnTableApiView(): Unit = {
val view1 = util.tableEnv.sqlQuery("select * from t1 join t2 on t1.a = t2.d")
util.tableEnv.createTemporaryView("view1", view1)
// The table hints on view expect to be ignored.
// The table hints on view expect to be prohibited
val sql = "select * from view1/*+ OPTIONS(k1='#v1', k2='#v2', k3='#v3', k4='#v4') */"
util.verifyExecPlan(sql)
assertThatThrownBy(() => util.verifyExecPlan(sql))
.hasMessageContaining("View '`default_catalog`.`default_database`.`view1`' " +
"cannot be enriched with new options. Hints can only be applied to tables.")
.isInstanceOf[ValidationException]
}

@Test
Expand Down Expand Up @@ -175,9 +179,12 @@ class OptionsHintTest(param: Param)
new ObjectPath(util.tableEnv.getCurrentDatabase, "view1"),
view1,
false)
// The table hints on view expect to be ignored.
// The table hints on view expect to be prohibited
val sql = "select * from view1/*+ OPTIONS(k1='#v1', k2='#v2', k3='#v3', k4='#v4') */"
util.verifyExecPlan(sql)
assertThatThrownBy(() => util.verifyExecPlan(sql))
.hasMessageContaining("View '`default_catalog`.`default_database`.`view1`' " +
"cannot be enriched with new options. Hints can only be applied to tables.")
.isInstanceOf[ValidationException]
}
}

Expand Down

0 comments on commit e32c04c

Please sign in to comment.