From dd201d80e6c3093416531a0901742e7524e68ac8 Mon Sep 17 00:00:00 2001 From: tyrantlucifer Date: Sun, 9 Apr 2023 02:26:48 +0800 Subject: [PATCH 1/3] [Improve][Starter] Optimize code structure & remove redundant code --- .../seatunnel-flink-starter/pom.xml | 1 + .../seatunnel-flink-13-starter/pom.xml | 14 +- .../seatunnel-flink-15-starter/pom.xml | 6 + .../core/starter/flink/FlinkStarter.java | 97 ----- .../core/starter/flink/SeaTunnelFlink.java | 36 -- .../starter/flink/args/FlinkCommandArgs.java | 138 ------- .../command/FlinkConfValidateCommand.java | 44 --- .../command/FlinkTaskExecuteCommand.java | 66 ---- .../FlinkAbstractPluginExecuteProcessor.java | 91 ----- .../flink/execution/FlinkExecution.java | 207 ---------- .../execution/SourceExecuteProcessor.java | 147 ------- .../execution/TransformExecuteProcessor.java | 138 ------- .../starter/flink/utils/ConfigKeyName.java | 49 --- .../starter/flink/utils/EnvironmentUtil.java | 120 ------ .../core/starter/flink/utils/TableUtil.java | 50 --- .../seatunnel-flink-starter-common/pom.xml | 73 ++++ .../core/starter/flink/FlinkStarter.java | 0 .../core/starter/flink/SeaTunnelFlink.java | 0 .../starter/flink/args/FlinkCommandArgs.java | 0 .../command/FlinkConfValidateCommand.java | 0 .../command/FlinkTaskExecuteCommand.java | 0 .../FlinkAbstractPluginExecuteProcessor.java | 0 .../flink/execution/FlinkExecution.java | 0 .../execution/FlinkRuntimeEnvironment.java | 0 .../flink/execution/SinkExecuteProcessor.java | 0 .../execution/SourceExecuteProcessor.java | 0 .../execution/TransformExecuteProcessor.java | 0 .../starter/flink/utils/ConfigKeyName.java | 0 .../starter/flink/utils/EnvironmentUtil.java | 0 .../core/starter/flink/utils/TableUtil.java | 0 .../seatunnel-spark-starter/pom.xml | 1 + .../seatunnel-spark-2-starter/pom.xml | 12 + .../seatunnel-spark-3-starter/pom.xml | 18 +- .../core/starter/spark/SeaTunnelSpark.java | 37 -- .../core/starter/spark/SparkStarter.java | 361 ------------------ .../starter/spark/args/SparkCommandArgs.java | 89 ----- .../command/SparkConfValidateCommand.java | 44 --- .../command/SparkTaskExecuteCommand.java | 66 ---- .../execution/SourceExecuteProcessor.java | 111 ------ .../SparkAbstractPluginExecuteProcessor.java | 78 ---- .../spark/execution/SparkExecution.java | 81 ---- .../execution/SparkRuntimeEnvironment.java | 179 --------- .../execution/TransformExecuteProcessor.java | 173 --------- .../seatunnel-spark-starter-common/pom.xml | 69 ++++ .../core/starter/spark/SeaTunnelSpark.java | 0 .../core/starter/spark/SparkStarter.java | 0 .../starter/spark/args/SparkCommandArgs.java | 0 .../command/SparkConfValidateCommand.java | 0 .../command/SparkTaskExecuteCommand.java | 0 .../spark/execution/SinkExecuteProcessor.java | 0 .../execution/SourceExecuteProcessor.java | 0 .../SparkAbstractPluginExecuteProcessor.java | 0 .../spark/execution/SparkExecution.java | 0 .../execution/SparkRuntimeEnvironment.java | 0 .../execution/TransformExecuteProcessor.java | 0 .../example/flink/v2/SeaTunnelApiExample.java | 1 - 56 files changed, 187 insertions(+), 2410 deletions(-) delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/args/FlinkCommandArgs.java delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkConfValidateCommand.java delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkTaskExecuteCommand.java delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/ConfigKeyName.java delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/EnvironmentUtil.java delete mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/TableUtil.java create mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/pom.xml rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/args/FlinkCommandArgs.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkConfValidateCommand.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkTaskExecuteCommand.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-15-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-15-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/utils/ConfigKeyName.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/utils/EnvironmentUtil.java (100%) rename seatunnel-core/seatunnel-flink-starter/{seatunnel-flink-13-starter => seatunnel-flink-starter-common}/src/main/java/org/apache/seatunnel/core/starter/flink/utils/TableUtil.java (100%) delete mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java delete mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java delete mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/args/SparkCommandArgs.java delete mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkConfValidateCommand.java delete mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkTaskExecuteCommand.java delete mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java delete mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java delete mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java delete mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java delete mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java create mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/pom.xml rename seatunnel-core/seatunnel-spark-starter/{seatunnel-spark-2-starter => seatunnel-spark-starter-common}/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java (100%) rename seatunnel-core/seatunnel-spark-starter/{seatunnel-spark-2-starter => seatunnel-spark-starter-common}/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java (100%) rename seatunnel-core/seatunnel-spark-starter/{seatunnel-spark-2-starter => seatunnel-spark-starter-common}/src/main/java/org/apache/seatunnel/core/starter/spark/args/SparkCommandArgs.java (100%) rename seatunnel-core/seatunnel-spark-starter/{seatunnel-spark-2-starter => seatunnel-spark-starter-common}/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkConfValidateCommand.java (100%) rename seatunnel-core/seatunnel-spark-starter/{seatunnel-spark-2-starter => seatunnel-spark-starter-common}/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkTaskExecuteCommand.java (100%) rename seatunnel-core/seatunnel-spark-starter/{seatunnel-spark-3-starter => seatunnel-spark-starter-common}/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java (100%) rename seatunnel-core/seatunnel-spark-starter/{seatunnel-spark-2-starter => seatunnel-spark-starter-common}/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java (100%) rename seatunnel-core/seatunnel-spark-starter/{seatunnel-spark-2-starter => seatunnel-spark-starter-common}/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java (100%) rename seatunnel-core/seatunnel-spark-starter/{seatunnel-spark-2-starter => seatunnel-spark-starter-common}/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java (100%) rename seatunnel-core/seatunnel-spark-starter/{seatunnel-spark-2-starter => seatunnel-spark-starter-common}/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java (100%) rename seatunnel-core/seatunnel-spark-starter/{seatunnel-spark-2-starter => seatunnel-spark-starter-common}/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java (100%) diff --git a/seatunnel-core/seatunnel-flink-starter/pom.xml b/seatunnel-core/seatunnel-flink-starter/pom.xml index 969397088bc..3b4f5442c32 100644 --- a/seatunnel-core/seatunnel-flink-starter/pom.xml +++ b/seatunnel-core/seatunnel-flink-starter/pom.xml @@ -35,6 +35,7 @@ seatunnel-flink-13-starter seatunnel-flink-15-starter + seatunnel-flink-starter-common diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/pom.xml b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/pom.xml index c41e48bc188..3eb93c5c00b 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/pom.xml +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/pom.xml @@ -31,6 +31,18 @@ + + org.apache.seatunnel + seatunnel-flink-starter-common + ${project.version} + + + * + * + + + + org.apache.seatunnel @@ -66,7 +78,7 @@ org.apache.flink flink-statebackend-rocksdb_${scala.binary.version} ${flink.1.13.6.version} - provided + ${flink.scope} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/pom.xml b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/pom.xml index 97fb280c68d..c92ae6c964e 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/pom.xml +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/pom.xml @@ -31,6 +31,12 @@ + + org.apache.seatunnel + seatunnel-flink-starter-common + ${revision} + + org.apache.seatunnel seatunnel-translation-flink-15 diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java deleted file mode 100644 index 2f9021c68bd..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink; - -import org.apache.seatunnel.common.config.Common; -import org.apache.seatunnel.core.starter.Starter; -import org.apache.seatunnel.core.starter.enums.EngineType; -import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; -import org.apache.seatunnel.core.starter.utils.CommandLineUtils; - -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; - -/** The SeaTunnel flink starter, used to generate the final flink job execute command. */ -public class FlinkStarter implements Starter { - private static final String APP_NAME = SeaTunnelFlink.class.getName(); - public static final String APP_JAR_NAME = EngineType.FLINK15.getStarterJarName(); - public static final String SHELL_NAME = EngineType.FLINK15.getStarterShellName(); - private final FlinkCommandArgs flinkCommandArgs; - private final String appJar; - - FlinkStarter(String[] args) { - this.flinkCommandArgs = - CommandLineUtils.parse(args, new FlinkCommandArgs(), SHELL_NAME, true); - // set the deployment mode, used to get the job jar path. - Common.setDeployMode(flinkCommandArgs.getDeployMode()); - Common.setStarter(true); - this.appJar = Common.appStarterDir().resolve(APP_JAR_NAME).toString(); - } - - @SuppressWarnings("checkstyle:RegexpSingleline") - public static void main(String[] args) { - FlinkStarter flinkStarter = new FlinkStarter(args); - System.out.println(String.join(" ", flinkStarter.buildCommands())); - } - - @Override - public List buildCommands() { - List command = new ArrayList<>(); - // set start command - command.add("${FLINK_HOME}/bin/flink"); - // set deploy mode, run or run-application - command.add(flinkCommandArgs.getDeployMode().getDeployMode()); - // set submitted target master - if (flinkCommandArgs.getMasterType() != null) { - command.add("--target"); - command.add(flinkCommandArgs.getMasterType().getMaster()); - } - // set flink original parameters - command.addAll(flinkCommandArgs.getOriginalParameters()); - // set main class name - command.add("-c"); - command.add(APP_NAME); - // set main jar name - command.add(appJar); - // set config file path - command.add("--config"); - command.add(flinkCommandArgs.getConfigFile()); - // set check config flag - if (flinkCommandArgs.isCheckConfig()) { - command.add("--check"); - } - // set job name - command.add("--name"); - command.add(flinkCommandArgs.getJobName()); - // set encryption - if (flinkCommandArgs.isEncrypt()) { - command.add("--encrypt"); - } - // set decryption - if (flinkCommandArgs.isDecrypt()) { - command.add("--decrypt"); - } - // set extra system properties - flinkCommandArgs.getVariables().stream() - .filter(Objects::nonNull) - .map(String::trim) - .forEach(variable -> command.add("-D" + variable)); - return command; - } -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java deleted file mode 100644 index 1595da686a7..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink; - -import org.apache.seatunnel.core.starter.SeaTunnel; -import org.apache.seatunnel.core.starter.enums.EngineType; -import org.apache.seatunnel.core.starter.exception.CommandException; -import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; -import org.apache.seatunnel.core.starter.utils.CommandLineUtils; - -public class SeaTunnelFlink { - public static void main(String[] args) throws CommandException { - FlinkCommandArgs flinkCommandArgs = - CommandLineUtils.parse( - args, - new FlinkCommandArgs(), - EngineType.FLINK15.getStarterShellName(), - true); - SeaTunnel.run(flinkCommandArgs.buildCommand()); - } -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/args/FlinkCommandArgs.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/args/FlinkCommandArgs.java deleted file mode 100644 index ff098b9df97..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/args/FlinkCommandArgs.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink.args; - -import org.apache.seatunnel.common.config.Common; -import org.apache.seatunnel.common.config.DeployMode; -import org.apache.seatunnel.core.starter.command.AbstractCommandArgs; -import org.apache.seatunnel.core.starter.command.Command; -import org.apache.seatunnel.core.starter.command.ConfDecryptCommand; -import org.apache.seatunnel.core.starter.command.ConfEncryptCommand; -import org.apache.seatunnel.core.starter.enums.MasterType; -import org.apache.seatunnel.core.starter.flink.command.FlinkConfValidateCommand; -import org.apache.seatunnel.core.starter.flink.command.FlinkTaskExecuteCommand; - -import com.beust.jcommander.IStringConverter; -import com.beust.jcommander.Parameter; -import lombok.Data; -import lombok.EqualsAndHashCode; - -import java.util.ArrayList; -import java.util.List; - -@EqualsAndHashCode(callSuper = true) -@Data -public class FlinkCommandArgs extends AbstractCommandArgs { - - @Parameter( - names = {"-e", "--deploy-mode"}, - converter = FlinkDeployModeConverter.class, - description = "Flink job deploy mode, support [run, run-application]") - private DeployMode deployMode = DeployMode.RUN; - - @Parameter( - names = {"--master", "--target"}, - converter = FlinkMasterTargetConverter.class, - description = - "Flink job submitted target master, support [local, remote, yarn-session, yarn-per-job, " - + "kubernetes-session, yarn-application, kubernetes-application]") - private MasterType masterType; - - @Override - public Command buildCommand() { - Common.setDeployMode(getDeployMode()); - if (checkConfig) { - return new FlinkConfValidateCommand(this); - } - if (encrypt) { - return new ConfEncryptCommand(this); - } - if (decrypt) { - return new ConfDecryptCommand(this); - } - return new FlinkTaskExecuteCommand(this); - } - - @Override - public String toString() { - return "FlinkCommandArgs{" - + "deployMode=" - + deployMode - + ", masterType=" - + masterType - + ", configFile='" - + configFile - + '\'' - + ", variables=" - + variables - + ", jobName='" - + jobName - + '\'' - + ", originalParameters=" - + originalParameters - + '}'; - } - - public static class FlinkMasterTargetConverter implements IStringConverter { - private static final List MASTER_TYPE_LIST = new ArrayList<>(); - - static { - MASTER_TYPE_LIST.add(MasterType.LOCAL); - MASTER_TYPE_LIST.add(MasterType.REMOTE); - MASTER_TYPE_LIST.add(MasterType.YARN_SESSION); - MASTER_TYPE_LIST.add(MasterType.YARN_PER_JOB); - MASTER_TYPE_LIST.add(MasterType.KUBERNETES_SESSION); - MASTER_TYPE_LIST.add(MasterType.YARN_APPLICATION); - MASTER_TYPE_LIST.add(MasterType.KUBERNETES_APPLICATION); - } - - @Override - public MasterType convert(String value) { - MasterType masterType = MasterType.valueOf(value.toUpperCase().replaceAll("-", "_")); - if (MASTER_TYPE_LIST.contains(masterType)) { - return masterType; - } else { - throw new IllegalArgumentException( - "SeaTunnel job on flink engine submitted target only " - + "support these options: [local, remote, yarn-session, yarn-per-job, kubernetes-session, " - + "yarn-application, kubernetes-application]"); - } - } - } - - public static class FlinkDeployModeConverter implements IStringConverter { - private static final List DEPLOY_MODE_TYPE_LIST = new ArrayList<>(); - - static { - DEPLOY_MODE_TYPE_LIST.add(DeployMode.RUN); - DEPLOY_MODE_TYPE_LIST.add(DeployMode.RUN_APPLICATION); - } - - @Override - public DeployMode convert(String value) { - DeployMode deployMode = DeployMode.valueOf(value.toUpperCase().replaceAll("-", "_")); - if (DEPLOY_MODE_TYPE_LIST.contains(deployMode)) { - return deployMode; - } else { - throw new IllegalArgumentException( - "SeaTunnel job on flink engine deploy mode only " - + "support these options: [run, run-application]"); - } - } - } -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkConfValidateCommand.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkConfValidateCommand.java deleted file mode 100644 index 78921e69802..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkConfValidateCommand.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink.command; - -import org.apache.seatunnel.core.starter.command.Command; -import org.apache.seatunnel.core.starter.exception.ConfigCheckException; -import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; -import org.apache.seatunnel.core.starter.utils.FileUtils; - -import lombok.extern.slf4j.Slf4j; - -import java.nio.file.Path; - -/** Use to validate the configuration of the SeaTunnel API. */ -@Slf4j -public class FlinkConfValidateCommand implements Command { - - private final FlinkCommandArgs flinkCommandArgs; - - public FlinkConfValidateCommand(FlinkCommandArgs flinkCommandArgs) { - this.flinkCommandArgs = flinkCommandArgs; - } - - @Override - public void execute() throws ConfigCheckException { - Path configPath = FileUtils.getConfigPath(flinkCommandArgs); - // TODO: validate the config by new api - } -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkTaskExecuteCommand.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkTaskExecuteCommand.java deleted file mode 100644 index f8539af752f..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkTaskExecuteCommand.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink.command; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigUtil; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory; - -import org.apache.seatunnel.common.Constants; -import org.apache.seatunnel.core.starter.command.Command; -import org.apache.seatunnel.core.starter.exception.CommandExecuteException; -import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; -import org.apache.seatunnel.core.starter.flink.execution.FlinkExecution; -import org.apache.seatunnel.core.starter.utils.ConfigBuilder; -import org.apache.seatunnel.core.starter.utils.FileUtils; - -import lombok.extern.slf4j.Slf4j; - -import java.nio.file.Path; - -import static org.apache.seatunnel.core.starter.utils.FileUtils.checkConfigExist; - -@Slf4j -public class FlinkTaskExecuteCommand implements Command { - - private final FlinkCommandArgs flinkCommandArgs; - - public FlinkTaskExecuteCommand(FlinkCommandArgs flinkCommandArgs) { - this.flinkCommandArgs = flinkCommandArgs; - } - - @Override - public void execute() throws CommandExecuteException { - Path configFile = FileUtils.getConfigPath(flinkCommandArgs); - checkConfigExist(configFile); - Config config = ConfigBuilder.of(configFile); - // if user specified job name using command line arguments, override config option - if (!flinkCommandArgs.getJobName().equals(Constants.LOGO)) { - config = - config.withValue( - ConfigUtil.joinPath("env", "job.name"), - ConfigValueFactory.fromAnyRef(flinkCommandArgs.getJobName())); - } - FlinkExecution seaTunnelTaskExecution = new FlinkExecution(config); - try { - seaTunnelTaskExecution.execute(); - } catch (Exception e) { - throw new CommandExecuteException("Flink job executed failed", e); - } - } -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java deleted file mode 100644 index e9d36ba068e..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink.execution; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.common.utils.ReflectionUtils; -import org.apache.seatunnel.core.starter.execution.PluginExecuteProcessor; -import org.apache.seatunnel.core.starter.flink.utils.TableUtil; - -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.types.Row; - -import java.net.URL; -import java.net.URLClassLoader; -import java.util.List; -import java.util.Optional; -import java.util.function.BiConsumer; - -public abstract class FlinkAbstractPluginExecuteProcessor - implements PluginExecuteProcessor, FlinkRuntimeEnvironment> { - protected static final String ENGINE_TYPE = "seatunnel"; - protected static final String PLUGIN_NAME = "plugin_name"; - protected static final String SOURCE_TABLE_NAME = "source_table_name"; - - protected static final BiConsumer ADD_URL_TO_CLASSLOADER = - (classLoader, url) -> { - if (classLoader.getClass().getName().endsWith("SafetyNetWrapperClassLoader")) { - URLClassLoader c = - (URLClassLoader) ReflectionUtils.getField(classLoader, "inner").get(); - ReflectionUtils.invoke(c, "addURL", url); - } else if (classLoader instanceof URLClassLoader) { - ReflectionUtils.invoke(classLoader, "addURL", url); - } else { - throw new RuntimeException( - "Unsupported classloader: " + classLoader.getClass().getName()); - } - }; - - protected FlinkRuntimeEnvironment flinkRuntimeEnvironment; - protected final List pluginConfigs; - protected JobContext jobContext; - protected final List plugins; - - protected FlinkAbstractPluginExecuteProcessor( - List jarPaths, List pluginConfigs, JobContext jobContext) { - this.pluginConfigs = pluginConfigs; - this.jobContext = jobContext; - this.plugins = initializePlugins(jarPaths, pluginConfigs); - } - - @Override - public void setRuntimeEnvironment(FlinkRuntimeEnvironment flinkRuntimeEnvironment) { - this.flinkRuntimeEnvironment = flinkRuntimeEnvironment; - } - - protected Optional> fromSourceTable(Config pluginConfig) { - if (pluginConfig.hasPath(SOURCE_TABLE_NAME)) { - StreamTableEnvironment tableEnvironment = - flinkRuntimeEnvironment.getStreamTableEnvironment(); - Table table = tableEnvironment.from(pluginConfig.getString(SOURCE_TABLE_NAME)); - return Optional.ofNullable(TableUtil.tableToDataStream(tableEnvironment, table, true)); - } - return Optional.empty(); - } - - protected void registerResultTable(Config pluginConfig, DataStream dataStream) { - flinkRuntimeEnvironment.registerResultTable(pluginConfig, dataStream); - } - - protected abstract List initializePlugins( - List jarPaths, List pluginConfigs); -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java deleted file mode 100644 index a3282cc4a1e..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java +++ /dev/null @@ -1,207 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink.execution; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigUtil; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory; - -import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.api.env.EnvCommonOptions; -import org.apache.seatunnel.common.Constants; -import org.apache.seatunnel.common.config.Common; -import org.apache.seatunnel.common.config.TypesafeConfigUtils; -import org.apache.seatunnel.common.utils.SeaTunnelException; -import org.apache.seatunnel.core.starter.exception.TaskExecuteException; -import org.apache.seatunnel.core.starter.execution.PluginExecuteProcessor; -import org.apache.seatunnel.core.starter.execution.RuntimeEnvironment; -import org.apache.seatunnel.core.starter.execution.TaskExecution; -import org.apache.seatunnel.core.starter.flink.FlinkStarter; - -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.types.Row; - -import lombok.extern.slf4j.Slf4j; - -import java.io.File; -import java.net.MalformedURLException; -import java.net.URL; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -/** Used to execute a SeaTunnelTask. */ -@Slf4j -public class FlinkExecution implements TaskExecution { - private final FlinkRuntimeEnvironment flinkRuntimeEnvironment; - private final PluginExecuteProcessor, FlinkRuntimeEnvironment> - sourcePluginExecuteProcessor; - private final PluginExecuteProcessor, FlinkRuntimeEnvironment> - transformPluginExecuteProcessor; - private final PluginExecuteProcessor, FlinkRuntimeEnvironment> - sinkPluginExecuteProcessor; - private final List jarPaths; - - public FlinkExecution(Config config) { - try { - jarPaths = - new ArrayList<>( - Collections.singletonList( - new File( - Common.appStarterDir() - .resolve(FlinkStarter.APP_JAR_NAME) - .toString()) - .toURI() - .toURL())); - } catch (MalformedURLException e) { - throw new SeaTunnelException("load flink starter error.", e); - } - registerPlugin(config.getConfig("env")); - JobContext jobContext = new JobContext(); - jobContext.setJobMode(RuntimeEnvironment.getJobMode(config)); - - this.sourcePluginExecuteProcessor = - new SourceExecuteProcessor( - jarPaths, config.getConfigList(Constants.SOURCE), jobContext); - this.transformPluginExecuteProcessor = - new TransformExecuteProcessor( - jarPaths, - TypesafeConfigUtils.getConfigList( - config, Constants.TRANSFORM, Collections.emptyList()), - jobContext); - this.sinkPluginExecuteProcessor = - new SinkExecuteProcessor( - jarPaths, config.getConfigList(Constants.SINK), jobContext); - - this.flinkRuntimeEnvironment = - FlinkRuntimeEnvironment.getInstance(this.registerPlugin(config, jarPaths)); - - this.sourcePluginExecuteProcessor.setRuntimeEnvironment(flinkRuntimeEnvironment); - this.transformPluginExecuteProcessor.setRuntimeEnvironment(flinkRuntimeEnvironment); - this.sinkPluginExecuteProcessor.setRuntimeEnvironment(flinkRuntimeEnvironment); - } - - @Override - public void execute() throws TaskExecuteException { - List> dataStreams = new ArrayList<>(); - dataStreams = sourcePluginExecuteProcessor.execute(dataStreams); - dataStreams = transformPluginExecuteProcessor.execute(dataStreams); - sinkPluginExecuteProcessor.execute(dataStreams); - log.info( - "Flink Execution Plan: {}", - flinkRuntimeEnvironment.getStreamExecutionEnvironment().getExecutionPlan()); - log.info("Flink job name: {}", flinkRuntimeEnvironment.getJobName()); - try { - flinkRuntimeEnvironment - .getStreamExecutionEnvironment() - .execute(flinkRuntimeEnvironment.getJobName()); - } catch (Exception e) { - throw new TaskExecuteException("Execute Flink job error", e); - } - } - - private void registerPlugin(Config envConfig) { - List thirdPartyJars = new ArrayList<>(); - if (envConfig.hasPath(EnvCommonOptions.JARS.key())) { - thirdPartyJars = - new ArrayList<>( - Common.getThirdPartyJars( - envConfig.getString(EnvCommonOptions.JARS.key()))); - } - thirdPartyJars.addAll(Common.getPluginsJarDependencies()); - List jarDependencies = - Stream.concat(thirdPartyJars.stream(), Common.getLibJars().stream()) - .map(Path::toUri) - .map( - uri -> { - try { - return uri.toURL(); - } catch (MalformedURLException e) { - throw new RuntimeException( - "the uri of jar illegal:" + uri, e); - } - }) - .collect(Collectors.toList()); - jarDependencies.forEach( - url -> - FlinkAbstractPluginExecuteProcessor.ADD_URL_TO_CLASSLOADER.accept( - Thread.currentThread().getContextClassLoader(), url)); - jarPaths.addAll(jarDependencies); - } - - private Config registerPlugin(Config config, List jars) { - config = - this.injectJarsToConfig( - config, ConfigUtil.joinPath("env", "pipeline", "jars"), jars); - return this.injectJarsToConfig( - config, ConfigUtil.joinPath("env", "pipeline", "classpaths"), jars); - } - - private Config injectJarsToConfig(Config config, String path, List jars) { - List validJars = new ArrayList<>(); - for (URL jarUrl : jars) { - if (new File(jarUrl.getFile()).exists()) { - validJars.add(jarUrl); - log.info("Inject jar to config: {}", jarUrl); - } else { - log.warn("Remove invalid jar when inject jars into config: {}", jarUrl); - } - } - - if (config.hasPath(path)) { - Set paths = - Arrays.stream(config.getString(path).split(";")) - .map( - uri -> { - try { - return new URL(uri); - } catch (MalformedURLException e) { - throw new RuntimeException( - "the uri of jar illegal:" + uri, e); - } - }) - .collect(Collectors.toSet()); - paths.addAll(validJars); - - config = - config.withValue( - path, - ConfigValueFactory.fromAnyRef( - paths.stream() - .map(URL::toString) - .distinct() - .collect(Collectors.joining(";")))); - - } else { - config = - config.withValue( - path, - ConfigValueFactory.fromAnyRef( - validJars.stream() - .map(URL::toString) - .distinct() - .collect(Collectors.joining(";")))); - } - return config; - } -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java deleted file mode 100644 index a3897a526e9..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink.execution; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.common.CommonOptions; -import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.source.SupportCoordinate; -import org.apache.seatunnel.common.constants.JobMode; -import org.apache.seatunnel.core.starter.enums.PluginType; -import org.apache.seatunnel.plugin.discovery.PluginIdentifier; -import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSourcePluginDiscovery; -import org.apache.seatunnel.translation.flink.source.BaseSeaTunnelSourceFunction; -import org.apache.seatunnel.translation.flink.source.SeaTunnelCoordinatedSource; -import org.apache.seatunnel.translation.flink.source.SeaTunnelParallelSource; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.source.ParallelSourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; -import org.apache.flink.types.Row; - -import com.google.common.collect.Lists; - -import java.net.URL; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -public class SourceExecuteProcessor extends FlinkAbstractPluginExecuteProcessor { - private static final String PLUGIN_TYPE = PluginType.SOURCE.getType(); - - public SourceExecuteProcessor( - List jarPaths, List sourceConfigs, JobContext jobContext) { - super(jarPaths, sourceConfigs, jobContext); - } - - @Override - public List> execute(List> upstreamDataStreams) { - StreamExecutionEnvironment executionEnvironment = - flinkRuntimeEnvironment.getStreamExecutionEnvironment(); - List> sources = new ArrayList<>(); - for (int i = 0; i < plugins.size(); i++) { - SeaTunnelSource internalSource = plugins.get(i); - BaseSeaTunnelSourceFunction sourceFunction; - if (internalSource instanceof SupportCoordinate) { - sourceFunction = new SeaTunnelCoordinatedSource(internalSource); - } else { - sourceFunction = new SeaTunnelParallelSource(internalSource); - } - DataStreamSource sourceStream = - addSource( - executionEnvironment, - sourceFunction, - "SeaTunnel " + internalSource.getClass().getSimpleName(), - internalSource.getBoundedness() - == org.apache.seatunnel.api.source.Boundedness.BOUNDED); - Config pluginConfig = pluginConfigs.get(i); - if (pluginConfig.hasPath(CommonOptions.PARALLELISM.key())) { - int parallelism = pluginConfig.getInt(CommonOptions.PARALLELISM.key()); - sourceStream.setParallelism(parallelism); - } - registerResultTable(pluginConfig, sourceStream); - sources.add(sourceStream); - } - return sources; - } - - private DataStreamSource addSource( - StreamExecutionEnvironment streamEnv, - BaseSeaTunnelSourceFunction function, - String sourceName, - boolean bounded) { - checkNotNull(function); - checkNotNull(sourceName); - checkNotNull(bounded); - - TypeInformation resolvedTypeInfo = function.getProducedType(); - - boolean isParallel = function instanceof ParallelSourceFunction; - - streamEnv.clean(function); - - final StreamSource sourceOperator = new StreamSource<>(function); - return new DataStreamSource<>( - streamEnv, - resolvedTypeInfo, - sourceOperator, - isParallel, - sourceName, - bounded ? Boundedness.BOUNDED : Boundedness.CONTINUOUS_UNBOUNDED); - } - - @Override - protected List initializePlugins( - List jarPaths, List pluginConfigs) { - SeaTunnelSourcePluginDiscovery sourcePluginDiscovery = - new SeaTunnelSourcePluginDiscovery(ADD_URL_TO_CLASSLOADER); - List sources = new ArrayList<>(); - Set jars = new HashSet<>(); - for (Config sourceConfig : pluginConfigs) { - PluginIdentifier pluginIdentifier = - PluginIdentifier.of( - ENGINE_TYPE, PLUGIN_TYPE, sourceConfig.getString(PLUGIN_NAME)); - jars.addAll( - sourcePluginDiscovery.getPluginJarPaths(Lists.newArrayList(pluginIdentifier))); - SeaTunnelSource seaTunnelSource = - sourcePluginDiscovery.createPluginInstance(pluginIdentifier); - seaTunnelSource.prepare(sourceConfig); - seaTunnelSource.setJobContext(jobContext); - if (jobContext.getJobMode() == JobMode.BATCH - && seaTunnelSource.getBoundedness() - == org.apache.seatunnel.api.source.Boundedness.UNBOUNDED) { - throw new UnsupportedOperationException( - String.format( - "'%s' source don't support off-line job.", - seaTunnelSource.getPluginName())); - } - sources.add(seaTunnelSource); - } - jarPaths.addAll(jars); - return sources; - } -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java deleted file mode 100644 index a358fb6f330..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink.execution; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.transform.SeaTunnelTransform; -import org.apache.seatunnel.core.starter.exception.TaskExecuteException; -import org.apache.seatunnel.plugin.discovery.PluginIdentifier; -import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelTransformPluginDiscovery; -import org.apache.seatunnel.translation.flink.serialization.FlinkRowConverter; -import org.apache.seatunnel.translation.flink.utils.TypeConverterUtils; - -import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.types.Row; -import org.apache.flink.util.Collector; - -import com.google.common.collect.Lists; - -import java.net.URL; -import java.util.ArrayList; -import java.util.List; -import java.util.stream.Collectors; - -public class TransformExecuteProcessor - extends FlinkAbstractPluginExecuteProcessor { - - private static final String PLUGIN_TYPE = "transform"; - - protected TransformExecuteProcessor( - List jarPaths, List pluginConfigs, JobContext jobContext) { - super(jarPaths, pluginConfigs, jobContext); - } - - @Override - protected List initializePlugins( - List jarPaths, List pluginConfigs) { - SeaTunnelTransformPluginDiscovery transformPluginDiscovery = - new SeaTunnelTransformPluginDiscovery(); - List pluginJars = new ArrayList<>(); - List transforms = - pluginConfigs.stream() - .map( - transformConfig -> { - PluginIdentifier pluginIdentifier = - PluginIdentifier.of( - ENGINE_TYPE, - PLUGIN_TYPE, - transformConfig.getString(PLUGIN_NAME)); - List pluginJarPaths = - transformPluginDiscovery.getPluginJarPaths( - Lists.newArrayList(pluginIdentifier)); - SeaTunnelTransform seaTunnelTransform = - transformPluginDiscovery.createPluginInstance( - pluginIdentifier); - jarPaths.addAll(pluginJarPaths); - seaTunnelTransform.prepare(transformConfig); - seaTunnelTransform.setJobContext(jobContext); - return seaTunnelTransform; - }) - .distinct() - .collect(Collectors.toList()); - jarPaths.addAll(pluginJars); - return transforms; - } - - @Override - public List> execute(List> upstreamDataStreams) - throws TaskExecuteException { - if (plugins.isEmpty()) { - return upstreamDataStreams; - } - DataStream input = upstreamDataStreams.get(0); - List> result = new ArrayList<>(); - for (int i = 0; i < plugins.size(); i++) { - try { - SeaTunnelTransform transform = plugins.get(i); - Config pluginConfig = pluginConfigs.get(i); - DataStream stream = fromSourceTable(pluginConfig).orElse(input); - input = flinkTransform(transform, stream); - registerResultTable(pluginConfig, input); - result.add(input); - } catch (Exception e) { - throw new TaskExecuteException( - String.format( - "SeaTunnel transform task: %s execute error", - plugins.get(i).getPluginName()), - e); - } - } - return result; - } - - protected DataStream flinkTransform(SeaTunnelTransform transform, DataStream stream) { - SeaTunnelDataType seaTunnelDataType = TypeConverterUtils.convert(stream.getType()); - transform.setTypeInfo(seaTunnelDataType); - TypeInformation rowTypeInfo = TypeConverterUtils.convert(transform.getProducedType()); - FlinkRowConverter transformInputRowConverter = new FlinkRowConverter(seaTunnelDataType); - FlinkRowConverter transformOutputRowConverter = - new FlinkRowConverter(transform.getProducedType()); - DataStream output = - stream.flatMap( - new FlatMapFunction() { - @Override - public void flatMap(Row value, Collector out) throws Exception { - SeaTunnelRow seaTunnelRow = - transformInputRowConverter.reconvert(value); - SeaTunnelRow dataRow = (SeaTunnelRow) transform.map(seaTunnelRow); - if (dataRow != null) { - Row copy = transformOutputRowConverter.convert(dataRow); - out.collect(copy); - } - } - }, - rowTypeInfo); - return output; - } -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/ConfigKeyName.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/ConfigKeyName.java deleted file mode 100644 index cc8229f26dc..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/ConfigKeyName.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink.utils; - -public class ConfigKeyName { - - private ConfigKeyName() { - throw new IllegalStateException("Utility class"); - } - - public static final String TIME_CHARACTERISTIC = "execution.time-characteristic"; - public static final String BUFFER_TIMEOUT_MILLIS = "execution.buffer.timeout"; - public static final String PARALLELISM = "execution.parallelism"; - public static final String MAX_PARALLELISM = "execution.max-parallelism"; - @Deprecated public static final String CHECKPOINT_INTERVAL = "execution.checkpoint.interval"; - public static final String CHECKPOINT_MODE = "execution.checkpoint.mode"; - public static final String CHECKPOINT_TIMEOUT = "execution.checkpoint.timeout"; - public static final String CHECKPOINT_DATA_URI = "execution.checkpoint.data-uri"; - public static final String MAX_CONCURRENT_CHECKPOINTS = "execution.max-concurrent-checkpoints"; - public static final String CHECKPOINT_CLEANUP_MODE = "execution.checkpoint.cleanup-mode"; - public static final String MIN_PAUSE_BETWEEN_CHECKPOINTS = "execution.checkpoint.min-pause"; - public static final String FAIL_ON_CHECKPOINTING_ERRORS = "execution.checkpoint.fail-on-error"; - public static final String RESTART_STRATEGY = "execution.restart.strategy"; - public static final String RESTART_ATTEMPTS = "execution.restart.attempts"; - public static final String RESTART_DELAY_BETWEEN_ATTEMPTS = - "execution.restart.delayBetweenAttempts"; - public static final String RESTART_FAILURE_INTERVAL = "execution.restart.failureInterval"; - public static final String RESTART_FAILURE_RATE = "execution.restart.failureRate"; - public static final String RESTART_DELAY_INTERVAL = "execution.restart.delayInterval"; - public static final String MAX_STATE_RETENTION_TIME = "execution.query.state.max-retention"; - public static final String MIN_STATE_RETENTION_TIME = "execution.query.state.min-retention"; - public static final String STATE_BACKEND = "execution.state.backend"; - public static final String PLANNER = "execution.planner"; -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/EnvironmentUtil.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/EnvironmentUtil.java deleted file mode 100644 index f15638cbb1f..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/EnvironmentUtil.java +++ /dev/null @@ -1,120 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink.utils; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.common.config.CheckResult; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.PipelineOptions; - -import lombok.extern.slf4j.Slf4j; - -import java.util.concurrent.TimeUnit; - -@Slf4j -public final class EnvironmentUtil { - - private EnvironmentUtil() {} - - public static void setRestartStrategy(Config config, ExecutionConfig executionConfig) { - try { - if (config.hasPath(ConfigKeyName.RESTART_STRATEGY)) { - String restartStrategy = config.getString(ConfigKeyName.RESTART_STRATEGY); - switch (restartStrategy.toLowerCase()) { - case "no": - executionConfig.setRestartStrategy(RestartStrategies.noRestart()); - break; - case "fixed-delay": - int attempts = config.getInt(ConfigKeyName.RESTART_ATTEMPTS); - long delay = config.getLong(ConfigKeyName.RESTART_DELAY_BETWEEN_ATTEMPTS); - executionConfig.setRestartStrategy( - RestartStrategies.fixedDelayRestart(attempts, delay)); - break; - case "failure-rate": - long failureInterval = - config.getLong(ConfigKeyName.RESTART_FAILURE_INTERVAL); - int rate = config.getInt(ConfigKeyName.RESTART_FAILURE_RATE); - long delayInterval = config.getLong(ConfigKeyName.RESTART_DELAY_INTERVAL); - executionConfig.setRestartStrategy( - RestartStrategies.failureRateRestart( - rate, - Time.of(failureInterval, TimeUnit.MILLISECONDS), - Time.of(delayInterval, TimeUnit.MILLISECONDS))); - break; - default: - log.warn( - "set restart.strategy failed, unknown restart.strategy [{}],only support no,fixed-delay,failure-rate", - restartStrategy); - } - } - } catch (Exception e) { - log.warn("set restart.strategy in config '{}' exception", config, e); - } - } - - public static CheckResult checkRestartStrategy(Config config) { - if (config.hasPath(ConfigKeyName.RESTART_STRATEGY)) { - String restartStrategy = config.getString(ConfigKeyName.RESTART_STRATEGY); - switch (restartStrategy.toLowerCase()) { - case "fixed-delay": - if (!(config.hasPath(ConfigKeyName.RESTART_ATTEMPTS) - && config.hasPath(ConfigKeyName.RESTART_DELAY_BETWEEN_ATTEMPTS))) { - return CheckResult.error( - String.format( - "fixed-delay restart strategy must set [%s],[%s]", - ConfigKeyName.RESTART_ATTEMPTS, - ConfigKeyName.RESTART_DELAY_BETWEEN_ATTEMPTS)); - } - break; - case "failure-rate": - if (!(config.hasPath(ConfigKeyName.RESTART_FAILURE_INTERVAL) - && config.hasPath(ConfigKeyName.RESTART_FAILURE_RATE) - && config.hasPath(ConfigKeyName.RESTART_DELAY_INTERVAL))) { - return CheckResult.error( - String.format( - "failure-rate restart strategy must set [%s],[%s],[%s]", - ConfigKeyName.RESTART_FAILURE_INTERVAL, - ConfigKeyName.RESTART_FAILURE_RATE, - ConfigKeyName.RESTART_DELAY_INTERVAL)); - } - break; - default: - return CheckResult.success(); - } - } - return CheckResult.success(); - } - - public static void initConfiguration(Config config, Configuration configuration) { - if (config.hasPath("pipeline")) { - Config pipeline = config.getConfig("pipeline"); - if (pipeline.hasPath("jars")) { - configuration.setString(PipelineOptions.JARS.key(), pipeline.getString("jars")); - } - if (pipeline.hasPath("classpaths")) { - configuration.setString( - PipelineOptions.CLASSPATHS.key(), pipeline.getString("classpaths")); - } - } - } -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/TableUtil.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/TableUtil.java deleted file mode 100644 index ca1603cdf99..00000000000 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/TableUtil.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.seatunnel.core.starter.flink.utils; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.types.Row; - -import java.util.Arrays; - -public final class TableUtil { - - private TableUtil() {} - - public static DataStream tableToDataStream( - StreamTableEnvironment tableEnvironment, Table table, boolean isAppend) { - - TypeInformation typeInfo = table.getSchema().toRowType(); - if (isAppend) { - return tableEnvironment.toAppendStream(table, typeInfo); - } - return tableEnvironment - .toRetractStream(table, typeInfo) - .filter(row -> row.f0) - .map(row -> row.f1) - .returns(typeInfo); - } - - public static boolean tableExists(TableEnvironment tableEnvironment, String name) { - return Arrays.asList(tableEnvironment.listTables()).contains(name); - } -} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/pom.xml b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/pom.xml new file mode 100644 index 00000000000..938c088a2b3 --- /dev/null +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/pom.xml @@ -0,0 +1,73 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-flink-starter + ${revision} + + + seatunnel-flink-starter-common + jar + + SeaTunnel : Core : Flink Starter : Common + + + + + org.apache.seatunnel + seatunnel-translation-flink-15 + ${project.version} + provided + + + + org.apache.flink + flink-java + ${flink.1.15.3.version} + provided + + + + org.apache.flink + flink-streaming-java + ${flink.1.15.3.version} + provided + + + + org.apache.flink + flink-table-api-java-bridge + ${flink.1.15.3.version} + provided + + + + org.apache.flink + flink-statebackend-rocksdb + ${flink.1.15.3.version} + provided + + + + + diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/args/FlinkCommandArgs.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/args/FlinkCommandArgs.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/args/FlinkCommandArgs.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/args/FlinkCommandArgs.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkConfValidateCommand.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkConfValidateCommand.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkConfValidateCommand.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkConfValidateCommand.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkTaskExecuteCommand.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkTaskExecuteCommand.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkTaskExecuteCommand.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/command/FlinkTaskExecuteCommand.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkAbstractPluginExecuteProcessor.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkExecution.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/FlinkRuntimeEnvironment.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-15-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SourceExecuteProcessor.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/TransformExecuteProcessor.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/ConfigKeyName.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/utils/ConfigKeyName.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/ConfigKeyName.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/utils/ConfigKeyName.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/EnvironmentUtil.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/utils/EnvironmentUtil.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/EnvironmentUtil.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/utils/EnvironmentUtil.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/TableUtil.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/utils/TableUtil.java similarity index 100% rename from seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/utils/TableUtil.java rename to seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/utils/TableUtil.java diff --git a/seatunnel-core/seatunnel-spark-starter/pom.xml b/seatunnel-core/seatunnel-spark-starter/pom.xml index 13f5ded0c1f..4502d72c976 100644 --- a/seatunnel-core/seatunnel-spark-starter/pom.xml +++ b/seatunnel-core/seatunnel-spark-starter/pom.xml @@ -33,6 +33,7 @@ seatunnel-spark-2-starter seatunnel-spark-3-starter + seatunnel-spark-starter-common diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/pom.xml b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/pom.xml index e3773cff93d..1dee3c5a35d 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/pom.xml +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/pom.xml @@ -58,6 +58,18 @@ ${project.version} + + org.apache.seatunnel + seatunnel-spark-starter-common + ${project.version} + + + * + * + + + + diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/pom.xml b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/pom.xml index 0e7fb1bb305..838723c7b02 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/pom.xml +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/pom.xml @@ -35,12 +35,6 @@ - - org.apache.seatunnel - seatunnel-translation-spark-3.3 - ${project.version} - - org.apache.spark spark-streaming_${scala.binary.version} @@ -62,6 +56,18 @@ ${spark.scope} + + org.apache.seatunnel + seatunnel-translation-spark-3.3 + ${project.version} + + + + org.apache.seatunnel + seatunnel-spark-starter-common + ${project.version} + + diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java deleted file mode 100644 index 9b3fde6fd10..00000000000 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.seatunnel.core.starter.spark; - -import org.apache.seatunnel.core.starter.SeaTunnel; -import org.apache.seatunnel.core.starter.enums.EngineType; -import org.apache.seatunnel.core.starter.exception.CommandException; -import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; -import org.apache.seatunnel.core.starter.utils.CommandLineUtils; - -public class SeaTunnelSpark { - - public static void main(String[] args) throws CommandException { - SparkCommandArgs sparkCommandArgs = - CommandLineUtils.parse( - args, - new SparkCommandArgs(), - EngineType.SPARK3.getStarterShellName(), - true); - SeaTunnel.run(sparkCommandArgs.buildCommand()); - } -} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java deleted file mode 100644 index 16a3dacad2e..00000000000 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java +++ /dev/null @@ -1,361 +0,0 @@ -/* - * 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.seatunnel.core.starter.spark; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigResolveOptions; - -import org.apache.seatunnel.api.env.EnvCommonOptions; -import org.apache.seatunnel.common.config.Common; -import org.apache.seatunnel.common.config.DeployMode; -import org.apache.seatunnel.core.starter.Starter; -import org.apache.seatunnel.core.starter.enums.EngineType; -import org.apache.seatunnel.core.starter.enums.PluginType; -import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; -import org.apache.seatunnel.core.starter.utils.CommandLineUtils; -import org.apache.seatunnel.core.starter.utils.CompressionUtils; -import org.apache.seatunnel.core.starter.utils.ConfigBuilder; -import org.apache.seatunnel.plugin.discovery.PluginIdentifier; -import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSinkPluginDiscovery; -import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSourcePluginDiscovery; - -import org.apache.commons.lang3.StringUtils; - -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -/** A Starter to generate spark-submit command for SeaTunnel job on spark. */ -public class SparkStarter implements Starter { - - /** original commandline args */ - protected String[] args; - - /** args parsed from {@link #args} */ - protected SparkCommandArgs commandArgs; - - /** jars to include on the spark driver and executor classpaths */ - protected List jars = new ArrayList<>(); - - /** files to be placed in the working directory of each spark executor */ - protected List files = new ArrayList<>(); - - /** spark configuration properties */ - protected Map sparkConf; - - private SparkStarter(String[] args, SparkCommandArgs commandArgs) { - this.args = args; - this.commandArgs = commandArgs; - } - - @SuppressWarnings("checkstyle:RegexpSingleline") - public static void main(String[] args) throws IOException { - SparkStarter starter = getInstance(args); - List command = starter.buildCommands(); - System.out.println(String.join(" ", command)); - } - - /** - * method to get SparkStarter instance, will return {@link ClusterModeSparkStarter} or {@link - * ClientModeSparkStarter} depending on deploy mode. - */ - static SparkStarter getInstance(String[] args) { - SparkCommandArgs commandArgs = - CommandLineUtils.parse( - args, - new SparkCommandArgs(), - EngineType.SPARK3.getStarterShellName(), - true); - DeployMode deployMode = commandArgs.getDeployMode(); - switch (deployMode) { - case CLUSTER: - return new ClusterModeSparkStarter(args, commandArgs); - case CLIENT: - return new ClientModeSparkStarter(args, commandArgs); - default: - throw new IllegalArgumentException("Deploy mode " + deployMode + " not supported"); - } - } - - @Override - public List buildCommands() throws IOException { - setSparkConf(); - Common.setDeployMode(commandArgs.getDeployMode()); - Common.setStarter(true); - this.jars.addAll(Common.getPluginsJarDependencies()); - this.jars.addAll(Common.getLibJars()); - this.jars.addAll(getConnectorJarDependencies()); - this.jars.addAll( - new ArrayList<>( - Common.getThirdPartyJars( - sparkConf.getOrDefault(EnvCommonOptions.JARS.key(), "")))); - // TODO: override job name in command args, because in spark cluster deploy mode - // command-line arguments are read first - // if user has not specified job with command line, the job name config in file will not - // work - return buildFinal(); - } - - /** parse spark configurations from SeaTunnel config file */ - private void setSparkConf() throws FileNotFoundException { - commandArgs.getVariables().stream() - .filter(Objects::nonNull) - .map(variable -> variable.split("=", 2)) - .filter(pair -> pair.length == 2) - .forEach(pair -> System.setProperty(pair[0], pair[1])); - this.sparkConf = getSparkConf(commandArgs.getConfigFile()); - String driverJavaOpts = this.sparkConf.getOrDefault("spark.driver.extraJavaOptions", ""); - String executorJavaOpts = - this.sparkConf.getOrDefault("spark.executor.extraJavaOptions", ""); - if (!commandArgs.getVariables().isEmpty()) { - String properties = - commandArgs.getVariables().stream() - .map(v -> "-D" + v) - .collect(Collectors.joining(" ")); - driverJavaOpts += " " + properties; - executorJavaOpts += " " + properties; - this.sparkConf.put("spark.driver.extraJavaOptions", driverJavaOpts.trim()); - this.sparkConf.put("spark.executor.extraJavaOptions", executorJavaOpts.trim()); - } - } - - /** Get spark configurations from SeaTunnel job config file. */ - static Map getSparkConf(String configFile) throws FileNotFoundException { - File file = new File(configFile); - if (!file.exists()) { - throw new FileNotFoundException("config file '" + file + "' does not exists!"); - } - Config appConfig = - ConfigFactory.parseFile(file) - .resolve(ConfigResolveOptions.defaults().setAllowUnresolved(true)) - .resolveWith( - ConfigFactory.systemProperties(), - ConfigResolveOptions.defaults().setAllowUnresolved(true)); - - return appConfig.getConfig("env").entrySet().stream() - .collect( - Collectors.toMap( - Map.Entry::getKey, e -> e.getValue().unwrapped().toString())); - } - - /** return connector's jars, which located in 'connectors/spark/*'. */ - private List getConnectorJarDependencies() { - Path pluginRootDir = Common.connectorJarDir("seatunnel"); - if (!Files.exists(pluginRootDir) || !Files.isDirectory(pluginRootDir)) { - return Collections.emptyList(); - } - Config config = ConfigBuilder.of(commandArgs.getConfigFile()); - Set pluginJars = new HashSet<>(); - SeaTunnelSourcePluginDiscovery seaTunnelSourcePluginDiscovery = - new SeaTunnelSourcePluginDiscovery(); - SeaTunnelSinkPluginDiscovery seaTunnelSinkPluginDiscovery = - new SeaTunnelSinkPluginDiscovery(); - pluginJars.addAll( - seaTunnelSourcePluginDiscovery.getPluginJarPaths( - getPluginIdentifiers(config, PluginType.SOURCE))); - pluginJars.addAll( - seaTunnelSinkPluginDiscovery.getPluginJarPaths( - getPluginIdentifiers(config, PluginType.SINK))); - return pluginJars.stream() - .map(url -> new File(url.getPath()).toPath()) - .collect(Collectors.toList()); - } - - /** build final spark-submit commands */ - protected List buildFinal() { - List commands = new ArrayList<>(); - commands.add("${SPARK_HOME}/bin/spark-submit"); - appendOption(commands, "--class", SeaTunnelSpark.class.getName()); - appendOption(commands, "--name", this.commandArgs.getJobName()); - appendOption(commands, "--master", this.commandArgs.getMaster()); - appendOption(commands, "--deploy-mode", this.commandArgs.getDeployMode().getDeployMode()); - appendJars(commands, this.jars); - appendFiles(commands, this.files); - appendSparkConf(commands, this.sparkConf); - appendAppJar(commands); - appendOption(commands, "--config", this.commandArgs.getConfigFile()); - appendOption(commands, "--master", this.commandArgs.getMaster()); - appendOption(commands, "--deploy-mode", this.commandArgs.getDeployMode().getDeployMode()); - appendOption(commands, "--name", this.commandArgs.getJobName()); - if (commandArgs.isEncrypt()) { - commands.add("--encrypt"); - } - if (commandArgs.isDecrypt()) { - commands.add("--decrypt"); - } - if (this.commandArgs.isCheckConfig()) { - commands.add("--check"); - } - return commands; - } - - /** append option to StringBuilder */ - protected void appendOption(List commands, String option, String value) { - commands.add(option); - commands.add("\"" + value.replace("\"", "\\\"") + "\""); - } - - /** append jars option to StringBuilder */ - protected void appendJars(List commands, List paths) { - appendPaths(commands, "--jars", paths); - } - - /** append files option to StringBuilder */ - protected void appendFiles(List commands, List paths) { - appendPaths(commands, "--files", paths); - } - - /** append comma-split paths option to StringBuilder */ - protected void appendPaths(List commands, String option, List paths) { - if (!paths.isEmpty()) { - String values = paths.stream().map(Path::toString).collect(Collectors.joining(",")); - appendOption(commands, option, values); - } - } - - /** append spark configurations to StringBuilder */ - protected void appendSparkConf(List commands, Map sparkConf) { - for (Map.Entry entry : sparkConf.entrySet()) { - String key = entry.getKey(); - String value = entry.getValue(); - appendOption(commands, "--conf", key + "=" + value); - } - } - - /** append appJar to StringBuilder */ - protected void appendAppJar(List commands) { - commands.add( - Common.appStarterDir().resolve(EngineType.SPARK3.getStarterJarName()).toString()); - } - - @SuppressWarnings("checkstyle:Indentation") - private List getPluginIdentifiers(Config config, PluginType... pluginTypes) { - return Arrays.stream(pluginTypes) - .flatMap( - (Function>) - pluginType -> { - List configList = - config.getConfigList(pluginType.getType()); - return configList.stream() - .map( - pluginConfig -> - PluginIdentifier.of( - "seatunnel", - pluginType.getType(), - pluginConfig.getString( - "plugin_name"))); - }) - .collect(Collectors.toList()); - } - - /** a Starter for building spark-submit commands with client mode options */ - private static class ClientModeSparkStarter extends SparkStarter { - - /** client mode specified spark options */ - private enum ClientModeSparkConfigs { - - /** Memory for driver in client mode */ - DriverMemory("--driver-memory", "spark.driver.memory"), - - /** Extra Java options to pass to the driver in client mode */ - DriverJavaOptions("--driver-java-options", "spark.driver.extraJavaOptions"), - - /** Extra library path entries to pass to the driver in client mode */ - DriverLibraryPath(" --driver-library-path", "spark.driver.extraLibraryPath"), - - /** Extra class path entries to pass to the driver in client mode */ - DriverClassPath("--driver-class-path", "spark.driver.extraClassPath"); - - private final String optionName; - - private final String propertyName; - - private static final Map PROPERTY_NAME_MAP = - new HashMap<>(); - - static { - for (ClientModeSparkConfigs config : values()) { - PROPERTY_NAME_MAP.put(config.propertyName, config); - } - } - - ClientModeSparkConfigs(String optionName, String propertyName) { - this.optionName = optionName; - this.propertyName = propertyName; - } - } - - private ClientModeSparkStarter(String[] args, SparkCommandArgs commandArgs) { - super(args, commandArgs); - } - - @Override - protected void appendSparkConf(List commands, Map sparkConf) { - for (ClientModeSparkConfigs config : ClientModeSparkConfigs.values()) { - String driverJavaOptions = this.sparkConf.get(config.propertyName); - if (StringUtils.isNotBlank(driverJavaOptions)) { - appendOption(commands, config.optionName, driverJavaOptions); - } - } - for (Map.Entry entry : sparkConf.entrySet()) { - String key = entry.getKey(); - String value = entry.getValue(); - if (ClientModeSparkConfigs.PROPERTY_NAME_MAP.containsKey(key)) { - continue; - } - appendOption(commands, "--conf", key + "=" + value); - } - } - } - - /** a Starter for building spark-submit commands with cluster mode options */ - private static class ClusterModeSparkStarter extends SparkStarter { - - private ClusterModeSparkStarter(String[] args, SparkCommandArgs commandArgs) { - super(args, commandArgs); - } - - @Override - public List buildCommands() throws IOException { - Common.setDeployMode(commandArgs.getDeployMode()); - Common.setStarter(true); - Path pluginTarball = Common.pluginTarball(); - CompressionUtils.tarGzip(Common.pluginRootDir(), pluginTarball); - this.files.add(pluginTarball); - this.files.add(Paths.get(commandArgs.getConfigFile())); - return super.buildCommands(); - } - } -} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/args/SparkCommandArgs.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/args/SparkCommandArgs.java deleted file mode 100644 index 84138392726..00000000000 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/args/SparkCommandArgs.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.seatunnel.core.starter.spark.args; - -import org.apache.seatunnel.common.config.Common; -import org.apache.seatunnel.common.config.DeployMode; -import org.apache.seatunnel.core.starter.command.AbstractCommandArgs; -import org.apache.seatunnel.core.starter.command.Command; -import org.apache.seatunnel.core.starter.command.ConfDecryptCommand; -import org.apache.seatunnel.core.starter.command.ConfEncryptCommand; -import org.apache.seatunnel.core.starter.spark.command.SparkConfValidateCommand; -import org.apache.seatunnel.core.starter.spark.command.SparkTaskExecuteCommand; - -import com.beust.jcommander.IStringConverter; -import com.beust.jcommander.Parameter; -import lombok.Data; -import lombok.EqualsAndHashCode; - -import java.util.ArrayList; -import java.util.List; - -@EqualsAndHashCode(callSuper = true) -@Data -public class SparkCommandArgs extends AbstractCommandArgs { - - @Parameter( - names = {"-e", "--deploy-mode"}, - description = "Spark deploy mode, support [cluster, client]", - converter = SparkDeployModeConverter.class) - private DeployMode deployMode = DeployMode.CLIENT; - - @Parameter( - names = {"-m", "--master"}, - description = - "Spark master, support [spark://host:port, mesos://host:port, yarn, " - + "k8s://https://host:port, local], default local[*]") - private String master = "local[*]"; - - @Override - public Command buildCommand() { - Common.setDeployMode(getDeployMode()); - if (checkConfig) { - return new SparkConfValidateCommand(this); - } - if (encrypt) { - return new ConfEncryptCommand(this); - } - if (decrypt) { - return new ConfDecryptCommand(this); - } - return new SparkTaskExecuteCommand(this); - } - - public static class SparkDeployModeConverter implements IStringConverter { - private static final List DEPLOY_MODE_TYPE_LIST = new ArrayList<>(); - - static { - DEPLOY_MODE_TYPE_LIST.add(DeployMode.CLIENT); - DEPLOY_MODE_TYPE_LIST.add(DeployMode.CLUSTER); - } - - @Override - public DeployMode convert(String value) { - DeployMode deployMode = DeployMode.valueOf(value.toUpperCase()); - if (DEPLOY_MODE_TYPE_LIST.contains(deployMode)) { - return deployMode; - } else { - throw new IllegalArgumentException( - "SeaTunnel job on spark engine deploy mode only " - + "support these options: [cluster, client]"); - } - } - } -} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkConfValidateCommand.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkConfValidateCommand.java deleted file mode 100644 index 9da96797031..00000000000 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkConfValidateCommand.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.seatunnel.core.starter.spark.command; - -import org.apache.seatunnel.core.starter.command.Command; -import org.apache.seatunnel.core.starter.exception.ConfigCheckException; -import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; -import org.apache.seatunnel.core.starter.utils.FileUtils; - -import lombok.extern.slf4j.Slf4j; - -import java.nio.file.Path; - -/** Use to validate the configuration of the SeaTunnel API. */ -@Slf4j -public class SparkConfValidateCommand implements Command { - - private final SparkCommandArgs sparkCommandArgs; - - public SparkConfValidateCommand(SparkCommandArgs sparkCommandArgs) { - this.sparkCommandArgs = sparkCommandArgs; - } - - @Override - public void execute() throws ConfigCheckException { - Path configPath = FileUtils.getConfigPath(sparkCommandArgs); - // TODO: validate the config by new api - } -} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkTaskExecuteCommand.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkTaskExecuteCommand.java deleted file mode 100644 index 463c1dc6581..00000000000 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkTaskExecuteCommand.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * 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.seatunnel.core.starter.spark.command; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigUtil; -import org.apache.seatunnel.shade.com.typesafe.config.ConfigValueFactory; - -import org.apache.seatunnel.common.Constants; -import org.apache.seatunnel.core.starter.command.Command; -import org.apache.seatunnel.core.starter.exception.CommandExecuteException; -import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; -import org.apache.seatunnel.core.starter.spark.execution.SparkExecution; -import org.apache.seatunnel.core.starter.utils.ConfigBuilder; -import org.apache.seatunnel.core.starter.utils.FileUtils; - -import lombok.extern.slf4j.Slf4j; - -import java.nio.file.Path; - -import static org.apache.seatunnel.core.starter.utils.FileUtils.checkConfigExist; - -@Slf4j -public class SparkTaskExecuteCommand implements Command { - - private final SparkCommandArgs sparkCommandArgs; - - public SparkTaskExecuteCommand(SparkCommandArgs sparkCommandArgs) { - this.sparkCommandArgs = sparkCommandArgs; - } - - @Override - public void execute() throws CommandExecuteException { - Path configFile = FileUtils.getConfigPath(sparkCommandArgs); - checkConfigExist(configFile); - Config config = ConfigBuilder.of(configFile); - if (!sparkCommandArgs.getJobName().equals(Constants.LOGO)) { - config = - config.withValue( - ConfigUtil.joinPath("env", "job.name"), - ConfigValueFactory.fromAnyRef(sparkCommandArgs.getJobName())); - } - try { - SparkExecution seaTunnelTaskExecution = new SparkExecution(config); - seaTunnelTaskExecution.execute(); - } catch (Exception e) { - log.error("Run SeaTunnel on spark failed.", e); - throw new CommandExecuteException(e.getMessage()); - } - } -} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java deleted file mode 100644 index d68aec3c232..00000000000 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * 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.seatunnel.core.starter.spark.execution; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.common.CommonOptions; -import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.common.Constants; -import org.apache.seatunnel.common.utils.SerializationUtils; -import org.apache.seatunnel.plugin.discovery.PluginIdentifier; -import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSourcePluginDiscovery; -import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; - -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.types.StructType; - -import com.google.common.collect.Lists; - -import java.net.URL; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -public class SourceExecuteProcessor - extends SparkAbstractPluginExecuteProcessor> { - private static final String PLUGIN_TYPE = "source"; - - public SourceExecuteProcessor( - SparkRuntimeEnvironment sparkEnvironment, - JobContext jobContext, - List sourceConfigs) { - super(sparkEnvironment, jobContext, sourceConfigs); - } - - @Override - public List> execute(List> upstreamDataStreams) { - List> sources = new ArrayList<>(); - for (int i = 0; i < plugins.size(); i++) { - SeaTunnelSource source = plugins.get(i); - Config pluginConfig = pluginConfigs.get(i); - int parallelism; - if (pluginConfig.hasPath(CommonOptions.PARALLELISM.key())) { - parallelism = pluginConfig.getInt(CommonOptions.PARALLELISM.key()); - } else { - parallelism = - sparkRuntimeEnvironment - .getSparkConf() - .getInt( - CommonOptions.PARALLELISM.key(), - CommonOptions.PARALLELISM.defaultValue()); - } - Dataset dataset = - sparkRuntimeEnvironment - .getSparkSession() - .read() - .format(SeaTunnelSource.class.getSimpleName()) - .option(CommonOptions.PARALLELISM.key(), parallelism) - .option( - Constants.SOURCE_SERIALIZATION, - SerializationUtils.objectToString(source)) - .schema( - (StructType) - TypeConverterUtils.convert(source.getProducedType())) - .load(); - sources.add(dataset); - registerInputTempView(pluginConfigs.get(i), dataset); - } - return sources; - } - - @Override - protected List> initializePlugins( - List pluginConfigs) { - SeaTunnelSourcePluginDiscovery sourcePluginDiscovery = new SeaTunnelSourcePluginDiscovery(); - List> sources = new ArrayList<>(); - Set jars = new HashSet<>(); - for (Config sourceConfig : pluginConfigs) { - PluginIdentifier pluginIdentifier = - PluginIdentifier.of( - ENGINE_TYPE, PLUGIN_TYPE, sourceConfig.getString(PLUGIN_NAME)); - jars.addAll( - sourcePluginDiscovery.getPluginJarPaths(Lists.newArrayList(pluginIdentifier))); - SeaTunnelSource seaTunnelSource = - sourcePluginDiscovery.createPluginInstance(pluginIdentifier); - seaTunnelSource.prepare(sourceConfig); - seaTunnelSource.setJobContext(jobContext); - sources.add(seaTunnelSource); - } - sparkRuntimeEnvironment.registerPlugin(new ArrayList<>(jars)); - return sources; - } -} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java deleted file mode 100644 index ebfcaf6e91f..00000000000 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * 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.seatunnel.core.starter.spark.execution; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.core.starter.execution.PluginExecuteProcessor; - -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; - -import java.util.List; -import java.util.Optional; - -public abstract class SparkAbstractPluginExecuteProcessor - implements PluginExecuteProcessor, SparkRuntimeEnvironment> { - protected SparkRuntimeEnvironment sparkRuntimeEnvironment; - protected final List pluginConfigs; - protected final JobContext jobContext; - protected final List plugins; - protected static final String ENGINE_TYPE = "seatunnel"; - protected static final String PLUGIN_NAME = "plugin_name"; - protected static final String RESULT_TABLE_NAME = "result_table_name"; - protected static final String SOURCE_TABLE_NAME = "source_table_name"; - - protected SparkAbstractPluginExecuteProcessor( - SparkRuntimeEnvironment sparkRuntimeEnvironment, - JobContext jobContext, - List pluginConfigs) { - this.sparkRuntimeEnvironment = sparkRuntimeEnvironment; - this.jobContext = jobContext; - this.pluginConfigs = pluginConfigs; - this.plugins = initializePlugins(pluginConfigs); - } - - @Override - public void setRuntimeEnvironment(SparkRuntimeEnvironment sparkRuntimeEnvironment) { - this.sparkRuntimeEnvironment = sparkRuntimeEnvironment; - } - - protected abstract List initializePlugins(List pluginConfigs); - - protected void registerInputTempView(Config pluginConfig, Dataset dataStream) { - if (pluginConfig.hasPath(RESULT_TABLE_NAME)) { - String tableName = pluginConfig.getString(RESULT_TABLE_NAME); - registerTempView(tableName, dataStream); - } - } - - protected Optional> fromSourceTable( - Config pluginConfig, SparkRuntimeEnvironment sparkRuntimeEnvironment) { - if (!pluginConfig.hasPath(SOURCE_TABLE_NAME)) { - return Optional.empty(); - } - String sourceTableName = pluginConfig.getString(SOURCE_TABLE_NAME); - return Optional.of(sparkRuntimeEnvironment.getSparkSession().read().table(sourceTableName)); - } - - private void registerTempView(String tableName, Dataset ds) { - ds.createOrReplaceTempView(tableName); - } -} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java deleted file mode 100644 index dab00cbdb43..00000000000 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.seatunnel.core.starter.spark.execution; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.common.Constants; -import org.apache.seatunnel.common.config.TypesafeConfigUtils; -import org.apache.seatunnel.core.starter.exception.TaskExecuteException; -import org.apache.seatunnel.core.starter.execution.PluginExecuteProcessor; -import org.apache.seatunnel.core.starter.execution.RuntimeEnvironment; -import org.apache.seatunnel.core.starter.execution.TaskExecution; - -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; - -import lombok.extern.slf4j.Slf4j; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -@Slf4j -public class SparkExecution implements TaskExecution { - private final SparkRuntimeEnvironment sparkRuntimeEnvironment; - private final PluginExecuteProcessor, SparkRuntimeEnvironment> - sourcePluginExecuteProcessor; - private final PluginExecuteProcessor, SparkRuntimeEnvironment> - transformPluginExecuteProcessor; - private final PluginExecuteProcessor, SparkRuntimeEnvironment> - sinkPluginExecuteProcessor; - - public SparkExecution(Config config) { - this.sparkRuntimeEnvironment = SparkRuntimeEnvironment.getInstance(config); - JobContext jobContext = new JobContext(); - jobContext.setJobMode(RuntimeEnvironment.getJobMode(config)); - this.sourcePluginExecuteProcessor = - new SourceExecuteProcessor( - sparkRuntimeEnvironment, - jobContext, - config.getConfigList(Constants.SOURCE)); - this.transformPluginExecuteProcessor = - new TransformExecuteProcessor( - sparkRuntimeEnvironment, - jobContext, - TypesafeConfigUtils.getConfigList( - config, Constants.TRANSFORM, Collections.emptyList())); - this.sinkPluginExecuteProcessor = - new SinkExecuteProcessor( - sparkRuntimeEnvironment, jobContext, config.getConfigList(Constants.SINK)); - } - - @Override - public void execute() throws TaskExecuteException { - List> datasets = new ArrayList<>(); - datasets = sourcePluginExecuteProcessor.execute(datasets); - datasets = transformPluginExecuteProcessor.execute(datasets); - sinkPluginExecuteProcessor.execute(datasets); - log.info("Spark Execution started"); - } - - public SparkRuntimeEnvironment getSparkRuntimeEnvironment() { - return sparkRuntimeEnvironment; - } -} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java deleted file mode 100644 index 7e31ca463bc..00000000000 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * 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.seatunnel.core.starter.spark.execution; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.common.Constants; -import org.apache.seatunnel.common.config.CheckResult; -import org.apache.seatunnel.common.constants.JobMode; -import org.apache.seatunnel.core.starter.enums.PluginType; -import org.apache.seatunnel.core.starter.execution.RuntimeEnvironment; - -import org.apache.spark.SparkConf; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.streaming.Seconds; -import org.apache.spark.streaming.StreamingContext; - -import lombok.extern.slf4j.Slf4j; - -import java.net.URL; -import java.util.List; - -@Slf4j -public class SparkRuntimeEnvironment implements RuntimeEnvironment { - private static final long DEFAULT_SPARK_STREAMING_DURATION = 5; - private static final String PLUGIN_NAME_KEY = "plugin_name"; - private static volatile SparkRuntimeEnvironment INSTANCE = null; - - private SparkConf sparkConf; - - private SparkSession sparkSession; - - private StreamingContext streamingContext; - - private Config config; - - private boolean enableHive = false; - - private JobMode jobMode; - - private String jobName = Constants.LOGO; - - private SparkRuntimeEnvironment(Config config) { - this.setEnableHive(checkIsContainHive(config)); - this.initialize(config); - } - - public void setEnableHive(boolean enableHive) { - this.enableHive = enableHive; - } - - @Override - public RuntimeEnvironment setConfig(Config config) { - this.config = config; - return this; - } - - @Override - public RuntimeEnvironment setJobMode(JobMode mode) { - this.jobMode = mode; - return this; - } - - @Override - public JobMode getJobMode() { - return jobMode; - } - - @Override - public Config getConfig() { - return this.config; - } - - @Override - public CheckResult checkConfig() { - return CheckResult.success(); - } - - @Override - public void registerPlugin(List pluginPaths) { - log.info("register plugins :" + pluginPaths); - // TODO we use --jar parameter to support submit multi-jar in spark cluster at now. Refactor - // it to - // support submit multi-jar in code or remove this logic. - // this.sparkSession.conf().set("spark.jars",pluginPaths.stream().map(URL::getPath).collect(Collectors.joining(","))); - } - - @Override - public SparkRuntimeEnvironment prepare() { - if (config.hasPath("job.name")) { - this.jobName = config.getString("job.name"); - } - sparkConf = createSparkConf(); - SparkSession.Builder builder = SparkSession.builder().config(sparkConf); - if (enableHive) { - builder.enableHiveSupport(); - } - this.sparkSession = builder.getOrCreate(); - createStreamingContext(); - return this; - } - - public SparkSession getSparkSession() { - return this.sparkSession; - } - - public StreamingContext getStreamingContext() { - return this.streamingContext; - } - - public SparkConf getSparkConf() { - return this.sparkConf; - } - - private SparkConf createSparkConf() { - SparkConf sparkConf = new SparkConf(); - this.config - .entrySet() - .forEach( - entry -> - sparkConf.set( - entry.getKey(), - String.valueOf(entry.getValue().unwrapped()))); - sparkConf.setAppName(jobName); - return sparkConf; - } - - private void createStreamingContext() { - SparkConf conf = this.sparkSession.sparkContext().getConf(); - long duration = - conf.getLong("spark.stream.batchDuration", DEFAULT_SPARK_STREAMING_DURATION); - if (this.streamingContext == null) { - this.streamingContext = - new StreamingContext(sparkSession.sparkContext(), Seconds.apply(duration)); - } - } - - protected boolean checkIsContainHive(Config config) { - List sourceConfigList = config.getConfigList(PluginType.SOURCE.getType()); - for (Config c : sourceConfigList) { - if (c.getString(PLUGIN_NAME_KEY).toLowerCase().contains("hive")) { - return true; - } - } - List sinkConfigList = config.getConfigList(PluginType.SINK.getType()); - for (Config c : sinkConfigList) { - if (c.getString(PLUGIN_NAME_KEY).toLowerCase().contains("hive")) { - return true; - } - } - return false; - } - - public static SparkRuntimeEnvironment getInstance(Config config) { - if (INSTANCE == null) { - synchronized (SparkRuntimeEnvironment.class) { - if (INSTANCE == null) { - INSTANCE = new SparkRuntimeEnvironment(config); - } - } - } - return INSTANCE; - } -} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java deleted file mode 100644 index f82c465ec9d..00000000000 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java +++ /dev/null @@ -1,173 +0,0 @@ -/* - * 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.seatunnel.core.starter.spark.execution; - -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.common.JobContext; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.transform.SeaTunnelTransform; -import org.apache.seatunnel.core.starter.exception.TaskExecuteException; -import org.apache.seatunnel.plugin.discovery.PluginIdentifier; -import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelTransformPluginDiscovery; -import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; - -import org.apache.spark.api.java.function.MapPartitionsFunction; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; -import org.apache.spark.sql.catalyst.encoders.RowEncoder; -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; -import org.apache.spark.sql.types.StructType; - -import com.google.common.collect.Lists; -import lombok.extern.slf4j.Slf4j; - -import java.io.IOException; -import java.io.Serializable; -import java.net.URL; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.stream.Collectors; - -@Slf4j -public class TransformExecuteProcessor - extends SparkAbstractPluginExecuteProcessor { - - private static final String PLUGIN_TYPE = "transform"; - - protected TransformExecuteProcessor( - SparkRuntimeEnvironment sparkRuntimeEnvironment, - JobContext jobContext, - List pluginConfigs) { - super(sparkRuntimeEnvironment, jobContext, pluginConfigs); - } - - @Override - protected List initializePlugins(List pluginConfigs) { - SeaTunnelTransformPluginDiscovery transformPluginDiscovery = - new SeaTunnelTransformPluginDiscovery(); - List pluginJars = new ArrayList<>(); - List transforms = - pluginConfigs.stream() - .map( - transformConfig -> { - PluginIdentifier pluginIdentifier = - PluginIdentifier.of( - ENGINE_TYPE, - PLUGIN_TYPE, - transformConfig.getString(PLUGIN_NAME)); - pluginJars.addAll( - transformPluginDiscovery.getPluginJarPaths( - Lists.newArrayList(pluginIdentifier))); - SeaTunnelTransform pluginInstance = - transformPluginDiscovery.createPluginInstance( - pluginIdentifier); - pluginInstance.prepare(transformConfig); - pluginInstance.setJobContext(jobContext); - return pluginInstance; - }) - .distinct() - .collect(Collectors.toList()); - sparkRuntimeEnvironment.registerPlugin(pluginJars); - return transforms; - } - - @Override - public List> execute(List> upstreamDataStreams) - throws TaskExecuteException { - if (plugins.isEmpty()) { - return upstreamDataStreams; - } - Dataset input = upstreamDataStreams.get(0); - List> result = new ArrayList<>(); - for (int i = 0; i < plugins.size(); i++) { - try { - SeaTunnelTransform transform = plugins.get(i); - Config pluginConfig = pluginConfigs.get(i); - Dataset stream = - fromSourceTable(pluginConfig, sparkRuntimeEnvironment).orElse(input); - input = sparkTransform(transform, stream); - registerInputTempView(pluginConfig, input); - result.add(input); - } catch (Exception e) { - throw new TaskExecuteException( - String.format( - "SeaTunnel transform task: %s execute error", - plugins.get(i).getPluginName()), - e); - } - } - return result; - } - - private Dataset sparkTransform(SeaTunnelTransform transform, Dataset stream) - throws IOException { - SeaTunnelDataType seaTunnelDataType = TypeConverterUtils.convert(stream.schema()); - transform.setTypeInfo(seaTunnelDataType); - StructType structType = - (StructType) TypeConverterUtils.convert(transform.getProducedType()); - ExpressionEncoder encoder = RowEncoder.apply(structType); - return stream.mapPartitions( - (MapPartitionsFunction) - (Iterator rowIterator) -> { - TransformIterator iterator = - new TransformIterator( - rowIterator, transform, structType); - return iterator; - }, - encoder) - .filter( - (Row row) -> { - return row != null; - }); - } - - private static class TransformIterator implements Iterator, Serializable { - private Iterator sourceIterator; - private SeaTunnelTransform transform; - private StructType structType; - - public TransformIterator( - Iterator sourceIterator, - SeaTunnelTransform transform, - StructType structType) { - this.sourceIterator = sourceIterator; - this.transform = transform; - this.structType = structType; - } - - @Override - public boolean hasNext() { - return sourceIterator.hasNext(); - } - - @Override - public Row next() { - Row row = sourceIterator.next(); - SeaTunnelRow seaTunnelRow = new SeaTunnelRow(((GenericRowWithSchema) row).values()); - seaTunnelRow = (SeaTunnelRow) transform.map(seaTunnelRow); - if (seaTunnelRow == null) { - return null; - } - return new GenericRowWithSchema(seaTunnelRow.getFields(), structType); - } - } -} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/pom.xml b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/pom.xml new file mode 100644 index 00000000000..ec4df4bfa94 --- /dev/null +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/pom.xml @@ -0,0 +1,69 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-spark-starter + ${revision} + + + seatunnel-spark-starter-common + jar + SeaTunnel : Core : Spark Starter : Common + + + 2.12 + + + + + + org.apache.seatunnel + seatunnel-translation-spark-3.3 + ${project.version} + provided + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${spark.3.3.0.version} + ${spark.scope} + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.3.3.0.version} + ${spark.scope} + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.3.3.0.version} + ${spark.scope} + + + + + diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java similarity index 100% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java rename to seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java similarity index 100% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java rename to seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/args/SparkCommandArgs.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/args/SparkCommandArgs.java similarity index 100% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/args/SparkCommandArgs.java rename to seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/args/SparkCommandArgs.java diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkConfValidateCommand.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkConfValidateCommand.java similarity index 100% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkConfValidateCommand.java rename to seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkConfValidateCommand.java diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkTaskExecuteCommand.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkTaskExecuteCommand.java similarity index 100% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkTaskExecuteCommand.java rename to seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/command/SparkTaskExecuteCommand.java diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java similarity index 100% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-3-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java rename to seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java similarity index 100% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java rename to seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java similarity index 100% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java rename to seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java similarity index 100% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java rename to seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkExecution.java diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java similarity index 100% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java rename to seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkRuntimeEnvironment.java diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java similarity index 100% rename from seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java rename to seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java index 553c1963622..857b3b0e50e 100644 --- a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java +++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.core.starter.SeaTunnel; import org.apache.seatunnel.core.starter.exception.CommandException; -import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; import java.io.FileNotFoundException; import java.net.URISyntaxException; From c0610a6369200d13bad95809980aefa0ee770abc Mon Sep 17 00:00:00 2001 From: tyrantlucifer Date: Sun, 9 Apr 2023 02:28:54 +0800 Subject: [PATCH 2/3] [Improve][Starter] Revert SeaTunnelApiExample --- .../apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java | 1 + 1 file changed, 1 insertion(+) diff --git a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java index 857b3b0e50e..553c1963622 100644 --- a/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java +++ b/seatunnel-examples/seatunnel-flink-connector-v2-example/src/main/java/org/apache/seatunnel/example/flink/v2/SeaTunnelApiExample.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.core.starter.SeaTunnel; import org.apache.seatunnel.core.starter.exception.CommandException; +import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; import java.io.FileNotFoundException; import java.net.URISyntaxException; From 7748ebc03b35f06b58da1cc1176205d2c4ad489a Mon Sep 17 00:00:00 2001 From: tyrantlucifer Date: Sun, 9 Apr 2023 03:09:40 +0800 Subject: [PATCH 3/3] [Improve][Starter] Fix ci error --- .../core/starter/flink/FlinkStarter.java | 97 +++++ .../core/starter/flink/SeaTunnelFlink.java | 36 ++ .../core/starter/flink/FlinkStarter.java | 4 +- .../core/starter/flink/SeaTunnelFlink.java | 2 +- .../core/starter/spark/SeaTunnelSpark.java | 37 ++ .../core/starter/spark/SparkStarter.java | 361 ++++++++++++++++++ .../core/starter/spark/SeaTunnelSpark.java | 2 +- .../core/starter/spark/SparkStarter.java | 4 +- 8 files changed, 537 insertions(+), 6 deletions(-) create mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java create mode 100644 seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java create mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java create mode 100644 seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java new file mode 100644 index 00000000000..078c29bbfaf --- /dev/null +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java @@ -0,0 +1,97 @@ +/* + * 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.seatunnel.core.starter.flink; + +import org.apache.seatunnel.common.config.Common; +import org.apache.seatunnel.core.starter.Starter; +import org.apache.seatunnel.core.starter.enums.EngineType; +import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; +import org.apache.seatunnel.core.starter.utils.CommandLineUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +/** The SeaTunnel flink starter, used to generate the final flink job execute command. */ +public class FlinkStarter implements Starter { + private static final String APP_NAME = SeaTunnelFlink.class.getName(); + public static final String APP_JAR_NAME = EngineType.FLINK13.getStarterJarName(); + public static final String SHELL_NAME = EngineType.FLINK13.getStarterShellName(); + private final FlinkCommandArgs flinkCommandArgs; + private final String appJar; + + FlinkStarter(String[] args) { + this.flinkCommandArgs = + CommandLineUtils.parse(args, new FlinkCommandArgs(), SHELL_NAME, true); + // set the deployment mode, used to get the job jar path. + Common.setDeployMode(flinkCommandArgs.getDeployMode()); + Common.setStarter(true); + this.appJar = Common.appStarterDir().resolve(APP_JAR_NAME).toString(); + } + + @SuppressWarnings("checkstyle:RegexpSingleline") + public static void main(String[] args) { + FlinkStarter flinkStarter = new FlinkStarter(args); + System.out.println(String.join(" ", flinkStarter.buildCommands())); + } + + @Override + public List buildCommands() { + List command = new ArrayList<>(); + // set start command + command.add("${FLINK_HOME}/bin/flink"); + // set deploy mode, run or run-application + command.add(flinkCommandArgs.getDeployMode().getDeployMode()); + // set submitted target master + if (flinkCommandArgs.getMasterType() != null) { + command.add("--target"); + command.add(flinkCommandArgs.getMasterType().getMaster()); + } + // set flink original parameters + command.addAll(flinkCommandArgs.getOriginalParameters()); + // set main class name + command.add("-c"); + command.add(APP_NAME); + // set main jar name + command.add(appJar); + // set config file path + command.add("--config"); + command.add(flinkCommandArgs.getConfigFile()); + // set check config flag + if (flinkCommandArgs.isCheckConfig()) { + command.add("--check"); + } + // set job name + command.add("--name"); + command.add(flinkCommandArgs.getJobName()); + // set encryption + if (flinkCommandArgs.isEncrypt()) { + command.add("--encrypt"); + } + // set decryption + if (flinkCommandArgs.isDecrypt()) { + command.add("--decrypt"); + } + // set extra system properties + flinkCommandArgs.getVariables().stream() + .filter(Objects::nonNull) + .map(String::trim) + .forEach(variable -> command.add("-D" + variable)); + return command; + } +} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java new file mode 100644 index 00000000000..8d1b434801d --- /dev/null +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java @@ -0,0 +1,36 @@ +/* + * 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.seatunnel.core.starter.flink; + +import org.apache.seatunnel.core.starter.SeaTunnel; +import org.apache.seatunnel.core.starter.enums.EngineType; +import org.apache.seatunnel.core.starter.exception.CommandException; +import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs; +import org.apache.seatunnel.core.starter.utils.CommandLineUtils; + +public class SeaTunnelFlink { + public static void main(String[] args) throws CommandException { + FlinkCommandArgs flinkCommandArgs = + CommandLineUtils.parse( + args, + new FlinkCommandArgs(), + EngineType.FLINK13.getStarterShellName(), + true); + SeaTunnel.run(flinkCommandArgs.buildCommand()); + } +} diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java index 078c29bbfaf..2f9021c68bd 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/FlinkStarter.java @@ -30,8 +30,8 @@ /** The SeaTunnel flink starter, used to generate the final flink job execute command. */ public class FlinkStarter implements Starter { private static final String APP_NAME = SeaTunnelFlink.class.getName(); - public static final String APP_JAR_NAME = EngineType.FLINK13.getStarterJarName(); - public static final String SHELL_NAME = EngineType.FLINK13.getStarterShellName(); + public static final String APP_JAR_NAME = EngineType.FLINK15.getStarterJarName(); + public static final String SHELL_NAME = EngineType.FLINK15.getStarterShellName(); private final FlinkCommandArgs flinkCommandArgs; private final String appJar; diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java index 8d1b434801d..1595da686a7 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/SeaTunnelFlink.java @@ -29,7 +29,7 @@ public static void main(String[] args) throws CommandException { CommandLineUtils.parse( args, new FlinkCommandArgs(), - EngineType.FLINK13.getStarterShellName(), + EngineType.FLINK15.getStarterShellName(), true); SeaTunnel.run(flinkCommandArgs.buildCommand()); } diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java new file mode 100644 index 00000000000..ca7b2ed4be1 --- /dev/null +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java @@ -0,0 +1,37 @@ +/* + * 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.seatunnel.core.starter.spark; + +import org.apache.seatunnel.core.starter.SeaTunnel; +import org.apache.seatunnel.core.starter.enums.EngineType; +import org.apache.seatunnel.core.starter.exception.CommandException; +import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; +import org.apache.seatunnel.core.starter.utils.CommandLineUtils; + +public class SeaTunnelSpark { + + public static void main(String[] args) throws CommandException { + SparkCommandArgs sparkCommandArgs = + CommandLineUtils.parse( + args, + new SparkCommandArgs(), + EngineType.SPARK2.getStarterShellName(), + true); + SeaTunnel.run(sparkCommandArgs.buildCommand()); + } +} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java new file mode 100644 index 00000000000..c187cceb6c4 --- /dev/null +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java @@ -0,0 +1,361 @@ +/* + * 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.seatunnel.core.starter.spark; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; +import org.apache.seatunnel.shade.com.typesafe.config.ConfigResolveOptions; + +import org.apache.seatunnel.api.env.EnvCommonOptions; +import org.apache.seatunnel.common.config.Common; +import org.apache.seatunnel.common.config.DeployMode; +import org.apache.seatunnel.core.starter.Starter; +import org.apache.seatunnel.core.starter.enums.EngineType; +import org.apache.seatunnel.core.starter.enums.PluginType; +import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs; +import org.apache.seatunnel.core.starter.utils.CommandLineUtils; +import org.apache.seatunnel.core.starter.utils.CompressionUtils; +import org.apache.seatunnel.core.starter.utils.ConfigBuilder; +import org.apache.seatunnel.plugin.discovery.PluginIdentifier; +import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSinkPluginDiscovery; +import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSourcePluginDiscovery; + +import org.apache.commons.lang3.StringUtils; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** A Starter to generate spark-submit command for SeaTunnel job on spark. */ +public class SparkStarter implements Starter { + + /** original commandline args */ + protected String[] args; + + /** args parsed from {@link #args} */ + protected SparkCommandArgs commandArgs; + + /** jars to include on the spark driver and executor classpaths */ + protected List jars = new ArrayList<>(); + + /** files to be placed in the working directory of each spark executor */ + protected List files = new ArrayList<>(); + + /** spark configuration properties */ + protected Map sparkConf; + + private SparkStarter(String[] args, SparkCommandArgs commandArgs) { + this.args = args; + this.commandArgs = commandArgs; + } + + @SuppressWarnings("checkstyle:RegexpSingleline") + public static void main(String[] args) throws IOException { + SparkStarter starter = getInstance(args); + List command = starter.buildCommands(); + System.out.println(String.join(" ", command)); + } + + /** + * method to get SparkStarter instance, will return {@link ClusterModeSparkStarter} or {@link + * ClientModeSparkStarter} depending on deploy mode. + */ + static SparkStarter getInstance(String[] args) { + SparkCommandArgs commandArgs = + CommandLineUtils.parse( + args, + new SparkCommandArgs(), + EngineType.SPARK2.getStarterShellName(), + true); + DeployMode deployMode = commandArgs.getDeployMode(); + switch (deployMode) { + case CLUSTER: + return new ClusterModeSparkStarter(args, commandArgs); + case CLIENT: + return new ClientModeSparkStarter(args, commandArgs); + default: + throw new IllegalArgumentException("Deploy mode " + deployMode + " not supported"); + } + } + + @Override + public List buildCommands() throws IOException { + setSparkConf(); + Common.setDeployMode(commandArgs.getDeployMode()); + Common.setStarter(true); + this.jars.addAll(Common.getPluginsJarDependencies()); + this.jars.addAll(Common.getLibJars()); + this.jars.addAll(getConnectorJarDependencies()); + this.jars.addAll( + new ArrayList<>( + Common.getThirdPartyJars( + sparkConf.getOrDefault(EnvCommonOptions.JARS.key(), "")))); + // TODO: override job name in command args, because in spark cluster deploy mode + // command-line arguments are read first + // if user has not specified job with command line, the job name config in file will not + // work + return buildFinal(); + } + + /** parse spark configurations from SeaTunnel config file */ + private void setSparkConf() throws FileNotFoundException { + commandArgs.getVariables().stream() + .filter(Objects::nonNull) + .map(variable -> variable.split("=", 2)) + .filter(pair -> pair.length == 2) + .forEach(pair -> System.setProperty(pair[0], pair[1])); + this.sparkConf = getSparkConf(commandArgs.getConfigFile()); + String driverJavaOpts = this.sparkConf.getOrDefault("spark.driver.extraJavaOptions", ""); + String executorJavaOpts = + this.sparkConf.getOrDefault("spark.executor.extraJavaOptions", ""); + if (!commandArgs.getVariables().isEmpty()) { + String properties = + commandArgs.getVariables().stream() + .map(v -> "-D" + v) + .collect(Collectors.joining(" ")); + driverJavaOpts += " " + properties; + executorJavaOpts += " " + properties; + this.sparkConf.put("spark.driver.extraJavaOptions", driverJavaOpts.trim()); + this.sparkConf.put("spark.executor.extraJavaOptions", executorJavaOpts.trim()); + } + } + + /** Get spark configurations from SeaTunnel job config file. */ + static Map getSparkConf(String configFile) throws FileNotFoundException { + File file = new File(configFile); + if (!file.exists()) { + throw new FileNotFoundException("config file '" + file + "' does not exists!"); + } + Config appConfig = + ConfigFactory.parseFile(file) + .resolve(ConfigResolveOptions.defaults().setAllowUnresolved(true)) + .resolveWith( + ConfigFactory.systemProperties(), + ConfigResolveOptions.defaults().setAllowUnresolved(true)); + + return appConfig.getConfig("env").entrySet().stream() + .collect( + Collectors.toMap( + Map.Entry::getKey, e -> e.getValue().unwrapped().toString())); + } + + /** return connector's jars, which located in 'connectors/spark/*'. */ + private List getConnectorJarDependencies() { + Path pluginRootDir = Common.connectorJarDir("seatunnel"); + if (!Files.exists(pluginRootDir) || !Files.isDirectory(pluginRootDir)) { + return Collections.emptyList(); + } + Config config = ConfigBuilder.of(commandArgs.getConfigFile()); + Set pluginJars = new HashSet<>(); + SeaTunnelSourcePluginDiscovery seaTunnelSourcePluginDiscovery = + new SeaTunnelSourcePluginDiscovery(); + SeaTunnelSinkPluginDiscovery seaTunnelSinkPluginDiscovery = + new SeaTunnelSinkPluginDiscovery(); + pluginJars.addAll( + seaTunnelSourcePluginDiscovery.getPluginJarPaths( + getPluginIdentifiers(config, PluginType.SOURCE))); + pluginJars.addAll( + seaTunnelSinkPluginDiscovery.getPluginJarPaths( + getPluginIdentifiers(config, PluginType.SINK))); + return pluginJars.stream() + .map(url -> new File(url.getPath()).toPath()) + .collect(Collectors.toList()); + } + + /** build final spark-submit commands */ + protected List buildFinal() { + List commands = new ArrayList<>(); + commands.add("${SPARK_HOME}/bin/spark-submit"); + appendOption(commands, "--class", SeaTunnelSpark.class.getName()); + appendOption(commands, "--name", this.commandArgs.getJobName()); + appendOption(commands, "--master", this.commandArgs.getMaster()); + appendOption(commands, "--deploy-mode", this.commandArgs.getDeployMode().getDeployMode()); + appendJars(commands, this.jars); + appendFiles(commands, this.files); + appendSparkConf(commands, this.sparkConf); + appendAppJar(commands); + appendOption(commands, "--config", this.commandArgs.getConfigFile()); + appendOption(commands, "--master", this.commandArgs.getMaster()); + appendOption(commands, "--deploy-mode", this.commandArgs.getDeployMode().getDeployMode()); + appendOption(commands, "--name", this.commandArgs.getJobName()); + if (commandArgs.isEncrypt()) { + commands.add("--encrypt"); + } + if (commandArgs.isDecrypt()) { + commands.add("--decrypt"); + } + if (this.commandArgs.isCheckConfig()) { + commands.add("--check"); + } + return commands; + } + + /** append option to StringBuilder */ + protected void appendOption(List commands, String option, String value) { + commands.add(option); + commands.add("\"" + value.replace("\"", "\\\"") + "\""); + } + + /** append jars option to StringBuilder */ + protected void appendJars(List commands, List paths) { + appendPaths(commands, "--jars", paths); + } + + /** append files option to StringBuilder */ + protected void appendFiles(List commands, List paths) { + appendPaths(commands, "--files", paths); + } + + /** append comma-split paths option to StringBuilder */ + protected void appendPaths(List commands, String option, List paths) { + if (!paths.isEmpty()) { + String values = paths.stream().map(Path::toString).collect(Collectors.joining(",")); + appendOption(commands, option, values); + } + } + + /** append spark configurations to StringBuilder */ + protected void appendSparkConf(List commands, Map sparkConf) { + for (Map.Entry entry : sparkConf.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + appendOption(commands, "--conf", key + "=" + value); + } + } + + /** append appJar to StringBuilder */ + protected void appendAppJar(List commands) { + commands.add( + Common.appStarterDir().resolve(EngineType.SPARK2.getStarterJarName()).toString()); + } + + @SuppressWarnings("checkstyle:Indentation") + private List getPluginIdentifiers(Config config, PluginType... pluginTypes) { + return Arrays.stream(pluginTypes) + .flatMap( + (Function>) + pluginType -> { + List configList = + config.getConfigList(pluginType.getType()); + return configList.stream() + .map( + pluginConfig -> + PluginIdentifier.of( + "seatunnel", + pluginType.getType(), + pluginConfig.getString( + "plugin_name"))); + }) + .collect(Collectors.toList()); + } + + /** a Starter for building spark-submit commands with client mode options */ + private static class ClientModeSparkStarter extends SparkStarter { + + /** client mode specified spark options */ + private enum ClientModeSparkConfigs { + + /** Memory for driver in client mode */ + DriverMemory("--driver-memory", "spark.driver.memory"), + + /** Extra Java options to pass to the driver in client mode */ + DriverJavaOptions("--driver-java-options", "spark.driver.extraJavaOptions"), + + /** Extra library path entries to pass to the driver in client mode */ + DriverLibraryPath(" --driver-library-path", "spark.driver.extraLibraryPath"), + + /** Extra class path entries to pass to the driver in client mode */ + DriverClassPath("--driver-class-path", "spark.driver.extraClassPath"); + + private final String optionName; + + private final String propertyName; + + private static final Map PROPERTY_NAME_MAP = + new HashMap<>(); + + static { + for (ClientModeSparkConfigs config : values()) { + PROPERTY_NAME_MAP.put(config.propertyName, config); + } + } + + ClientModeSparkConfigs(String optionName, String propertyName) { + this.optionName = optionName; + this.propertyName = propertyName; + } + } + + private ClientModeSparkStarter(String[] args, SparkCommandArgs commandArgs) { + super(args, commandArgs); + } + + @Override + protected void appendSparkConf(List commands, Map sparkConf) { + for (ClientModeSparkConfigs config : ClientModeSparkConfigs.values()) { + String driverJavaOptions = this.sparkConf.get(config.propertyName); + if (StringUtils.isNotBlank(driverJavaOptions)) { + appendOption(commands, config.optionName, driverJavaOptions); + } + } + for (Map.Entry entry : sparkConf.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (ClientModeSparkConfigs.PROPERTY_NAME_MAP.containsKey(key)) { + continue; + } + appendOption(commands, "--conf", key + "=" + value); + } + } + } + + /** a Starter for building spark-submit commands with cluster mode options */ + private static class ClusterModeSparkStarter extends SparkStarter { + + private ClusterModeSparkStarter(String[] args, SparkCommandArgs commandArgs) { + super(args, commandArgs); + } + + @Override + public List buildCommands() throws IOException { + Common.setDeployMode(commandArgs.getDeployMode()); + Common.setStarter(true); + Path pluginTarball = Common.pluginTarball(); + CompressionUtils.tarGzip(Common.pluginRootDir(), pluginTarball); + this.files.add(pluginTarball); + this.files.add(Paths.get(commandArgs.getConfigFile())); + return super.buildCommands(); + } + } +} diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java index ca7b2ed4be1..9b3fde6fd10 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SeaTunnelSpark.java @@ -30,7 +30,7 @@ public static void main(String[] args) throws CommandException { CommandLineUtils.parse( args, new SparkCommandArgs(), - EngineType.SPARK2.getStarterShellName(), + EngineType.SPARK3.getStarterShellName(), true); SeaTunnel.run(sparkCommandArgs.buildCommand()); } diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java index c187cceb6c4..16a3dacad2e 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/SparkStarter.java @@ -96,7 +96,7 @@ static SparkStarter getInstance(String[] args) { CommandLineUtils.parse( args, new SparkCommandArgs(), - EngineType.SPARK2.getStarterShellName(), + EngineType.SPARK3.getStarterShellName(), true); DeployMode deployMode = commandArgs.getDeployMode(); switch (deployMode) { @@ -257,7 +257,7 @@ protected void appendSparkConf(List commands, Map sparkC /** append appJar to StringBuilder */ protected void appendAppJar(List commands) { commands.add( - Common.appStarterDir().resolve(EngineType.SPARK2.getStarterJarName()).toString()); + Common.appStarterDir().resolve(EngineType.SPARK3.getStarterJarName()).toString()); } @SuppressWarnings("checkstyle:Indentation")