fsk119 commented on code in PR #20931:
URL: https://github.com/apache/flink/pull/20931#discussion_r985739642


##########
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/ClientParser.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.parser;
+
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImplConstants;
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImplTokenManager;
+import org.apache.flink.sql.parser.impl.SimpleCharStream;
+import org.apache.flink.sql.parser.impl.Token;
+import org.apache.flink.table.api.SqlParserEOFException;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.operations.Operation;
+
+import javax.annotation.Nonnull;
+
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * ClientParser use {@link FlinkSqlParserImplTokenManager} to do lexical 
analysis. It cannot
+ * recognize special hive keywords yet.
+ */
+public class ClientParser implements SqlCommandParser, 
FlinkSqlParserImplConstants {
+
+    /** A dumb implementation. TODO: remove this after unifying the 
SqlMultiLineParser. */
+    @Override
+    public Optional<Operation> parseCommand(String command) {
+        return Optional.empty();

Review Comment:
   I think it should be
   
   ```
   parseStatement(statement);
   return Optional.empty();
   ```



##########
flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/parser/ClientParserTest.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.parser;
+
+import org.apache.flink.table.api.SqlParserEOFException;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
+import static 
org.apache.flink.table.client.cli.parser.StatementType.BEGIN_STATEMENT_SET;
+import static org.apache.flink.table.client.cli.parser.StatementType.CLEAR;
+import static org.apache.flink.table.client.cli.parser.StatementType.END;
+import static org.apache.flink.table.client.cli.parser.StatementType.EXPLAIN;
+import static org.apache.flink.table.client.cli.parser.StatementType.HELP;
+import static org.apache.flink.table.client.cli.parser.StatementType.OTHER;
+import static org.apache.flink.table.client.cli.parser.StatementType.QUIT;
+import static 
org.apache.flink.table.client.cli.parser.StatementType.SHOW_CREATE;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Testing whether {@link ClientParser} can parse statement to get {@link 
StatementType} correctly.
+ */
+@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+public class ClientParserTest {
+
+    private final ClientParser clientParser = new ClientParser();
+
+    @ParameterizedTest
+    @MethodSource("generateTestData")
+    public void testParseStatement(TestSpec testData) {
+        Optional<StatementType> type = 
clientParser.parseStatement(testData.statement);
+        assertThat(type).isEqualTo(testData.type);
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"", "\n", " ", "-- comment;", "SHOW TABLES -- 
comment;", "SHOW TABLES"})

Review Comment:
   Add a case `EXPLAIN STATEMENT SET BEGIN
   INSERT INTO StreamingTable SELECT * FROM (VALUES (1, 'Hello World'), (2, 
'Hi'), (2, 'Hi'), (3, 'Hello'), (3, 'World'), (4, 'ADD'), (5, 'LINE'));`



##########
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/ClientParser.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.parser;
+
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImplConstants;
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImplTokenManager;
+import org.apache.flink.sql.parser.impl.SimpleCharStream;
+import org.apache.flink.sql.parser.impl.Token;
+import org.apache.flink.table.api.SqlParserEOFException;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.operations.Operation;
+
+import javax.annotation.Nonnull;
+
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * ClientParser use {@link FlinkSqlParserImplTokenManager} to do lexical 
analysis. It cannot
+ * recognize special hive keywords yet.
+ */
+public class ClientParser implements SqlCommandParser, 
FlinkSqlParserImplConstants {
+
+    /** A dumb implementation. TODO: remove this after unifying the 
SqlMultiLineParser. */
+    @Override
+    public Optional<Operation> parseCommand(String command) {
+        return Optional.empty();
+    }
+
+    public Optional<StatementType> parseStatement(@Nonnull String statement)
+            throws SqlExecutionException {
+        String trimmedStatement = statement.trim();
+        FlinkSqlParserImplTokenManager tokenManager =
+                new FlinkSqlParserImplTokenManager(
+                        new SimpleCharStream(new 
StringReader(trimmedStatement)));
+        // this means to switch to "BACK TICK IDENTIFIER" state to support '`' 
in Flink SQL
+        tokenManager.SwitchTo(2);
+        List<Token> tokens = new ArrayList<>();
+        Token token = tokenManager.getNextToken();
+        while (token.kind != EOF) {
+            tokens.add(token);
+            token = tokenManager.getNextToken();
+        }
+
+        if (tokens.size() == 0 || tokens.get(tokens.size() - 1).kind != 
SEMICOLON) {
+            // throw this to notify the terminal to continue reading input
+            throw new SqlExecutionException("", new SqlParserEOFException(""));

Review Comment:
   Add meaningful exception msg here.



##########
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/ClientParser.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.parser;
+
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImplConstants;
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImplTokenManager;
+import org.apache.flink.sql.parser.impl.SimpleCharStream;
+import org.apache.flink.sql.parser.impl.Token;
+import org.apache.flink.table.api.SqlParserEOFException;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.operations.Operation;
+
+import javax.annotation.Nonnull;
+
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * ClientParser use {@link FlinkSqlParserImplTokenManager} to do lexical 
analysis. It cannot
+ * recognize special hive keywords yet.

Review Comment:
   Because Hive has a slightly different vocabulary compared to Flink 
vocabulary, which causes the `ClientParser` will misunderstand Hive's keywords 
to IDENTIFIER. But the `ClientParser` is only responsible to check whether the 
statement is completed or not and only cares about a few statements. So it's 
acceptable to tolerate the inaccuracy here.
   



##########
flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/parser/ClientParserTest.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.parser;
+
+import org.apache.flink.table.api.SqlParserEOFException;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
+import static 
org.apache.flink.table.client.cli.parser.StatementType.BEGIN_STATEMENT_SET;
+import static org.apache.flink.table.client.cli.parser.StatementType.CLEAR;
+import static org.apache.flink.table.client.cli.parser.StatementType.END;
+import static org.apache.flink.table.client.cli.parser.StatementType.EXPLAIN;
+import static org.apache.flink.table.client.cli.parser.StatementType.HELP;
+import static org.apache.flink.table.client.cli.parser.StatementType.OTHER;
+import static org.apache.flink.table.client.cli.parser.StatementType.QUIT;
+import static 
org.apache.flink.table.client.cli.parser.StatementType.SHOW_CREATE;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Testing whether {@link ClientParser} can parse statement to get {@link 
StatementType} correctly.
+ */
+@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+public class ClientParserTest {
+
+    private final ClientParser clientParser = new ClientParser();
+
+    @ParameterizedTest
+    @MethodSource("generateTestData")
+    public void testParseStatement(TestSpec testData) {
+        Optional<StatementType> type = 
clientParser.parseStatement(testData.statement);
+        assertThat(type).isEqualTo(testData.type);

Review Comment:
   nit: I think it's better to use
   
   ```
   assertThat(type.orElse(null)).isEqualTo(testData.type);
   ```
   
   We can also remove SuppressWarnings above.



##########
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/ClientParser.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.parser;
+
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImplConstants;
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImplTokenManager;
+import org.apache.flink.sql.parser.impl.SimpleCharStream;
+import org.apache.flink.sql.parser.impl.Token;
+import org.apache.flink.table.api.SqlParserEOFException;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.operations.Operation;
+
+import javax.annotation.Nonnull;
+
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * ClientParser use {@link FlinkSqlParserImplTokenManager} to do lexical 
analysis. It cannot
+ * recognize special hive keywords yet.
+ */
+public class ClientParser implements SqlCommandParser, 
FlinkSqlParserImplConstants {
+
+    /** A dumb implementation. TODO: remove this after unifying the 
SqlMultiLineParser. */
+    @Override
+    public Optional<Operation> parseCommand(String command) {
+        return Optional.empty();
+    }
+
+    public Optional<StatementType> parseStatement(@Nonnull String statement)

Review Comment:
   we don't need to add `@Nonnull` annoation. By default, we assume the 
parameter is not null and we only add `@Nullable` if it is nullable.



##########
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/StatementType.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.parser;
+
+/** Enumerates the possible types of input statements. */
+public enum StatementType {
+    QUIT,

Review Comment:
   nit: It's better we can add detailed msg for every types...



##########
flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/parser/ClientParserTest.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.parser;
+
+import org.apache.flink.table.api.SqlParserEOFException;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
+import static 
org.apache.flink.table.client.cli.parser.StatementType.BEGIN_STATEMENT_SET;
+import static org.apache.flink.table.client.cli.parser.StatementType.CLEAR;
+import static org.apache.flink.table.client.cli.parser.StatementType.END;
+import static org.apache.flink.table.client.cli.parser.StatementType.EXPLAIN;
+import static org.apache.flink.table.client.cli.parser.StatementType.HELP;
+import static org.apache.flink.table.client.cli.parser.StatementType.OTHER;
+import static org.apache.flink.table.client.cli.parser.StatementType.QUIT;
+import static 
org.apache.flink.table.client.cli.parser.StatementType.SHOW_CREATE;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Testing whether {@link ClientParser} can parse statement to get {@link 
StatementType} correctly.
+ */
+@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+public class ClientParserTest {
+
+    private final ClientParser clientParser = new ClientParser();
+
+    @ParameterizedTest
+    @MethodSource("generateTestData")
+    public void testParseStatement(TestSpec testData) {
+        Optional<StatementType> type = 
clientParser.parseStatement(testData.statement);
+        assertThat(type).isEqualTo(testData.type);
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"", "\n", " ", "-- comment;", "SHOW TABLES -- 
comment;", "SHOW TABLES"})
+    public void testParseIncompleteStatement(String statement) {
+        assertThatThrownBy(() -> clientParser.parseStatement(statement))
+                .satisfies(anyCauseMatches(SqlExecutionException.class))
+                .cause()
+                .satisfies(anyCauseMatches(SqlParserEOFException.class));
+    }
+
+    private static List<TestSpec> generateTestData() {
+        return Arrays.asList(
+                TestSpec.of(";", null),

Review Comment:
   Why we stilll need `null`? What's the difference between `OTHER` and `null`? 



##########
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/ClientParser.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.parser;
+
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImplConstants;
+import org.apache.flink.sql.parser.impl.FlinkSqlParserImplTokenManager;
+import org.apache.flink.sql.parser.impl.SimpleCharStream;
+import org.apache.flink.sql.parser.impl.Token;
+import org.apache.flink.table.api.SqlParserEOFException;
+import org.apache.flink.table.client.gateway.SqlExecutionException;
+import org.apache.flink.table.operations.Operation;
+
+import javax.annotation.Nonnull;
+
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * ClientParser use {@link FlinkSqlParserImplTokenManager} to do lexical 
analysis. It cannot
+ * recognize special hive keywords yet.

Review Comment:
   Hive has a slightly different vocabulary compared to Flink vocabulary, which 
causes the `ClientParser` will misunderstand Hive's keywords to IDENTIFIER. But 
the `ClientParser` is only responsible to check whether the statement is 
completed or not and only cares about a few statements. So it's acceptable to 
tolerate the inaccuracy 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.

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

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

Reply via email to