You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/05/16 05:58:29 UTC

[GitHub] [flink] godfreyhe opened a new pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

godfreyhe opened a new pull request #12188:
URL: https://github.com/apache/flink/pull/12188


   
   ## What is the purpose of the change
   
   *currently, all statements in sql client are parsed via regex matching, which has many limitations, such as it can't handle comments. To avoid that limitations, we should try best to use sql parser to parse a statement. There are many statement can't be handle by sql parser, such as: set, reset. So they are still handle through regex matching.
   
   statements handled through regex matching:
   quit, exit, clear, help, desc, explain, set, reset source, show modules
   
   statements handled through sql parser:
   show catalogs, show databases, show tables, show functions, use catalog, use, describe, explain plan for, select, insert, DDLs*
   
   
   ## Brief change log
   
     - *SqlCommandParser parses a statement via sql parse, and then via regex matching*
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
     - *extended SqlCommandParserTest to verify all cases*
   
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes / **no**)
     - If yes, how is the feature documented? (not applicable / docs / JavaDocs / **not documented**)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582",
       "triggerID" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1608",
       "triggerID" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aae4f45fd18320222d790dffcda117e02cb64e5e",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1752",
       "triggerID" : "aae4f45fd18320222d790dffcda117e02cb64e5e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * aae4f45fd18320222d790dffcda117e02cb64e5e Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1752) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629594206


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit aae4f45fd18320222d790dffcda117e02cb64e5e (Fri Oct 16 10:50:21 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] danny0405 commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426150317



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,29 +58,131 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Function<String, List<Operation>> sqlParserFunction, String stmt) {
 		// normalize
 		stmt = stmt.trim();
 		// remove ';' at the end
 		if (stmt.endsWith(";")) {
 			stmt = stmt.substring(0, stmt.length() - 1).trim();
 		}
 
-		// parse
+		// parse statement via sql parser first
+		Optional<SqlCommandCall> callOpt = parseBySqlParser(sqlParserFunction, stmt);
+		if (callOpt.isPresent()) {
+			return callOpt;
+		}
+
+		// parse statement via regex match
 		for (SqlCommand cmd : SqlCommand.values()) {
-			final Matcher matcher = cmd.pattern.matcher(stmt);
-			if (matcher.matches()) {
-				final String[] groups = new String[matcher.groupCount()];
-				for (int i = 0; i < groups.length; i++) {
-					groups[i] = matcher.group(i + 1);
+			if (cmd.hasRegexPattern()) {
+				final Matcher matcher = cmd.pattern.matcher(stmt);
+				if (matcher.matches()) {
+					final String[] groups = new String[matcher.groupCount()];
+					for (int i = 0; i < groups.length; i++) {
+						groups[i] = matcher.group(i + 1);
+					}
+					return cmd.operandConverter.apply(groups)
+							.map((operands) -> {
+								String[] newOperands = operands;
+								if (cmd == SqlCommand.EXPLAIN) {
+									// convert `explain xx` to `explain plan for xx`

Review comment:
       After a offline discussion, we have a TODO to refactor the commands in SQL-CLI to keep them more in line with TableEnv, so it's not a big deal currently.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426140762



##########
File path: flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/ParserUtils.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.client.cli.utils;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.internal.TableEnvironmentInternal;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.operations.Operation;
+
+import java.util.List;
+
+/**
+ * An utility class that provides abilities to parse sql statements.
+ */
+public class ParserUtils {
+
+	private static final TableEnvironment tableEnv = TableEnvironment.create(EnvironmentSettings.newInstance().build());
+
+	static {
+		tableEnv.executeSql("create table MyTable (a int, b bigint, c varchar(32)) " +
+				"with ('connector' = 'filesystem', 'path' = '/non')");
+		tableEnv.executeSql("create table MyOtherTable (a int, b bigint) " +
+				"with ('connector' = 'filesystem', 'path' = '/non')");

Review comment:
       prepare some tables for testing. now `Parser#parse` contains validation logic, we should make sure the table in a statement exists. I can add some comments here




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] danny0405 commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426335767



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,24 +61,130 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Parser sqlParser, String stmt) {
 		// normalize
 		stmt = stmt.trim();
 		// remove ';' at the end
 		if (stmt.endsWith(";")) {
 			stmt = stmt.substring(0, stmt.length() - 1).trim();
 		}
 
-		// parse
+		// parse statement via sql parser first
+		Optional<SqlCommandCall> callOpt = parseBySqlParser(sqlParser, stmt);
+		if (callOpt.isPresent()) {
+			return callOpt;
+		} else {
+			return parseByRegexMatching(stmt);
+		}
+	}
+
+	private static Optional<SqlCommandCall> parseBySqlParser(Parser sqlParser, String stmt) {
+		List<Operation> operations;
+		try {
+			operations = sqlParser.parse(stmt);
+		} catch (Throwable e) {
+			if (e instanceof ValidationException) {
+				// can be parsed via sql parser, but is not validated.
+				// throw exception directly
+				throw new SqlExecutionException("Invalidate SQL statement.", e);
+			}
+			return Optional.empty();
+		}
+		if (operations.size() != 1) {
+			throw new SqlExecutionException("Only single statement is supported now.");
+		}
+
+		final SqlCommand cmd;
+		String[] operands = new String[0];
+		Operation operation = operations.get(0);
+		if (operation instanceof CatalogSinkModifyOperation) {
+			boolean overwrite = ((CatalogSinkModifyOperation) operation).isOverwrite();
+			cmd = overwrite ? SqlCommand.INSERT_OVERWRITE : SqlCommand.INSERT_INTO;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateTableOperation) {
+			cmd = SqlCommand.CREATE_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropTableOperation) {
+			cmd = SqlCommand.DROP_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterTableOperation) {
+			cmd = SqlCommand.ALTER_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateViewOperation) {
+			cmd = SqlCommand.CREATE_VIEW;
+			CreateViewOperation op = (CreateViewOperation) operation;
+			operands = new String[] { op.getViewIdentifier().asSerializableString(),
+					op.getCatalogView().getOriginalQuery() };
+		} else if (operation instanceof DropViewOperation) {
+			cmd = SqlCommand.DROP_VIEW;
+			operands = new String[] { ((DropViewOperation) operation).getViewIdentifier().asSerializableString() };
+		} else if (operation instanceof CreateDatabaseOperation) {
+			cmd = SqlCommand.CREATE_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropDatabaseOperation) {
+			cmd = SqlCommand.DROP_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterDatabaseOperation) {
+			cmd = SqlCommand.ALTER_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateCatalogOperation) {
+			cmd = SqlCommand.CREATE_CATALOG;
+			operands = new String[] { stmt };
+		} else if (operation instanceof UseCatalogOperation) {
+			cmd = SqlCommand.USE_CATALOG;
+			operands = new String[] { String.format("`%s`", ((UseCatalogOperation) operation).getCatalogName()) };
+		} else if (operation instanceof UseDatabaseOperation) {
+			cmd = SqlCommand.USE;
+			UseDatabaseOperation op = ((UseDatabaseOperation) operation);
+			operands = new String[] { String.format("`%s`.`%s`", op.getCatalogName(), op.getDatabaseName()) };
+		} else if (operation instanceof ShowCatalogsOperation) {
+			cmd = SqlCommand.SHOW_CATALOGS;
+		} else if (operation instanceof ShowDatabasesOperation) {
+			cmd = SqlCommand.SHOW_DATABASES;
+		} else if (operation instanceof ShowTablesOperation) {
+			cmd = SqlCommand.SHOW_TABLES;
+		} else if (operation instanceof ShowFunctionsOperation) {
+			cmd = SqlCommand.SHOW_FUNCTIONS;
+		} else if (operation instanceof ExplainOperation) {
+			cmd = SqlCommand.EXPLAIN;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DescribeTableOperation) {

Review comment:
       Most of the operations has the same operands `new String[] { stmt };` can we merge the handling of them ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] wuchong merged pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
wuchong merged pull request #12188:
URL: https://github.com/apache/flink/pull/12188


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582",
       "triggerID" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1608",
       "triggerID" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aae4f45fd18320222d790dffcda117e02cb64e5e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "aae4f45fd18320222d790dffcda117e02cb64e5e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 48e61b51442e40d0de50c315a9850aae80c4396b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1608) 
   * aae4f45fd18320222d790dffcda117e02cb64e5e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] danny0405 commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426338099



##########
File path: flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/SqlParserHelper.java
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.client.cli.utils;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.internal.TableEnvironmentInternal;
+import org.apache.flink.table.delegation.Parser;
+
+/**
+ * An utility class that provides pre-prepared tables and sql parser.
+ */
+public class SqlParserHelper {
+	// return the sql parser instance hold by this table evn.
+	private final TableEnvironment tableEnv;
+
+	public SqlParserHelper() {
+		tableEnv = TableEnvironment.create(EnvironmentSettings.newInstance().build());
+	}
+
+	/**
+	 * prepare some tables for testing.
+	 */
+	public void registerTables() {
+		tableEnv.executeSql("create table MyTable (a int, b bigint, c varchar(32)) " +
+				"with ('connector' = 'filesystem', 'path' = '/non')");

Review comment:
       Can invoke `registerTable(String createTableStmt)` instead.

##########
File path: flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java
##########
@@ -90,9 +92,9 @@ public void testFailedUpdateSubmission() throws Exception {
 
 	@Test
 	public void testSqlCompletion() throws IOException {
-		verifySqlCompletion("", 0, Arrays.asList("SELECT", "QUIT;", "RESET;"), Collections.emptyList());
-		verifySqlCompletion("SELEC", 5, Collections.singletonList("SELECT"), Collections.singletonList("QUIT;"));
-		verifySqlCompletion("SELE", 0, Collections.singletonList("SELECT"), Collections.singletonList("QUIT;"));
+		verifySqlCompletion("", 0, Arrays.asList("SOURCE", "QUIT;", "RESET;"), Collections.emptyList());

Review comment:
       Why drop the `SELEC` completion test ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] danny0405 commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426133545



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -194,6 +283,11 @@ private SqlCommandParser() {
 		public final Pattern pattern;
 		public final Function<String[], Optional<String[]>> operandConverter;
 
+		SqlCommand() {
+			this.pattern = null;
+			this.operandConverter = null;
+		}
+
 		SqlCommand(String matchingRegex, Function<String[], Optional<String[]>> operandConverter) {
 			this.pattern = Pattern.compile(matchingRegex, DEFAULT_PATTERN_FLAGS);

Review comment:
       Mark both operands as nullable.

##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -207,6 +301,10 @@ public String toString() {
 		public boolean hasOperands() {
 			return operandConverter != NO_OPERANDS;
 		}

Review comment:
       We should use equals instead of `!=` to compare the operands.

##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,29 +58,131 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Function<String, List<Operation>> sqlParserFunction, String stmt) {
 		// normalize
 		stmt = stmt.trim();
 		// remove ';' at the end
 		if (stmt.endsWith(";")) {
 			stmt = stmt.substring(0, stmt.length() - 1).trim();
 		}
 
-		// parse
+		// parse statement via sql parser first
+		Optional<SqlCommandCall> callOpt = parseBySqlParser(sqlParserFunction, stmt);
+		if (callOpt.isPresent()) {
+			return callOpt;
+		}
+
+		// parse statement via regex match
 		for (SqlCommand cmd : SqlCommand.values()) {
-			final Matcher matcher = cmd.pattern.matcher(stmt);
-			if (matcher.matches()) {
-				final String[] groups = new String[matcher.groupCount()];
-				for (int i = 0; i < groups.length; i++) {
-					groups[i] = matcher.group(i + 1);
+			if (cmd.hasRegexPattern()) {
+				final Matcher matcher = cmd.pattern.matcher(stmt);
+				if (matcher.matches()) {
+					final String[] groups = new String[matcher.groupCount()];
+					for (int i = 0; i < groups.length; i++) {
+						groups[i] = matcher.group(i + 1);
+					}
+					return cmd.operandConverter.apply(groups)
+							.map((operands) -> {
+								String[] newOperands = operands;
+								if (cmd == SqlCommand.EXPLAIN) {
+									// convert `explain xx` to `explain plan for xx`
+									newOperands = new String[] { "EXPLAIN PLAN FOR " + operands[0] };
+								}
+								return new SqlCommandCall(cmd, newOperands);
+							});
 				}
-				return cmd.operandConverter.apply(groups)
-					.map((operands) -> new SqlCommandCall(cmd, operands));
 			}
 		}
 		return Optional.empty();
 	}
 
+	private static Optional<SqlCommandCall> parseBySqlParser(
+			Function<String, List<Operation>> sqlParserFunction, String stmt) {
+		List<Operation> operations;
+		try {
+			operations = sqlParserFunction.apply(stmt);
+		} catch (SqlExecutionException e) {
+			if (e.getCause() instanceof ValidationException) {
+				// can be parsed via sql parser, but is not validated.
+				// throw exception directly
+				throw e;
+			}
+			return Optional.empty();
+		}
+		if (operations.size() != 1) {
+			throw new SqlExecutionException("Only single statement is supported now.");
+		}
+
+		final SqlCommand cmd;
+		String[] operands = new String[0];
+		Operation operation = operations.get(0);
+		if (operation instanceof CatalogSinkModifyOperation) {
+			boolean overwrite = ((CatalogSinkModifyOperation) operation).isOverwrite();
+			cmd = overwrite ? SqlCommand.INSERT_OVERWRITE : SqlCommand.INSERT_INTO;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateTableOperation) {
+			cmd = SqlCommand.CREATE_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropTableOperation) {
+			cmd = SqlCommand.DROP_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterTableOperation) {
+			cmd = SqlCommand.ALTER_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateViewOperation) {
+			cmd = SqlCommand.CREATE_VIEW;
+			CreateViewOperation op = (CreateViewOperation) operation;
+			operands = new String[] { op.getViewIdentifier().asSerializableString(),
+					op.getCatalogView().getOriginalQuery() };
+		} else if (operation instanceof DropViewOperation) {
+			cmd = SqlCommand.DROP_VIEW;
+			operands = new String[] { ((DropViewOperation) operation).getViewIdentifier().asSerializableString() };
+		} else if (operation instanceof CreateDatabaseOperation) {
+			cmd = SqlCommand.CREATE_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropDatabaseOperation) {
+			cmd = SqlCommand.DROP_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterDatabaseOperation) {
+			cmd = SqlCommand.ALTER_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateCatalogOperation) {
+			cmd = SqlCommand.CREATE_CATALOG;
+			operands = new String[] { stmt };
+		} else if (operation instanceof UseCatalogOperation) {
+			cmd = SqlCommand.USE_CATALOG;
+			operands = new String[] { String.format("`%s`", ((UseCatalogOperation) operation).getCatalogName()) };

Review comment:
       Move the whole if else block to a single class named `OperationToSqlCommandConverter` ? We also need to add tests for the mapping.

##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,29 +58,131 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Function<String, List<Operation>> sqlParserFunction, String stmt) {
 		// normalize
 		stmt = stmt.trim();

Review comment:
       How about we just split 2 methods `parseByExecutor` and `parseByCli`, parse a function here seems weird, because the first operand of the `sqlParserFunction ` is just the `stmt`.

##########
File path: flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/ParserUtils.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.client.cli.utils;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.internal.TableEnvironmentInternal;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.operations.Operation;
+
+import java.util.List;
+
+/**
+ * An utility class that provides abilities to parse sql statements.
+ */
+public class ParserUtils {
+
+	private static final TableEnvironment tableEnv = TableEnvironment.create(EnvironmentSettings.newInstance().build());
+
+	static {
+		tableEnv.executeSql("create table MyTable (a int, b bigint, c varchar(32)) " +
+				"with ('connector' = 'filesystem', 'path' = '/non')");
+		tableEnv.executeSql("create table MyOtherTable (a int, b bigint) " +
+				"with ('connector' = 'filesystem', 'path' = '/non')");

Review comment:
       It's a `ParserUtils` but why we register table there ?

##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,29 +58,131 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Function<String, List<Operation>> sqlParserFunction, String stmt) {
 		// normalize
 		stmt = stmt.trim();
 		// remove ';' at the end
 		if (stmt.endsWith(";")) {
 			stmt = stmt.substring(0, stmt.length() - 1).trim();
 		}
 
-		// parse
+		// parse statement via sql parser first
+		Optional<SqlCommandCall> callOpt = parseBySqlParser(sqlParserFunction, stmt);
+		if (callOpt.isPresent()) {
+			return callOpt;
+		}
+
+		// parse statement via regex match
 		for (SqlCommand cmd : SqlCommand.values()) {
-			final Matcher matcher = cmd.pattern.matcher(stmt);
-			if (matcher.matches()) {
-				final String[] groups = new String[matcher.groupCount()];
-				for (int i = 0; i < groups.length; i++) {
-					groups[i] = matcher.group(i + 1);
+			if (cmd.hasRegexPattern()) {
+				final Matcher matcher = cmd.pattern.matcher(stmt);
+				if (matcher.matches()) {
+					final String[] groups = new String[matcher.groupCount()];
+					for (int i = 0; i < groups.length; i++) {
+						groups[i] = matcher.group(i + 1);
+					}
+					return cmd.operandConverter.apply(groups)
+							.map((operands) -> {
+								String[] newOperands = operands;
+								if (cmd == SqlCommand.EXPLAIN) {
+									// convert `explain xx` to `explain plan for xx`

Review comment:
       It is hacky to support a non-common syntax and replace it silently, i think support `EXPLAIN PLAN FOR` is enough.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426428163



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,24 +61,130 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Parser sqlParser, String stmt) {
 		// normalize
 		stmt = stmt.trim();
 		// remove ';' at the end
 		if (stmt.endsWith(";")) {
 			stmt = stmt.substring(0, stmt.length() - 1).trim();
 		}
 
-		// parse
+		// parse statement via sql parser first
+		Optional<SqlCommandCall> callOpt = parseBySqlParser(sqlParser, stmt);
+		if (callOpt.isPresent()) {
+			return callOpt;
+		} else {
+			return parseByRegexMatching(stmt);
+		}
+	}
+
+	private static Optional<SqlCommandCall> parseBySqlParser(Parser sqlParser, String stmt) {
+		List<Operation> operations;
+		try {
+			operations = sqlParser.parse(stmt);
+		} catch (Throwable e) {
+			if (e instanceof ValidationException) {
+				// can be parsed via sql parser, but is not validated.
+				// throw exception directly
+				throw new SqlExecutionException("Invalidate SQL statement.", e);
+			}
+			return Optional.empty();
+		}
+		if (operations.size() != 1) {
+			throw new SqlExecutionException("Only single statement is supported now.");
+		}
+
+		final SqlCommand cmd;
+		String[] operands = new String[0];
+		Operation operation = operations.get(0);
+		if (operation instanceof CatalogSinkModifyOperation) {
+			boolean overwrite = ((CatalogSinkModifyOperation) operation).isOverwrite();
+			cmd = overwrite ? SqlCommand.INSERT_OVERWRITE : SqlCommand.INSERT_INTO;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateTableOperation) {
+			cmd = SqlCommand.CREATE_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropTableOperation) {
+			cmd = SqlCommand.DROP_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterTableOperation) {
+			cmd = SqlCommand.ALTER_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateViewOperation) {
+			cmd = SqlCommand.CREATE_VIEW;
+			CreateViewOperation op = (CreateViewOperation) operation;
+			operands = new String[] { op.getViewIdentifier().asSerializableString(),
+					op.getCatalogView().getOriginalQuery() };
+		} else if (operation instanceof DropViewOperation) {
+			cmd = SqlCommand.DROP_VIEW;
+			operands = new String[] { ((DropViewOperation) operation).getViewIdentifier().asSerializableString() };
+		} else if (operation instanceof CreateDatabaseOperation) {
+			cmd = SqlCommand.CREATE_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropDatabaseOperation) {
+			cmd = SqlCommand.DROP_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterDatabaseOperation) {
+			cmd = SqlCommand.ALTER_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateCatalogOperation) {
+			cmd = SqlCommand.CREATE_CATALOG;
+			operands = new String[] { stmt };
+		} else if (operation instanceof UseCatalogOperation) {
+			cmd = SqlCommand.USE_CATALOG;
+			operands = new String[] { String.format("`%s`", ((UseCatalogOperation) operation).getCatalogName()) };
+		} else if (operation instanceof UseDatabaseOperation) {
+			cmd = SqlCommand.USE;
+			UseDatabaseOperation op = ((UseDatabaseOperation) operation);
+			operands = new String[] { String.format("`%s`.`%s`", op.getCatalogName(), op.getDatabaseName()) };
+		} else if (operation instanceof ShowCatalogsOperation) {
+			cmd = SqlCommand.SHOW_CATALOGS;
+		} else if (operation instanceof ShowDatabasesOperation) {
+			cmd = SqlCommand.SHOW_DATABASES;
+		} else if (operation instanceof ShowTablesOperation) {
+			cmd = SqlCommand.SHOW_TABLES;
+		} else if (operation instanceof ShowFunctionsOperation) {
+			cmd = SqlCommand.SHOW_FUNCTIONS;
+		} else if (operation instanceof ExplainOperation) {
+			cmd = SqlCommand.EXPLAIN;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DescribeTableOperation) {

Review comment:
       we can change the default value from `new String[0]` to `new String[] { stmt }`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582",
       "triggerID" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1608",
       "triggerID" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 48e61b51442e40d0de50c315a9850aae80c4396b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1608) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426141169



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,29 +58,131 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Function<String, List<Operation>> sqlParserFunction, String stmt) {
 		// normalize
 		stmt = stmt.trim();

Review comment:
       for the `SqlCommandParser` users, they do not need to know how to parse a statement, so I think we should only provide one entry point method. I can extract the regex matching part into a method. for `sqlParserFunction`, how about pass it into `SqlCommandParser` through constructor, that means we should make `parse` method as a non-static method ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582",
       "triggerID" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1608",
       "triggerID" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aae4f45fd18320222d790dffcda117e02cb64e5e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1752",
       "triggerID" : "aae4f45fd18320222d790dffcda117e02cb64e5e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 48e61b51442e40d0de50c315a9850aae80c4396b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1608) 
   * aae4f45fd18320222d790dffcda117e02cb64e5e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1752) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582",
       "triggerID" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 615cd355c672d2275c8daa85a9cf4d68ae465211 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] danny0405 commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426150335



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,29 +58,131 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Function<String, List<Operation>> sqlParserFunction, String stmt) {
 		// normalize
 		stmt = stmt.trim();
 		// remove ';' at the end
 		if (stmt.endsWith(";")) {
 			stmt = stmt.substring(0, stmt.length() - 1).trim();
 		}
 
-		// parse
+		// parse statement via sql parser first
+		Optional<SqlCommandCall> callOpt = parseBySqlParser(sqlParserFunction, stmt);
+		if (callOpt.isPresent()) {
+			return callOpt;
+		}
+
+		// parse statement via regex match
 		for (SqlCommand cmd : SqlCommand.values()) {
-			final Matcher matcher = cmd.pattern.matcher(stmt);
-			if (matcher.matches()) {
-				final String[] groups = new String[matcher.groupCount()];
-				for (int i = 0; i < groups.length; i++) {
-					groups[i] = matcher.group(i + 1);
+			if (cmd.hasRegexPattern()) {
+				final Matcher matcher = cmd.pattern.matcher(stmt);
+				if (matcher.matches()) {
+					final String[] groups = new String[matcher.groupCount()];
+					for (int i = 0; i < groups.length; i++) {
+						groups[i] = matcher.group(i + 1);
+					}
+					return cmd.operandConverter.apply(groups)
+							.map((operands) -> {
+								String[] newOperands = operands;
+								if (cmd == SqlCommand.EXPLAIN) {
+									// convert `explain xx` to `explain plan for xx`
+									newOperands = new String[] { "EXPLAIN PLAN FOR " + operands[0] };
+								}
+								return new SqlCommandCall(cmd, newOperands);
+							});
 				}
-				return cmd.operandConverter.apply(groups)
-					.map((operands) -> new SqlCommandCall(cmd, operands));
 			}
 		}
 		return Optional.empty();
 	}
 
+	private static Optional<SqlCommandCall> parseBySqlParser(
+			Function<String, List<Operation>> sqlParserFunction, String stmt) {
+		List<Operation> operations;
+		try {
+			operations = sqlParserFunction.apply(stmt);
+		} catch (SqlExecutionException e) {
+			if (e.getCause() instanceof ValidationException) {
+				// can be parsed via sql parser, but is not validated.
+				// throw exception directly
+				throw e;
+			}
+			return Optional.empty();
+		}
+		if (operations.size() != 1) {
+			throw new SqlExecutionException("Only single statement is supported now.");
+		}
+
+		final SqlCommand cmd;
+		String[] operands = new String[0];
+		Operation operation = operations.get(0);
+		if (operation instanceof CatalogSinkModifyOperation) {
+			boolean overwrite = ((CatalogSinkModifyOperation) operation).isOverwrite();
+			cmd = overwrite ? SqlCommand.INSERT_OVERWRITE : SqlCommand.INSERT_INTO;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateTableOperation) {
+			cmd = SqlCommand.CREATE_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropTableOperation) {
+			cmd = SqlCommand.DROP_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterTableOperation) {
+			cmd = SqlCommand.ALTER_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateViewOperation) {
+			cmd = SqlCommand.CREATE_VIEW;
+			CreateViewOperation op = (CreateViewOperation) operation;
+			operands = new String[] { op.getViewIdentifier().asSerializableString(),
+					op.getCatalogView().getOriginalQuery() };
+		} else if (operation instanceof DropViewOperation) {
+			cmd = SqlCommand.DROP_VIEW;
+			operands = new String[] { ((DropViewOperation) operation).getViewIdentifier().asSerializableString() };
+		} else if (operation instanceof CreateDatabaseOperation) {
+			cmd = SqlCommand.CREATE_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropDatabaseOperation) {
+			cmd = SqlCommand.DROP_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterDatabaseOperation) {
+			cmd = SqlCommand.ALTER_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateCatalogOperation) {
+			cmd = SqlCommand.CREATE_CATALOG;
+			operands = new String[] { stmt };
+		} else if (operation instanceof UseCatalogOperation) {
+			cmd = SqlCommand.USE_CATALOG;
+			operands = new String[] { String.format("`%s`", ((UseCatalogOperation) operation).getCatalogName()) };

Review comment:
       Let's see if we can make some refactoring here, the code now is a mess.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426140506



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,29 +58,131 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Function<String, List<Operation>> sqlParserFunction, String stmt) {
 		// normalize
 		stmt = stmt.trim();
 		// remove ';' at the end
 		if (stmt.endsWith(";")) {
 			stmt = stmt.substring(0, stmt.length() - 1).trim();
 		}
 
-		// parse
+		// parse statement via sql parser first
+		Optional<SqlCommandCall> callOpt = parseBySqlParser(sqlParserFunction, stmt);
+		if (callOpt.isPresent()) {
+			return callOpt;
+		}
+
+		// parse statement via regex match
 		for (SqlCommand cmd : SqlCommand.values()) {
-			final Matcher matcher = cmd.pattern.matcher(stmt);
-			if (matcher.matches()) {
-				final String[] groups = new String[matcher.groupCount()];
-				for (int i = 0; i < groups.length; i++) {
-					groups[i] = matcher.group(i + 1);
+			if (cmd.hasRegexPattern()) {
+				final Matcher matcher = cmd.pattern.matcher(stmt);
+				if (matcher.matches()) {
+					final String[] groups = new String[matcher.groupCount()];
+					for (int i = 0; i < groups.length; i++) {
+						groups[i] = matcher.group(i + 1);
+					}
+					return cmd.operandConverter.apply(groups)
+							.map((operands) -> {
+								String[] newOperands = operands;
+								if (cmd == SqlCommand.EXPLAIN) {
+									// convert `explain xx` to `explain plan for xx`
+									newOperands = new String[] { "EXPLAIN PLAN FOR " + operands[0] };
+								}
+								return new SqlCommandCall(cmd, newOperands);
+							});
 				}
-				return cmd.operandConverter.apply(groups)
-					.map((operands) -> new SqlCommandCall(cmd, operands));
 			}
 		}
 		return Optional.empty();
 	}
 
+	private static Optional<SqlCommandCall> parseBySqlParser(
+			Function<String, List<Operation>> sqlParserFunction, String stmt) {
+		List<Operation> operations;
+		try {
+			operations = sqlParserFunction.apply(stmt);
+		} catch (SqlExecutionException e) {
+			if (e.getCause() instanceof ValidationException) {
+				// can be parsed via sql parser, but is not validated.
+				// throw exception directly
+				throw e;
+			}
+			return Optional.empty();
+		}
+		if (operations.size() != 1) {
+			throw new SqlExecutionException("Only single statement is supported now.");
+		}
+
+		final SqlCommand cmd;
+		String[] operands = new String[0];
+		Operation operation = operations.get(0);
+		if (operation instanceof CatalogSinkModifyOperation) {
+			boolean overwrite = ((CatalogSinkModifyOperation) operation).isOverwrite();
+			cmd = overwrite ? SqlCommand.INSERT_OVERWRITE : SqlCommand.INSERT_INTO;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateTableOperation) {
+			cmd = SqlCommand.CREATE_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropTableOperation) {
+			cmd = SqlCommand.DROP_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterTableOperation) {
+			cmd = SqlCommand.ALTER_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateViewOperation) {
+			cmd = SqlCommand.CREATE_VIEW;
+			CreateViewOperation op = (CreateViewOperation) operation;
+			operands = new String[] { op.getViewIdentifier().asSerializableString(),
+					op.getCatalogView().getOriginalQuery() };
+		} else if (operation instanceof DropViewOperation) {
+			cmd = SqlCommand.DROP_VIEW;
+			operands = new String[] { ((DropViewOperation) operation).getViewIdentifier().asSerializableString() };
+		} else if (operation instanceof CreateDatabaseOperation) {
+			cmd = SqlCommand.CREATE_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropDatabaseOperation) {
+			cmd = SqlCommand.DROP_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterDatabaseOperation) {
+			cmd = SqlCommand.ALTER_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateCatalogOperation) {
+			cmd = SqlCommand.CREATE_CATALOG;
+			operands = new String[] { stmt };
+		} else if (operation instanceof UseCatalogOperation) {
+			cmd = SqlCommand.USE_CATALOG;
+			operands = new String[] { String.format("`%s`", ((UseCatalogOperation) operation).getCatalogName()) };

Review comment:
       The mapping logic is tested in `SqlCommandParserTest`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426426306



##########
File path: flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java
##########
@@ -90,9 +92,9 @@ public void testFailedUpdateSubmission() throws Exception {
 
 	@Test
 	public void testSqlCompletion() throws IOException {
-		verifySqlCompletion("", 0, Arrays.asList("SELECT", "QUIT;", "RESET;"), Collections.emptyList());
-		verifySqlCompletion("SELEC", 5, Collections.singletonList("SELECT"), Collections.singletonList("QUIT;"));
-		verifySqlCompletion("SELE", 0, Collections.singletonList("SELECT"), Collections.singletonList("QUIT;"));
+		verifySqlCompletion("", 0, Arrays.asList("SOURCE", "QUIT;", "RESET;"), Collections.emptyList());

Review comment:
       before this pr, all commands are hint candidates. after this refactor,  only the commands who has regex pattern are hint candidates, or fallback to Table API hinting (will delegate to `tableEnv.getCompletionHints` in LocalExecutor). in `MockExecutor`, `completeStatement` method only returns `HintA` and `Hint B` 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582",
       "triggerID" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504) 
   * 615cd355c672d2275c8daa85a9cf4d68ae465211 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] danny0405 commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426335546



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,24 +61,130 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Parser sqlParser, String stmt) {
 		// normalize
 		stmt = stmt.trim();
 		// remove ';' at the end
 		if (stmt.endsWith(";")) {
 			stmt = stmt.substring(0, stmt.length() - 1).trim();
 		}
 
-		// parse
+		// parse statement via sql parser first
+		Optional<SqlCommandCall> callOpt = parseBySqlParser(sqlParser, stmt);
+		if (callOpt.isPresent()) {
+			return callOpt;
+		} else {
+			return parseByRegexMatching(stmt);
+		}
+	}
+
+	private static Optional<SqlCommandCall> parseBySqlParser(Parser sqlParser, String stmt) {
+		List<Operation> operations;
+		try {
+			operations = sqlParser.parse(stmt);
+		} catch (Throwable e) {
+			if (e instanceof ValidationException) {
+				// can be parsed via sql parser, but is not validated.
+				// throw exception directly
+				throw new SqlExecutionException("Invalidate SQL statement.", e);
+			}
+			return Optional.empty();
+		}
+		if (operations.size() != 1) {
+			throw new SqlExecutionException("Only single statement is supported now.");
+		}
+
+		final SqlCommand cmd;
+		String[] operands = new String[0];
+		Operation operation = operations.get(0);
+		if (operation instanceof CatalogSinkModifyOperation) {
+			boolean overwrite = ((CatalogSinkModifyOperation) operation).isOverwrite();
+			cmd = overwrite ? SqlCommand.INSERT_OVERWRITE : SqlCommand.INSERT_INTO;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateTableOperation) {
+			cmd = SqlCommand.CREATE_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropTableOperation) {
+			cmd = SqlCommand.DROP_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterTableOperation) {
+			cmd = SqlCommand.ALTER_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateViewOperation) {
+			cmd = SqlCommand.CREATE_VIEW;
+			CreateViewOperation op = (CreateViewOperation) operation;
+			operands = new String[] { op.getViewIdentifier().asSerializableString(),
+					op.getCatalogView().getOriginalQuery() };
+		} else if (operation instanceof DropViewOperation) {
+			cmd = SqlCommand.DROP_VIEW;
+			operands = new String[] { ((DropViewOperation) operation).getViewIdentifier().asSerializableString() };
+		} else if (operation instanceof CreateDatabaseOperation) {
+			cmd = SqlCommand.CREATE_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropDatabaseOperation) {
+			cmd = SqlCommand.DROP_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterDatabaseOperation) {
+			cmd = SqlCommand.ALTER_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateCatalogOperation) {
+			cmd = SqlCommand.CREATE_CATALOG;
+			operands = new String[] { stmt };
+		} else if (operation instanceof UseCatalogOperation) {
+			cmd = SqlCommand.USE_CATALOG;
+			operands = new String[] { String.format("`%s`", ((UseCatalogOperation) operation).getCatalogName()) };
+		} else if (operation instanceof UseDatabaseOperation) {
+			cmd = SqlCommand.USE;
+			UseDatabaseOperation op = ((UseDatabaseOperation) operation);
+			operands = new String[] { String.format("`%s`.`%s`", op.getCatalogName(), op.getDatabaseName()) };
+		} else if (operation instanceof ShowCatalogsOperation) {
+			cmd = SqlCommand.SHOW_CATALOGS;
+		} else if (operation instanceof ShowDatabasesOperation) {
+			cmd = SqlCommand.SHOW_DATABASES;
+		} else if (operation instanceof ShowTablesOperation) {
+			cmd = SqlCommand.SHOW_TABLES;
+		} else if (operation instanceof ShowFunctionsOperation) {
+			cmd = SqlCommand.SHOW_FUNCTIONS;
+		} else if (operation instanceof ExplainOperation) {
+			cmd = SqlCommand.EXPLAIN;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DescribeTableOperation) {
+			cmd = SqlCommand.DESCRIBE;
+			operands = new String[] { ((DescribeTableOperation) operation).getSqlIdentifier().asSerializableString() };
+		} else if (operation instanceof QueryOperation) {
+			cmd = SqlCommand.SELECT;
+			operands = new String[] { stmt };
+		} else {
+			cmd = null;
+		}
+		if (cmd == null) {
+			return Optional.empty();
+		} else {
+			return Optional.of(new SqlCommandCall(cmd, operands));
+		}

Review comment:
       `return cmd = null ? Optional.empty() : Optional.of(new SqlCommandCall(cmd, operands))`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] danny0405 commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426150259



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,29 +58,131 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Function<String, List<Operation>> sqlParserFunction, String stmt) {
 		// normalize
 		stmt = stmt.trim();

Review comment:
       How about just add method `#parse(Parser, String)`, and add a new interface `Executor#getParser`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504) 
   * 615cd355c672d2275c8daa85a9cf4d68ae465211 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426140427



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -207,6 +301,10 @@ public String toString() {
 		public boolean hasOperands() {
 			return operandConverter != NO_OPERANDS;
 		}

Review comment:
       `NO_OPERANDS ` is internal variable and NO_OPERANDS is a static final instance. we can use `!=`  to do compare




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582",
       "triggerID" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1608",
       "triggerID" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 615cd355c672d2275c8daa85a9cf4d68ae465211 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582) 
   * 48e61b51442e40d0de50c315a9850aae80c4396b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1608) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426140607



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,29 +58,131 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Function<String, List<Operation>> sqlParserFunction, String stmt) {
 		// normalize
 		stmt = stmt.trim();
 		// remove ';' at the end
 		if (stmt.endsWith(";")) {
 			stmt = stmt.substring(0, stmt.length() - 1).trim();
 		}
 
-		// parse
+		// parse statement via sql parser first
+		Optional<SqlCommandCall> callOpt = parseBySqlParser(sqlParserFunction, stmt);
+		if (callOpt.isPresent()) {
+			return callOpt;
+		}
+
+		// parse statement via regex match
 		for (SqlCommand cmd : SqlCommand.values()) {
-			final Matcher matcher = cmd.pattern.matcher(stmt);
-			if (matcher.matches()) {
-				final String[] groups = new String[matcher.groupCount()];
-				for (int i = 0; i < groups.length; i++) {
-					groups[i] = matcher.group(i + 1);
+			if (cmd.hasRegexPattern()) {
+				final Matcher matcher = cmd.pattern.matcher(stmt);
+				if (matcher.matches()) {
+					final String[] groups = new String[matcher.groupCount()];
+					for (int i = 0; i < groups.length; i++) {
+						groups[i] = matcher.group(i + 1);
+					}
+					return cmd.operandConverter.apply(groups)
+							.map((operands) -> {
+								String[] newOperands = operands;
+								if (cmd == SqlCommand.EXPLAIN) {
+									// convert `explain xx` to `explain plan for xx`

Review comment:
       I want do that, but we should treat `EXPLAIN` as public interface, we can't remove this directly




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629594206


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e (Sat May 16 06:02:05 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
    * **This pull request references an unassigned [Jira ticket](https://issues.apache.org/jira/browse/FLINK-17728).** According to the [code contribution guide](https://flink.apache.org/contributing/contribute-code.html), tickets need to be assigned before starting with the implementation work.
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] danny0405 commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426335656



##########
File path: flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/SqlCommandParser.java
##########
@@ -34,24 +61,130 @@ private SqlCommandParser() {
 		// private
 	}
 
-	public static Optional<SqlCommandCall> parse(String stmt) {
+	public static Optional<SqlCommandCall> parse(Parser sqlParser, String stmt) {
 		// normalize
 		stmt = stmt.trim();
 		// remove ';' at the end
 		if (stmt.endsWith(";")) {
 			stmt = stmt.substring(0, stmt.length() - 1).trim();
 		}
 
-		// parse
+		// parse statement via sql parser first
+		Optional<SqlCommandCall> callOpt = parseBySqlParser(sqlParser, stmt);
+		if (callOpt.isPresent()) {
+			return callOpt;
+		} else {
+			return parseByRegexMatching(stmt);
+		}
+	}
+
+	private static Optional<SqlCommandCall> parseBySqlParser(Parser sqlParser, String stmt) {
+		List<Operation> operations;
+		try {
+			operations = sqlParser.parse(stmt);
+		} catch (Throwable e) {
+			if (e instanceof ValidationException) {
+				// can be parsed via sql parser, but is not validated.
+				// throw exception directly
+				throw new SqlExecutionException("Invalidate SQL statement.", e);
+			}
+			return Optional.empty();
+		}
+		if (operations.size() != 1) {
+			throw new SqlExecutionException("Only single statement is supported now.");
+		}
+
+		final SqlCommand cmd;
+		String[] operands = new String[0];
+		Operation operation = operations.get(0);
+		if (operation instanceof CatalogSinkModifyOperation) {
+			boolean overwrite = ((CatalogSinkModifyOperation) operation).isOverwrite();
+			cmd = overwrite ? SqlCommand.INSERT_OVERWRITE : SqlCommand.INSERT_INTO;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateTableOperation) {
+			cmd = SqlCommand.CREATE_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropTableOperation) {
+			cmd = SqlCommand.DROP_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterTableOperation) {
+			cmd = SqlCommand.ALTER_TABLE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateViewOperation) {
+			cmd = SqlCommand.CREATE_VIEW;
+			CreateViewOperation op = (CreateViewOperation) operation;
+			operands = new String[] { op.getViewIdentifier().asSerializableString(),
+					op.getCatalogView().getOriginalQuery() };
+		} else if (operation instanceof DropViewOperation) {
+			cmd = SqlCommand.DROP_VIEW;
+			operands = new String[] { ((DropViewOperation) operation).getViewIdentifier().asSerializableString() };
+		} else if (operation instanceof CreateDatabaseOperation) {
+			cmd = SqlCommand.CREATE_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DropDatabaseOperation) {
+			cmd = SqlCommand.DROP_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof AlterDatabaseOperation) {
+			cmd = SqlCommand.ALTER_DATABASE;
+			operands = new String[] { stmt };
+		} else if (operation instanceof CreateCatalogOperation) {
+			cmd = SqlCommand.CREATE_CATALOG;
+			operands = new String[] { stmt };
+		} else if (operation instanceof UseCatalogOperation) {
+			cmd = SqlCommand.USE_CATALOG;
+			operands = new String[] { String.format("`%s`", ((UseCatalogOperation) operation).getCatalogName()) };
+		} else if (operation instanceof UseDatabaseOperation) {
+			cmd = SqlCommand.USE;
+			UseDatabaseOperation op = ((UseDatabaseOperation) operation);
+			operands = new String[] { String.format("`%s`.`%s`", op.getCatalogName(), op.getDatabaseName()) };
+		} else if (operation instanceof ShowCatalogsOperation) {
+			cmd = SqlCommand.SHOW_CATALOGS;
+		} else if (operation instanceof ShowDatabasesOperation) {
+			cmd = SqlCommand.SHOW_DATABASES;
+		} else if (operation instanceof ShowTablesOperation) {
+			cmd = SqlCommand.SHOW_TABLES;
+		} else if (operation instanceof ShowFunctionsOperation) {
+			cmd = SqlCommand.SHOW_FUNCTIONS;
+		} else if (operation instanceof ExplainOperation) {
+			cmd = SqlCommand.EXPLAIN;
+			operands = new String[] { stmt };
+		} else if (operation instanceof DescribeTableOperation) {
+			cmd = SqlCommand.DESCRIBE;
+			operands = new String[] { ((DescribeTableOperation) operation).getSqlIdentifier().asSerializableString() };
+		} else if (operation instanceof QueryOperation) {
+			cmd = SqlCommand.SELECT;
+			operands = new String[] { stmt };
+		} else {
+			cmd = null;

Review comment:
       `return Optional.empty();` directly. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #12188:
URL: https://github.com/apache/flink/pull/12188#issuecomment-629595773


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1504",
       "triggerID" : "7111bcfcc655d8d6b0ee5f9ff66e80e732f7272e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582",
       "triggerID" : "615cd355c672d2275c8daa85a9cf4d68ae465211",
       "triggerType" : "PUSH"
     }, {
       "hash" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "48e61b51442e40d0de50c315a9850aae80c4396b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 615cd355c672d2275c8daa85a9cf4d68ae465211 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=1582) 
   * 48e61b51442e40d0de50c315a9850aae80c4396b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] danny0405 commented on a change in pull request #12188: [FLINK-17728] [sql-client] sql client supports parser statements via sql parser

Posted by GitBox <gi...@apache.org>.
danny0405 commented on a change in pull request #12188:
URL: https://github.com/apache/flink/pull/12188#discussion_r426150425



##########
File path: flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/utils/ParserUtils.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.client.cli.utils;
+
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.internal.TableEnvironmentInternal;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.operations.Operation;
+
+import java.util.List;
+
+/**
+ * An utility class that provides abilities to parse sql statements.
+ */
+public class ParserUtils {
+
+	private static final TableEnvironment tableEnv = TableEnvironment.create(EnvironmentSettings.newInstance().build());
+
+	static {
+		tableEnv.executeSql("create table MyTable (a int, b bigint, c varchar(32)) " +
+				"with ('connector' = 'filesystem', 'path' = '/non')");
+		tableEnv.executeSql("create table MyOtherTable (a int, b bigint) " +
+				"with ('connector' = 'filesystem', 'path' = '/non')");

Review comment:
       I think the tool class should be stateless, or we make a common data structure there with pre-registered tables there only for testing.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org