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