From 2e8248211b3f026f4b50a230de9417a958c9a778 Mon Sep 17 00:00:00 2001 From: CheneyYin Date: Sun, 1 Sep 2024 10:31:05 +0800 Subject: [PATCH 1/2] [WIP][feature][spark] Support streaming --- .../api/table/type/SeaTunnelRow.java | 13 + .../apache/seatunnel/common/Constants.java | 2 + .../spark/execution/SinkExecuteProcessor.java | 21 +- .../spark/execution/SinkExecuteProcessor.java | 31 +- .../execution/SourceExecuteProcessor.java | 37 ++- .../SparkAbstractPluginExecuteProcessor.java | 12 + .../execution/TransformExecuteProcessor.java | 17 +- .../reader/micro/MicroBatchSourceReader.java | 1 + .../spark/sink/SeaTunnelSinkTable.java | 6 +- .../sink/{ => write}/SeaTunnelBatchWrite.java | 18 +- .../sink/write/SeaTunnelSparkDataWriter.java | 120 ++++++- .../SeaTunnelSparkDataWriterFactory.java | 45 ++- .../spark/sink/write/SeaTunnelWrite.java | 18 +- .../sink/write/SeaTunnelWriteBuilder.java | 15 +- .../spark/source/SeaTunnelSourceTable.java | 7 +- .../batch/InternalParallelBatchSource.java | 50 +++ .../batch/ParallelBatchPartitionReader.java | 25 +- .../SeaTunnelContinuousPartitionReader.java | 105 ++++++ ...unnelContinuousPartitionReaderFactory.java | 36 ++ .../continuous/SeaTunnelContinuousStream.java | 89 +++++ .../continuous/SeaTunnelInputPartition.java | 22 ++ .../partition/continuous/SeaTunnelOffset.java | 27 ++ .../continuous/SeaTunnelPartitionOffset.java | 22 ++ .../micro/CheckpointDataLogManager.java | 110 ++++++ .../CoordinatedMicroBatchPartitionReader.java | 75 +---- .../micro/InternalCoordinatedSource.java | 99 ++++++ .../InternalParallelMicroBatchSource.java | 51 +++ .../ParallelMicroBatchPartitionReader.java | 312 ++++++++++++++---- .../partition/micro/SeaTunnelMicroBatch.java | 27 +- .../SeaTunnelMicroBatchPartitionReader.java | 6 +- ...unnelMicroBatchPartitionReaderFactory.java | 5 +- .../partition/micro/SeaTunnelOffset.java | 4 +- .../spark/source/scan/SeaTunnelScan.java | 15 +- .../source/scan/SeaTunnelScanBuilder.java | 2 +- .../rpc/SplitEnumeratorEndPoint.java | 38 +++ .../translation/spark/sink/SparkSinkTest.java | 15 + .../spark/execution/CheckpointMetadata.java | 90 +++++ .../serialization/InternalRowConverter.java | 85 ++++- .../serialization/SeaTunnelRowConverter.java | 38 ++- .../spark/utils/CaseInsensitiveStringMap.java | 190 +++++++++++ .../spark/utils/TypeConverterUtils.java | 15 +- .../execution/MultiTableManagerTest.java | 232 ++++++------- 42 files changed, 1790 insertions(+), 358 deletions(-) rename seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/{ => write}/SeaTunnelBatchWrite.java (88%) create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/InternalParallelBatchSource.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReader.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReaderFactory.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousStream.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelInputPartition.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelOffset.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelPartitionOffset.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CheckpointDataLogManager.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/InternalCoordinatedSource.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/InternalParallelMicroBatchSource.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndPoint.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/CheckpointMetadata.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/CaseInsensitiveStringMap.java diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java index 10a5b33a935..2672380ef4c 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/type/SeaTunnelRow.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -30,6 +31,8 @@ public final class SeaTunnelRow implements Serializable { private String tableId = ""; /** The kind of change that a row describes in a changelog. */ private RowKind rowKind = RowKind.INSERT; + + private Map metadata = new HashMap<>(); /** The array to store the actual internal format values. */ private final Object[] fields; @@ -67,6 +70,14 @@ public RowKind getRowKind() { return this.rowKind; } + public Map getMetadata() { + return this.metadata; + } + + public void setMetadata(Map metadata) { + this.metadata = metadata; + } + public Object[] getFields() { return fields; } @@ -81,6 +92,7 @@ public SeaTunnelRow copy() { SeaTunnelRow newRow = new SeaTunnelRow(newFields); newRow.setRowKind(this.getRowKind()); newRow.setTableId(this.getTableId()); + newRow.setMetadata(this.getMetadata()); return newRow; } @@ -92,6 +104,7 @@ public SeaTunnelRow copy(int[] indexMapping) { SeaTunnelRow newRow = new SeaTunnelRow(newFields); newRow.setRowKind(this.getRowKind()); newRow.setTableId(this.getTableId()); + newRow.setMetadata(this.getMetadata()); return newRow; } diff --git a/seatunnel-common/src/main/java/org/apache/seatunnel/common/Constants.java b/seatunnel-common/src/main/java/org/apache/seatunnel/common/Constants.java index 602143e100c..1d0ae8ac129 100644 --- a/seatunnel-common/src/main/java/org/apache/seatunnel/common/Constants.java +++ b/seatunnel-common/src/main/java/org/apache/seatunnel/common/Constants.java @@ -39,6 +39,8 @@ public final class Constants { public static final String CHECKPOINT_ID = "checkpoint.id"; + public static final String CHECKPOINT_LOCATION = "checkpointLocation"; + public static final String UUID = "uuid"; public static final String NOW = "now"; diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java index 48f8cab8e1d..26148d2579a 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java @@ -41,6 +41,8 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.streaming.StreamingQuery; +import org.apache.spark.sql.streaming.StreamingQueryException; import java.net.URL; import java.util.ArrayList; @@ -124,9 +126,22 @@ public List execute(List upstreamDataStreams sparkRuntimeEnvironment.getSparkSession().sparkContext().applicationId(); CatalogTable[] catalogTables = datasetTableInfo.getCatalogTables().toArray(new CatalogTable[0]); - SparkSinkInjector.inject(dataset.write(), sink, catalogTables, applicationId) - .option("checkpointLocation", "/tmp") - .save(); + if (isStreaming()) { + StreamingQuery streamingQuery = + SparkSinkInjector.inject( + dataset.writeStream(), sink, catalogTables, applicationId) + .option("checkpointLocation", "/tmp/test-spark-seatunnel") + .start(); + try { + streamingQuery.awaitTermination(); + } catch (StreamingQueryException e) { + throw new RuntimeException(e); + } + } else { + SparkSinkInjector.inject(dataset.write(), sink, catalogTables, applicationId) + .option("checkpointLocation", "/tmp/test-spark-seatunnel") + .save(); + } } // the sink is the last stream return null; diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java index 0b54e2a115d..3d6e9a3f394 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java @@ -42,12 +42,16 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.streaming.OutputMode; +import org.apache.spark.sql.streaming.StreamingQuery; +import org.apache.spark.sql.streaming.StreamingQueryException; import java.net.URL; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import static org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode.HANDLE_SAVE_MODE_FAILED; @@ -124,10 +128,29 @@ public List execute(List upstreamDataStreams sparkRuntimeEnvironment.getStreamingContext().sparkContext().applicationId(); CatalogTable[] catalogTables = datasetTableInfo.getCatalogTables().toArray(new CatalogTable[0]); - SparkSinkInjector.inject(dataset.write(), sink, catalogTables, applicationId) - .option("checkpointLocation", "/tmp") - .mode(SaveMode.Append) - .save(); + if (isStreaming()) { + try { + StreamingQuery streamingQuery = + SparkSinkInjector.inject( + dataset.writeStream(), + sink, + catalogTables, + applicationId) + .option("checkpointLocation", "/tmp/test-spark-seatunnel") + .outputMode(OutputMode.Append()) + .start(); + streamingQuery.awaitTermination(); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } catch (StreamingQueryException e) { + throw new RuntimeException(e); + } + } else { + SparkSinkInjector.inject(dataset.write(), sink, catalogTables, applicationId) + .option("checkpointLocation", "/tmp/test-spark-seatunnel") + .mode(SaveMode.Append) + .save(); + } } // the sink is the last stream return null; diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java index d4a4038e485..459c8016a1b 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SourceExecuteProcessor.java @@ -35,6 +35,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; import com.google.common.collect.Lists; @@ -86,17 +87,31 @@ public List execute(List upstreamDataStreams CommonOptions.PARALLELISM.key(), CommonOptions.PARALLELISM.defaultValue()); } - Dataset dataset = - sparkRuntimeEnvironment - .getSparkSession() - .read() - .format(SeaTunnelSource.class.getSimpleName()) - .option(CommonOptions.PARALLELISM.key(), parallelism) - .option( - Constants.SOURCE_SERIALIZATION, - SerializationUtils.objectToString(source)) - .options(envOption) - .load(); + SparkSession sparkSession = sparkRuntimeEnvironment.getSparkSession(); + Dataset dataset = null; + if (isStreaming()) { + dataset = + sparkSession + .readStream() + .format(SeaTunnelSource.class.getSimpleName()) + .option(CommonOptions.PARALLELISM.key(), parallelism) + .option( + Constants.SOURCE_SERIALIZATION, + SerializationUtils.objectToString(source)) + .options(envOption) + .load(); + } else { + dataset = + sparkSession + .read() + .format(SeaTunnelSource.class.getSimpleName()) + .option(CommonOptions.PARALLELISM.key(), parallelism) + .option( + Constants.SOURCE_SERIALIZATION, + SerializationUtils.objectToString(source)) + .options(envOption) + .load(); + } sources.add( new DatasetTableInfo( dataset, diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java index f1c138285ad..31d0d4c1243 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SparkAbstractPluginExecuteProcessor.java @@ -20,6 +20,8 @@ import org.apache.seatunnel.shade.com.typesafe.config.Config; import org.apache.seatunnel.api.common.JobContext; +import org.apache.seatunnel.api.env.EnvCommonOptions; +import org.apache.seatunnel.common.constants.JobMode; import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.core.starter.execution.PluginExecuteProcessor; import org.apache.seatunnel.translation.spark.execution.DatasetTableInfo; @@ -51,6 +53,16 @@ protected SparkAbstractPluginExecuteProcessor( this.plugins = initializePlugins(pluginConfigs); } + public boolean isStreaming() { + if (sparkRuntimeEnvironment.getConfig().hasPath(EnvCommonOptions.JOB_MODE.key())) { + return sparkRuntimeEnvironment + .getConfig() + .getString(EnvCommonOptions.JOB_MODE.key()) + .equalsIgnoreCase(JobMode.STREAMING.name()); + } + return false; + } + @Override public void setRuntimeEnvironment(SparkRuntimeEnvironment sparkRuntimeEnvironment) { this.sparkRuntimeEnvironment = sparkRuntimeEnvironment; diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java index 3736576817d..23ad27272e3 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java @@ -31,6 +31,7 @@ import org.apache.seatunnel.core.starter.execution.PluginUtil; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelFactoryDiscovery; import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelTransformPluginDiscovery; +import org.apache.seatunnel.translation.spark.execution.CheckpointMetadata; import org.apache.seatunnel.translation.spark.execution.DatasetTableInfo; import org.apache.seatunnel.translation.spark.serialization.SeaTunnelRowConverter; import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; @@ -180,19 +181,19 @@ private Dataset sparkTransform(SeaTunnelTransform transform, DatasetTableIn private static class TransformIterator implements Iterator, Serializable { private Iterator sourceIterator; private SeaTunnelTransform transform; - private StructType structType; + private StructType outputSchema; private SeaTunnelRowConverter inputRowConverter; private SeaTunnelRowConverter outputRowConverter; public TransformIterator( Iterator sourceIterator, SeaTunnelTransform transform, - StructType structType, + StructType outputSchema, SeaTunnelRowConverter inputRowConverter, SeaTunnelRowConverter outputRowConverter) { this.sourceIterator = sourceIterator; this.transform = transform; - this.structType = structType; + this.outputSchema = outputSchema; this.inputRowConverter = inputRowConverter; this.outputRowConverter = outputRowConverter; } @@ -207,6 +208,16 @@ public Row next() { try { Row row = sourceIterator.next(); SeaTunnelRow seaTunnelRow = inputRowConverter.unpack((GenericRowWithSchema) row); + if (CheckpointMetadata.of(seaTunnelRow.getMetadata()).isCheckpoint()) { + /* + * Skip checkpoint event + */ + return outputRowConverter.checkpointEvent( + outputSchema, + seaTunnelRow.getRowKind(), + seaTunnelRow.getTableId(), + CheckpointMetadata.of(seaTunnelRow.getMetadata())); + } seaTunnelRow = (SeaTunnelRow) transform.map(seaTunnelRow); if (seaTunnelRow == null) { return null; diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/MicroBatchSourceReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/MicroBatchSourceReader.java index f28c8c1b794..d8ca8da6d50 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/MicroBatchSourceReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-2.4/src/main/java/org/apache/seatunnel/translation/spark/source/reader/micro/MicroBatchSourceReader.java @@ -102,6 +102,7 @@ public Offset deserializeOffset(String microBatchState) { @Override public void commit(Offset end) { // nothing + System.out.println(end); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelSinkTable.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelSinkTable.java index f3652be673f..193e03867ef 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelSinkTable.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelSinkTable.java @@ -47,10 +47,13 @@ public class SeaTunnelSinkTable implements Table, SupportsWrite { private final SeaTunnelSink sink; private final CatalogTable[] catalogTables; + private final String checkpointLocation; private final String jobId; public SeaTunnelSinkTable(Map properties) { this.properties = properties; + this.checkpointLocation = + properties.getOrDefault(Constants.CHECKPOINT_LOCATION, "/tmp/seatunnel"); String sinkSerialization = properties.getOrDefault(Constants.SINK_SERIALIZATION, ""); if (StringUtils.isBlank(sinkSerialization)) { throw new IllegalArgumentException(Constants.SINK_SERIALIZATION + " must be specified"); @@ -68,7 +71,8 @@ public SeaTunnelSinkTable(Map properties) { @Override public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { - return new SeaTunnelWriteBuilder<>(sink, catalogTables, jobId); + return new SeaTunnelWriteBuilder<>( + sink, properties, catalogTables, info.schema(), checkpointLocation, jobId); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelBatchWrite.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelBatchWrite.java similarity index 88% rename from seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelBatchWrite.java rename to seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelBatchWrite.java index f1c343ba56f..b7205275bf0 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/SeaTunnelBatchWrite.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelBatchWrite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.translation.spark.sink; +package org.apache.seatunnel.translation.spark.sink.write; import org.apache.seatunnel.api.sink.MultiTableResourceManager; import org.apache.seatunnel.api.sink.SeaTunnelSink; @@ -23,8 +23,6 @@ import org.apache.seatunnel.api.sink.SupportResourceShare; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.translation.spark.sink.write.SeaTunnelSparkDataWriterFactory; -import org.apache.seatunnel.translation.spark.sink.write.SeaTunnelSparkWriterCommitMessage; import org.apache.spark.sql.connector.write.BatchWrite; import org.apache.spark.sql.connector.write.DataWriterFactory; @@ -32,11 +30,13 @@ import org.apache.spark.sql.connector.write.WriterCommitMessage; import org.apache.spark.sql.connector.write.streaming.StreamingDataWriterFactory; import org.apache.spark.sql.connector.write.streaming.StreamingWrite; +import org.apache.spark.sql.types.StructType; import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; @@ -48,18 +48,27 @@ public class SeaTunnelBatchWrite private final SinkAggregatedCommitter aggregatedCommitter; private MultiTableResourceManager resourceManager; + private final Map properties; private final CatalogTable[] catalogTables; + private final StructType schema; + private final String checkpointLocation; private final String jobId; public SeaTunnelBatchWrite( SeaTunnelSink sink, + Map properties, CatalogTable[] catalogTables, + StructType schema, + String checkpointLocation, String jobId) throws IOException { this.sink = sink; + this.properties = properties; this.catalogTables = catalogTables; + this.schema = schema; + this.checkpointLocation = checkpointLocation; this.jobId = jobId; this.aggregatedCommitter = sink.createAggregatedCommitter().orElse(null); if (aggregatedCommitter != null) { @@ -78,7 +87,8 @@ public SeaTunnelBatchWrite( @Override public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - return new SeaTunnelSparkDataWriterFactory<>(sink, catalogTables, jobId); + return new SeaTunnelSparkDataWriterFactory<>( + sink, properties, catalogTables, schema, checkpointLocation, jobId); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java index c2c24aa9147..e5dc02fde3c 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriter.java @@ -23,18 +23,31 @@ import org.apache.seatunnel.api.sink.SupportResourceShare; import org.apache.seatunnel.api.sink.event.WriterCloseEvent; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.Constants; +import org.apache.seatunnel.translation.spark.execution.CheckpointMetadata; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; +import org.apache.seatunnel.translation.spark.serialization.InternalRowConverter; +import org.apache.seatunnel.translation.spark.source.partition.micro.CheckpointDataLogManager; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.write.DataWriter; import org.apache.spark.sql.connector.write.WriterCommitMessage; +import org.apache.spark.sql.execution.streaming.CheckpointFileManager; +import org.apache.spark.sql.types.StructType; import lombok.extern.slf4j.Slf4j; import javax.annotation.Nullable; import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; import java.util.Collections; +import java.util.Map; import java.util.Optional; @Slf4j @@ -46,29 +59,118 @@ public class SeaTunnelSparkDataWriter implements DataWriter protected CommitInfoT latestCommitInfoT; protected long epochId; protected volatile MultiTableResourceManager resourceManager; + private final Map properties; private final MultiTableManager multiTableManager; private final SinkWriter.Context context; + private final StructType schema; + private final Path checkpointLocation; + private final int partitionId; + + private FileSystem fs; + + private CheckpointDataLogManager logManager = null; public SeaTunnelSparkDataWriter( SinkWriter sinkWriter, @Nullable SinkCommitter sinkCommitter, MultiTableManager multiTableManager, + StructType schema, + Map properties, + String checkpointLocation, + int partitionId, long epochId, SinkWriter.Context context) { this.sinkWriter = sinkWriter; this.sinkCommitter = sinkCommitter; this.multiTableManager = multiTableManager; + this.checkpointLocation = new Path(checkpointLocation, "sinks"); + this.partitionId = partitionId; this.epochId = epochId == 0 ? 1 : epochId; + this.properties = properties; + this.schema = schema; this.context = context; initResourceManger(); } @Override public void write(InternalRow record) throws IOException { + Map metadata = InternalRowConverter.unpackMetadata(record); + CheckpointMetadata checkpointMetadata = CheckpointMetadata.of(metadata); + if (checkpointMetadata.isCheckpoint()) { + String location = checkpointMetadata.location(); + int batchId = checkpointMetadata.batchId(); + int subTaskId = checkpointMetadata.subTaskId(); + int checkpointId = checkpointMetadata.checkpointId(); + ackCheckpoint(location, batchId, subTaskId, checkpointId); + return; + } sinkWriter.write(multiTableManager.reconvert(record)); } + private void ackCheckpoint(String location, int batchId, int subTaskId, int checkpointId) { + try { + Optional commitInfoTOptional = sinkWriter.prepareCommit(); + sinkWriter.snapshotState(epochId++); + if (logManager == null) { + Configuration configuration = new Configuration(); + String hdfsRoot = + properties.getOrDefault( + Constants.HDFS_ROOT, + FileSystem.getDefaultUri(configuration).toString()); + String hdfsUser = properties.getOrDefault(Constants.HDFS_USER, ""); + this.logManager = + new CheckpointDataLogManager( + CheckpointFileManager.create( + this.checkpointLocation, configuration)); + } + if (fs == null) { + fs = getFileSystem(); + } + Path preCommitPath = + logManager.preparedCommittedFilePath( + location, batchId, subTaskId, checkpointId); + if (!fs.exists(preCommitPath)) { + throw new IllegalStateException( + String.format( + "The source's snapshot(%s)[batchId=%d,subTaskId=%d,checkpointId=%d] must be existed.", + preCommitPath.toString(), batchId, subTaskId, checkpointId)); + } + + if (!fs.getFileStatus(preCommitPath).isFile()) { + throw new IllegalStateException( + String.format( + "The source's snapshot(%s)[batchId=%d,subTaskId=%d,checkpointId=%d] must be one file.", + preCommitPath.toString(), batchId, subTaskId, checkpointId)); + } + + Path ackCommitFilePath = + logManager.committedFilePath(location, batchId, subTaskId, checkpointId); + fs.rename(preCommitPath, ackCommitFilePath); + } catch (IOException | URISyntaxException | InterruptedException e) { + throw new IllegalStateException( + String.format( + "Fail to ack the source's snapshot[batchId=%d,subTaskId=%d,checkpointId=%d].", + batchId, subTaskId, checkpointId), + e); + } + } + + private FileSystem getFileSystem() + throws URISyntaxException, IOException, InterruptedException { + Configuration configuration = new Configuration(); + String hdfsRoot = + properties.getOrDefault( + Constants.HDFS_ROOT, FileSystem.getDefaultUri(configuration).toString()); + String hdfsUser = properties.getOrDefault(Constants.HDFS_USER, ""); + configuration.set("fs.defaultFS", hdfsRoot); + if (StringUtils.isNotBlank(hdfsUser)) { + return FileSystem.get(new URI(hdfsRoot), configuration, hdfsUser); + } else { + return FileSystem.get(new URI(hdfsRoot), configuration); + } + } + protected void initResourceManger() { if (sinkWriter instanceof SupportResourceShare) { resourceManager = @@ -115,6 +217,10 @@ public void abort() throws IOException { } } cleanCommitInfo(); + if (this.fs != null) { + this.fs.close(); + } + sinkWriter.close(); } private void cleanCommitInfo() { @@ -122,5 +228,17 @@ private void cleanCommitInfo() { } @Override - public void close() throws IOException {} + public void close() throws IOException { + if (fs != null) { + this.fs.close(); + } + if (sinkCommitter != null) { + if (latestCommitInfoT == null) { + sinkCommitter.commit(Collections.emptyList()); + } else { + sinkCommitter.commit(Collections.singletonList(latestCommitInfoT)); + } + } + cleanCommitInfo(); + } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java index 255a9cd339f..a92ff34ad7f 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelSparkDataWriterFactory.java @@ -29,22 +29,33 @@ import org.apache.spark.sql.connector.write.DataWriter; import org.apache.spark.sql.connector.write.DataWriterFactory; import org.apache.spark.sql.connector.write.streaming.StreamingDataWriterFactory; +import org.apache.spark.sql.types.StructType; import java.io.IOException; +import java.util.Map; public class SeaTunnelSparkDataWriterFactory implements DataWriterFactory, StreamingDataWriterFactory { private final SeaTunnelSink sink; + private final Map properties; private final CatalogTable[] catalogTables; + private final StructType schema; + private final String checkpointLocation; private final String jobId; public SeaTunnelSparkDataWriterFactory( SeaTunnelSink sink, + Map properties, CatalogTable[] catalogTables, + StructType schema, + String checkpointLocation, String jobId) { this.sink = sink; + this.properties = properties; this.catalogTables = catalogTables; + this.schema = schema; + this.checkpointLocation = checkpointLocation; this.jobId = jobId; } @@ -64,11 +75,41 @@ public DataWriter createWriter(int partitionId, long taskId) { throw new RuntimeException("Failed to create SinkCommitter.", e); } return new SeaTunnelSparkDataWriter<>( - writer, committer, new MultiTableManager(catalogTables), 0, context); + writer, + committer, + new MultiTableManager(catalogTables), + schema, + properties, + checkpointLocation, + partitionId, + 0, + context); } @Override public DataWriter createWriter(int partitionId, long taskId, long epochId) { - return createWriter(partitionId, taskId); + SinkWriter.Context context = new DefaultSinkWriterContext(jobId, (int) taskId); + SinkWriter writer; + SinkCommitter committer; + try { + writer = sink.createWriter(context); + } catch (IOException e) { + throw new RuntimeException("Failed to create SinkWriter.", e); + } + try { + committer = sink.createCommitter().orElse(null); + } catch (IOException e) { + throw new RuntimeException("Failed to create SinkCommitter.", e); + } + return new SeaTunnelSparkDataWriter<>( + writer, + committer, + new MultiTableManager(catalogTables), + schema, + properties, + checkpointLocation, + partitionId, + epochId, + context); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWrite.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWrite.java index 2bd4388cc22..9721f3723ec 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWrite.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWrite.java @@ -20,33 +20,44 @@ import org.apache.seatunnel.api.sink.SeaTunnelSink; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.translation.spark.sink.SeaTunnelBatchWrite; import org.apache.spark.sql.connector.write.BatchWrite; import org.apache.spark.sql.connector.write.Write; import org.apache.spark.sql.connector.write.streaming.StreamingWrite; +import org.apache.spark.sql.types.StructType; import java.io.IOException; +import java.util.Map; public class SeaTunnelWrite implements Write { private final SeaTunnelSink sink; + private final Map properties; private final CatalogTable[] catalogTables; + private final StructType schema; + private final String checkpointLocation; private final String jobId; public SeaTunnelWrite( SeaTunnelSink sink, + Map properties, CatalogTable[] catalogTables, + StructType schema, + String checkpointLocation, String jobId) { this.sink = sink; + this.properties = properties; this.catalogTables = catalogTables; + this.schema = schema; + this.checkpointLocation = checkpointLocation; this.jobId = jobId; } @Override public BatchWrite toBatch() { try { - return new SeaTunnelBatchWrite<>(sink, catalogTables, jobId); + return new SeaTunnelBatchWrite<>( + sink, properties, catalogTables, schema, checkpointLocation, jobId); } catch (IOException e) { throw new RuntimeException("SeaTunnel Spark sink create batch failed", e); } @@ -55,7 +66,8 @@ public BatchWrite toBatch() { @Override public StreamingWrite toStreaming() { try { - return new SeaTunnelBatchWrite<>(sink, catalogTables, jobId); + return new SeaTunnelBatchWrite<>( + sink, properties, catalogTables, schema, checkpointLocation, jobId); } catch (IOException e) { throw new RuntimeException("SeaTunnel Spark sink create batch failed", e); } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWriteBuilder.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWriteBuilder.java index 896982c3516..73d2000b220 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWriteBuilder.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/sink/write/SeaTunnelWriteBuilder.java @@ -23,25 +23,38 @@ import org.apache.spark.sql.connector.write.Write; import org.apache.spark.sql.connector.write.WriteBuilder; +import org.apache.spark.sql.types.StructType; + +import java.util.Map; public class SeaTunnelWriteBuilder implements WriteBuilder { private final SeaTunnelSink sink; + private final Map properties; private final CatalogTable[] catalogTables; + private final StructType schema; + private final String checkpointLocation; private final String jobId; public SeaTunnelWriteBuilder( SeaTunnelSink sink, + Map properties, CatalogTable[] catalogTables, + StructType schema, + String checkpointLocation, String jobId) { this.sink = sink; + this.properties = properties; this.catalogTables = catalogTables; + this.schema = schema; + this.checkpointLocation = checkpointLocation; this.jobId = jobId; } @Override public Write build() { - return new SeaTunnelWrite<>(sink, catalogTables, jobId); + return new SeaTunnelWrite<>( + sink, properties, catalogTables, schema, checkpointLocation, jobId); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceTable.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceTable.java index ef707095e63..02e5ce7ddd5 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceTable.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/SeaTunnelSourceTable.java @@ -85,7 +85,12 @@ public ScanBuilder newScanBuilder(CaseInsensitiveStringMap caseInsensitiveString Integer.parseInt(properties.getOrDefault(CommonOptions.PARALLELISM.key(), "1")); String applicationId = SparkSession.getActiveSession().get().sparkContext().applicationId(); return new SeaTunnelScanBuilder( - source, parallelism, applicationId, caseInsensitiveStringMap, multiTableManager); + source, + parallelism, + applicationId, + new org.apache.seatunnel.translation.spark.utils.CaseInsensitiveStringMap( + caseInsensitiveStringMap.asCaseSensitiveMap()), + multiTableManager); } /** A name to identify this table */ diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/InternalParallelBatchSource.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/InternalParallelBatchSource.java new file mode 100644 index 00000000000..f9c44a991f3 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/InternalParallelBatchSource.java @@ -0,0 +1,50 @@ +/* + * 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.partition.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 java.io.Serializable; +import java.util.List; +import java.util.Map; + +public class InternalParallelBatchSource + extends ParallelSource { + + private final ParallelBatchPartitionReader parallelBatchPartitionReader; + + public InternalParallelBatchSource( + ParallelBatchPartitionReader parallelBatchPartitionReader, + SeaTunnelSource source, + Map> restoredState, + int parallelism, + String jobId, + int subtaskId) { + super(source, restoredState, parallelism, jobId, subtaskId); + this.parallelBatchPartitionReader = parallelBatchPartitionReader; + } + + @Override + protected void handleNoMoreElement() { + super.handleNoMoreElement(); + parallelBatchPartitionReader.running = false; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/ParallelBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/ParallelBatchPartitionReader.java index 27ab9f42d2c..52afeeeaabe 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/ParallelBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/batch/ParallelBatchPartitionReader.java @@ -18,11 +18,9 @@ package org.apache.seatunnel.translation.spark.source.partition.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.common.Handover; import org.apache.seatunnel.translation.source.BaseSourceFunction; -import org.apache.seatunnel.translation.source.ParallelSource; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.serialization.InternalRowCollector; import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory; @@ -32,8 +30,6 @@ import lombok.extern.slf4j.Slf4j; import java.io.IOException; -import java.io.Serializable; -import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -125,7 +121,7 @@ protected void prepare() { } protected BaseSourceFunction createInternalSource() { - return new InternalParallelSource<>(source, null, parallelism, jobId, subtaskId); + return new InternalParallelBatchSource<>(this, source, null, parallelism, jobId, subtaskId); } public InternalRow get() { @@ -147,23 +143,4 @@ public void close() throws IOException { } executorService.shutdown(); } - - public class InternalParallelSource - extends ParallelSource { - - public InternalParallelSource( - SeaTunnelSource source, - Map> restoredState, - int parallelism, - String jobId, - int subtaskId) { - super(source, restoredState, parallelism, jobId, subtaskId); - } - - @Override - protected void handleNoMoreElement() { - super.handleNoMoreElement(); - running = false; - } - } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReader.java new file mode 100644 index 00000000000..4f0fc401607 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReader.java @@ -0,0 +1,105 @@ +/* + * 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.partition.continuous; + +import org.apache.seatunnel.api.serialization.Serializer; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.Handover; +import org.apache.seatunnel.translation.spark.serialization.InternalMultiRowCollector; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.read.streaming.ContinuousPartitionReader; +import org.apache.spark.sql.connector.read.streaming.PartitionOffset; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; + +public class SeaTunnelContinuousPartitionReader< + SplitT extends SourceSplit, StateT extends Serializable> + implements ContinuousPartitionReader { + private final SeaTunnelSource source; + private final String jobId; + private final Integer subtaskId; + private final Integer parallelism; + private final SourceSplitEnumerator splitEnumerator; + private final SourceSplitEnumerator.Context splitEnumeratorCtx; + protected final List restoredSplitState; + protected final SourceReader reader; + + protected final Serializer splitSerializer; + protected final Serializer enumeratorStateSerializer; + private InternalMultiRowCollector collector; + Handover handover; + + public SeaTunnelContinuousPartitionReader( + SeaTunnelSource source, + String jobId, + Integer subtaskId, + Integer parallelism, + SourceSplitEnumerator splitEnumerator, + SourceSplitEnumerator.Context splitEnumeratorCtx, + List restoredSplitState, + SourceReader reader, + Serializer splitSerializer, + Serializer enumeratorStateSerializer, + int subTaskId) { + this.source = source; + this.jobId = jobId; + this.subtaskId = subtaskId; + this.parallelism = parallelism; + this.splitEnumerator = splitEnumerator; + this.splitEnumeratorCtx = splitEnumeratorCtx; + this.restoredSplitState = restoredSplitState; + this.reader = reader; + this.splitSerializer = splitSerializer; + this.enumeratorStateSerializer = enumeratorStateSerializer; + } + + @Override + public PartitionOffset getOffset() { + return null; + } + + @Override + public boolean next() throws IOException { + try { + if (handover.isEmpty()) { + reader.pollNext(collector); + if (handover.isEmpty()) { + // splitEnumeratorCtx.assignSplit(); + } + } + return true; + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public InternalRow get() { + return null; + } + + @Override + public void close() throws IOException {} +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReaderFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReaderFactory.java new file mode 100644 index 00000000000..d8cd730bc34 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReaderFactory.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.translation.spark.source.partition.continuous; + +import org.apache.spark.SparkEnv; +import org.apache.spark.rpc.RpcEndpointRef; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.streaming.ContinuousPartitionReader; +import org.apache.spark.sql.connector.read.streaming.ContinuousPartitionReaderFactory; +import org.apache.spark.util.RpcUtils; + +public class SeaTunnelContinuousPartitionReaderFactory implements ContinuousPartitionReaderFactory { + @Override + public ContinuousPartitionReader createReader(InputPartition partition) { + RpcEndpointRef endpointRef = + RpcUtils.makeDriverRef("", SparkEnv.get().conf(), SparkEnv.get().rpcEnv()); + + return null; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousStream.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousStream.java new file mode 100644 index 00000000000..6285ab91011 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousStream.java @@ -0,0 +1,89 @@ +/* + * 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.partition.continuous; + +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.spark.execution.MultiTableManager; +import org.apache.seatunnel.translation.spark.utils.CaseInsensitiveStringMap; + +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.streaming.ContinuousPartitionReaderFactory; +import org.apache.spark.sql.connector.read.streaming.ContinuousStream; +import org.apache.spark.sql.connector.read.streaming.Offset; +import org.apache.spark.sql.connector.read.streaming.PartitionOffset; + +public class SeaTunnelContinuousStream implements ContinuousStream { + private final SeaTunnelSource source; + private final int parallelism; + private final String jobId; + private final String checkpointLocation; + private final CaseInsensitiveStringMap caseInsensitiveStringMap; + private final MultiTableManager multiTableManager; + + public SeaTunnelContinuousStream( + SeaTunnelSource source, + int parallelism, + String jobId, + String checkpointLocation, + CaseInsensitiveStringMap caseInsensitiveStringMap, + MultiTableManager multiTableManager) { + this.source = source; + this.parallelism = parallelism; + this.jobId = jobId; + this.checkpointLocation = checkpointLocation; + this.caseInsensitiveStringMap = caseInsensitiveStringMap; + this.multiTableManager = multiTableManager; + } + + @Override + public InputPartition[] planInputPartitions(Offset start) { + return new InputPartition[0]; + } + + @Override + public ContinuousPartitionReaderFactory createContinuousReaderFactory() { + return null; + } + + @Override + public Offset mergeOffsets(PartitionOffset[] offsets) { + return null; + } + + @Override + public boolean needsReconfiguration() { + return ContinuousStream.super.needsReconfiguration(); + } + + @Override + public Offset initialOffset() { + return null; + } + + @Override + public Offset deserializeOffset(String json) { + return null; + } + + @Override + public void commit(Offset end) {} + + @Override + public void stop() {} +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelInputPartition.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelInputPartition.java new file mode 100644 index 00000000000..44f88ec664b --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelInputPartition.java @@ -0,0 +1,22 @@ +/* + * 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.partition.continuous; + +import org.apache.spark.sql.connector.read.InputPartition; + +public class SeaTunnelInputPartition implements InputPartition {} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelOffset.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelOffset.java new file mode 100644 index 00000000000..74702af5310 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelOffset.java @@ -0,0 +1,27 @@ +/* + * 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.partition.continuous; + +import org.apache.spark.sql.connector.read.streaming.Offset; + +public class SeaTunnelOffset extends Offset { + @Override + public String json() { + return ""; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelPartitionOffset.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelPartitionOffset.java new file mode 100644 index 00000000000..6e014c479c6 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelPartitionOffset.java @@ -0,0 +1,22 @@ +/* + * 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.partition.continuous; + +import org.apache.spark.sql.connector.read.streaming.PartitionOffset; + +public class SeaTunnelPartitionOffset implements PartitionOffset {} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CheckpointDataLogManager.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CheckpointDataLogManager.java new file mode 100644 index 00000000000..8bcf88e3a9e --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CheckpointDataLogManager.java @@ -0,0 +1,110 @@ +/* + * 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.partition.micro; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.spark.sql.execution.streaming.CheckpointFileManager; + +import java.util.Arrays; +import java.util.Optional; +import java.util.regex.Pattern; +import java.util.stream.Stream; + +public class CheckpointDataLogManager { + public static final String SUFFIX_PREPARED = ".prepared"; + public static final String SUFFIX_COMMITTED = ".committed"; + private final CheckpointFileManager checkpointFileManager; + + public CheckpointDataLogManager(CheckpointFileManager checkpointFileManager) { + this.checkpointFileManager = checkpointFileManager; + } + + public CheckpointFileManager getCheckpointFileManager() { + return this.checkpointFileManager; + } + + public Path logDataPath(String root, int batchId) { + return new Path(root, String.valueOf(batchId)); + } + + public Path logDataPath(Path root, int batchId) { + return new Path(root, String.valueOf(batchId)); + } + + public Path subTaskLogDataPath(String root, int batchId, int subTaskId) { + return new Path(logDataPath(root, batchId), String.valueOf(subTaskId)); + } + + public Path subTaskLogDataPath(Path root, int batchId, int subTaskId) { + return new Path(logDataPath(root, batchId), String.valueOf(subTaskId)); + } + + public Path logFilePath(String root, int batchId, int subTaskId, int checkpointId) { + return new Path(subTaskLogDataPath(root, batchId, subTaskId), String.valueOf(checkpointId)); + } + + public Path logFilePath(Path root, int batchId, int subTaskId, int checkpointId) { + return new Path(subTaskLogDataPath(root, batchId, subTaskId), String.valueOf(checkpointId)); + } + + public Path preparedCommittedFilePath(Path root, int batchId, int subTaskId, int checkpointId) { + return new Path( + subTaskLogDataPath(root, batchId, subTaskId), + String.valueOf(checkpointId) + SUFFIX_PREPARED); + } + + public Path committedFilePath(Path root, int batchId, int subTaskId, int checkpointId) { + return new Path( + subTaskLogDataPath(root, batchId, subTaskId), + String.valueOf(checkpointId) + SUFFIX_COMMITTED); + } + + public Path preparedCommittedFilePath( + String root, int batchId, int subTaskId, int checkpointId) { + return new Path( + subTaskLogDataPath(root, batchId, subTaskId), + String.valueOf(checkpointId) + SUFFIX_PREPARED); + } + + public Path committedFilePath(String root, int batchId, int subTaskId, int checkpointId) { + return new Path( + subTaskLogDataPath(root, batchId, subTaskId), + String.valueOf(checkpointId) + SUFFIX_COMMITTED); + } + + public int maxNum(Path path, Optional suffix) { + Pattern pattern = Pattern.compile("^\\d+$"); + if (checkpointFileManager.exists(path)) { + FileStatus[] fileStatuses = checkpointFileManager.list(path); + Stream nameStream = + Arrays.stream(fileStatuses) + .map((FileStatus fileStatus) -> fileStatus.getPath().getName().trim()); + if (suffix.isPresent()) { + String suffixVal = suffix.get(); + nameStream = + nameStream + .filter((String name) -> name.endsWith(suffixVal)) + .map((String name) -> name.split(suffixVal)[0]); + } + + return nameStream.map(Integer::parseInt).max(Integer::compare).orElse(-1); + } + return -1; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CoordinatedMicroBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CoordinatedMicroBatchPartitionReader.java index ccf71692dcc..8804c9fde7b 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CoordinatedMicroBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/CoordinatedMicroBatchPartitionReader.java @@ -17,22 +17,15 @@ package org.apache.seatunnel.translation.spark.source.partition.micro; -import org.apache.seatunnel.api.source.Collector; import org.apache.seatunnel.api.source.SeaTunnelSource; -import org.apache.seatunnel.api.source.SourceReader; -import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.source.BaseSourceFunction; -import org.apache.seatunnel.translation.source.CoordinatedSource; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.serialization.InternalRowCollector; -import java.io.Serializable; import java.util.HashMap; import java.util.Iterator; -import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; public class CoordinatedMicroBatchPartitionReader extends ParallelMicroBatchPartitionReader { protected final Map collectorMap; @@ -113,7 +106,7 @@ private void internalCheckpoint(Iterator iterator, int loo } // Block #next() method synchronized (handover) { - final int currentCheckpoint = checkpointId; + final int currentCheckpoint = batchId; ReaderState readerState = snapshotState(); saveState(readerState, currentCheckpoint); internalSource.notifyCheckpointComplete(currentCheckpoint); @@ -129,70 +122,6 @@ protected String getEnumeratorThreadName() { @Override protected BaseSourceFunction createInternalSource() { - return new InternalCoordinatedSource<>(source, null, parallelism, jobId); - } - - public class InternalCoordinatedSource - extends CoordinatedSource { - - public InternalCoordinatedSource( - SeaTunnelSource source, - Map> restoredState, - int parallelism, - String jobId) { - super(source, restoredState, parallelism, jobId); - } - - @Override - public void run(Collector collector) throws Exception { - readerMap - .entrySet() - .parallelStream() - .forEach( - entry -> { - final AtomicBoolean flag = readerRunningMap.get(entry.getKey()); - final SourceReader reader = entry.getValue(); - final Collector rowCollector = - collectorMap.get(entry.getKey()); - executorService.execute( - () -> { - while (flag.get()) { - try { - reader.pollNext(rowCollector); - if (rowCollector.isEmptyThisPollNext()) { - Thread.sleep(100); - } else { - rowCollector.resetEmptyThisPollNext(); - /** - * sleep(0) is used to prevent the current - * thread from occupying CPU resources for a - * long time, thus blocking the checkpoint - * thread for a long time. It is mentioned - * in this - * https://github.com/apache/seatunnel/issues/5694 - */ - Thread.sleep(0L); - } - } catch (Exception e) { - this.running = false; - flag.set(false); - throw new RuntimeException(e); - } - } - }); - }); - splitEnumerator.run(); - while (this.running) { - Thread.sleep(SLEEP_TIME_INTERVAL); - } - } - - @Override - protected void handleNoMoreElement(int subtaskId) { - super.handleNoMoreElement(subtaskId); - if (!this.running) { - CoordinatedMicroBatchPartitionReader.this.running = false; - } - } + return new InternalCoordinatedSource<>(this, source, null, parallelism, jobId); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/InternalCoordinatedSource.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/InternalCoordinatedSource.java new file mode 100644 index 00000000000..eaeb8d29c18 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/InternalCoordinatedSource.java @@ -0,0 +1,99 @@ +/* + * 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.partition.micro; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.translation.source.CoordinatedSource; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; + +public class InternalCoordinatedSource + extends CoordinatedSource { + + private final CoordinatedMicroBatchPartitionReader coordinatedMicroBatchPartitionReader; + + public InternalCoordinatedSource( + CoordinatedMicroBatchPartitionReader coordinatedMicroBatchPartitionReader, + SeaTunnelSource source, + Map> restoredState, + int parallelism, + String jobId) { + super(source, restoredState, parallelism, jobId); + this.coordinatedMicroBatchPartitionReader = coordinatedMicroBatchPartitionReader; + } + + @Override + public void run(Collector collector) throws Exception { + readerMap + .entrySet() + .parallelStream() + .forEach( + entry -> { + final AtomicBoolean flag = readerRunningMap.get(entry.getKey()); + final SourceReader reader = entry.getValue(); + final Collector rowCollector = + coordinatedMicroBatchPartitionReader.collectorMap.get( + entry.getKey()); + executorService.execute( + () -> { + while (flag.get()) { + try { + reader.pollNext(rowCollector); + if (rowCollector.isEmptyThisPollNext()) { + Thread.sleep(100); + } else { + rowCollector.resetEmptyThisPollNext(); + /** + * sleep(0) is used to prevent the current + * thread from occupying CPU resources for a + * long time, thus blocking the checkpoint + * thread for a long time. It is mentioned in + * this + * https://github.com/apache/seatunnel/issues/5694 + */ + Thread.sleep(0L); + } + } catch (Exception e) { + this.running = false; + flag.set(false); + throw new RuntimeException(e); + } + } + }); + }); + splitEnumerator.run(); + while (this.running) { + Thread.sleep(SLEEP_TIME_INTERVAL); + } + } + + @Override + protected void handleNoMoreElement(int subtaskId) { + super.handleNoMoreElement(subtaskId); + if (!this.running) { + coordinatedMicroBatchPartitionReader.setRunning(false); + } + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/InternalParallelMicroBatchSource.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/InternalParallelMicroBatchSource.java new file mode 100644 index 00000000000..9efcdc13db8 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/InternalParallelMicroBatchSource.java @@ -0,0 +1,51 @@ +/* + * 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.partition.micro; + +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 java.io.Serializable; +import java.util.List; +import java.util.Map; + +public class InternalParallelMicroBatchSource< + SplitT extends SourceSplit, StateT extends Serializable> + extends ParallelSource { + + private final ParallelMicroBatchPartitionReader parallelBatchPartitionReader; + + public InternalParallelMicroBatchSource( + ParallelMicroBatchPartitionReader parallelBatchPartitionReader, + SeaTunnelSource source, + Map> restoredState, + int parallelism, + String jobId, + int subtaskId) { + super(source, restoredState, parallelism, jobId, subtaskId); + this.parallelBatchPartitionReader = parallelBatchPartitionReader; + } + + @Override + protected void handleNoMoreElement() { + super.handleNoMoreElement(); + parallelBatchPartitionReader.running = false; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/ParallelMicroBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/ParallelMicroBatchPartitionReader.java index bfa7aa6079d..748a58b498f 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/ParallelMicroBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/ParallelMicroBatchPartitionReader.java @@ -18,11 +18,16 @@ package org.apache.seatunnel.translation.spark.source.partition.micro; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.type.RowKind; import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.Handover; import org.apache.seatunnel.common.utils.SerializationUtils; import org.apache.seatunnel.translation.source.BaseSourceFunction; +import org.apache.seatunnel.translation.spark.execution.CheckpointMetadata; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; -import org.apache.seatunnel.translation.spark.source.partition.batch.ParallelBatchPartitionReader; +import org.apache.seatunnel.translation.spark.serialization.InternalRowCollector; +import org.apache.seatunnel.translation.spark.serialization.InternalRowConverter; import org.apache.seatunnel.translation.util.ThreadPoolExecutorFactory; import org.apache.commons.lang3.StringUtils; @@ -31,65 +36,177 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.execution.streaming.CheckpointFileManager; + +import lombok.extern.slf4j.Slf4j; import java.io.ByteArrayOutputStream; -import java.io.File; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; -public class ParallelMicroBatchPartitionReader extends ParallelBatchPartitionReader { +@Slf4j +public class ParallelMicroBatchPartitionReader { protected static final Integer CHECKPOINT_SLEEP_INTERVAL = 10; protected static final Integer CHECKPOINT_RETRIES = 3; - protected volatile Integer checkpointId; + protected volatile Integer batchId; protected final Integer checkpointInterval; protected final String checkpointPath; protected final String hdfsRoot; protected final String hdfsUser; protected Map> restoredState; - protected ScheduledThreadPoolExecutor executor; + protected ScheduledThreadPoolExecutor snapshotExecutor; protected FileSystem fileSystem; + protected static final Integer INTERVAL = 1000; + + protected final SeaTunnelSource source; + protected final Integer parallelism; + protected final String jobId; + protected final Integer subtaskId; + + protected final ExecutorService collectorExecutor; + protected final Handover handover; + + protected final Object checkpointLock = new Object(); + + protected volatile boolean running = true; + protected volatile boolean prepared = false; + + protected volatile BaseSourceFunction internalSource; + protected volatile InternalRowCollector internalRowCollector; + private final Map envOptions; + + private final MultiTableManager multiTableManager; + private volatile int checkpointId = 0; + private final CheckpointDataLogManager logManager; + private volatile boolean shouldCheckpoint = false; + private volatile long totalCollectRowSize = 0; + public ParallelMicroBatchPartitionReader( SeaTunnelSource source, Integer parallelism, String jobId, Integer subtaskId, - Integer checkpointId, + Integer batchId, Integer checkpointInterval, String checkpointPath, String hdfsRoot, String hdfsUser, Map envOptions, MultiTableManager multiTableManager) { - super(source, parallelism, jobId, subtaskId, envOptions, multiTableManager); - this.checkpointId = checkpointId; + this.source = source; + this.parallelism = parallelism; + this.jobId = jobId; + this.subtaskId = subtaskId; + this.collectorExecutor = + ThreadPoolExecutorFactory.createScheduledThreadPoolExecutor( + 1, getEnumeratorThreadName()); + this.handover = new Handover<>(); + this.envOptions = envOptions; + this.multiTableManager = multiTableManager; + this.batchId = batchId; this.checkpointInterval = checkpointInterval; this.checkpointPath = checkpointPath; this.hdfsRoot = hdfsRoot; this.hdfsUser = hdfsUser; + Configuration configuration = new Configuration(); + configuration.set("fs.defaultFS", hdfsRoot); + this.logManager = + new CheckpointDataLogManager( + CheckpointFileManager.create(new Path(this.checkpointPath), configuration)); + } + + protected String getEnumeratorThreadName() { + return String.format("parallel-split-enumerator-executor-%s", subtaskId); } - @Override protected BaseSourceFunction createInternalSource() { - return new InternalParallelSource<>(source, restoredState, parallelism, jobId, subtaskId); + return new InternalParallelMicroBatchSource<>( + this, source, restoredState, parallelism, jobId, subtaskId); } - @Override protected void prepare() { + if (prepared) { + return; + } try { this.fileSystem = getFileSystem(); - this.restoredState = restoreState(checkpointId - 1); + int committedCheckpointId = this.latestCommittedCheckpointId(this.batchId); + this.checkpointId = committedCheckpointId + 1; + Path latestPreCommittedPath = + logManager.preparedCommittedFilePath( + this.checkpointPath, batchId, subtaskId, this.checkpointId); + this.restoredState = restoreState(latestPreCommittedPath); + if (this.restoredState == null) { + /* + * If fail to restore state from the latest prepared checkpoint at the current batch, it will try restore state from the committed checkpoint. + */ + Path latestCommittedPath = + logManager.committedFilePath( + this.checkpointPath, batchId, subtaskId, committedCheckpointId); + this.restoredState = restoreState(latestCommittedPath); + } + + int prevBatchId = batchId; + while (this.restoredState == null & --prevBatchId >= 0) { + /* + * If fail to restore state from the latest committed checkpoint at the current batch, it will try restore state from the previous committed batch. + */ + int prevBatchCommittedCheckpointId = this.latestCommittedCheckpointId(prevBatchId); + int prevBatchPreparedCheckpointId = this.latestPreparedCheckpointId(prevBatchId); + Path recentCheckpointPath = null; + if (prevBatchPreparedCheckpointId > prevBatchCommittedCheckpointId) { + recentCheckpointPath = + logManager.preparedCommittedFilePath( + this.checkpointPath, + prevBatchId, + subtaskId, + prevBatchPreparedCheckpointId); + } else { + recentCheckpointPath = + logManager.committedFilePath( + this.checkpointPath, + prevBatchId, + subtaskId, + prevBatchCommittedCheckpointId); + } + this.restoredState = restoreState(recentCheckpointPath); + } } catch (Exception e) { throw new RuntimeException(e); } - super.prepare(); - prepareCheckpoint(); + + this.internalSource = createInternalSource(); + try { + this.internalSource.open(); + } catch (Exception e) { + running = false; + throw new RuntimeException("Failed to open internal source.", e); + } + + this.internalRowCollector = + multiTableManager.getInternalRowCollector(handover, checkpointLock, envOptions); + collectorExecutor.execute( + () -> { + try { + internalSource.run(internalRowCollector); + } catch (Exception e) { + handover.reportError(e); + log.error("BatchPartitionReader execute failed.", e); + running = false; + } + }); + startCheckpointConditionService(); + prepared = true; } protected FileSystem getFileSystem() @@ -113,50 +230,42 @@ protected ReaderState snapshotState() { return new ReaderState(bytes, subtaskId, checkpointId++); } - public void prepareCheckpoint() { - executor = - ThreadPoolExecutorFactory.createScheduledThreadPoolExecutor( - 1, String.format("parallel-reader-checkpoint-executor-%s", subtaskId)); - executor.schedule(this::virtualCheckpoint, checkpointInterval, TimeUnit.MILLISECONDS); + public void startCheckpointConditionService() { + if (snapshotExecutor == null) { + snapshotExecutor = + ThreadPoolExecutorFactory.createScheduledThreadPoolExecutor( + 1, + String.format( + "parallel-reader-checkpoint-condition-executor-%s-%s", + batchId, subtaskId)); + } + snapshotExecutor.schedule( + this::checkpointCondition, checkpointInterval, TimeUnit.MILLISECONDS); } - public void virtualCheckpoint() { - try { - int checkpointRetries = Math.max(1, CHECKPOINT_RETRIES); - do { - checkpointRetries--; - if (internalRowCollector.collectTotalCount() == 0) { - Thread.sleep(CHECKPOINT_SLEEP_INTERVAL); - } - synchronized (checkpointLock) { - if (internalRowCollector.collectTotalCount() != 0 || checkpointRetries == 0) { - checkpointRetries = 0; - - while (!handover.isEmpty()) { - Thread.sleep(CHECKPOINT_SLEEP_INTERVAL); - } - // Block #next() method - synchronized (handover) { - final int currentCheckpoint = checkpointId; - ReaderState readerState = snapshotState(); - saveState(readerState, currentCheckpoint); - internalSource.notifyCheckpointComplete(currentCheckpoint); - running = false; - } - } - } - } while (checkpointRetries > 0); - } catch (Exception e) { - throw new RuntimeException("An error occurred in virtual checkpoint execution.", e); + private void checkpointCondition() { + long current = internalRowCollector.collectTotalCount(); + if (current > totalCollectRowSize) { + shouldCheckpoint = true; } + totalCollectRowSize = current; } - private Map> restoreState(int checkpointId) throws IOException { - Path hdfsPath = getCheckpointPathWithId(checkpointId); - if (!fileSystem.exists(hdfsPath)) { + private int latestCommittedCheckpointId(int batchId) throws IOException { + Path path = logManager.subTaskLogDataPath(this.checkpointPath, batchId, this.subtaskId); + return logManager.maxNum(path, Optional.of(CheckpointDataLogManager.SUFFIX_COMMITTED)); + } + + private int latestPreparedCheckpointId(int batchId) throws IOException { + Path path = logManager.subTaskLogDataPath(this.checkpointPath, batchId, this.subtaskId); + return logManager.maxNum(path, Optional.of(CheckpointDataLogManager.SUFFIX_PREPARED)); + } + + private Map> restoreState(Path statePath) throws IOException { + if (!fileSystem.exists(statePath)) { return null; } - try (FSDataInputStream inputStream = fileSystem.open(hdfsPath); + try (FSDataInputStream inputStream = fileSystem.open(statePath); ByteArrayOutputStream out = new ByteArrayOutputStream()) { int i = 0; final int defaultLen = 1024; @@ -173,31 +282,104 @@ private Map> restoreState(int checkpointId) throws IOExcep protected void saveState(ReaderState readerState, int checkpointId) throws IOException { byte[] bytes = SerializationUtils.serialize(readerState); - Path hdfsPath = getCheckpointPathWithId(checkpointId); + Path hdfsPath = + logManager.preparedCommittedFilePath( + this.checkpointPath, batchId, this.subtaskId, checkpointId); + if (!fileSystem.exists(hdfsPath)) { - fileSystem.createNewFile(hdfsPath); + try (FSDataOutputStream outputStream = fileSystem.create(hdfsPath)) { + outputStream.write(bytes); + } catch (Exception e) { + throw new RuntimeException(e); + } + return; } - try (FSDataOutputStream outputStream = fileSystem.append(hdfsPath)) { + try (FSDataOutputStream outputStream = fileSystem.appendFile(hdfsPath).build()) { outputStream.write(bytes); } catch (Exception e) { throw new RuntimeException(e); } } - private Path getCheckpointPathWithId(int checkpointId) { - return new Path( - this.checkpointPath - + File.separator - + this.subtaskId - + File.separator - + checkpointId); + public boolean next() throws IOException { + prepare(); + if (!running) { + return false; + } + + if (shouldCheckpoint) { + try { + final int currentCheckpoint = checkpointId; + ReaderState readerState = snapshotState(); + saveState(readerState, currentCheckpoint); + internalSource.notifyCheckpointComplete(currentCheckpoint); + InternalRow checkpointEvent = + checkpointEvent(batchId, subtaskId, currentCheckpoint); + handover.produce(checkpointEvent); + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + shouldCheckpoint = false; + } + } + + int retry = 20; + while (retry-- > 0 & handover.isEmpty()) { + try { + Thread.sleep(INTERVAL); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + return !handover.isEmpty(); + } + + public InternalRow get() { + try { + return handover.pollNext().get(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private InternalRow checkpointEvent(int batchId, int subtaskId, int checkpointId) { + CatalogTable catalogTable = source.getProducedCatalogTables().get(0); + return InternalRowConverter.checkpointEvent( + catalogTable.getSeaTunnelRowType(), + RowKind.INSERT, + catalogTable.getTableId().toTablePath().toString(), + CheckpointMetadata.of(this.checkpointPath, batchId, subtaskId, checkpointId)); } - @Override public void close() throws IOException { - fileSystem.close(); - executor.shutdown(); - super.close(); + try { + snapshotExecutor.awaitTermination(3, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } finally { + if (fileSystem != null) { + fileSystem.close(); + } + running = false; + try { + if (internalSource != null) { + internalSource.close(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + collectorExecutor.shutdown(); + } + } + + public boolean isRunning() { + return this.running; + } + + public void setRunning(boolean status) { + this.running = status; } + + public void virtualCheckpoint() {} } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatch.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatch.java index 11eec4d7fc5..fbb6f661e46 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatch.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatch.java @@ -24,20 +24,23 @@ import org.apache.seatunnel.common.Constants; import org.apache.seatunnel.common.utils.JsonUtils; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; +import org.apache.seatunnel.translation.spark.utils.CaseInsensitiveStringMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.read.InputPartition; import org.apache.spark.sql.connector.read.PartitionReaderFactory; import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; import org.apache.spark.sql.connector.read.streaming.Offset; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.apache.spark.sql.execution.streaming.CheckpointFileManager; import lombok.Getter; import java.util.ArrayList; import java.util.List; +import java.util.Optional; @Getter public class SeaTunnelMicroBatch implements MicroBatchStream { @@ -58,6 +61,7 @@ public class SeaTunnelMicroBatch implements MicroBatchStream { private Offset currentOffset = initialOffset; private final MultiTableManager multiTableManager; + private CheckpointDataLogManager logManager; public SeaTunnelMicroBatch( SeaTunnelSource source, @@ -72,15 +76,26 @@ public SeaTunnelMicroBatch( this.checkpointLocation = checkpointLocation; this.caseInsensitiveStringMap = caseInsensitiveStringMap; this.multiTableManager = multiTableManager; + SparkSession sparkSession = SparkSession.getActiveSession().get(); + this.logManager = + new CheckpointDataLogManager( + CheckpointFileManager.create( + new Path(this.checkpointLocation).getParent().getParent(), + sparkSession.sessionState().newHadoopConf())); } @Override public Offset latestOffset() { - return currentOffset; + // TODO + Path commitsPath = + new Path(new Path(checkpointLocation).getParent().getParent(), "commits"); + int maxBatchId = this.logManager.maxNum(commitsPath, Optional.empty()); + return new SeaTunnelOffset(maxBatchId + 1); } @Override public InputPartition[] planInputPartitions(Offset start, Offset end) { + SeaTunnelOffset startOffset = (SeaTunnelOffset) start; int checkpointInterval = caseInsensitiveStringMap.getInt( EnvCommonOptions.CHECKPOINT_INTERVAL.key(), CHECKPOINT_INTERVAL_DEFAULT); @@ -98,7 +113,7 @@ public InputPartition[] planInputPartitions(Offset start, Offset end) { source, parallelism, 0, - 1, + (int) startOffset.getCheckpointId(), checkpointInterval, checkpointLocation, hdfsRoot, @@ -111,14 +126,14 @@ public InputPartition[] planInputPartitions(Offset start, Offset end) { source, parallelism, subtaskId, - 1, + (int) startOffset.getCheckpointId(), checkpointInterval, checkpointLocation, hdfsRoot, hdfsUser)); } } - return virtualPartitions.toArray(new InputPartition[0]); + return virtualPartitions.toArray(new InputPartition[parallelism]); } @Override @@ -144,7 +159,7 @@ public Offset deserializeOffset(String json) { @Override public void commit(Offset end) { - this.currentOffset = ((SeaTunnelOffset) end).inc(); + this.currentOffset = ((SeaTunnelOffset) this.currentOffset).inc(); } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReader.java index 61d466d946d..07f45c87198 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReader.java @@ -17,8 +17,6 @@ package org.apache.seatunnel.translation.spark.source.partition.micro; -import org.apache.seatunnel.translation.spark.source.partition.batch.ParallelBatchPartitionReader; - import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.read.PartitionReader; @@ -26,9 +24,9 @@ public class SeaTunnelMicroBatchPartitionReader implements PartitionReader { - private final ParallelBatchPartitionReader partitionReader; + private final ParallelMicroBatchPartitionReader partitionReader; - public SeaTunnelMicroBatchPartitionReader(ParallelBatchPartitionReader partitionReader) { + public SeaTunnelMicroBatchPartitionReader(ParallelMicroBatchPartitionReader partitionReader) { this.partitionReader = partitionReader; } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReaderFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReaderFactory.java index a92a7dfb3bf..cbfe93bcd10 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReaderFactory.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelMicroBatchPartitionReaderFactory.java @@ -21,13 +21,12 @@ import org.apache.seatunnel.api.source.SupportCoordinate; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; -import org.apache.seatunnel.translation.spark.source.partition.batch.ParallelBatchPartitionReader; +import org.apache.seatunnel.translation.spark.utils.CaseInsensitiveStringMap; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.read.InputPartition; import org.apache.spark.sql.connector.read.PartitionReader; import org.apache.spark.sql.connector.read.PartitionReaderFactory; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; import java.util.Map; @@ -63,13 +62,13 @@ public SeaTunnelMicroBatchPartitionReaderFactory( public PartitionReader createReader(InputPartition partition) { SeaTunnelMicroBatchInputPartition seaTunnelPartition = (SeaTunnelMicroBatchInputPartition) partition; - ParallelBatchPartitionReader partitionReader; Integer subtaskId = seaTunnelPartition.getSubtaskId(); Integer checkpointId = seaTunnelPartition.getCheckpointId(); Integer checkpointInterval = seaTunnelPartition.getCheckpointInterval(); String hdfsRoot = seaTunnelPartition.getHdfsRoot(); String hdfsUser = seaTunnelPartition.getHdfsUser(); Map envOptions = caseInsensitiveStringMap.asCaseSensitiveMap(); + ParallelMicroBatchPartitionReader partitionReader = null; if (source instanceof SupportCoordinate) { partitionReader = new CoordinatedMicroBatchPartitionReader( diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelOffset.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelOffset.java index 134c0a5c893..fe778994026 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelOffset.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/micro/SeaTunnelOffset.java @@ -30,7 +30,9 @@ @Setter public class SeaTunnelOffset extends Offset implements Serializable { - private final long checkpointId; + private long checkpointId; + + public SeaTunnelOffset() {} public SeaTunnelOffset(long checkpointId) { this.checkpointId = checkpointId; diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java index c2085d91270..4107853ff35 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java @@ -21,13 +21,15 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.partition.batch.SeaTunnelBatch; +import org.apache.seatunnel.translation.spark.source.partition.continuous.SeaTunnelContinuousStream; import org.apache.seatunnel.translation.spark.source.partition.micro.SeaTunnelMicroBatch; +import org.apache.seatunnel.translation.spark.utils.CaseInsensitiveStringMap; import org.apache.spark.sql.connector.read.Batch; import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.streaming.ContinuousStream; import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; import org.apache.spark.sql.types.StructType; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; import java.util.Map; @@ -76,4 +78,15 @@ public MicroBatchStream toMicroBatchStream(String checkpointLocation) { caseInsensitiveStringMap, multiTableManager); } + + @Override + public ContinuousStream toContinuousStream(String checkpointLocation) { + return new SeaTunnelContinuousStream( + source, + parallelism, + jobId, + checkpointLocation, + caseInsensitiveStringMap, + multiTableManager); + } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScanBuilder.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScanBuilder.java index 932f83dac9e..565c5654d20 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScanBuilder.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScanBuilder.java @@ -20,10 +20,10 @@ import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; +import org.apache.seatunnel.translation.spark.utils.CaseInsensitiveStringMap; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.ScanBuilder; -import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** The builder for {@link SeaTunnelScan} used to build {@link SeaTunnelScan} */ public class SeaTunnelScanBuilder implements ScanBuilder { diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndPoint.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndPoint.java new file mode 100644 index 00000000000..224d1d3053a --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndPoint.java @@ -0,0 +1,38 @@ +/* + * 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.spark.sql.execution.streaming.continuous.seatunnel.rpc; + +import org.apache.spark.rpc.RpcCallContext; +import org.apache.spark.rpc.RpcEnv; +import org.apache.spark.rpc.ThreadSafeRpcEndpoint; + +import scala.PartialFunction; +import scala.runtime.BoxedUnit; + +public class SplitEnumeratorEndPoint implements ThreadSafeRpcEndpoint { + + @Override + public RpcEnv rpcEnv() { + return null; + } + + @Override + public PartialFunction receiveAndReply(RpcCallContext context) { + return null; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/test/java/org/apache/seatunnel/translation/spark/sink/SparkSinkTest.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/test/java/org/apache/seatunnel/translation/spark/sink/SparkSinkTest.java index 73084fb76b3..4d41c092628 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/test/java/org/apache/seatunnel/translation/spark/sink/SparkSinkTest.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/test/java/org/apache/seatunnel/translation/spark/sink/SparkSinkTest.java @@ -135,6 +135,11 @@ public void testSparkSinkWriteDataWithCopy() { new Object[] { (byte) 1, "test.test.test", + new HashMap() { + { + put("k", "v"); + } + }, 42, "string1", true, @@ -202,6 +207,11 @@ public void testSparkSinkWriteDataWithCopy() { new Object[] { (byte) 1, "test.test.test", + new HashMap() { + { + put("k", "v"); + } + }, 12, "string2", false, @@ -269,6 +279,11 @@ public void testSparkSinkWriteDataWithCopy() { new Object[] { (byte) 1, "test.test.test", + new HashMap() { + { + put("k", "v"); + } + }, 233, "string3", true, diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/CheckpointMetadata.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/CheckpointMetadata.java new file mode 100644 index 00000000000..f6cc3c37079 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/execution/CheckpointMetadata.java @@ -0,0 +1,90 @@ +/* + * 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.execution; + +import java.util.HashMap; +import java.util.Map; + +public class CheckpointMetadata { + + public static final String CHECKPOINT = "checkpoint"; + public static final String CHECKPOINT_ID = "checkpointId"; + public static final String BATCH_ID = "batchId"; + public static final String LOCATION = "location"; + public static final String SUBTASK_ID = "subTaskId"; + + public static Map create( + String location, int batchId, int subTaskId, int checkpointId) { + Map metadata = new HashMap<>(); + metadata.put(CHECKPOINT, String.valueOf(true)); + metadata.put(CHECKPOINT_ID, String.valueOf(checkpointId)); + metadata.put(BATCH_ID, String.valueOf(batchId)); + metadata.put(LOCATION, location); + metadata.put(SUBTASK_ID, String.valueOf(subTaskId)); + return metadata; + } + + private final boolean checkpoint; + private final String location; + private final int checkpointId; + private final int batchId; + private final int subTaskId; + + private CheckpointMetadata( + boolean checkpoint, String location, int batchId, int subTaskId, int checkpointId) { + this.checkpoint = checkpoint; + this.location = location; + this.batchId = batchId; + this.checkpointId = checkpointId; + this.subTaskId = subTaskId; + } + + public static CheckpointMetadata of(Map data) { + boolean isCheckpoint = Boolean.parseBoolean(data.getOrDefault(CHECKPOINT, "false")); + String location = data.getOrDefault(LOCATION, ""); + int batchId = Integer.parseInt(data.getOrDefault(BATCH_ID, "-1")); + int subTaskId = Integer.parseInt(data.getOrDefault(SUBTASK_ID, "-1")); + int checkpointId = Integer.parseInt(data.getOrDefault(CHECKPOINT_ID, "-1")); + return new CheckpointMetadata(isCheckpoint, location, batchId, subTaskId, checkpointId); + } + + public static CheckpointMetadata of( + String location, int batchId, int subTaskId, int checkpointId) { + return of(create(location, batchId, subTaskId, checkpointId)); + } + + public boolean isCheckpoint() { + return this.checkpoint; + } + + public String location() { + return this.location; + } + + public int checkpointId() { + return this.checkpointId; + } + + public int batchId() { + return this.batchId; + } + + public int subTaskId() { + return this.subTaskId; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowConverter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowConverter.java index 8dbca1975a7..5a610ff4333 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowConverter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/InternalRowConverter.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.translation.serialization.RowConverter; +import org.apache.seatunnel.translation.spark.execution.CheckpointMetadata; import org.apache.seatunnel.translation.spark.utils.InstantConverterUtils; import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; @@ -145,20 +146,27 @@ private static InternalRow convert(SeaTunnelRow seaTunnelRow, SeaTunnelRowType r } private InternalRow parcel(SeaTunnelRow seaTunnelRow, SeaTunnelRowType rowType) { - // 0 -> row kind, 1 -> table id + // 0 -> row kind, 1 -> table id, 3 -> metadata int arity = rowType.getTotalFields(); - MutableValue[] values = new MutableValue[arity + 2]; + MutableValue[] values = new MutableValue[arity + 3]; for (int i = 0; i < indexes.length; i++) { - values[indexes[i] + 2] = createMutableValue(rowType.getFieldType(indexes[i])); + values[indexes[i] + 3] = createMutableValue(rowType.getFieldType(indexes[i])); Object fieldValue = convert(seaTunnelRow.getField(i), rowType.getFieldType(indexes[i])); if (fieldValue != null) { - values[indexes[i] + 2].update(fieldValue); + values[indexes[i] + 3].update(fieldValue); } } values[0] = new MutableByte(); values[0].update(seaTunnelRow.getRowKind().toByteValue()); values[1] = new MutableAny(); values[1].update(UTF8String.fromString(seaTunnelRow.getTableId())); + values[2] = new MutableAny(); + Map metadata = seaTunnelRow.getMetadata(); + Object metadataVal = + convert( + metadata, + new MapType(BasicType.STRING_TYPE, BasicType.STRING_TYPE)); + values[2].update(metadataVal); // Fill any remaining null values with MutableAny for (int i = 0; i < values.length; i++) { if (values[i] == null) { @@ -168,6 +176,54 @@ private InternalRow parcel(SeaTunnelRow seaTunnelRow, SeaTunnelRowType rowType) return new SpecificInternalRow(values); } + public static InternalRow attachMetadata(InternalRow engineRow, Map metadata) { + SpecificInternalRow row = (SpecificInternalRow) engineRow; + MutableValue[] values = row.values(); + Object metadataVal = + convert( + metadata, + new MapType(BasicType.STRING_TYPE, BasicType.STRING_TYPE)); + values[2].update(metadataVal); + // Fill any remaining null values with MutableAny + for (int i = 0; i < values.length; i++) { + if (values[i] == null) { + values[i] = new MutableAny(); + } + } + return new SpecificInternalRow(values); + } + + public static InternalRow checkpointEvent( + SeaTunnelRowType rowType, + RowKind rowKind, + String tableId, + CheckpointMetadata metadata) { + int fieldSize = rowType.getTotalFields(); + MutableValue[] values = new MutableValue[fieldSize + 3]; + values[0] = new MutableByte(); + values[0].update(RowKind.INSERT.toByteValue()); + values[1] = new MutableAny(); + values[1].update(UTF8String.fromString(tableId)); + values[2] = new MutableAny(); + Map data = + CheckpointMetadata.create( + metadata.location(), + metadata.batchId(), + metadata.subTaskId(), + metadata.checkpointId()); + Object metadataVal = + convert( + data, + new MapType(BasicType.STRING_TYPE, BasicType.STRING_TYPE)); + values[2].update(metadataVal); + for (int i = 0; i < values.length; i++) { + if (values[i] == null) { + values[i] = new MutableAny(); + } + } + return new SpecificInternalRow(values); + } + private static ArrayBasedMapData convertMap(Map mapData, MapType mapType) { if (mapData == null || mapData.size() == 0) { return ArrayBasedMapData.apply(new Object[] {}, new Object[] {}); @@ -248,21 +304,38 @@ private static MutableValue createMutableValue(SeaTunnelDataType dataType) { public SeaTunnelRow unpack(InternalRow engineRow, SeaTunnelRowType rowType) throws IOException { RowKind rowKind = RowKind.fromByteValue(engineRow.getByte(0)); String tableId = engineRow.getString(1); + Map metadata = + (Map) + reconvert( + engineRow.getMap(2), + new MapType( + BasicType.STRING_TYPE, BasicType.STRING_TYPE)); Object[] fields = new Object[indexes.length]; for (int i = 0; i < indexes.length; i++) { fields[i] = reconvert( engineRow.get( - indexes[i] + 2, + indexes[i] + 3, TypeConverterUtils.convert(rowType.getFieldType(indexes[i]))), rowType.getFieldType(indexes[i])); } SeaTunnelRow seaTunnelRow = new SeaTunnelRow(fields); seaTunnelRow.setRowKind(rowKind); seaTunnelRow.setTableId(tableId); + seaTunnelRow.setMetadata(metadata); return seaTunnelRow; } + public static Map unpackMetadata(InternalRow engineRow) { + Map metadata = + (Map) + reconvert( + engineRow.getMap(2), + new MapType( + BasicType.STRING_TYPE, BasicType.STRING_TYPE)); + return metadata; + } + @Override public SeaTunnelRow reconvert(InternalRow engineRow) throws IOException { return unpack(engineRow, (SeaTunnelRowType) dataType); @@ -403,7 +476,7 @@ private Object convertToField(Object internalRowField, DataType dataType) { keys[i] = convertToField(keys[i], mapType.keyType()); values[i] = convertToField(values[i], mapType.valueType()); Tuple2 tuple2 = new Tuple2<>(keys[i], values[i]); - newMap = newMap.$plus(tuple2); + newMap = newMap.$plus(tuple2); } return newMap; } else if (dataType instanceof org.apache.spark.sql.types.ArrayType diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java index 33d1b49fb1c..2b5d62a2c9b 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java @@ -25,6 +25,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.translation.serialization.RowConverter; +import org.apache.seatunnel.translation.spark.execution.CheckpointMetadata; import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; @@ -61,21 +62,46 @@ public SeaTunnelRow convert(SeaTunnelRow seaTunnelRow) throws IOException { } public GenericRowWithSchema parcel(SeaTunnelRow seaTunnelRow) { + // 0 -> RowKind, 1 -> TableId, 2 -> Metadata + int offset = 3; SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; int arity = rowType.getTotalFields(); - Object[] fields = new Object[arity + 2]; + Object[] fields = new Object[arity + offset]; fields[0] = seaTunnelRow.getRowKind().toByteValue(); fields[1] = seaTunnelRow.getTableId(); + fields[2] = + convertMap( + seaTunnelRow.getMetadata(), + new MapType(BasicType.STRING_TYPE, BasicType.STRING_TYPE)); StructType schema = (StructType) TypeConverterUtils.parcel(rowType); for (int i = 0; i < arity; i++) { Object fieldValue = convert(seaTunnelRow.getField(i), rowType.getFieldType(i)); if (fieldValue != null) { - fields[i + 2] = fieldValue; + fields[i + offset] = fieldValue; } } return new GenericRowWithSchema(fields, schema); } + public GenericRowWithSchema checkpointEvent( + StructType schema, RowKind rowKind, String tableId, CheckpointMetadata metadata) { + // 0 -> RowKind, 1 -> TableId, 2 -> Metadata + Object[] fields = new Object[schema.fields().length]; + fields[0] = rowKind.toByteValue(); + fields[1] = tableId; + fields[2] = + CheckpointMetadata.create( + metadata.location(), + metadata.batchId(), + metadata.subTaskId(), + metadata.checkpointId()); + fields[2] = + convertMap( + (Map) fields[2], + new MapType(BasicType.STRING_TYPE, BasicType.STRING_TYPE)); + return new GenericRowWithSchema(fields, schema); + } + private Object convert(Object field, SeaTunnelDataType dataType) { if (field == null) { return null; @@ -148,7 +174,7 @@ private scala.collection.immutable.HashMap convertMap( keys[i] = convert(keys[i], mapType.getKeyType()); values[i] = convert(values[i], mapType.getValueType()); Tuple2 tuple2 = new Tuple2<>(keys[i], values[i]); - newMap = newMap.$plus(tuple2); + newMap = newMap.$plus(tuple2); } return newMap; @@ -172,16 +198,20 @@ public SeaTunnelRow reconvert(SeaTunnelRow engineRow) throws IOException { } public SeaTunnelRow unpack(GenericRowWithSchema engineRow) throws IOException { + // 0 -> RowKind, 1 -> TableId, 2 -> Metadata + int offset = 3; SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; RowKind rowKind = RowKind.fromByteValue(engineRow.getByte(0)); String tableId = engineRow.getString(1); + Map metadata = engineRow.getJavaMap(2); Object[] fields = new Object[rowType.getTotalFields()]; for (int i = 0; i < fields.length; i++) { - fields[i] = reconvert(engineRow.get(i + 2), rowType.getFieldType(i)); + fields[i] = reconvert(engineRow.get(i + offset), rowType.getFieldType(i)); } SeaTunnelRow seaTunnelRow = new SeaTunnelRow(fields); seaTunnelRow.setRowKind(rowKind); seaTunnelRow.setTableId(tableId); + seaTunnelRow.setMetadata(metadata); return seaTunnelRow; } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/CaseInsensitiveStringMap.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/CaseInsensitiveStringMap.java new file mode 100644 index 00000000000..d08b0d870cc --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/CaseInsensitiveStringMap.java @@ -0,0 +1,190 @@ +/* + * 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.utils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +public class CaseInsensitiveStringMap implements Map, Serializable { + private final Logger logger = LoggerFactory.getLogger(CaseInsensitiveStringMap.class); + + private String unsupportedOperationMsg = "CaseInsensitiveStringMap is read-only."; + + public static CaseInsensitiveStringMap empty() { + return new CaseInsensitiveStringMap(new HashMap<>(0)); + } + + private final Map original; + + private final Map delegate; + + public CaseInsensitiveStringMap(Map originalMap) { + original = new HashMap<>(originalMap); + delegate = new HashMap<>(originalMap.size()); + for (Map.Entry entry : originalMap.entrySet()) { + String key = toLowerCase(entry.getKey()); + if (delegate.containsKey(key)) { + logger.warn( + "Converting duplicated key " + + entry.getKey() + + " into CaseInsensitiveStringMap."); + } + delegate.put(key, entry.getValue()); + } + } + + @Override + public int size() { + return delegate.size(); + } + + @Override + public boolean isEmpty() { + return delegate.isEmpty(); + } + + private String toLowerCase(Object key) { + return key.toString().toLowerCase(Locale.ROOT); + } + + @Override + public boolean containsKey(Object key) { + return delegate.containsKey(toLowerCase(key)); + } + + @Override + public boolean containsValue(Object value) { + return delegate.containsValue(value); + } + + @Override + public String get(Object key) { + return delegate.get(toLowerCase(key)); + } + + @Override + public String put(String key, String value) { + throw new UnsupportedOperationException(unsupportedOperationMsg); + } + + @Override + public String remove(Object key) { + throw new UnsupportedOperationException(unsupportedOperationMsg); + } + + @Override + public void putAll(Map m) { + throw new UnsupportedOperationException(unsupportedOperationMsg); + } + + @Override + public void clear() { + throw new UnsupportedOperationException(unsupportedOperationMsg); + } + + @Override + public Set keySet() { + return delegate.keySet(); + } + + @Override + public Collection values() { + return delegate.values(); + } + + @Override + public Set> entrySet() { + return delegate.entrySet(); + } + + /** + * Returns the boolean value to which the specified key is mapped, or defaultValue if there is + * no mapping for the key. The key match is case-insensitive. + */ + public boolean getBoolean(String key, boolean defaultValue) { + String value = get(key); + // We can't use `Boolean.parseBoolean` here, as it returns false for invalid strings. + if (value == null) { + return defaultValue; + } else if (value.equalsIgnoreCase("true")) { + return true; + } else if (value.equalsIgnoreCase("false")) { + return false; + } else { + throw new IllegalArgumentException(value + " is not a boolean string."); + } + } + + /** + * Returns the integer value to which the specified key is mapped, or defaultValue if there is + * no mapping for the key. The key match is case-insensitive. + */ + public int getInt(String key, int defaultValue) { + String value = get(key); + return value == null ? defaultValue : Integer.parseInt(value); + } + + /** + * Returns the long value to which the specified key is mapped, or defaultValue if there is no + * mapping for the key. The key match is case-insensitive. + */ + public long getLong(String key, long defaultValue) { + String value = get(key); + return value == null ? defaultValue : Long.parseLong(value); + } + + /** + * Returns the double value to which the specified key is mapped, or defaultValue if there is no + * mapping for the key. The key match is case-insensitive. + */ + public double getDouble(String key, double defaultValue) { + String value = get(key); + return value == null ? defaultValue : Double.parseDouble(value); + } + + /** Returns the original case-sensitive map. */ + public Map asCaseSensitiveMap() { + return Collections.unmodifiableMap(original); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CaseInsensitiveStringMap that = (CaseInsensitiveStringMap) o; + return delegate.equals(that.delegate); + } + + @Override + public int hashCode() { + return Objects.hash(delegate); + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/TypeConverterUtils.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/TypeConverterUtils.java index 627cde28340..52616ba9181 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/TypeConverterUtils.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/utils/TypeConverterUtils.java @@ -46,6 +46,7 @@ public class TypeConverterUtils { public static final String ROW_KIND_FIELD = "seatunnel_row_kind"; public static final String ROW = "row"; public static final String TABLE_ID = "seatunnel_table_id"; + public static final String METADATA = "seatunnel_metadata"; public static final String LOGICAL_TIME_TYPE_FLAG = "logical_time_type"; static { @@ -132,14 +133,22 @@ private static StructType convert(SeaTunnelRowType rowType) { } public static DataType parcel(SeaTunnelDataType dataType) { - // 0 -> row kind, 1 -> table id + // 0 -> row kind, 1 -> table id, 2 -> metadata SeaTunnelRowType seaTunnelRowType = (SeaTunnelRowType) dataType; - StructField[] fields = new StructField[2 + seaTunnelRowType.getTotalFields()]; + StructField[] fields = new StructField[3 + seaTunnelRowType.getTotalFields()]; fields[0] = new StructField(ROW_KIND_FIELD, DataTypes.ByteType, true, Metadata.empty()); fields[1] = new StructField(TABLE_ID, DataTypes.StringType, true, Metadata.empty()); + fields[2] = + new StructField( + METADATA, + convert( + new MapType( + BasicType.STRING_TYPE, BasicType.STRING_TYPE)), + true, + Metadata.empty()); StructType structType = (StructType) convert(dataType); for (int i = 0; i < seaTunnelRowType.getTotalFields(); i++) { - fields[i + 2] = structType.fields()[i]; + fields[i + 3] = structType.fields()[i]; } return new StructType(fields); } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/test/java/org/apache/seatunnel/translation/spark/execution/MultiTableManagerTest.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/test/java/org/apache/seatunnel/translation/spark/execution/MultiTableManagerTest.java index ad5fdfbea53..7c04378821b 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/test/java/org/apache/seatunnel/translation/spark/execution/MultiTableManagerTest.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/test/java/org/apache/seatunnel/translation/spark/execution/MultiTableManagerTest.java @@ -61,6 +61,7 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.seatunnel.translation.spark.utils.TypeConverterUtils.METADATA; import static org.apache.seatunnel.translation.spark.utils.TypeConverterUtils.ROW_KIND_FIELD; import static org.apache.seatunnel.translation.spark.utils.TypeConverterUtils.TABLE_ID; import static org.apache.spark.sql.types.DataTypes.BooleanType; @@ -180,45 +181,32 @@ public void testMultiReaderConverter() throws IOException { internalMultiRowCollector.getRowSerializationMap(); InternalRow internalRow = rowSerializationMap.get(seaTunnelRow1.getTableId()).convert(seaTunnelRow1); - for (int v = 0; v < specificInternalRow2.numFields(); v++) { - if (specificInternalRow2.genericGet(v) instanceof ArrayBasedMapData) { + assertRowEquals(specificInternalRow2, internalRow); + InternalRow internalRow3 = + rowSerializationMap.get(seaTunnelRow3.getTableId()).convert(seaTunnelRow3); + assertRowEquals(specificInternalRow3, internalRow3); + } + + private void assertRowEquals(SpecificInternalRow expectedRow, InternalRow actualRow) { + Assertions.assertEquals(expectedRow.numFields(), actualRow.numFields()); + for (int v = 0; v < expectedRow.numFields(); v++) { + if (expectedRow.isNullAt(v)) { + Assertions.assertTrue(actualRow.isNullAt(v)); + } else if (expectedRow.genericGet(v) instanceof ArrayBasedMapData) { Assertions.assertEquals( - specificInternalRow2.getMap(v).keyArray(), - internalRow.getMap(v).keyArray()); + expectedRow.getMap(v).keyArray(), actualRow.getMap(v).keyArray()); Assertions.assertEquals( - specificInternalRow2.getMap(v).valueArray(), - internalRow.getMap(v).valueArray()); - } else if (specificInternalRow2.genericGet(v) instanceof SpecificInternalRow) { - SpecificInternalRow expected = - (SpecificInternalRow) specificInternalRow2.genericGet(v); + expectedRow.getMap(v).valueArray(), actualRow.getMap(v).valueArray()); + } else if (expectedRow.genericGet(v) instanceof SpecificInternalRow) { + SpecificInternalRow expected = (SpecificInternalRow) expectedRow.genericGet(v); SpecificInternalRow actual = - (SpecificInternalRow) ((SpecificInternalRow) internalRow).genericGet(v); - for (int o = 0; v < expected.numFields(); v++) { - if (expected.genericGet(o) instanceof ArrayBasedMapData) { - Assertions.assertEquals( - expected.getMap(o).keyArray(), actual.getMap(o).keyArray()); - Assertions.assertEquals( - expected.getMap(o).valueArray(), actual.getMap(o).valueArray()); - } else { - Assertions.assertEquals( - expected.genericGet(v), - ((SpecificInternalRow) actual).genericGet(v)); - } - } + (SpecificInternalRow) ((SpecificInternalRow) actualRow).genericGet(v); + assertRowEquals(expected, actual); } else { Assertions.assertEquals( - specificInternalRow2.genericGet(v), - ((SpecificInternalRow) internalRow).genericGet(v)); + expectedRow.genericGet(v), ((SpecificInternalRow) actualRow).genericGet(v)); } } - InternalRow internalRow3 = - rowSerializationMap.get(seaTunnelRow3.getTableId()).convert(seaTunnelRow3); - Assertions.assertEquals(specificInternalRow3, internalRow3); - for (int v = 0; v < specificInternalRow3.numFields(); v++) { - Assertions.assertEquals( - specificInternalRow3.genericGet(v), - ((SpecificInternalRow) internalRow3).genericGet(v)); - } } @Test @@ -231,37 +219,7 @@ public void testReaderConverter() throws IOException { multiTableManager.getInternalRowCollector(null, null, null); InternalRowConverter rowSerialization = internalRowCollector.getRowSerialization(); InternalRow internalRow = rowSerialization.convert(seaTunnelRow1); - for (int v = 0; v < specificInternalRow1.numFields(); v++) { - if (specificInternalRow1.genericGet(v) instanceof ArrayBasedMapData) { - Assertions.assertEquals( - specificInternalRow1.getMap(v).keyArray(), - internalRow.getMap(v).keyArray()); - Assertions.assertEquals( - specificInternalRow1.getMap(v).valueArray(), - internalRow.getMap(v).valueArray()); - } else if (specificInternalRow1.genericGet(v) instanceof SpecificInternalRow) { - SpecificInternalRow expected = - (SpecificInternalRow) specificInternalRow1.genericGet(v); - SpecificInternalRow actual = - (SpecificInternalRow) ((SpecificInternalRow) internalRow).genericGet(v); - for (int o = 0; v < expected.numFields(); v++) { - if (expected.genericGet(o) instanceof ArrayBasedMapData) { - Assertions.assertEquals( - expected.getMap(o).keyArray(), actual.getMap(o).keyArray()); - Assertions.assertEquals( - expected.getMap(o).valueArray(), actual.getMap(o).valueArray()); - } else { - Assertions.assertEquals( - expected.genericGet(v), - ((SpecificInternalRow) actual).genericGet(v)); - } - } - } else { - Assertions.assertEquals( - specificInternalRow1.genericGet(v), - ((SpecificInternalRow) internalRow).genericGet(v)); - } - } + assertRowEquals(specificInternalRow1, internalRow); } public void initSchema() { @@ -550,6 +508,7 @@ public void initSchema() { new StructType() .add(ROW_KIND_FIELD, DataTypes.ByteType) .add(TABLE_ID, DataTypes.StringType) + .add(METADATA, new MapType(StringType, StringType, true)) .add("column0", IntegerType) .add("column1", StringType) .add("column2", BooleanType) @@ -577,6 +536,7 @@ public void initSchema() { new StructType() .add(ROW_KIND_FIELD, DataTypes.ByteType) .add(TABLE_ID, DataTypes.StringType) + .add(METADATA, new MapType(StringType, StringType, true)) .add("column0", IntegerType) .add("column1", StringType) .add("column2", BooleanType) @@ -607,6 +567,7 @@ public void initSchema() { new StructType() .add(ROW_KIND_FIELD, DataTypes.ByteType) .add(TABLE_ID, DataTypes.StringType) + .add(METADATA, new MapType(StringType, StringType, true)) .add("int", IntegerType) .add("string", StringType) .add("boolean", BooleanType) @@ -774,39 +735,41 @@ public void initData() { SpecificInternalRow specificInternalRow = new SpecificInternalRow(mutableValues); - MutableValue[] mutableValues1 = new MutableValue[24]; + MutableValue[] mutableValues1 = new MutableValue[25]; mutableValues1[0] = new MutableByte(); mutableValues1[0].update(RowKind.INSERT.toByteValue()); mutableValues1[1] = new MutableAny(); mutableValues1[1].update(UTF8String.fromString("test.test.test1")); - mutableValues1[2] = new MutableInt(); - mutableValues1[2].update(233); - mutableValues1[3] = new MutableAny(); - mutableValues1[3].update(UTF8String.fromString("string3")); - mutableValues1[4] = new MutableBoolean(); - mutableValues1[4].update(true); - mutableValues1[5] = new MutableFloat(); - mutableValues1[5].update(231.1f); - mutableValues1[6] = new MutableDouble(); - mutableValues1[6].update(3533.33); - mutableValues1[7] = new MutableByte(); - mutableValues1[7].update((byte) 7); - mutableValues1[8] = new MutableShort(); - mutableValues1[8].update((short) 2); - mutableValues1[9] = new MutableLong(); - mutableValues1[9].update(Long.MAX_VALUE - 2); - mutableValues1[10] = new MutableAny(); - mutableValues1[10].update(Decimal.apply(new BigDecimal("65.55"))); - mutableValues1[11] = new MutableInt(); - mutableValues1[11].update((int) LocalDate.parse("2001-01-01").toEpochDay()); - mutableValues1[12] = new MutableAny(); - mutableValues1[12].update( + mutableValues1[2] = new MutableAny(); + mutableValues1[2].update(ArrayBasedMapData.apply(new Object[] {}, new Object[] {})); + mutableValues1[3] = new MutableInt(); + mutableValues1[3].update(233); + mutableValues1[4] = new MutableAny(); + mutableValues1[4].update(UTF8String.fromString("string3")); + mutableValues1[5] = new MutableBoolean(); + mutableValues1[5].update(true); + mutableValues1[6] = new MutableFloat(); + mutableValues1[6].update(231.1f); + mutableValues1[7] = new MutableDouble(); + mutableValues1[7].update(3533.33); + mutableValues1[8] = new MutableByte(); + mutableValues1[8].update((byte) 7); + mutableValues1[9] = new MutableShort(); + mutableValues1[9].update((short) 2); + mutableValues1[10] = new MutableLong(); + mutableValues1[10].update(Long.MAX_VALUE - 2); + mutableValues1[11] = new MutableAny(); + mutableValues1[11].update(Decimal.apply(new BigDecimal("65.55"))); + mutableValues1[12] = new MutableInt(); + mutableValues1[12].update((int) LocalDate.parse("2001-01-01").toEpochDay()); + mutableValues1[13] = new MutableAny(); + mutableValues1[13].update( InstantConverterUtils.toEpochMicro( Timestamp.valueOf(LocalDateTime.parse("2031-01-01T00:00:00")).toInstant())); - mutableValues1[13] = new MutableAny(); mutableValues1[14] = new MutableAny(); - mutableValues1[14].update( + mutableValues1[15] = new MutableAny(); + mutableValues1[15].update( ArrayData.toArrayData( new Object[] { UTF8String.fromString("string1fsa"), @@ -814,31 +777,31 @@ public void initData() { UTF8String.fromString("strfdsaing3") })); - mutableValues1[15] = new MutableAny(); - mutableValues1[15].update(ArrayData.toArrayData(new Object[] {false, true, true})); - mutableValues1[16] = new MutableAny(); - mutableValues1[16].update( - ArrayData.toArrayData(new Object[] {(byte) 6, (byte) 2, (byte) 1})); + mutableValues1[16].update(ArrayData.toArrayData(new Object[] {false, true, true})); mutableValues1[17] = new MutableAny(); mutableValues1[17].update( - ArrayData.toArrayData(new Object[] {(short) 7, (short) 8, (short) 9})); + ArrayData.toArrayData(new Object[] {(byte) 6, (byte) 2, (byte) 1})); mutableValues1[18] = new MutableAny(); - mutableValues1[18].update(ArrayData.toArrayData(new Object[] {3, 77, 22})); + mutableValues1[18].update( + ArrayData.toArrayData(new Object[] {(short) 7, (short) 8, (short) 9})); mutableValues1[19] = new MutableAny(); - mutableValues1[19].update(ArrayData.toArrayData(new Object[] {143L, 642L, 533L})); + mutableValues1[19].update(ArrayData.toArrayData(new Object[] {3, 77, 22})); mutableValues1[20] = new MutableAny(); - mutableValues1[20].update(ArrayData.toArrayData(new Object[] {24.1f, 54.2f, 1.3f})); + mutableValues1[20].update(ArrayData.toArrayData(new Object[] {143L, 642L, 533L})); mutableValues1[21] = new MutableAny(); - mutableValues1[21].update(ArrayData.toArrayData(new Object[] {431.11, 2422.22, 3243.33})); + mutableValues1[21].update(ArrayData.toArrayData(new Object[] {24.1f, 54.2f, 1.3f})); mutableValues1[22] = new MutableAny(); - mutableValues1[22].update( + mutableValues1[22].update(ArrayData.toArrayData(new Object[] {431.11, 2422.22, 3243.33})); + + mutableValues1[23] = new MutableAny(); + mutableValues1[23].update( ArrayBasedMapData.apply( new Object[] { UTF8String.fromString("kefdsay3"), @@ -851,19 +814,19 @@ public void initData() { UTF8String.fromString("vafdslue2") })); - mutableValues1[23] = new MutableAny(); - mutableValues1[23].update(specificInternalRow); + mutableValues1[24] = new MutableAny(); + mutableValues1[24].update(specificInternalRow); specificInternalRow1 = new SpecificInternalRow(mutableValues1); - MutableValue[] mutableValues2 = new MutableValue[27]; + MutableValue[] mutableValues2 = new MutableValue[28]; for (int i = 0; i < mutableValues1.length; i++) { mutableValues2[i] = mutableValues1[i].copy(); } - mutableValues2[24] = new MutableAny(); mutableValues2[25] = new MutableAny(); mutableValues2[26] = new MutableAny(); + mutableValues2[27] = new MutableAny(); specificInternalRow2 = new SpecificInternalRow(mutableValues2); @@ -888,57 +851,60 @@ public void initData() { }); seaTunnelRow3.setRowKind(RowKind.INSERT); seaTunnelRow3.setTableId("test.test.test3"); + seaTunnelRow3.setMetadata(new HashMap<>()); - // [0, 1, 3, 22, 2, 23, 4, 5, 24, 7, 6, 8, 10, 9, 11] - MutableValue[] mutableValues3 = new MutableValue[27]; + // [0, 1, 2, 4, 23, 3, 24, 5, 6, 25, 8, 7, 9, 11, 10, 12] + MutableValue[] mutableValues3 = new MutableValue[28]; mutableValues3[0] = new MutableByte(); mutableValues3[0].update(RowKind.INSERT.toByteValue()); mutableValues3[1] = new MutableAny(); mutableValues3[1].update(UTF8String.fromString("test.test.test3")); + mutableValues3[2] = new MutableAny(); + mutableValues3[2].update(ArrayBasedMapData.apply(new Object[] {}, new Object[] {})); - mutableValues3[2] = new MutableInt(); - mutableValues3[2].update(233); + mutableValues3[3] = new MutableInt(); + mutableValues3[3].update(233); - mutableValues3[3] = new MutableAny(); - mutableValues3[3].update(UTF8String.fromString("string3")); + mutableValues3[4] = new MutableAny(); + mutableValues3[4].update(UTF8String.fromString("string3")); - mutableValues3[5] = new MutableFloat(); - mutableValues3[5].update(231.1f); + mutableValues3[6] = new MutableFloat(); + mutableValues3[6].update(231.1f); - mutableValues3[24] = new MutableFloat(); - mutableValues3[24].update(231.1f); + mutableValues3[25] = new MutableFloat(); + mutableValues3[25].update(231.1f); - mutableValues3[4] = new MutableBoolean(); - mutableValues3[4].update(true); + mutableValues3[5] = new MutableBoolean(); + mutableValues3[5].update(true); - mutableValues3[25] = new MutableBoolean(); - mutableValues3[25].update(true); + mutableValues3[26] = new MutableBoolean(); + mutableValues3[26].update(true); - mutableValues3[6] = new MutableDouble(); - mutableValues3[6].update(3533.33); + mutableValues3[7] = new MutableDouble(); + mutableValues3[7].update(3533.33); - mutableValues3[7] = new MutableByte(); - mutableValues3[7].update((byte) 7); + mutableValues3[8] = new MutableByte(); + mutableValues3[8].update((byte) 7); - mutableValues3[26] = new MutableByte(); - mutableValues3[26].update((byte) 7); + mutableValues3[27] = new MutableByte(); + mutableValues3[27].update((byte) 7); - mutableValues3[9] = new MutableLong(); - mutableValues3[9].update(Long.MAX_VALUE - 2); + mutableValues3[10] = new MutableLong(); + mutableValues3[10].update(Long.MAX_VALUE - 2); - mutableValues3[8] = new MutableShort(); - mutableValues3[8].update((short) 2); + mutableValues3[9] = new MutableShort(); + mutableValues3[9].update((short) 2); - mutableValues3[10] = new MutableAny(); - mutableValues3[10].update(Decimal.apply(new BigDecimal("65.55"))); + mutableValues3[11] = new MutableAny(); + mutableValues3[11].update(Decimal.apply(new BigDecimal("65.55"))); - mutableValues3[12] = new MutableLong(); - mutableValues3[12].update( + mutableValues3[13] = new MutableLong(); + mutableValues3[13].update( InstantConverterUtils.toEpochMicro( Timestamp.valueOf(LocalDateTime.parse("2031-01-01T00:00:00")).toInstant())); - mutableValues3[11] = new MutableInt(); - mutableValues3[11].update((int) LocalDate.parse("2001-01-01").toEpochDay()); + mutableValues3[12] = new MutableInt(); + mutableValues3[12].update((int) LocalDate.parse("2001-01-01").toEpochDay()); for (int i = 0; i < mutableValues3.length; i++) { if (mutableValues3[i] == null) { From b522c35b76122a3fcbdee7f1c5e2e900bb60d1a3 Mon Sep 17 00:00:00 2001 From: CheneyYin Date: Tue, 10 Sep 2024 13:59:55 +0800 Subject: [PATCH 2/2] [WIP][feature][spark] Support streaming - Support Spark Continuous Processing --- ...ontinuousStreamSplitEnumeratorContext.java | 76 +++++++++++ .../SeaTunnelContinuousPartitionReader.java | 64 ++++----- ...unnelContinuousPartitionReaderFactory.java | 20 ++- .../continuous/SeaTunnelContinuousStream.java | 28 +++- .../continuous/SeaTunnelInputPartition.java | 18 ++- .../source/endpoint/EndpointSource.java | 67 ++++++++++ .../endpoint/EndpointSplitEnumerator.java} | 22 +--- .../EndpointSplitEnumeratorContext.java | 97 ++++++++++++++ .../source/rpc/RpcSourceReaderContext.java | 63 +++++++++ .../source/rpc/RpcSplitEnumerator.java | 67 ++++++++++ .../source/rpc/RpcSplitEnumeratorContext.java | 67 ++++++++++ .../spark/source/scan/SeaTunnelScan.java | 3 +- .../rpc/SplitEnumeratorEndpoint.java | 123 ++++++++++++++++++ 13 files changed, 652 insertions(+), 63 deletions(-) create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/ContinuousStreamSplitEnumeratorContext.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSource.java rename seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/{spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndPoint.java => seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSplitEnumerator.java} (59%) create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSplitEnumeratorContext.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSourceReaderContext.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSplitEnumerator.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSplitEnumeratorContext.java create mode 100644 seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndpoint.java diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/ContinuousStreamSplitEnumeratorContext.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/ContinuousStreamSplitEnumeratorContext.java new file mode 100644 index 00000000000..3ca5b1cc7da --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/ContinuousStreamSplitEnumeratorContext.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.partition.continuous; + +import org.apache.seatunnel.api.common.metrics.AbstractMetricsContext; +import org.apache.seatunnel.api.common.metrics.MetricsContext; +import org.apache.seatunnel.api.event.EventListener; +import org.apache.seatunnel.api.source.SourceEvent; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class ContinuousStreamSplitEnumeratorContext + implements SourceSplitEnumerator.Context { + private final int parallelism; + private final EventListener eventListener; + private final Set readers = new HashSet<>(); + + public ContinuousStreamSplitEnumeratorContext(int parallelism, EventListener eventListener) { + this.parallelism = parallelism; + this.eventListener = eventListener; + } + + @Override + public int currentParallelism() { + return this.parallelism; + } + + @Override + public Set registeredReaders() { + return this.readers; + } + + @Override + public void assignSplit(int subtaskId, List splits) {} + + @Override + public void signalNoMoreSplits(int subtask) {} + + @Override + public void sendEventToSourceReader(int subtaskId, SourceEvent event) { + throw new UnsupportedOperationException( + "Flink ParallelSource don't support sending SourceEvent. " + + "Please implement the `SupportCoordinate` marker interface on the SeaTunnel source."); + } + + @Override + public MetricsContext getMetricsContext() { + // TODO Waiting for Flink and Spark to implement MetricsContext + // https://github.com/apache/seatunnel/issues/3431 + return new AbstractMetricsContext() {}; + } + + @Override + public EventListener getEventListener() { + return eventListener; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReader.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReader.java index 4f0fc401607..84d203fc1bf 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReader.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReader.java @@ -25,10 +25,15 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.common.Handover; import org.apache.seatunnel.translation.spark.serialization.InternalMultiRowCollector; +import org.apache.seatunnel.translation.spark.source.partition.continuous.source.rpc.RpcSourceReaderContext; +import org.apache.seatunnel.translation.spark.source.partition.continuous.source.rpc.RpcSplitEnumeratorContext; +import org.apache.spark.SparkEnv; +import org.apache.spark.rpc.RpcEndpointRef; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.read.streaming.ContinuousPartitionReader; import org.apache.spark.sql.connector.read.streaming.PartitionOffset; +import org.apache.spark.util.RpcUtils; import java.io.IOException; import java.io.Serializable; @@ -37,42 +42,39 @@ public class SeaTunnelContinuousPartitionReader< SplitT extends SourceSplit, StateT extends Serializable> implements ContinuousPartitionReader { - private final SeaTunnelSource source; - private final String jobId; - private final Integer subtaskId; - private final Integer parallelism; - private final SourceSplitEnumerator splitEnumerator; - private final SourceSplitEnumerator.Context splitEnumeratorCtx; - protected final List restoredSplitState; - protected final SourceReader reader; + private final SeaTunnelSource source; + private final SeaTunnelInputPartition inputPartition; + private final RpcEndpointRef driverRef; + protected List restoredSplitState; + protected SourceReader reader; + protected SourceSplitEnumerator splitEnumerator; - protected final Serializer splitSerializer; - protected final Serializer enumeratorStateSerializer; + protected Serializer splitSerializer; + protected Serializer enumeratorStateSerializer; private InternalMultiRowCollector collector; Handover handover; public SeaTunnelContinuousPartitionReader( - SeaTunnelSource source, - String jobId, - Integer subtaskId, - Integer parallelism, - SourceSplitEnumerator splitEnumerator, - SourceSplitEnumerator.Context splitEnumeratorCtx, - List restoredSplitState, - SourceReader reader, - Serializer splitSerializer, - Serializer enumeratorStateSerializer, - int subTaskId) { + SeaTunnelSource source, + SeaTunnelInputPartition inputPartition) { this.source = source; - this.jobId = jobId; - this.subtaskId = subtaskId; - this.parallelism = parallelism; - this.splitEnumerator = splitEnumerator; - this.splitEnumeratorCtx = splitEnumeratorCtx; - this.restoredSplitState = restoredSplitState; - this.reader = reader; - this.splitSerializer = splitSerializer; - this.enumeratorStateSerializer = enumeratorStateSerializer; + this.inputPartition = inputPartition; + this.driverRef = + RpcUtils.makeDriverRef( + inputPartition.getEndpointName(), + SparkEnv.get().conf(), + SparkEnv.get().rpcEnv()); + RpcSourceReaderContext readerCtx = new RpcSourceReaderContext(this.driverRef); + RpcSplitEnumeratorContext splitEnumeratorContext = + new RpcSplitEnumeratorContext(this.driverRef); + try { + reader = (SourceReader) this.source.createReader(readerCtx); + splitEnumerator = + (SourceSplitEnumerator) + this.source.createEnumerator(splitEnumeratorContext); + } catch (Exception e) { + throw new RuntimeException(e); + } } @Override @@ -86,7 +88,7 @@ public boolean next() throws IOException { if (handover.isEmpty()) { reader.pollNext(collector); if (handover.isEmpty()) { - // splitEnumeratorCtx.assignSplit(); + splitEnumerator.run(); } } return true; diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReaderFactory.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReaderFactory.java index d8cd730bc34..c706ef0de45 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReaderFactory.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousPartitionReaderFactory.java @@ -17,20 +17,26 @@ package org.apache.seatunnel.translation.spark.source.partition.continuous; -import org.apache.spark.SparkEnv; -import org.apache.spark.rpc.RpcEndpointRef; +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.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.read.InputPartition; import org.apache.spark.sql.connector.read.streaming.ContinuousPartitionReader; import org.apache.spark.sql.connector.read.streaming.ContinuousPartitionReaderFactory; -import org.apache.spark.util.RpcUtils; public class SeaTunnelContinuousPartitionReaderFactory implements ContinuousPartitionReaderFactory { + private final SeaTunnelSource source; + + public SeaTunnelContinuousPartitionReaderFactory(SeaTunnelSource source) { + this.source = source; + } + @Override public ContinuousPartitionReader createReader(InputPartition partition) { - RpcEndpointRef endpointRef = - RpcUtils.makeDriverRef("", SparkEnv.get().conf(), SparkEnv.get().rpcEnv()); - - return null; + SeaTunnelInputPartition inputPartition = (SeaTunnelInputPartition) partition; + return new SeaTunnelContinuousPartitionReader<>( + (SeaTunnelSource) source, inputPartition); } } diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousStream.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousStream.java index 6285ab91011..b74acbad8f8 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousStream.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelContinuousStream.java @@ -18,26 +18,32 @@ package org.apache.seatunnel.translation.spark.source.partition.continuous; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.translation.spark.execution.MultiTableManager; +import org.apache.seatunnel.translation.spark.source.partition.continuous.source.endpoint.EndpointSplitEnumeratorContext; import org.apache.seatunnel.translation.spark.utils.CaseInsensitiveStringMap; +import org.apache.spark.rpc.RpcEndpointRef; import org.apache.spark.sql.connector.read.InputPartition; import org.apache.spark.sql.connector.read.streaming.ContinuousPartitionReaderFactory; import org.apache.spark.sql.connector.read.streaming.ContinuousStream; import org.apache.spark.sql.connector.read.streaming.Offset; import org.apache.spark.sql.connector.read.streaming.PartitionOffset; +import org.apache.spark.sql.execution.streaming.continuous.seatunnel.rpc.SplitEnumeratorEndpoint; public class SeaTunnelContinuousStream implements ContinuousStream { - private final SeaTunnelSource source; + private final SeaTunnelSource source; private final int parallelism; private final String jobId; private final String checkpointLocation; private final CaseInsensitiveStringMap caseInsensitiveStringMap; private final MultiTableManager multiTableManager; + private RpcEndpointRef endpointRef; public SeaTunnelContinuousStream( - SeaTunnelSource source, + SeaTunnelSource source, int parallelism, String jobId, String checkpointLocation, @@ -53,7 +59,23 @@ public SeaTunnelContinuousStream( @Override public InputPartition[] planInputPartitions(Offset start) { - return new InputPartition[0]; + SourceSplitEnumerator.Context enumeratorContext = + new EndpointSplitEnumeratorContext<>(parallelism, jobId); + + try { + SourceSplitEnumerator enumerator = source.createEnumerator(enumeratorContext); + } catch (Exception e) { + throw new RuntimeException(e); + } + + SplitEnumeratorEndpoint enumeratorEndpoint = new SplitEnumeratorEndpoint(); + String endpointName = "SplitEnumeratorEndpoint-" + java.util.UUID.randomUUID(); + endpointRef = enumeratorEndpoint.rpcEnv().setupEndpoint(endpointName, enumeratorEndpoint); + InputPartition[] inputPartitions = new SeaTunnelInputPartition[parallelism]; + for (int idx = 0; idx < inputPartitions.length; idx++) { + inputPartitions[idx] = new SeaTunnelInputPartition(endpointName, idx); + } + return inputPartitions; } @Override diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelInputPartition.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelInputPartition.java index 44f88ec664b..273a6765043 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelInputPartition.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/SeaTunnelInputPartition.java @@ -19,4 +19,20 @@ import org.apache.spark.sql.connector.read.InputPartition; -public class SeaTunnelInputPartition implements InputPartition {} +public class SeaTunnelInputPartition implements InputPartition { + private final String endpointName; + private final int partitionId; + + public SeaTunnelInputPartition(String endpointName, int partitionId) { + this.endpointName = endpointName; + this.partitionId = partitionId; + } + + public String getEndpointName() { + return endpointName; + } + + public int getPartitionId() { + return partitionId; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSource.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSource.java new file mode 100644 index 00000000000..241c39afb47 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSource.java @@ -0,0 +1,67 @@ +/* + * 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.partition.continuous.source.endpoint; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.translation.source.BaseSourceFunction; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; + +public class EndpointSource + implements BaseSourceFunction { + private final SeaTunnelSource source; + protected final SourceSplitEnumerator splitEnumerator; + protected final Integer parallelism; + protected final String jobId; + + public EndpointSource( + SeaTunnelSource source, Integer parallelism, String jobId) { + this.source = source; + this.parallelism = parallelism; + this.jobId = jobId; + SourceSplitEnumerator.Context enumeratorContext = + new EndpointSplitEnumeratorContext<>(parallelism, jobId); + try { + splitEnumerator = source.restoreEnumerator(enumeratorContext, null); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public void open() throws Exception {} + + @Override + public void run(Collector collector) throws Exception {} + + @Override + public Map> snapshotState(long checkpointId) throws Exception { + return null; + } + + @Override + public void close() throws Exception {} + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception {} +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndPoint.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSplitEnumerator.java similarity index 59% rename from seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndPoint.java rename to seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSplitEnumerator.java index 224d1d3053a..c0f1a0a6479 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndPoint.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSplitEnumerator.java @@ -15,24 +15,6 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming.continuous.seatunnel.rpc; +package org.apache.seatunnel.translation.spark.source.partition.continuous.source.endpoint; -import org.apache.spark.rpc.RpcCallContext; -import org.apache.spark.rpc.RpcEnv; -import org.apache.spark.rpc.ThreadSafeRpcEndpoint; - -import scala.PartialFunction; -import scala.runtime.BoxedUnit; - -public class SplitEnumeratorEndPoint implements ThreadSafeRpcEndpoint { - - @Override - public RpcEnv rpcEnv() { - return null; - } - - @Override - public PartialFunction receiveAndReply(RpcCallContext context) { - return null; - } -} +public class EndpointSplitEnumerator {} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSplitEnumeratorContext.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSplitEnumeratorContext.java new file mode 100644 index 00000000000..a4a0d76521c --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/endpoint/EndpointSplitEnumeratorContext.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.translation.spark.source.partition.continuous.source.endpoint; + +import org.apache.seatunnel.api.common.metrics.AbstractMetricsContext; +import org.apache.seatunnel.api.common.metrics.MetricsContext; +import org.apache.seatunnel.api.event.DefaultEventProcessor; +import org.apache.seatunnel.api.event.EventListener; +import org.apache.seatunnel.api.source.SourceEvent; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.LinkedBlockingDeque; + +public class EndpointSplitEnumeratorContext + implements SourceSplitEnumerator.Context { + protected final int parallelism; + protected final EventListener eventListener; + protected volatile boolean running = false; + private final LinkedBlockingDeque[] queues; + + public EndpointSplitEnumeratorContext(int parallelism, String jobId) { + this.parallelism = parallelism; + this.queues = new LinkedBlockingDeque[this.parallelism]; + this.eventListener = new DefaultEventProcessor(jobId); + } + + @Override + public int currentParallelism() { + return parallelism; + } + + @Override + public Set registeredReaders() { + Set readers = new HashSet<>(); + if (running) { + for (int i = 0; i < queues.length; i++) { + if (queues[i] != null) { + readers.add(i); + } + } + } + return readers; + } + + @Override + public void assignSplit(int subtaskId, List splits) { + if (subtaskId < 0 || subtaskId >= parallelism) { + return; + } + + LinkedBlockingDeque queue = queues[subtaskId]; + if (queue == null) { + queue = new LinkedBlockingDeque<>(); + } + + queue.addAll(splits); + } + + @Override + public void signalNoMoreSplits(int subtask) { + queues[subtask] = null; + } + + @Override + public void sendEventToSourceReader(int subtaskId, SourceEvent event) { + throw new UnsupportedOperationException("Unsupported"); + } + + @Override + public MetricsContext getMetricsContext() { + return new AbstractMetricsContext() {}; + } + + @Override + public EventListener getEventListener() { + return eventListener; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSourceReaderContext.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSourceReaderContext.java new file mode 100644 index 00000000000..fd3b0fdb042 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSourceReaderContext.java @@ -0,0 +1,63 @@ +/* + * 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.partition.continuous.source.rpc; + +import org.apache.seatunnel.api.common.metrics.MetricsContext; +import org.apache.seatunnel.api.event.EventListener; +import org.apache.seatunnel.api.source.Boundedness; +import org.apache.seatunnel.api.source.SourceEvent; +import org.apache.seatunnel.api.source.SourceReader; + +import org.apache.spark.rpc.RpcEndpointRef; + +public class RpcSourceReaderContext implements SourceReader.Context { + private final RpcEndpointRef driverRef; + + public RpcSourceReaderContext(RpcEndpointRef driverRef) { + this.driverRef = driverRef; + } + + @Override + public int getIndexOfSubtask() { + return 0; + } + + @Override + public Boundedness getBoundedness() { + return null; + } + + @Override + public void signalNoMoreElement() {} + + @Override + public void sendSplitRequest() {} + + @Override + public void sendSourceEventToEnumerator(SourceEvent sourceEvent) {} + + @Override + public MetricsContext getMetricsContext() { + return null; + } + + @Override + public EventListener getEventListener() { + return null; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSplitEnumerator.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSplitEnumerator.java new file mode 100644 index 00000000000..cfb9b63c1ac --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSplitEnumerator.java @@ -0,0 +1,67 @@ +/* + * 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.partition.continuous.source.rpc; + +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; + +import org.apache.spark.rpc.RpcEndpointRef; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; + +public class RpcSplitEnumerator + implements SourceSplitEnumerator { + private final RpcEndpointRef driverRef; + + public RpcSplitEnumerator(RpcEndpointRef driverRef) { + this.driverRef = driverRef; + } + + @Override + public void open() {} + + @Override + public void run() throws Exception {} + + @Override + public void close() throws IOException {} + + @Override + public void addSplitsBack(List splits, int subtaskId) {} + + @Override + public int currentUnassignedSplitSize() { + return 0; + } + + @Override + public void handleSplitRequest(int subtaskId) {} + + @Override + public void registerReader(int subtaskId) {} + + @Override + public StateT snapshotState(long checkpointId) throws Exception { + return null; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception {} +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSplitEnumeratorContext.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSplitEnumeratorContext.java new file mode 100644 index 00000000000..388fc6f111f --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/partition/continuous/source/rpc/RpcSplitEnumeratorContext.java @@ -0,0 +1,67 @@ +/* + * 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.partition.continuous.source.rpc; + +import org.apache.seatunnel.api.common.metrics.MetricsContext; +import org.apache.seatunnel.api.event.EventListener; +import org.apache.seatunnel.api.source.SourceEvent; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; + +import org.apache.spark.rpc.RpcEndpointRef; + +import java.util.List; +import java.util.Set; + +public class RpcSplitEnumeratorContext + implements SourceSplitEnumerator.Context { + private final RpcEndpointRef driverRef; + + public RpcSplitEnumeratorContext(RpcEndpointRef driverRef) { + this.driverRef = driverRef; + } + + @Override + public int currentParallelism() { + return 0; + } + + @Override + public Set registeredReaders() { + return null; + } + + @Override + public void assignSplit(int subtaskId, List splits) {} + + @Override + public void signalNoMoreSplits(int subtask) {} + + @Override + public void sendEventToSourceReader(int subtaskId, SourceEvent event) {} + + @Override + public MetricsContext getMetricsContext() { + return null; + } + + @Override + public EventListener getEventListener() { + return null; + } +} diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java index 4107853ff35..c4cf4803dde 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/seatunnel/translation/spark/source/scan/SeaTunnelScan.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.translation.spark.source.scan; 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.spark.execution.MultiTableManager; import org.apache.seatunnel.translation.spark.source.partition.batch.SeaTunnelBatch; @@ -82,7 +83,7 @@ public MicroBatchStream toMicroBatchStream(String checkpointLocation) { @Override public ContinuousStream toContinuousStream(String checkpointLocation) { return new SeaTunnelContinuousStream( - source, + (SeaTunnelSource) source, parallelism, jobId, checkpointLocation, diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndpoint.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndpoint.java new file mode 100644 index 00000000000..e4932f6ad44 --- /dev/null +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-3.3/src/main/java/org/apache/spark/sql/execution/streaming/continuous/seatunnel/rpc/SplitEnumeratorEndpoint.java @@ -0,0 +1,123 @@ +/* + * 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.spark.sql.execution.streaming.continuous.seatunnel.rpc; + +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; + +import org.apache.spark.SparkEnv; +import org.apache.spark.rpc.RpcCallContext; +import org.apache.spark.rpc.RpcEndpointRef; +import org.apache.spark.rpc.RpcEnv; +import org.apache.spark.rpc.ThreadSafeRpcEndpoint; + +import scala.PartialFunction; +import scala.reflect.ClassTag; +import scala.runtime.BoxedUnit; + +import java.io.Serializable; + +public class SplitEnumeratorEndpoint implements ThreadSafeRpcEndpoint { + + @Override + public RpcEnv rpcEnv() { + return SparkEnv.get().rpcEnv(); + } + + @Override + public PartialFunction receiveAndReply(RpcCallContext context) { + return new PartialFunction() { + @Override + public boolean isDefinedAt(Object x) { + return x instanceof PollNext; + } + + @Override + public BoxedUnit apply(Object v1) { + PollNext pollNextReq = (PollNext) v1; + context.reply(new PollResponse(pollNextReq.subTaskId, null)); + return BoxedUnit.UNIT; + } + }; + } + + public static class PollNext implements Serializable { + private final int subTaskId; + + public PollNext(int subTaskId) { + this.subTaskId = subTaskId; + } + + public int subTaskId() { + return this.subTaskId; + } + } + + public static class PollResponse implements Serializable { + private final int subTaskId; + private final SourceSplit split; + + public PollResponse(int subTaskId, SourceSplit split) { + this.subTaskId = subTaskId; + this.split = split; + } + + public int getSubTaskId() { + return subTaskId; + } + + public SourceSplit getSplit() { + return split; + } + } + + public static interface ISplitEnumerator { + public SourceSplit pollNext(int subTaskId); + } + + public static class SplitEnumeratorStub implements ISplitEnumerator { + private final RpcEndpointRef endpointRef; + + public SplitEnumeratorStub(RpcEndpointRef endpointRef) { + this.endpointRef = endpointRef; + } + + @Override + public SourceSplit pollNext(int subTaskId) { + PollNext rep = new PollNext(subTaskId); + PollResponse resp = + endpointRef.askSync(rep, ClassTag.apply(PollResponse.class)); + return resp.getSplit(); + } + } + + public static class SplitEnumeratorSrv + implements ISplitEnumerator { + private SourceSplitEnumerator splitEnumerator; + + @Override + public SourceSplit pollNext(int subTaskId) { + try { + splitEnumerator.run(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return null; + } + } +}