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/16 15:24:28 UTC

[incubator-seatunnel] branch api-draft updated: [Api-Draft] SeaTunnel Source support Spark engine. (#1871)

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 862c31e4 [Api-Draft] SeaTunnel Source support Spark engine. (#1871)
862c31e4 is described below

commit 862c31e4cb8841465610bae29439fa31da1baf24
Author: Zongwen Li <zo...@gmail.com>
AuthorDate: Mon May 16 23:24:21 2022 +0800

    [Api-Draft] SeaTunnel Source support Spark engine. (#1871)
---
 .../api/source/SourceSplitEnumerator.java          |   2 +
 .../fake/source/FakeSourceSplitEnumerator.java     |   5 +
 .../InternalRowSerialization.java}                 |  28 ++---
 .../translation/spark/source/Handover.java         |  88 ++++++++++++++
 .../{SparkState.java => InternalRowCollector.java} |  26 ++---
 .../spark/source/ParallelSourceSupport.java        |  61 ----------
 .../spark/source/SeaTunnelSourceSupport.java       |  92 +++++++++++++++
 .../source/batch/BatchParallelSourceReader.java    |  56 +++++++++
 .../{SparkState.java => batch/BatchPartition.java} |  35 +++---
 .../spark/source/batch/BatchPartitionReader.java   | 126 +++++++++++++++++++++
 .../continnous/ContinuousParallelSourceReader.java | 115 +++++++++++++++++++
 .../source/continnous/ContinuousPartition.java     |  52 +++++++++
 .../continnous/ContinuousPartitionReader.java      |  76 +++++++++++++
 .../CoordinationState.java}                        |  33 +++---
 .../ReaderState.java}                              |  39 ++++---
 .../micro/MicroBatchParallelSourceReader.java      |  84 ++++++++++++++
 ...org.apache.spark.sql.sources.DataSourceRegister |   1 +
 17 files changed, 769 insertions(+), 150 deletions(-)

diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SourceSplitEnumerator.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SourceSplitEnumerator.java
index 4ad8df6a..a091e641 100644
--- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SourceSplitEnumerator.java
+++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/source/SourceSplitEnumerator.java
@@ -52,6 +52,8 @@ public interface SourceSplitEnumerator<SplitT extends SourceSplit, StateT> exten
      */
     void addSplitsBack(List<SplitT> splits, int subtaskId);
 
+    int currentUnassignedSplitSize();
+
     void handleSplitRequest(int subtaskId);
 
     void registerReader(int subtaskId);
diff --git a/seatunnel-connectors/seatunnel-connectors-seatunnel/seatunnel-connectors-seatunnel-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java b/seatunnel-connectors/seatunnel-connectors-seatunnel/seatunnel-connectors-seatunnel-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java
index e41d96c0..9b2900c1 100644
--- a/seatunnel-connectors/seatunnel-connectors-seatunnel/seatunnel-connectors-seatunnel-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java
+++ b/seatunnel-connectors/seatunnel-connectors-seatunnel/seatunnel-connectors-seatunnel-fake/src/main/java/org/apache/seatunnel/connectors/seatunnel/fake/source/FakeSourceSplitEnumerator.java
@@ -51,6 +51,11 @@ public class FakeSourceSplitEnumerator implements SourceSplitEnumerator<FakeSour
 
     }
 
