You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by lz...@apache.org on 2020/06/18 03:35:30 UTC
[flink] branch release-1.11 updated: [FLINK-18300][sql-client] SQL
Client doesn't support ALTER VIEW
This is an automated email from the ASF dual-hosted git repository.
lzljs3620320 pushed a commit to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/release-1.11 by this push:
new 1830c1c [FLINK-18300][sql-client] SQL Client doesn't support ALTER VIEW
1830c1c is described below
commit 1830c1c47b8a985ec328a7332e92d21433c0a4df
Author: Rui Li <li...@apache.org>
AuthorDate: Thu Jun 18 11:34:21 2020 +0800
[FLINK-18300][sql-client] SQL Client doesn't support ALTER VIEW
This closes #12655
---
.../apache/flink/table/client/cli/CliClient.java | 3 +++
.../apache/flink/table/client/cli/CliStrings.java | 4 ++++
.../flink/table/client/cli/SqlCommandParser.java | 5 +++++
.../table/client/cli/SqlCommandParserTest.java | 21 +++++++++++++++++++++
.../table/client/cli/utils/SqlParserHelper.java | 5 +++++
5 files changed, 38 insertions(+)
diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java
index 5b0e197..8e9c2e8 100644
--- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java
+++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliClient.java
@@ -326,6 +326,9 @@ public class CliClient {
case DROP_VIEW:
callDdl(cmdCall.operands[0], CliStrings.MESSAGE_VIEW_REMOVED);
break;
+ case ALTER_VIEW:
+ callDdl(cmdCall.operands[0], CliStrings.MESSAGE_ALTER_VIEW_SUCCEEDED, CliStrings.MESSAGE_ALTER_VIEW_FAILED);
+ break;
case CREATE_FUNCTION:
callDdl(cmdCall.operands[0], CliStrings.MESSAGE_FUNCTION_CREATED);
break;
diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliStrings.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliStrings.java
index 7fb90e6..2d2dd19 100644
--- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliStrings.java
+++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/CliStrings.java
@@ -146,6 +146,10 @@ public final class CliStrings {
public static final String MESSAGE_VIEW_REMOVED = "View has been removed.";
+ public static final String MESSAGE_ALTER_VIEW_SUCCEEDED = "Alter view succeeded!";
+
+ public static final String MESSAGE_ALTER_VIEW_FAILED = "Alter view failed!";
+
public static final String MESSAGE_FUNCTION_CREATED = "Function has been created.";
public static final String MESSAGE_FUNCTION_REMOVED = "Function has been removed.";
diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
index c01d53c..63ee844 100644
--- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
+++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
@@ -34,6 +34,7 @@ import org.apache.flink.table.operations.UseDatabaseOperation;
import org.apache.flink.table.operations.ddl.AlterCatalogFunctionOperation;
import org.apache.flink.table.operations.ddl.AlterDatabaseOperation;
import org.apache.flink.table.operations.ddl.AlterTableOperation;
+import org.apache.flink.table.operations.ddl.AlterViewOperation;
import org.apache.flink.table.operations.ddl.CreateCatalogFunctionOperation;
import org.apache.flink.table.operations.ddl.CreateCatalogOperation;
import org.apache.flink.table.operations.ddl.CreateDatabaseOperation;
@@ -118,6 +119,8 @@ public final class SqlCommandParser {
cmd = SqlCommand.CREATE_VIEW;
} else if (operation instanceof DropViewOperation) {
cmd = SqlCommand.DROP_VIEW;
+ } else if (operation instanceof AlterViewOperation) {
+ cmd = SqlCommand.ALTER_VIEW;
} else if (operation instanceof CreateDatabaseOperation) {
cmd = SqlCommand.CREATE_DATABASE;
} else if (operation instanceof DropDatabaseOperation) {
@@ -273,6 +276,8 @@ public final class SqlCommandParser {
DROP_VIEW,
+ ALTER_VIEW,
+
CREATE_FUNCTION,
DROP_FUNCTION,
diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/SqlCommandParserTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/SqlCommandParserTest.java
index a8d2a8e..879d691 100644
--- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/SqlCommandParserTest.java
+++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/SqlCommandParserTest.java
@@ -19,6 +19,8 @@
package org.apache.flink.table.client.cli;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.table.api.SqlDialect;
+import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.client.cli.SqlCommandParser.SqlCommand;
import org.apache.flink.table.client.cli.SqlCommandParser.SqlCommandCall;
import org.apache.flink.table.client.cli.utils.SqlParserHelper;
@@ -44,12 +46,14 @@ import static org.junit.Assert.fail;
public class SqlCommandParserTest {
private Parser parser;
+ private TableEnvironment tableEnv;
@Before
public void setup() {
SqlParserHelper helper = new SqlParserHelper();
helper.registerTables();
parser = helper.getSqlParser();
+ tableEnv = helper.getTableEnv();
}
@Test
@@ -126,6 +130,11 @@ public class SqlCommandParserTest {
TestItem.invalidSql("DROP VIEW ", // missing name
SqlExecutionException.class,
"Encountered \"<EOF>\""),
+ // alter view
+ TestItem.validSql(SqlDialect.HIVE,
+ "ALTER VIEW MyView RENAME TO MyView1",
+ SqlCommand.ALTER_VIEW,
+ "ALTER VIEW MyView RENAME TO MyView1"),
// set
TestItem.validSql("SET", SqlCommand.SET).cannotParseComment(),
TestItem.validSql("SET x=y", SqlCommand.SET, "x", "y").cannotParseComment(),
@@ -279,6 +288,7 @@ public class SqlCommandParserTest {
"Alter temporary system function is not supported")
);
for (TestItem item : testItems) {
+ tableEnv.getConfig().setSqlDialect(item.sqlDialect);
runTestItem(item);
}
}
@@ -348,6 +358,7 @@ public class SqlCommandParserTest {
private String[] expectedOperands = new String[0];
private Class<? extends Throwable> expectedException = null;
private String expectedExceptionMsg = null;
+ private SqlDialect sqlDialect = SqlDialect.DEFAULT;
private TestItem(String sql) {
this.sql = sql;
@@ -372,6 +383,16 @@ public class SqlCommandParserTest {
return testItem;
}
+ public static TestItem validSql(
+ SqlDialect sqlDialect, String sql, SqlCommand expectedCmd, String... expectedOperands) {
+ TestItem testItem = new TestItem(sql);
+ testItem.expectedCmd = expectedCmd;
+ testItem.expectedOperands = expectedOperands;
+ testItem.cannotParseComment = false; // default is false
+ testItem.sqlDialect = sqlDialect;
+ return testItem;
+ }
+
public TestItem cannotParseComment() {
cannotParseComment = true;
return this;
diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/SqlParserHelper.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/SqlParserHelper.java
index ad6bc9b..fc8dfdc 100644
--- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/SqlParserHelper.java
+++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/SqlParserHelper.java
@@ -43,6 +43,7 @@ public class SqlParserHelper {
registerTable("create table MyOtherTable (a int, b bigint) " +
"with ('connector' = 'filesystem', 'path' = '/non')");
registerTable("create table MySink (a int, c varchar(32)) with ('connector' = 'COLLECTION' )");
+ registerTable("create view MyView as select * from MyTable");
}
public void registerTable(String createTableStmt) {
@@ -52,4 +53,8 @@ public class SqlParserHelper {
public Parser getSqlParser() {
return ((TableEnvironmentInternal) tableEnv).getParser();
}
+
+ public TableEnvironment getTableEnv() {
+ return tableEnv;
+ }
}