fsk119 commented on code in PR #20931: URL: https://github.com/apache/flink/pull/20931#discussion_r985252646
########## flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/StatementType.java: ########## @@ -0,0 +1,30 @@ +/* + * 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 { Review Comment: How about BEGIN STATEMENT SET/END? ########## 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: Test `SHOW TABLES -- comment ;` and muli-line cases: ``` SHOW\n create\t TABLE `tbl`; ``` Take a look at presto test cases `TestStatementSplitter` ########## 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: When the statement is not ends with ;, it is not a statement. But it seems we will always return OTHER? ########## 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); Review Comment: It's possible it is also an identitifer. The comment is confusing for others. ########## 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); + } + } + + private Optional<StatementType> getPotentialCommandType(String image) { + switch (image.toUpperCase()) { + case "QUIT": + case "EXIT": + return Optional.of(StatementType.QUIT); + case "CLEAR": + return Optional.of(StatementType.CLEAR); + case "HELP": + return Optional.of(StatementType.HELP); + default: + return Optional.of(StatementType.OTHER); + } + } + + private Optional<StatementType> getPotentialShowCreateType(List<Token> tokenList) { Review Comment: tokenList -> tokens ########## 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. */ Review Comment: Add some description about can not recognize hive keywords. ########## 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))); Review Comment: I think we should close the `StringReader` if the parse ends. ########## flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/cli/parser/SqlCommandParser.java: ########## @@ -16,17 +16,18 @@ * limitations under the License. */ -package org.apache.flink.table.client.cli; +package org.apache.flink.table.client.cli.parser; import org.apache.flink.annotation.Internal; +import org.apache.flink.sql.parser.impl.FlinkSqlParserImplConstants; import org.apache.flink.table.client.gateway.SqlExecutionException; import org.apache.flink.table.operations.Operation; import java.util.Optional; -/** SqlClient command parser. */ +/** SqlClient command parser. Extends FlinkSqlParserImplConstants to use defined token kinds. */ @Internal -interface SqlCommandParser { +public interface SqlCommandParser extends FlinkSqlParserImplConstants { Review Comment: Why this interface extends `FlinkSqlParserImplConstants`? I think ClientParser extends this is enough. ########## flink-table/flink-sql-client/pom.xml: ########## @@ -511,6 +511,12 @@ under the License. <scope>provided</scope> </dependency> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-sql-parser</artifactId> + <version>${project.version}</version> + </dependency> Review Comment: Take a look at the shade plugin below. We can do as table-planner module append the parser class into the final jar. But I think we don't need all classes in the parser jar and we can filter out if it is not needed. ########## 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: Use a TestSpec rather than Tuple2 here. TestSpec has better semantic. BTW, when user inputs `quit`, the terminal should contine reading from the input. ########## 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()); Review Comment: I think we can condition can be `token != null` ########## 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: Can we use itearor model here? We can reduce loop twice to once -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
