You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@seatunnel.apache.org by we...@apache.org on 2022/05/19 12:12:36 UTC
[incubator-seatunnel] branch api-draft updated: Add Transform to SeaTunnel Spark API (#1925)
This is an automated email from the ASF dual-hosted git repository.
wenjun pushed a commit to branch api-draft
in repository https://gitbox.apache.org/repos/asf/incubator-seatunnel.git
The following commit(s) were added to refs/heads/api-draft by this push:
new ad586962 Add Transform to SeaTunnel Spark API (#1925)
ad586962 is described below
commit ad586962530e5774ad28f8870b26d18be653d4fa
Author: Wenjun Ruan <we...@apache.org>
AuthorDate: Thu May 19 20:12:31 2022 +0800
Add Transform to SeaTunnel Spark API (#1925)
---
.../command/SeaTunnelApiTaskExecuteCommand.java | 21 ++----
.../execution/AbstractPluginExecuteProcessor.java | 68 +++++++++++++++++++
.../spark/execution/PluginExecuteProcessor.java | 34 ++++++++++
.../spark/execution/SeaTunnelTaskExecution.java | 79 ++++++++++++++++++++++
.../core/spark/execution/SinkExecuteProcessor.java | 77 +++++++++++++++++++++
.../spark/execution/SourceExecuteProcessor.java | 78 +++++++++++++++++++++
.../spark/execution/TransformExecuteProcessor.java | 79 ++++++++++++++++++++++
.../example/spark/SeaTunnelApiExample.java | 2 +-
.../main/resources/examples/seatunnel-spark.conf | 76 +++++++++++++++++++++
9 files changed, 496 insertions(+), 18 deletions(-)
diff --git a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/command/SeaTunnelApiTaskExecuteCommand.java b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/command/SeaTunnelApiTaskExecuteCommand.java
index 55f731d9..a12d23b4 100644
--- a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/command/SeaTunnelApiTaskExecuteCommand.java
+++ b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/command/SeaTunnelApiTaskExecuteCommand.java
@@ -19,9 +19,7 @@ package org.apache.seatunnel.core.spark.command;
import org.apache.seatunnel.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.source.SeaTunnelSource;
-import org.apache.seatunnel.common.config.Common;
import org.apache.seatunnel.common.constants.JobMode;
-import org.apache.seatunnel.common.utils.SerializationUtils;
import org.apache.seatunnel.core.base.command.Command;
import org.apache.seatunnel.core.base.config.ConfigBuilder;
import org.apache.seatunnel.core.base.config.EngineType;
@@ -29,22 +27,18 @@ import org.apache.seatunnel.core.base.config.EnvironmentFactory;
import org.apache.seatunnel.core.base.exception.CommandExecuteException;
import org.apache.seatunnel.core.base.utils.FileUtils;
import org.apache.seatunnel.core.spark.args.SparkCommandArgs;
+import org.apache.seatunnel.core.spark.execution.SeaTunnelTaskExecution;
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.seatunnel.spark.SparkEnvironment;
-import org.apache.seatunnel.translation.spark.sink.SparkSinkInjector;
-import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils;
import org.apache.seatunnel.shade.com.typesafe.config.Config;
-import org.apache.spark.sql.Dataset;
-import org.apache.spark.sql.Row;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.nio.file.Path;
-import java.util.HashMap;
/**
* todo: do we need to move these class to a new module? since this may cause version conflict with the old flink version.
@@ -63,19 +57,12 @@ public class SeaTunnelApiTaskExecuteCommand implements Command<SparkCommandArgs>
@Override
public void execute() throws CommandExecuteException {
Path configFile = FileUtils.getConfigPath(sparkCommandArgs);
-
Config config = new ConfigBuilder(configFile).getConfig();
- SparkEnvironment sparkEnvironment = getSparkEnvironment(config);
- SeaTunnelSource<?, ?, ?> source = getSource(config);
- Dataset<Row> dataset = sparkEnvironment.getSparkSession().read().format("SeaTunnelSource")
- .option("source.serialization", SerializationUtils.objectToString(source))
- .schema(TypeConverterUtils.convertRow(source.getRowTypeInfo())).load();
- SeaTunnelSink<?, ?, ?, ?> sink = getSink(config);
try {
- SparkSinkInjector.inject(dataset.write(), sink, new HashMap<>(Common.COLLECTION_SIZE)).option(
- "checkpointLocation", "/tmp").save();
+ SeaTunnelTaskExecution seaTunnelTaskExecution = new SeaTunnelTaskExecution(config);
+ seaTunnelTaskExecution.execute();
} catch (Exception e) {
- LOGGER.error("run seatunnel on spark failed.", e);
+ LOGGER.error("Run SeaTunnel on spark failed.", e);
}
}
diff --git a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/AbstractPluginExecuteProcessor.java b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/AbstractPluginExecuteProcessor.java
new file mode 100644
index 00000000..eb871aae
--- /dev/null
+++ b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/AbstractPluginExecuteProcessor.java
@@ -0,0 +1,68 @@
+/*
+ * 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.spark.execution;
+
+import static org.apache.seatunnel.apis.base.plugin.Plugin.RESULT_TABLE_NAME;
+import static org.apache.seatunnel.apis.base.plugin.Plugin.SOURCE_TABLE_NAME;
+
+import org.apache.seatunnel.spark.SparkEnvironment;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.util.List;
+import java.util.Optional;
+
+public abstract class AbstractPluginExecuteProcessor<T> implements PluginExecuteProcessor {
+
+ protected final SparkEnvironment sparkEnvironment;
+ protected final List<? extends Config> pluginConfigs;
+ protected final List<T> plugins;
+ protected static final String ENGINE_TYPE = "seatunnel";
+ protected static final String PLUGIN_NAME = "plugin_name";
+
+ protected AbstractPluginExecuteProcessor(SparkEnvironment flinkEnvironment,
+ List<? extends Config> pluginConfigs) {
+ this.sparkEnvironment = flinkEnvironment;
+ this.pluginConfigs = pluginConfigs;
+ this.plugins = initializePlugins(pluginConfigs);
+ }
+
+ protected abstract List<T> initializePlugins(List<? extends Config> pluginConfigs);
+
+ protected void registerInputTempView(Config pluginConfig, Dataset<Row> dataStream) {
+ if (pluginConfig.hasPath(RESULT_TABLE_NAME)) {
+ String tableName = pluginConfig.getString(RESULT_TABLE_NAME);
+ registerTempView(tableName, dataStream);
+ }
+ }
+
+ protected Optional<Dataset<Row>> fromSourceTable(Config pluginConfig, SparkEnvironment sparkEnvironment) {
+ if (!pluginConfig.hasPath(SOURCE_TABLE_NAME)) {
+ return Optional.empty();
+ }
+ String sourceTableName = pluginConfig.getString(SOURCE_TABLE_NAME);
+ return Optional.of(sparkEnvironment.getSparkSession().read().table(sourceTableName));
+ }
+
+ private void registerTempView(String tableName, Dataset<Row> ds) {
+ ds.createOrReplaceTempView(tableName);
+ }
+}
diff --git a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/PluginExecuteProcessor.java b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/PluginExecuteProcessor.java
new file mode 100644
index 00000000..48d18488
--- /dev/null
+++ b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/PluginExecuteProcessor.java
@@ -0,0 +1,34 @@
+/*
+ * 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.spark.execution;
+
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.util.List;
+
+public interface PluginExecuteProcessor {
+
+ /**
+ * Execute the current plugins, and return the result data stream.
+ *
+ * @param upstreamDataStreams the upstream data streams.
+ * @return the result data stream
+ */
+ List<Dataset<Row>> execute(List<Dataset<Row>> upstreamDataStreams) throws Exception;
+}
diff --git a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/SeaTunnelTaskExecution.java b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/SeaTunnelTaskExecution.java
new file mode 100644
index 00000000..d25d1764
--- /dev/null
+++ b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/SeaTunnelTaskExecution.java
@@ -0,0 +1,79 @@
+/*
+ * 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.spark.execution;
+
+import org.apache.seatunnel.api.common.SeaTunnelContext;
+import org.apache.seatunnel.common.constants.JobMode;
+import org.apache.seatunnel.core.base.config.EngineType;
+import org.apache.seatunnel.core.base.config.EnvironmentFactory;
+import org.apache.seatunnel.spark.SparkEnvironment;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class SeaTunnelTaskExecution {
+
+ private static final Logger LOGGER = LoggerFactory.getLogger(SeaTunnelTaskExecution.class);
+
+ private final Config config;
+ private final SparkEnvironment sparkEnvironment;
+ private final PluginExecuteProcessor sourcePluginExecuteProcessor;
+ private final PluginExecuteProcessor transformPluginExecuteProcessor;
+ private final PluginExecuteProcessor sinkPluginExecuteProcessor;
+
+ public SeaTunnelTaskExecution(Config config) {
+ this.config = config;
+ this.sparkEnvironment = getSparkEnvironment(config);
+ this.sourcePluginExecuteProcessor = new SourceExecuteProcessor(sparkEnvironment, config.getConfigList("source"));
+ this.transformPluginExecuteProcessor = new TransformExecuteProcessor(sparkEnvironment, config.getConfigList("transform"));
+ this.sinkPluginExecuteProcessor = new SinkExecuteProcessor(sparkEnvironment, config.getConfigList("sink"));
+ }
+
+ public void execute() throws Exception {
+ List<Dataset<Row>> datasets = new ArrayList<>();
+ datasets = sourcePluginExecuteProcessor.execute(datasets);
+ datasets = transformPluginExecuteProcessor.execute(datasets);
+ sinkPluginExecuteProcessor.execute(datasets);
+
+ LOGGER.info("Spark Execution started");
+ }
+
+ private SparkEnvironment getSparkEnvironment(Config config) {
+ SparkEnvironment sparkEnvironment = (SparkEnvironment) new EnvironmentFactory<>(config, EngineType.SPARK).getEnvironment();
+
+ Config envConfig = config.getConfig("env");
+ JobMode jobMode = JobMode.STREAMING;
+ if (envConfig.hasPath("job.mode")) {
+ jobMode = envConfig.getEnum(JobMode.class, "job.mode");
+ }
+ SeaTunnelContext.getContext().setJobMode(jobMode);
+
+ sparkEnvironment.setJobMode(JobMode.STREAMING)
+ .setConfig(config)
+ .prepare();
+
+ return sparkEnvironment;
+ }
+}
diff --git a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/SinkExecuteProcessor.java
new file mode 100644
index 00000000..08807f7c
--- /dev/null
+++ b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/SinkExecuteProcessor.java
@@ -0,0 +1,77 @@
+/*
+ * 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.spark.execution;
+
+import org.apache.seatunnel.api.sink.SeaTunnelSink;
+import org.apache.seatunnel.common.config.Common;
+import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
+import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSinkPluginDiscovery;
+import org.apache.seatunnel.spark.SparkEnvironment;
+import org.apache.seatunnel.translation.spark.sink.SparkSinkInjector;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import com.google.common.collect.Lists;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class SinkExecuteProcessor extends AbstractPluginExecuteProcessor<SeaTunnelSink<?, ?, ?, ?>> {
+
+ private static final String PLUGIN_TYPE = "sink";
+
+ protected SinkExecuteProcessor(SparkEnvironment sparkEnvironment,
+ List<? extends Config> pluginConfigs) {
+ super(sparkEnvironment, pluginConfigs);
+ }
+
+ @Override
+ protected List<SeaTunnelSink<?, ?, ?, ?>> initializePlugins(List<? extends Config> pluginConfigs) {
+ SeaTunnelSinkPluginDiscovery sinkPluginDiscovery = new SeaTunnelSinkPluginDiscovery();
+ List<URL> pluginJars = new ArrayList<>();
+ List<SeaTunnelSink<?, ?, ?, ?>> sinks = pluginConfigs.stream().map(sinkConfig -> {
+ PluginIdentifier pluginIdentifier = PluginIdentifier.of(ENGINE_TYPE, PLUGIN_TYPE, sinkConfig.getString(PLUGIN_NAME));
+ pluginJars.addAll(sinkPluginDiscovery.getPluginJarPaths(Lists.newArrayList(pluginIdentifier)));
+ SeaTunnelSink<?, ?, ?, ?> seaTunnelSink = sinkPluginDiscovery.getPluginInstance(pluginIdentifier);
+ seaTunnelSink.prepare(sinkConfig);
+ return seaTunnelSink;
+ }).collect(Collectors.toList());
+ sparkEnvironment.registerPlugin(pluginJars);
+ return sinks;
+ }
+
+ @Override
+ public List<Dataset<Row>> execute(List<Dataset<Row>> upstreamDataStreams) throws Exception {
+ Dataset<Row> input = upstreamDataStreams.get(0);
+ for (int i = 0; i < plugins.size(); i++) {
+ Config sinkConfig = pluginConfigs.get(i);
+ SeaTunnelSink<?, ?, ?, ?> seaTunnelSink = plugins.get(i);
+ Dataset<Row> dataset = fromSourceTable(sinkConfig, sparkEnvironment).orElse(input);
+ SparkSinkInjector.inject(dataset.write(), seaTunnelSink, new HashMap<>(Common.COLLECTION_SIZE)).option(
+ "checkpointLocation", "/tmp").save();
+ }
+ // the sink is the last stream
+ return null;
+ }
+
+}
diff --git a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/SourceExecuteProcessor.java
new file mode 100644
index 00000000..c6e42a0a
--- /dev/null
+++ b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/SourceExecuteProcessor.java
@@ -0,0 +1,78 @@
+/*
+ * 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.spark.execution;
+
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+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.spark.SparkEnvironment;
+import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import com.google.common.collect.Lists;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+
+public class SourceExecuteProcessor extends AbstractPluginExecuteProcessor<SeaTunnelSource<?, ?, ?>> {
+
+ private static final String PLUGIN_TYPE = "source";
+
+ public SourceExecuteProcessor(SparkEnvironment sparkEnvironment,
+ List<? extends Config> sourceConfigs) {
+ super(sparkEnvironment, sourceConfigs);
+ }
+
+ @Override
+ public List<Dataset<Row>> execute(List<Dataset<Row>> upstreamDataStreams) {
+ List<Dataset<Row>> sources = new ArrayList<>();
+ for (int i = 0; i < plugins.size(); i++) {
+ SeaTunnelSource<?, ?, ?> source = plugins.get(i);
+ Dataset<Row> dataset = sparkEnvironment.getSparkSession()
+ .read()
+ .format(SeaTunnelSource.class.getSimpleName())
+ .option("source.serialization", SerializationUtils.objectToString(source))
+ .schema(TypeConverterUtils.convertRow(source.getRowTypeInfo())).load();
+ sources.add(dataset);
+ registerInputTempView(pluginConfigs.get(i), dataset);
+ }
+ return sources;
+ }
+
+ @Override
+ protected List<SeaTunnelSource<?, ?, ?>> initializePlugins(List<? extends Config> pluginConfigs) {
+ SeaTunnelSourcePluginDiscovery sourcePluginDiscovery = new SeaTunnelSourcePluginDiscovery();
+ List<SeaTunnelSource<?, ?, ?>> sources = new ArrayList<>();
+ List<URL> jars = new ArrayList<>();
+ 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.getPluginInstance(pluginIdentifier);
+ seaTunnelSource.prepare(sourceConfig);
+ sources.add(seaTunnelSource);
+ }
+ sparkEnvironment.registerPlugin(jars);
+ return sources;
+ }
+}
diff --git a/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/TransformExecuteProcessor.java
new file mode 100644
index 00000000..aa6e56c8
--- /dev/null
+++ b/seatunnel-core/seatunnel-core-spark/src/main/java/org/apache/seatunnel/core/spark/execution/TransformExecuteProcessor.java
@@ -0,0 +1,79 @@
+/*
+ * 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.spark.execution;
+
+import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
+import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSparkTransformPluginDiscovery;
+import org.apache.seatunnel.spark.BaseSparkTransform;
+import org.apache.seatunnel.spark.SparkEnvironment;
+
+import org.apache.seatunnel.shade.com.typesafe.config.Config;
+
+import com.google.common.collect.Lists;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+public class TransformExecuteProcessor extends AbstractPluginExecuteProcessor<BaseSparkTransform> {
+
+ private static final String PLUGIN_TYPE = "transform";
+
+ protected TransformExecuteProcessor(SparkEnvironment sparkEnvironment,
+ List<? extends Config> pluginConfigs) {
+ super(sparkEnvironment, pluginConfigs);
+ }
+
+ @Override
+ protected List<BaseSparkTransform> initializePlugins(List<? extends Config> pluginConfigs) {
+ SeaTunnelSparkTransformPluginDiscovery transformPluginDiscovery = new SeaTunnelSparkTransformPluginDiscovery();
+ List<URL> pluginJars = new ArrayList<>();
+ List<BaseSparkTransform> transforms = pluginConfigs.stream()
+ .map(transformConfig -> {
+ PluginIdentifier pluginIdentifier = PluginIdentifier.of(ENGINE_TYPE, PLUGIN_TYPE, transformConfig.getString(PLUGIN_NAME));
+ pluginJars.addAll(transformPluginDiscovery.getPluginJarPaths(Lists.newArrayList(pluginIdentifier)));
+ BaseSparkTransform pluginInstance = transformPluginDiscovery.getPluginInstance(pluginIdentifier);
+ pluginInstance.setConfig(transformConfig);
+ pluginInstance.prepare(sparkEnvironment);
+ return pluginInstance;
+ }).collect(Collectors.toList());
+ sparkEnvironment.registerPlugin(pluginJars);
+ return transforms;
+ }
+
+ @Override
+ public List<Dataset<Row>> execute(List<Dataset<Row>> upstreamDataStreams) throws Exception {
+ if (plugins.isEmpty()) {
+ return upstreamDataStreams;
+ }
+ Dataset<Row> input = upstreamDataStreams.get(0);
+ List<Dataset<Row>> result = new ArrayList<>();
+ for (int i = 0; i < plugins.size(); i++) {
+ BaseSparkTransform transform = plugins.get(i);
+ Config pluginConfig = pluginConfigs.get(i);
+ Dataset<Row> stream = fromSourceTable(pluginConfig, sparkEnvironment).orElse(input);
+ input = transform.process(stream, sparkEnvironment);
+ registerInputTempView(pluginConfig, input);
+ result.add(input);
+ }
+ return result;
+ }
+}
diff --git a/seatunnel-examples/seatunnel-spark-examples/src/main/java/org/apache/seatunnel/example/spark/SeaTunnelApiExample.java b/seatunnel-examples/seatunnel-spark-examples/src/main/java/org/apache/seatunnel/example/spark/SeaTunnelApiExample.java
index f1d9b5b2..db28d1ea 100644
--- a/seatunnel-examples/seatunnel-spark-examples/src/main/java/org/apache/seatunnel/example/spark/SeaTunnelApiExample.java
+++ b/seatunnel-examples/seatunnel-spark-examples/src/main/java/org/apache/seatunnel/example/spark/SeaTunnelApiExample.java
@@ -33,7 +33,7 @@ import java.nio.file.Paths;
public class SeaTunnelApiExample {
public static void main(String[] args) throws FileNotFoundException, URISyntaxException, CommandException {
- String configFile = getTestConfigFile("/examples/spark.batch.conf");
+ String configFile = getTestConfigFile("/examples/seatunnel-spark.conf");
SparkCommandArgs sparkCommandArgs = new SparkCommandArgs();
sparkCommandArgs.setConfigFile(configFile);
sparkCommandArgs.setCheckConfig(false);
diff --git a/seatunnel-examples/seatunnel-spark-examples/src/main/resources/examples/seatunnel-spark.conf b/seatunnel-examples/seatunnel-spark-examples/src/main/resources/examples/seatunnel-spark.conf
new file mode 100644
index 00000000..45ab040b
--- /dev/null
+++ b/seatunnel-examples/seatunnel-spark-examples/src/main/resources/examples/seatunnel-spark.conf
@@ -0,0 +1,76 @@
+#
+# 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.
+#
+
+######
+###### This config file is a demonstration of batch processing in SeaTunnel config
+######
+
+env {
+ # You can set spark configuration here
+ # see available properties defined by spark: https://spark.apache.org/docs/latest/configuration.html#available-properties
+ #job.mode = BATCH
+ spark.app.name = "SeaTunnel"
+ spark.executor.instances = 2
+ spark.executor.cores = 1
+ spark.executor.memory = "1g"
+ spark.master = local
+}
+
+source {
+ # This is a example input plugin **only for test and demonstrate the feature input plugin**
+ FakeSource {
+ result_table_name = "fake"
+ field_name = "name,age,timestamp"
+ }
+
+ # You can also use other input plugins, such as hdfs
+ # hdfs {
+ # result_table_name = "accesslog"
+ # path = "hdfs://hadoop-cluster-01/nginx/accesslog"
+ # format = "json"
+ # }
+
+ # If you would like to get more information about how to configure seatunnel and see full list of input plugins,
+ # please go to https://seatunnel.apache.org/docs/spark/configuration/source-plugins/Fake
+}
+
+transform {
+ # split data by specific delimiter
+
+ # you can also use other transform plugins, such as sql
+ sql {
+ sql = "select name,age from fake"
+ result_table_name = "sql"
+ }
+
+ # If you would like to get more information about how to configure seatunnel and see full list of transform plugins,
+ # please go to https://seatunnel.apache.org/docs/spark/configuration/transform-plugins/Split
+}
+
+sink {
+ # choose stdout output plugin to output data to console
+ Console {}
+
+ # you can also you other output plugins, such as sql
+ # hdfs {
+ # path = "hdfs://hadoop-cluster-01/nginx/accesslog_processed"
+ # save_mode = "append"
+ # }
+
+ # If you would like to get more information about how to configure seatunnel and see full list of output plugins,
+ # please go to https://seatunnel.apache.org/docs/spark/configuration/sink-plugins/Console
+}