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