+    @Override
+    public int currentUnassignedSplitSize() {
+        return 0;
+    }
+
     @Override
     public void handleSplitRequest(int subtaskId) {
 
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowSerialization.java
similarity index 51%
copy from seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java
copy to seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowSerialization.java
index 09214146..0a0a4378 100644
--- a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowSerialization.java
@@ -15,33 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.seatunnel.translation.spark.source;
+package org.apache.seatunnel.translation.spark.serialization;
 
-import org.apache.seatunnel.api.source.SourceSplit;
 import org.apache.seatunnel.api.table.type.SeaTunnelRow;
-import org.apache.seatunnel.translation.source.ParallelSource;
+import org.apache.seatunnel.translation.serialization.RowSerialization;
 
-import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+import org.apache.spark.sql.catalyst.InternalRow;
 
-import java.util.List;
+import java.io.IOException;
 
-public class SparkState<SplitT extends SourceSplit, StateT> extends Offset {
+// TODO:
+public class InternalRowSerialization implements RowSerialization<InternalRow> {
 
-    protected final ParallelSource<SeaTunnelRow, SplitT, StateT> parallelSource;
-    protected volatile Integer checkpointId;
-
-    public SparkState(ParallelSource<SeaTunnelRow, SplitT, StateT> parallelSource, int checkpointId) {
-        this.parallelSource = parallelSource;
-        this.checkpointId = checkpointId;
+    @Override
+    public InternalRow serialize(SeaTunnelRow seaTunnelRow) throws IOException {
+        return null;
     }
 
     @Override
-    public String json() {
-        try {
-            List<byte[]> bytes = this.parallelSource.snapshotState(checkpointId);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
+    public SeaTunnelRow deserialize(InternalRow engineRow) throws IOException {
         return null;
     }
 }
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/Handover.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/Handover.java
new file mode 100644
index 00000000..0cb6556f
--- /dev/null
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/Handover.java
@@ -0,0 +1,88 @@
+/*
+ * 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.translation.spark.source;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.Closeable;
+import java.util.Optional;
+import java.util.concurrent.LinkedBlockingQueue;
+
+public final class Handover<T> implements Closeable {
+    private final Object lock = new Object();
+    private final LinkedBlockingQueue<T> blockingQueue =
+            new LinkedBlockingQueue<>();
+    private Throwable error;
+
+    public boolean isEmpty() {
+        return blockingQueue.isEmpty();
+    }
+
+    public Optional<T> pollNext() throws Exception {
+        if (error != null) {
+            rethrowException(error, error.getMessage());
+        } else if (!isEmpty()) {
+            return Optional.ofNullable(blockingQueue.poll());
+        }
+        return Optional.empty();
+    }
+
+    public void produce(final T element)
+            throws InterruptedException, ClosedException {
+        if (error != null) {
+            throw new ClosedException();
+        }
+        blockingQueue.put(element);
+    }
+
+    public void reportError(Throwable t) {
+        checkNotNull(t);
+
+        synchronized (lock) {
+            // do not override the initial exception
+            if (error == null) {
+                error = t;
+            }
+            lock.notifyAll();
+        }
+    }
+
+    @Override
+    public void close() {
+        synchronized (lock) {
+            if (error == null) {
+                error = new ClosedException();
+            }
+            lock.notifyAll();
+        }
+    }
+
+    public static void rethrowException(Throwable t, String parentMessage) throws Exception {
+        if (t instanceof Error) {
+            throw (Error) t;
+        } else if (t instanceof Exception) {
+            throw (Exception) t;
+        } else {
+            throw new Exception(parentMessage, t);
+        }
+    }
+
+    public static final class ClosedException extends Exception {
+        private static final long serialVersionUID = 1L;
+    }
+}
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/InternalRowCollector.java
similarity index 58%
copy from seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java
copy to seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/InternalRowCollector.java
index 09214146..593d679e 100644
--- a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/InternalRowCollector.java
@@ -17,31 +17,27 @@
 
 package org.apache.seatunnel.translation.spark.source;
 
-import org.apache.seatunnel.api.source.SourceSplit;
+import org.apache.seatunnel.api.source.Collector;
 import org.apache.seatunnel.api.table.type.SeaTunnelRow;
-import org.apache.seatunnel.translation.source.ParallelSource;
+import org.apache.seatunnel.translation.spark.serialization.InternalRowSerialization;
 
-import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+import org.apache.spark.sql.catalyst.InternalRow;
 
-import java.util.List;
+public class InternalRowCollector implements Collector<SeaTunnelRow> {
+    private final Handover<InternalRow> handover;
+    private final InternalRowSerialization rowSerialization = new InternalRowSerialization();
 
-public class SparkState<SplitT extends SourceSplit, StateT> extends Offset {
-
-    protected final ParallelSource<SeaTunnelRow, SplitT, StateT> parallelSource;
-    protected volatile Integer checkpointId;
-
-    public SparkState(ParallelSource<SeaTunnelRow, SplitT, StateT> parallelSource, int checkpointId) {
-        this.parallelSource = parallelSource;
-        this.checkpointId = checkpointId;
+    public InternalRowCollector(Handover<InternalRow> handover) {
+        this.handover = handover;
     }
 
     @Override
-    public String json() {
+    public void collect(SeaTunnelRow record) {
         try {
-            List<byte[]> bytes = this.parallelSource.snapshotState(checkpointId);
+            // TODO: Lock InternalRowCollector while checkpoint is running
+            handover.produce(rowSerialization.serialize(record));
         } catch (Exception e) {
             throw new RuntimeException(e);
         }
-        return null;
     }
 }
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/ParallelSourceSupport.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/ParallelSourceSupport.java
deleted file mode 100644
index 79f6aff2..00000000
--- a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/ParallelSourceSupport.java
+++ /dev/null
@@ -1,61 +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.translation.spark.source;
-
-import org.apache.spark.sql.sources.DataSourceRegister;
-import org.apache.spark.sql.sources.v2.ContinuousReadSupport;
-import org.apache.spark.sql.sources.v2.DataSourceOptions;
-import org.apache.spark.sql.sources.v2.DataSourceV2;
-import org.apache.spark.sql.sources.v2.MicroBatchReadSupport;
-import org.apache.spark.sql.sources.v2.ReadSupport;
-import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
-import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader;
-import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader;
-import org.apache.spark.sql.types.StructType;
-
-import java.util.Optional;
-
-public class ParallelSourceSupport implements DataSourceV2, ReadSupport, MicroBatchReadSupport, ContinuousReadSupport, DataSourceRegister {
-
-    public static final String SEA_TUNNEL_SOURCE_NAME = "SeaTunnel";
-
-    @Override
-    public String shortName() {
-        return SEA_TUNNEL_SOURCE_NAME;
-    }
-
-    @Override
-    public DataSourceReader createReader(StructType schema, DataSourceOptions options) {
-        return ReadSupport.super.createReader(schema, options);
-    }
-
-    @Override
-    public DataSourceReader createReader(DataSourceOptions options) {
-        return null;
-    }
-
-    @Override
-    public MicroBatchReader createMicroBatchReader(Optional<StructType> schema, String checkpointLocation, DataSourceOptions options) {
-        return null;
-    }
-
-    @Override
-    public ContinuousReader createContinuousReader(Optional<StructType> schema, String checkpointLocation, DataSourceOptions options) {
-        return null;
-    }
-}
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceSupport.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceSupport.java
new file mode 100644
index 00000000..82f467bb
--- /dev/null
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceSupport.java
@@ -0,0 +1,92 @@
+/*
+ * 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.translation.spark.source;
+
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.common.utils.SerializationUtils;
+import org.apache.seatunnel.translation.spark.source.batch.BatchParallelSourceReader;
+import org.apache.seatunnel.translation.spark.source.continnous.ContinuousParallelSourceReader;
+import org.apache.seatunnel.translation.spark.source.micro.MicroBatchParallelSourceReader;
+
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.sources.v2.ContinuousReadSupport;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.apache.spark.sql.sources.v2.DataSourceV2;
+import org.apache.spark.sql.sources.v2.MicroBatchReadSupport;
+import org.apache.spark.sql.sources.v2.ReadSupport;
+import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
+import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader;
+import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.Optional;
+
+public class SeaTunnelSourceSupport implements DataSourceV2, ReadSupport, MicroBatchReadSupport, ContinuousReadSupport, DataSourceRegister {
+
+    public static final String SEA_TUNNEL_SOURCE_NAME = "SeaTunnelSource";
+
+    @Override
+    public String shortName() {
+        return SEA_TUNNEL_SOURCE_NAME;
+    }
+
+    @Override
+    public DataSourceReader createReader(StructType rowType, DataSourceOptions options) {
+        SeaTunnelSource<SeaTunnelRow, ?, ?> seaTunnelSource = getSeaTunnelSource(options);
+        Integer parallelism = options.getInt("source.parallelism", 1);
+        // TODO: case coordinated source
+        return new BatchParallelSourceReader(seaTunnelSource, parallelism, rowType);
+    }
+
+    @Override
+    public DataSourceReader createReader(DataSourceOptions options) {
+        throw createUnspecifiedRowTypeException();
+    }
+
+    @Override
+    public MicroBatchReader createMicroBatchReader(Optional<StructType> rowTypeOptional, String checkpointLocation, DataSourceOptions options) {
+        StructType rowType = checkRowType(rowTypeOptional);
+        SeaTunnelSource<SeaTunnelRow, ?, ?> seaTunnelSource = getSeaTunnelSource(options);
+        Integer parallelism = options.getInt("source.parallelism", 1);
+        // TODO: case coordinated source
+        return new MicroBatchParallelSourceReader(seaTunnelSource, parallelism, rowType);
+    }
+
+    @Override
+    public ContinuousReader createContinuousReader(Optional<StructType> rowTypeOptional, String checkpointLocation, DataSourceOptions options) {
+        StructType rowType = checkRowType(rowTypeOptional);
+        SeaTunnelSource<SeaTunnelRow, ?, ?> seaTunnelSource = getSeaTunnelSource(options);
+        Integer parallelism = options.getInt("source.parallelism", 1);
+        // TODO: case coordinated source
+        return new ContinuousParallelSourceReader(seaTunnelSource, parallelism, rowType);
+    }
+
+    private SeaTunnelSource<SeaTunnelRow, ?, ?> getSeaTunnelSource(DataSourceOptions options) {
+        return SerializationUtils.stringToObject(options.get("source.serialization")
+                .orElseThrow(() -> new UnsupportedOperationException("Serialization information for the SeaTunnelSource is required")));
+    }
+
+    private static StructType checkRowType(Optional<StructType> rowTypeOptional) {
+        return rowTypeOptional.orElseThrow(SeaTunnelSourceSupport::createUnspecifiedRowTypeException);
+    }
+
+    private static RuntimeException createUnspecifiedRowTypeException() {
+        return new UnsupportedOperationException("SeaTunnel Spark source does not support user unspecified row type.");
+    }
+}
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/BatchParallelSourceReader.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/BatchParallelSourceReader.java
new file mode 100644
index 00000000..ae1d6c60
--- /dev/null
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/BatchParallelSourceReader.java
@@ -0,0 +1,56 @@
+/*
+ * 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.translation.spark.source.batch;
+
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
+import org.apache.spark.sql.sources.v2.reader.InputPartition;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class BatchParallelSourceReader implements DataSourceReader {
+
+    protected final SeaTunnelSource<SeaTunnelRow, ?, ?> source;
+    protected final Integer parallelism;
+    protected final StructType rowType;
+
+    public BatchParallelSourceReader(SeaTunnelSource<SeaTunnelRow, ?, ?> source, Integer parallelism, StructType rowType) {
+        this.source = source;
+        this.parallelism = parallelism;
+        this.rowType = rowType;
+    }
+
+    @Override
+    public StructType readSchema() {
+        return rowType;
+    }
+
+    @Override
+    public List<InputPartition<InternalRow>> planInputPartitions() {
+        List<InputPartition<InternalRow>> virtualPartitions = new ArrayList<>(parallelism);
+        for (int subtaskId = 0; subtaskId < parallelism; subtaskId++) {
+            virtualPartitions.add(new BatchPartition(source, parallelism, subtaskId));
+        }
+        return virtualPartitions;
+    }
+}
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/BatchPartition.java
similarity index 50%
copy from seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java
copy to seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/BatchPartition.java
index 09214146..0fd42477 100644
--- a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/BatchPartition.java
@@ -15,33 +15,28 @@
  * limitations under the License.
  */
 
-package org.apache.seatunnel.translation.spark.source;
+package org.apache.seatunnel.translation.spark.source.batch;
 
-import org.apache.seatunnel.api.source.SourceSplit;
+import org.apache.seatunnel.api.source.SeaTunnelSource;
 import org.apache.seatunnel.api.table.type.SeaTunnelRow;
-import org.apache.seatunnel.translation.source.ParallelSource;
 
-import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.reader.InputPartition;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
 
-import java.util.List;
+public class BatchPartition implements InputPartition<InternalRow> {
+    protected final SeaTunnelSource<SeaTunnelRow, ?, ?> source;
+    protected final Integer parallelism;
+    protected final Integer subtaskId;
 
-public class SparkState<SplitT extends SourceSplit, StateT> extends Offset {
-
-    protected final ParallelSource<SeaTunnelRow, SplitT, StateT> parallelSource;
-    protected volatile Integer checkpointId;
-
-    public SparkState(ParallelSource<SeaTunnelRow, SplitT, StateT> parallelSource, int checkpointId) {
-        this.parallelSource = parallelSource;
-        this.checkpointId = checkpointId;
+    public BatchPartition(SeaTunnelSource<SeaTunnelRow, ?, ?> source, Integer parallelism, Integer subtaskId) {
+        this.source = source;
+        this.parallelism = parallelism;
+        this.subtaskId = subtaskId;
     }
 
     @Override
-    public String json() {
-        try {
-            List<byte[]> bytes = this.parallelSource.snapshotState(checkpointId);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-        return null;
+    public InputPartitionReader<InternalRow> createPartitionReader() {
+        return new BatchPartitionReader(source, parallelism, subtaskId);
     }
 }
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/BatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/BatchPartitionReader.java
new file mode 100644
index 00000000..749ddddd
--- /dev/null
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/batch/BatchPartitionReader.java
@@ -0,0 +1,126 @@
+/*
+ * 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.translation.spark.source.batch;
+
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.source.SourceSplit;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.translation.source.ParallelSource;
+import org.apache.seatunnel.translation.spark.source.Handover;
+import org.apache.seatunnel.translation.spark.source.InternalRowCollector;
+import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+public class BatchPartitionReader implements InputPartitionReader<InternalRow> {
+    protected static final Integer INTERVAL = 100;
+
+    protected final SeaTunnelSource<SeaTunnelRow, ?, ?> source;
+    protected final Integer parallelism;
+    protected final Integer subtaskId;
+
+    protected final ExecutorService executorService;
+    protected final Handover<InternalRow> handover;
+
+    protected volatile boolean running = true;
+    protected volatile boolean prepare = true;
+
+    protected volatile InternalParallelSource<?, ?> parallelSource;
+
+    public BatchPartitionReader(SeaTunnelSource<SeaTunnelRow, ?, ?> source, Integer parallelism, Integer subtaskId) {
+        this.source = source;
+        this.parallelism = parallelism;
+        this.subtaskId = subtaskId;
+        this.executorService = ThreadPoolExecutorFactory.createScheduledThreadPoolExecutor(1, String.format("parallel-split-enumerator-executor-%s", subtaskId));
+        this.handover = new Handover<>();
+    }
+
+    @Override
+    public boolean next() throws IOException {
+        prepare();
+        while (running && handover.isEmpty()) {
+            try {
+                Thread.sleep(INTERVAL);
+            } catch (InterruptedException e) {
+                throw new RuntimeException(e);
+            }
+        }
+        return running;
+    }
+
+    private void prepare() {
+        if (!prepare) {
+            return;
+        }
+        this.parallelSource = createInternalParallelSource();
+        try {
+            this.parallelSource.open();
+        } catch (Exception e) {
+            running = false;
+            throw new RuntimeException("");
+        }
+        executorService.execute(() -> {
+            try {
+                parallelSource.run(new InternalRowCollector(handover));
+            } catch (Exception e) {
+                handover.reportError(e);
+            }
+        });
+        prepare = false;
+    }
+
+    protected InternalParallelSource<?, ?> createInternalParallelSource() {
+        return new InternalParallelSource<>(source,
+                null,
+                parallelism,
+                subtaskId);
+    }
+
+    @Override
+    public InternalRow get() {
+        try {
+            return handover.pollNext().get();
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        running = false;
+        parallelSource.close();
+    }
+
+    public class InternalParallelSource<SplitT extends SourceSplit, StateT> extends ParallelSource<SeaTunnelRow, SplitT, StateT> {
+
+        public InternalParallelSource(SeaTunnelSource<SeaTunnelRow, SplitT, StateT> source, List<byte[]> restoredState, int parallelism, int subtaskId) {
+            super(source, restoredState, parallelism, subtaskId);
+        }
+
+        @Override
+        protected void handleNoMoreElement() {
+            super.handleNoMoreElement();
+            running = false;
+        }
+    }
+}
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ContinuousParallelSourceReader.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ContinuousParallelSourceReader.java
new file mode 100644
index 00000000..f4f557d5
--- /dev/null
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ContinuousParallelSourceReader.java
@@ -0,0 +1,115 @@
+/*
+ * 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.translation.spark.source.continnous;
+
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.common.utils.SerializationUtils;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.reader.InputPartition;
+import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+public class ContinuousParallelSourceReader implements ContinuousReader {
+
+    private final SeaTunnelSource<SeaTunnelRow, ?, ?> source;
+    private final Integer parallelism;
+    private final StructType rowType;
+    private Map<Integer, ReaderState> readerStateMap = new HashMap<>();
+    private CoordinationState coordinationState;
+    private int checkpointId = 1;
+
+    public ContinuousParallelSourceReader(SeaTunnelSource<SeaTunnelRow, ?, ?> source, Integer parallelism, StructType rowType) {
+        this.source = source;
+        this.parallelism = parallelism;
+        this.rowType = rowType;
+    }
+
+    @Override
+    public Offset mergeOffsets(PartitionOffset[] subtaskStates) {
+        // aggregate state
+        List<ReaderState> readerStateList = new ArrayList<>(subtaskStates.length);
+        for (PartitionOffset subtaskState : subtaskStates) {
+            if (subtaskState instanceof ReaderState) {
+                ReaderState readerState = (ReaderState) subtaskState;
+                readerStateMap.put(readerState.getSubtaskId(), readerState);
+                readerStateList.add(readerState);
+            } else {
+                throw new UnsupportedOperationException(String.format("Unsupported state type: %s", subtaskState.getClass()));
+            }
+        }
+        return new CoordinationState(readerStateList, readerStateList.get(0).getCheckpointId());
+    }
+
+    @Override
+    public Offset deserializeOffset(String aggregatedState) {
+        return SerializationUtils.stringToObject(aggregatedState);
+    }
+
+    @Override
+    public void setStartOffset(Optional<Offset> start) {
+        // initialize or restore state
+        start.ifPresent(state -> {
+            CoordinationState restoreState = (CoordinationState) state;
+            checkpointId = restoreState.getCheckpointId();
+            for (ReaderState readerState : restoreState.getReaderStateList()) {
+                readerStateMap.put(readerState.getSubtaskId(), readerState);
+            }
+        });
+        coordinationState = (CoordinationState) start.orElse(new CoordinationState(new ArrayList<>(), 1));
+    }
+
+    @Override
+    public Offset getStartOffset() {
+        return coordinationState;
+    }
+
+    @Override
+    public void commit(Offset end) {
+        // TODO: rpc commit {@link ContinuousPartitionReader#notifyCheckpointComplete}
+    }
+
+    @Override
+    public void stop() {
+        // TODO: stop rpc
+    }
+
+    @Override
+    public StructType readSchema() {
+        return rowType;
+    }
+
+    @Override
+    public List<InputPartition<InternalRow>> planInputPartitions() {
+        List<InputPartition<InternalRow>> virtualPartitions = new ArrayList<>(parallelism);
+        for (int subtaskId = 0; subtaskId < parallelism; subtaskId++) {
+            ReaderState readerState = readerStateMap.get(subtaskId);
+            virtualPartitions.add(new ContinuousPartition(source, parallelism, subtaskId, checkpointId, readerState == null ? null : readerState.getBytes()));
+        }
+        return virtualPartitions;
+    }
+}
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ContinuousPartition.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ContinuousPartition.java
new file mode 100644
index 00000000..49200301
--- /dev/null
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ContinuousPartition.java
@@ -0,0 +1,52 @@
+/*
+ * 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.translation.spark.source.continnous;
+
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.reader.InputPartition;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
+
+import java.util.List;
+
+public class ContinuousPartition implements InputPartition<InternalRow> {
+    protected final SeaTunnelSource<SeaTunnelRow, ?, ?> source;
+    protected final Integer parallelism;
+    protected final Integer subtaskId;
+    protected final Integer checkpointId;
+    protected final List<byte[]> restoredState;
+
+    public ContinuousPartition(SeaTunnelSource<SeaTunnelRow, ?, ?> source,
+                               Integer parallelism,
+                               Integer subtaskId,
+                               Integer checkpointId,
+                               List<byte[]> restoredState) {
+        this.source = source;
+        this.parallelism = parallelism;
+        this.subtaskId = subtaskId;
+        this.checkpointId = checkpointId;
+        this.restoredState = restoredState;
+    }
+
+    @Override
+    public InputPartitionReader<InternalRow> createPartitionReader() {
+        return new ContinuousPartitionReader(source, parallelism, subtaskId, checkpointId, restoredState);
+    }
+}
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ContinuousPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ContinuousPartitionReader.java
new file mode 100644
index 00000000..ebd25d09
--- /dev/null
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ContinuousPartitionReader.java
@@ -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.
+ */
+
+package org.apache.seatunnel.translation.spark.source.continnous;
+
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+import org.apache.seatunnel.translation.spark.source.batch.BatchPartitionReader;
+
+import org.apache.spark.TaskContext;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousInputPartitionReader;
+import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset;
+
+import java.io.IOException;
+import java.util.List;
+
+public class ContinuousPartitionReader extends BatchPartitionReader implements ContinuousInputPartitionReader<InternalRow> {
+    protected volatile Integer checkpointId;
+    protected final List<byte[]> restoredState;
+
+    public ContinuousPartitionReader(SeaTunnelSource<SeaTunnelRow, ?, ?> source, Integer parallelism, Integer subtaskId, Integer checkpointId, List<byte[]> restoredState) {
+        super(source, parallelism, subtaskId);
+        this.checkpointId = checkpointId;
+        this.restoredState = restoredState;
+    }
+
+    @Override
+    protected InternalParallelSource<?, ?> createInternalParallelSource() {
+        return new InternalParallelSource<>(source,
+                restoredState,
+                parallelism,
+                subtaskId);
+    }
+
+    @Override
+    public PartitionOffset getOffset() {
+        List<byte[]> bytes;
+        try {
+            bytes = parallelSource.snapshotState(checkpointId);
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+        ReaderState readerState = new ReaderState(bytes, subtaskId, checkpointId++);
+        TaskContext.get();
+        return readerState;
+    }
+
+    // TODO: RPC call
+    /**
+     * The method is called by RPC
+     */
+    public void notifyCheckpointComplete(long checkpointId) throws Exception {
+        parallelSource.notifyCheckpointComplete(checkpointId);
+    }
+
+    @Override
+    public void close() throws IOException {
+        super.close();
+        // TODO: close rpc
+    }
+}
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/CoordinationState.java
similarity index 55%
copy from seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java
copy to seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/CoordinationState.java
index 09214146..b0e90414 100644
--- a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/CoordinationState.java
@@ -15,33 +15,34 @@
  * limitations under the License.
  */
 
-package org.apache.seatunnel.translation.spark.source;
+package org.apache.seatunnel.translation.spark.source.continnous;
 
-import org.apache.seatunnel.api.source.SourceSplit;
-import org.apache.seatunnel.api.table.type.SeaTunnelRow;
-import org.apache.seatunnel.translation.source.ParallelSource;
+import org.apache.seatunnel.common.utils.SerializationUtils;
 
 import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
 
+import java.io.Serializable;
 import java.util.List;
 
-public class SparkState<SplitT extends SourceSplit, StateT> extends Offset {
+public class CoordinationState extends Offset implements Serializable {
+    private List<ReaderState> readerStateList;
+    private Integer checkpointId;
 
-    protected final ParallelSource<SeaTunnelRow, SplitT, StateT> parallelSource;
-    protected volatile Integer checkpointId;
-
-    public SparkState(ParallelSource<SeaTunnelRow, SplitT, StateT> parallelSource, int checkpointId) {
-        this.parallelSource = parallelSource;
+    public CoordinationState(List<ReaderState> readerStateList, Integer checkpointId) {
+        this.readerStateList = readerStateList;
         this.checkpointId = checkpointId;
     }
 
     @Override
     public String json() {
-        try {
-            List<byte[]> bytes = this.parallelSource.snapshotState(checkpointId);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-        return null;
+        return SerializationUtils.objectToString(this);
+    }
+
+    public List<ReaderState> getReaderStateList() {
+        return readerStateList;
+    }
+
+    public Integer getCheckpointId() {
+        return checkpointId;
     }
 }
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ReaderState.java
similarity index 50%
rename from seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java
rename to seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ReaderState.java
index 09214146..becbd321 100644
--- a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/SparkState.java
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/continnous/ReaderState.java
@@ -15,33 +15,32 @@
  * limitations under the License.
  */
 
-package org.apache.seatunnel.translation.spark.source;
+package org.apache.seatunnel.translation.spark.source.continnous;
 
-import org.apache.seatunnel.api.source.SourceSplit;
-import org.apache.seatunnel.api.table.type.SeaTunnelRow;
-import org.apache.seatunnel.translation.source.ParallelSource;
-
-import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset;
 
 import java.util.List;
 
-public class SparkState<SplitT extends SourceSplit, StateT> extends Offset {
-
-    protected final ParallelSource<SeaTunnelRow, SplitT, StateT> parallelSource;
-    protected volatile Integer checkpointId;
+public class ReaderState implements PartitionOffset {
+    private final List<byte[]> bytes;
+    private final Integer subtaskId;
+    private final Integer checkpointId;
 
-    public SparkState(ParallelSource<SeaTunnelRow, SplitT, StateT> parallelSource, int checkpointId) {
-        this.parallelSource = parallelSource;
+    public ReaderState(List<byte[]> bytes, Integer subtaskId, Integer checkpointId) {
+        this.bytes = bytes;
+        this.subtaskId = subtaskId;
         this.checkpointId = checkpointId;
     }
 
-    @Override
-    public String json() {
-        try {
-            List<byte[]> bytes = this.parallelSource.snapshotState(checkpointId);
-        } catch (Exception e) {
-            throw new RuntimeException(e);
-        }
-        return null;
+    public List<byte[]> getBytes() {
+        return bytes;
+    }
+
+    public Integer getSubtaskId() {
+        return subtaskId;
+    }
+
+    public Integer getCheckpointId() {
+        return checkpointId;
     }
 }
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/micro/MicroBatchParallelSourceReader.java b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/micro/MicroBatchParallelSourceReader.java
new file mode 100644
index 00000000..5bc25186
--- /dev/null
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/java/org/apache/seatunnel/translation/spark/source/micro/MicroBatchParallelSourceReader.java
@@ -0,0 +1,84 @@
+/*
+ * 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.translation.spark.source.micro;
+
+import org.apache.seatunnel.api.source.SeaTunnelSource;
+import org.apache.seatunnel.api.table.type.SeaTunnelRow;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.reader.InputPartition;
+import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.Optional;
+
+public class MicroBatchParallelSourceReader implements MicroBatchReader {
+
+    protected final SeaTunnelSource<SeaTunnelRow, ?, ?> source;
+    protected final Integer parallelism;
+    protected final StructType rowType;
+
+    public MicroBatchParallelSourceReader(SeaTunnelSource<SeaTunnelRow, ?, ?> source, Integer parallelism, StructType rowType) {
+        this.source = source;
+        this.parallelism = parallelism;
+        this.rowType = rowType;
+    }
+
+    @Override
+    public void setOffsetRange(Optional<Offset> start, Optional<Offset> end) {
+
+    }
+
+    @Override
+    public Offset getStartOffset() {
+        return null;
+    }
+
+    @Override
+    public Offset getEndOffset() {
+        // TODO: How to implement micro-batch?
+        return null;
+    }
+
+    @Override
+    public Offset deserializeOffset(String json) {
+        return null;
+    }
+
+    @Override
+    public void commit(Offset end) {
+
+    }
+
+    @Override
+    public void stop() {
+
+    }
+
+    @Override
+    public StructType readSchema() {
+        return null;
+    }
+
+    @Override
+    public List<InputPartition<InternalRow>> planInputPartitions() {
+        return null;
+    }
+}
diff --git a/seatunnel-translation/seatunnel-translation-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/seatunnel-translation/seatunnel-translation-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
new file mode 100644
index 00000000..2a5363d0
--- /dev/null
+++ b/seatunnel-translation/seatunnel-translation-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -0,0 +1 @@
+org.apache.seatunnel.translation.spark.source.SeaTunnelSourceSupport
\ No newline at end of file