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


##########
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/ClientParser.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.FlinkSqlParserImplTokenManager;
+import org.apache.flink.sql.parser.impl.SimpleCharStream;
+import org.apache.flink.sql.parser.impl.Token;
+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. */
+public class ClientParser implements SqlCommandParser {
+
+    /** 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)));
+        List<Token> tokenList = new ArrayList<>();
+        Token token;
+        do {
+            token = tokenManager.getNextToken();
+            tokenList.add(token);
+        } while (token.endColumn != trimmedStatement.length());
+        return getStatementType(tokenList);
+    }
+
+    // 
---------------------------------------------------------------------------------------------
+    private Optional<StatementType> getStatementType(List<Token> tokenList) {
+        Token firstToken = tokenList.get(0);
+
+        if (firstToken.kind == EOF || firstToken.kind == EMPTY || 
firstToken.kind == SEMICOLON) {
+            return Optional.empty();
+        }
+
+        if (firstToken.kind == IDENTIFIER) {
+            // unrecognized token
+            return getPotentialCommandType(firstToken.image);
+        } else if (firstToken.kind == EXPLAIN) {
+            return Optional.of(StatementType.EXPLAIN);
+        } else if (firstToken.kind == SHOW) {
+            return getPotentialShowCreateType(tokenList);
+        } else {
+            return Optional.of(StatementType.OTHER);
+        }

Review Comment:
   After checking the implementation of `SqlMultiLineParser`, here when the 
statement is incomplete, an SqlExecutionException should be thrown. I added the 
codes.



##########
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/ClientParser.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.FlinkSqlParserImplTokenManager;
+import org.apache.flink.sql.parser.impl.SimpleCharStream;
+import org.apache.flink.sql.parser.impl.Token;
+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. */
+public class ClientParser implements SqlCommandParser {
+
+    /** 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)));
+        List<Token> tokenList = new ArrayList<>();
+        Token token;
+        do {
+            token = tokenManager.getNextToken();
+            tokenList.add(token);
+        } while (token.endColumn != trimmedStatement.length());
+        return getStatementType(tokenList);
+    }
+
+    // 
---------------------------------------------------------------------------------------------
+    private Optional<StatementType> getStatementType(List<Token> tokenList) {

Review Comment:
   I think use iterator model here won't save time here, but it will make the 
codes more complicated. I think an `ArrayList` here is just OK.



##########
flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/ClientParser.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.FlinkSqlParserImplTokenManager;
+import org.apache.flink.sql.parser.impl.SimpleCharStream;
+import org.apache.flink.sql.parser.impl.Token;
+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. */
+public class ClientParser implements SqlCommandParser {
+
+    /** 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)));
+        List<Token> tokenList = new ArrayList<>();
+        Token token;
+        do {
+            token = tokenManager.getNextToken();
+            tokenList.add(token);
+        } while (token.endColumn != trimmedStatement.length());
+        return getStatementType(tokenList);
+    }
+
+    // 
---------------------------------------------------------------------------------------------
+    private Optional<StatementType> getStatementType(List<Token> tokenList) {

Review Comment:
   I think use iterator model here won't save time, but it will make the codes 
more complicated. I think an `ArrayList` here is just OK.



##########
flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/parser/ClientParserTest.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+
+import org.junit.Ignore;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Testing whether {@link ClientParser} can parse statement to get {@link 
StatementType} correctly.
+ */
+@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+public class ClientParserTest {
+
+    private final ClientParser clientParser = new ClientParser();
+
+    private static final Optional<StatementType> QUIT = 
Optional.of(StatementType.QUIT);
+    private static final Optional<StatementType> CLEAR = 
Optional.of(StatementType.CLEAR);
+    private static final Optional<StatementType> HELP = 
Optional.of(StatementType.HELP);
+    private static final Optional<StatementType> EXPLAIN = 
Optional.of(StatementType.EXPLAIN);
+    private static final Optional<StatementType> SHOW_CREATE =
+            Optional.of(StatementType.SHOW_CREATE);
+    private static final Optional<StatementType> OTHER = 
Optional.of(StatementType.OTHER);
+    private static final Optional<StatementType> EMPTY = Optional.empty();
+
+    @Ignore
+    @ParameterizedTest
+    @MethodSource("generateTestData")
+    public void testParseStatement(Tuple2<String, Optional<StatementType>> 
testData) {
+        Optional<StatementType> type = 
clientParser.parseStatement(testData.f0);
+        assertThat(type).isEqualTo(testData.f1);
+    }
+
+    private static List<Tuple2<String, Optional<StatementType>>> 
generateTestData() {
+        return Arrays.asList(
+                Tuple2.of("quit;", QUIT),

Review Comment:
   Applied.



##########
flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/parser/ClientParserTest.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.api.java.tuple.Tuple2;
+
+import org.junit.Ignore;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Testing whether {@link ClientParser} can parse statement to get {@link 
StatementType} correctly.
+ */
+@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+public class ClientParserTest {
+
+    private final ClientParser clientParser = new ClientParser();
+
+    private static final Optional<StatementType> QUIT = 
Optional.of(StatementType.QUIT);
+    private static final Optional<StatementType> CLEAR = 
Optional.of(StatementType.CLEAR);
+    private static final Optional<StatementType> HELP = 
Optional.of(StatementType.HELP);
+    private static final Optional<StatementType> EXPLAIN = 
Optional.of(StatementType.EXPLAIN);
+    private static final Optional<StatementType> SHOW_CREATE =
+            Optional.of(StatementType.SHOW_CREATE);
+    private static final Optional<StatementType> OTHER = 
Optional.of(StatementType.OTHER);
+    private static final Optional<StatementType> EMPTY = Optional.empty();
+
+    @Ignore
+    @ParameterizedTest
+    @MethodSource("generateTestData")
+    public void testParseStatement(Tuple2<String, Optional<StatementType>> 
testData) {
+        Optional<StatementType> type = 
clientParser.parseStatement(testData.f0);
+        assertThat(type).isEqualTo(testData.f1);
+    }
+
+    private static List<Tuple2<String, Optional<StatementType>>> 
generateTestData() {
+        return Arrays.asList(
+                Tuple2.of("quit;", QUIT),
+                Tuple2.of("quit", QUIT),
+                Tuple2.of("QUIT", QUIT),
+                Tuple2.of("Quit", QUIT),
+                Tuple2.of("QuIt", QUIT),
+                Tuple2.of("clear;", CLEAR),
+                Tuple2.of("help;", HELP),
+                Tuple2.of("EXPLAIN PLAN FOR what_ever", EXPLAIN),
+                Tuple2.of("SHOW CREATE TABLE(what_ever);", SHOW_CREATE),
+                Tuple2.of("SHOW CREATE VIEW (what_ever)", SHOW_CREATE),
+                Tuple2.of("SHOW CREATE syntax_error;", OTHER),
+                Tuple2.of("--SHOW CREATE TABLE ignore_comment", EMPTY),

Review Comment:
   Added some tests.



-- 
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