forked from lakesoul-io/LakeSoul
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[Flink] Fix sql submit main entry (lakesoul-io#494)
* fix sql submit entry Signed-off-by: chenxu <[email protected]> * add arrow sink example Signed-off-by: chenxu <[email protected]> * fix build Signed-off-by: chenxu <[email protected]> * fix clippy action Signed-off-by: chenxu <[email protected]> * fix clippy deprecation errors Signed-off-by: chenxu <[email protected]> --------- Signed-off-by: chenxu <[email protected]> Co-authored-by: chenxu <[email protected]>
- Loading branch information
1 parent
9958faf
commit 2635e2b
Showing
22 changed files
with
1,038 additions
and
809 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
161 changes: 90 additions & 71 deletions
161
lakesoul-flink/src/main/java/org/apache/flink/lakesoul/entry/sql/flink/ExecuteSql.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,100 +1,119 @@ | ||
// THIS FILE IS OVERWRITE BY THE zhp8341/FLINK-STREAMING-PLATFORM-WEB PROJECT, UNDER MIT LICENSE. | ||
|
||
// SPDX-FileCopyrightText: 2023 LakeSoul Contributors | ||
// | ||
// SPDX-License-Identifier: Apache-2.0 | ||
// This file is modified from https://github.com/apache/flink-kubernetes-operator/blob/main/examples/flink-sql-runner-example/src/main/java/org/apache/flink/examples/SqlRunner.java | ||
|
||
package org.apache.flink.lakesoul.entry.sql.flink; | ||
|
||
import org.apache.commons.lang3.StringUtils; | ||
import org.apache.flink.api.common.RuntimeExecutionMode; | ||
import org.apache.flink.table.api.TableEnvironment; | ||
import org.apache.flink.table.api.internal.TableEnvironmentInternal; | ||
import org.apache.flink.table.delegation.Parser; | ||
import org.apache.flink.table.operations.BeginStatementSetOperation; | ||
import org.apache.flink.table.operations.ModifyOperation; | ||
import org.apache.flink.table.operations.Operation; | ||
import org.apache.flink.table.operations.command.SetOperation; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
import org.slf4j.helpers.MessageFormatter; | ||
|
||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.concurrent.ExecutionException; | ||
import java.util.regex.Pattern; | ||
|
||
public class ExecuteSql { | ||
public static void exeSql(List<String> sqlList, TableEnvironment tEnv) { | ||
Parser parser = ((TableEnvironmentInternal) tEnv).getParser(); | ||
List<ModifyOperation> modifyOperationList = new ArrayList<>(); | ||
import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; | ||
|
||
for (String stmtOri : sqlList) { | ||
String stmt = trimBlank(stmtOri); | ||
Operation operation = parser.parse(stmt).get(0); | ||
public class ExecuteSql { | ||
|
||
// flink version 1.14.5 | ||
switch (operation.getClass().getSimpleName()) { | ||
case "PlannerQueryOperation": | ||
case "ShowTablesOperation": | ||
case "ShowCatalogsOperation": | ||
case "ShowCreateTableOperation": | ||
case "ShowCurrentCatalogOperation": | ||
case "ShowCurrentDatabaseOperation": | ||
case "ShowDatabasesOperation": | ||
case "ShowFunctionsOperation": | ||
case "ShowModulesOperation": | ||
case "ShowPartitionsOperation": | ||
case "ShowViewsOperation": | ||
case "ExplainOperation": | ||
case "DescribeTableOperation": | ||
tEnv.executeSql(stmt).print(); | ||
break; | ||
private static final Logger LOG = LoggerFactory.getLogger(ExecuteSql.class); | ||
|
||
//set | ||
case "SetOperation": | ||
SetOperation setOperation = (SetOperation) operation; | ||
Configurations.setSingleConfiguration(tEnv, setOperation.getKey().get(), | ||
setOperation.getValue().get()); | ||
break; | ||
private static final String STATEMENT_DELIMITER = ";"; // a statement should end with `;` | ||
private static final String LINE_DELIMITER = "\n"; | ||
|
||
case "BeginStatementSetOperation": | ||
case "EndStatementSetOperation": | ||
break; | ||
private static final String COMMENT_PATTERN = "(--.*)|(((\\/\\*)+?[\\w\\W]+?(\\*\\/)+))"; | ||
|
||
case "DropTableOperation": | ||
case "DropCatalogFunctionOperation": | ||
case "DropTempSystemFunctionOperation": | ||
case "DropCatalogOperation": | ||
case "DropDatabaseOperation": | ||
case "DropViewOperation": | ||
case "CreateTableOperation": | ||
case "CreateViewOperation": | ||
case "CreateDatabaseOperation": | ||
case "CreateCatalogOperation": | ||
case "CreateTableASOperation": | ||
case "CreateCatalogFunctionOperation": | ||
case "CreateTempSystemFunctionOperation": | ||
case "AlterTableOperation": | ||
case "AlterViewOperation": | ||
case "AlterDatabaseOperation": | ||
case "AlterCatalogFunctionOperation": | ||
case "UseCatalogOperation": | ||
case "UseDatabaseOperation": | ||
case "LoadModuleOperation": | ||
case "UnloadModuleOperation": | ||
case "NopOperation": { | ||
((TableEnvironmentInternal) tEnv).executeInternal(operation).print(); | ||
break; | ||
public static void executeSqlFileContent(String script, TableEnvironment tableEnv) | ||
throws ExecutionException, InterruptedException { | ||
List<String> statements = parseStatements(script); | ||
Parser parser = ((TableEnvironmentInternal) tableEnv).getParser(); | ||
for (String statement : statements) { | ||
Operation operation = parser.parse(statement).get(0); | ||
if (operation instanceof SetOperation) { | ||
SetOperation setOperation = (SetOperation) operation; | ||
if (setOperation.getKey().isPresent() && setOperation.getValue().isPresent()) { | ||
System.out.println(MessageFormatter.format("\n======Setting config: {}={}", | ||
setOperation.getKey().get(), | ||
setOperation.getValue().get()).getMessage()); | ||
tableEnv.getConfig().getConfiguration() | ||
.setString(setOperation.getKey().get(), setOperation.getValue().get()); | ||
} else if (setOperation.getKey().isPresent()) { | ||
String value = tableEnv.getConfig().getConfiguration().getString(setOperation.getKey().get(), ""); | ||
System.out.println(MessageFormatter.format("Config {}={}", | ||
setOperation.getKey().get(), value).getMessage()); | ||
} else { | ||
System.out.println(MessageFormatter.format("All configs: {}", | ||
tableEnv.getConfig().getConfiguration()).getMessage()); | ||
} | ||
} else if (operation instanceof ModifyOperation || operation instanceof BeginStatementSetOperation) { | ||
System.out.println(MessageFormatter.format("\n======Executing insertion:\n{}", statement).getMessage()); | ||
// execute insertion and do not wait for results for stream mode | ||
if (tableEnv.getConfig().get(RUNTIME_MODE) == RuntimeExecutionMode.BATCH) { | ||
tableEnv.executeSql(statement).await(); | ||
} else { | ||
tableEnv.executeSql(statement); | ||
} | ||
// insert | ||
case "ModifyOperation": | ||
modifyOperationList.add((ModifyOperation) operation); | ||
break; | ||
default: | ||
throw new RuntimeException("not support sql=" + stmt); | ||
} else { | ||
// for all show/select/alter/create catalog/use, etc. statements | ||
// execute and print results | ||
System.out.println(MessageFormatter.format("\n======Executing:\n{}", statement).getMessage()); | ||
tableEnv.executeSql(statement).print(); | ||
} | ||
} | ||
int modifyOperationListLength = modifyOperationList.size(); | ||
if (modifyOperationListLength == 0) { | ||
return; | ||
} | ||
|
||
public static List<String> parseStatements(String script) { | ||
String formatted = | ||
formatSqlFile(script) | ||
.replaceAll(COMMENT_PATTERN, ""); | ||
|
||
List<String> statements = new ArrayList<String>(); | ||
|
||
StringBuilder current = null; | ||
boolean statementSet = false; | ||
for (String line : formatted.split("\n")) { | ||
String trimmed = line.trim(); | ||
if (StringUtils.isBlank(trimmed)) { | ||
continue; | ||
} | ||
if (current == null) { | ||
current = new StringBuilder(); | ||
} | ||
if (trimmed.startsWith("EXECUTE STATEMENT SET")) { | ||
statementSet = true; | ||
} | ||
current.append(trimmed); | ||
current.append("\n"); | ||
if (trimmed.endsWith(STATEMENT_DELIMITER)) { | ||
if (!statementSet || trimmed.equals("END;")) { | ||
statements.add(current.toString()); | ||
current = null; | ||
statementSet = false; | ||
} | ||
} | ||
} | ||
((TableEnvironmentInternal) tEnv).executeInternal(modifyOperationList).print(); | ||
return statements; | ||
} | ||
|
||
private static String trimBlank(String str) { | ||
return str.replace("\\n", " ").replaceAll("\\s+", " ").trim(); | ||
public static String formatSqlFile(String content) { | ||
String trimmed = content.trim(); | ||
StringBuilder formatted = new StringBuilder(); | ||
formatted.append(trimmed); | ||
if (!trimmed.endsWith(STATEMENT_DELIMITER)) { | ||
formatted.append(STATEMENT_DELIMITER); | ||
} | ||
formatted.append(LINE_DELIMITER); | ||
return formatted.toString(); | ||
} | ||
} |
Oops, something went wrong.