Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Improve] Remove use SeaTunnelSink::getConsumedType method and mark it as deprecated #5755

Merged
merged 4 commits into from
Nov 7, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -64,8 +64,10 @@ default void setTypeInfo(SeaTunnelRowType seaTunnelRowType) {
/**
* Get the data type of the records consumed by this sink.
*
* @deprecated instead by {@link org.apache.seatunnel.api.table.factory.Factory}
* @return SeaTunnel data type.
*/
@Deprecated
Hisoka-X marked this conversation as resolved.
Show resolved Hide resolved
SeaTunnelDataType<IN> getConsumedType();

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,9 @@ public List<DataStreamTableInfo> execute(List<DataStreamTableInfo> upstreamDataS
saveModeHandler.ifPresent(SaveModeHandler::handleSaveMode);
}
DataStreamSink<Row> dataStreamSink =
stream.getDataStream().sinkTo(new FlinkSink<>(sink)).name(sink.getPluginName());
stream.getDataStream()
.sinkTo(new FlinkSink<>(sink, stream.getCatalogTable()))
.name(sink.getPluginName());
if (sinkConfig.hasPath(CommonOptions.PARALLELISM.key())) {
int parallelism = sinkConfig.getInt(CommonOptions.PARALLELISM.key());
dataStreamSink.setParallelism(parallelism);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,9 @@ public List<DataStreamTableInfo> execute(List<DataStreamTableInfo> upstreamDataS
}
DataStreamSink<Row> dataStreamSink =
stream.getDataStream()
.sinkTo(SinkV1Adapter.wrap(new FlinkSink<>(sink)))
.sinkTo(
SinkV1Adapter.wrap(
new FlinkSink<>(sink, stream.getCatalogTable())))
.name(sink.getPluginName());
if (sinkConfig.hasPath(CommonOptions.PARALLELISM.key())) {
int parallelism = sinkConfig.getInt(CommonOptions.PARALLELISM.key());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -140,7 +140,7 @@ public List<DatasetTableInfo> execute(List<DatasetTableInfo> upstreamDataStreams
Optional<SaveModeHandler> saveModeHandler = saveModeSink.getSaveModeHandler();
saveModeHandler.ifPresent(SaveModeHandler::handleSaveMode);
}
SparkSinkInjector.inject(dataset.write(), sink)
SparkSinkInjector.inject(dataset.write(), sink, datasetTableInfo.getCatalogTable())
.option("checkpointLocation", "/tmp")
.save();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,7 @@ public List<DatasetTableInfo> execute(List<DatasetTableInfo> upstreamDataStreams
Optional<SaveModeHandler> saveModeHandler = saveModeSink.getSaveModeHandler();
saveModeHandler.ifPresent(SaveModeHandler::handleSaveMode);
}
SparkSinkInjector.inject(dataset.write(), sink)
SparkSinkInjector.inject(dataset.write(), sink, datasetTableInfo.getCatalogTable())
.option("checkpointLocation", "/tmp")
.mode(SaveMode.Append)
.save();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.seatunnel.api.sink.DefaultSinkWriterContext;
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.flink.serialization.CommitWrapperSerializer;
import org.apache.seatunnel.translation.flink.serialization.FlinkSimpleVersionedSerializer;
Expand Down Expand Up @@ -48,8 +49,13 @@ public class FlinkSink<InputT, CommT, WriterStateT, GlobalCommT>

private final SeaTunnelSink<SeaTunnelRow, WriterStateT, CommT, GlobalCommT> sink;

public FlinkSink(SeaTunnelSink<SeaTunnelRow, WriterStateT, CommT, GlobalCommT> sink) {
private final CatalogTable catalogTable;

public FlinkSink(
SeaTunnelSink<SeaTunnelRow, WriterStateT, CommT, GlobalCommT> sink,
CatalogTable catalogTable) {
this.sink = sink;
this.catalogTable = catalogTable;
}

@Override
Expand All @@ -60,14 +66,15 @@ public SinkWriter<InputT, CommitWrapper<CommT>, FlinkWriterState<WriterStateT>>
new DefaultSinkWriterContext(context.getSubtaskId());

if (states == null || states.isEmpty()) {
return new FlinkSinkWriter<>(sink.createWriter(stContext), 1, sink.getConsumedType());
return new FlinkSinkWriter<>(
sink.createWriter(stContext), 1, catalogTable.getSeaTunnelRowType());
} else {
List<WriterStateT> restoredState =
states.stream().map(FlinkWriterState::getState).collect(Collectors.toList());
return new FlinkSinkWriter<>(
sink.restoreWriter(stContext, restoredState),
states.get(0).getCheckpointId() + 1,
sink.getConsumedType());
catalogTable.getSeaTunnelRowType());
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.seatunnel.translation.spark.sink;

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.common.Constants;
import org.apache.seatunnel.common.utils.SerializationUtils;
Expand All @@ -42,6 +43,8 @@ public class SparkSink<StateT, CommitInfoT, AggregatedCommitInfoT>

private volatile SeaTunnelSink<SeaTunnelRow, StateT, CommitInfoT, AggregatedCommitInfoT> sink;

private volatile CatalogTable catalogTable;

private void init(DataSourceOptions options) {
if (sink == null) {
this.sink =
Expand All @@ -53,6 +56,16 @@ private void init(DataSourceOptions options) {
"can not find sink "
+ "class string in DataSourceOptions")));
}
if (catalogTable == null) {
this.catalogTable =
SerializationUtils.stringToObject(
options.get(SparkSinkInjector.SINK_CATALOG_TABLE)
.orElseThrow(
() ->
new IllegalArgumentException(
"can not find sink "
+ "catalog table string in DataSourceOptions")));
}
}

@Override
Expand All @@ -61,7 +74,7 @@ public StreamWriter createStreamWriter(
init(options);

try {
return new SparkStreamWriter<>(sink);
return new SparkStreamWriter<>(sink, catalogTable);
} catch (IOException e) {
throw new RuntimeException("find error when createStreamWriter", e);
}
Expand All @@ -73,7 +86,7 @@ public Optional<DataSourceWriter> createWriter(
init(options);

try {
return Optional.of(new SparkDataSourceWriter<>(sink));
return Optional.of(new SparkDataSourceWriter<>(sink, catalogTable));
} catch (IOException e) {
throw new RuntimeException("find error when createStreamWriter", e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.seatunnel.translation.spark.sink;

import org.apache.seatunnel.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.common.Constants;
import org.apache.seatunnel.common.utils.SerializationUtils;

Expand All @@ -31,16 +32,24 @@ public class SparkSinkInjector {
private static final String SPARK_SINK_CLASS_NAME =
"org.apache.seatunnel.translation.spark.sink.SparkSink";

public static final String SINK_CATALOG_TABLE = "sink.catalog.table";

public static DataStreamWriter<Row> inject(
DataStreamWriter<Row> dataset, SeaTunnelSink<?, ?, ?, ?> sink) {
DataStreamWriter<Row> dataset,
SeaTunnelSink<?, ?, ?, ?> sink,
CatalogTable catalogTable) {
return dataset.format(SPARK_SINK_CLASS_NAME)
.outputMode(OutputMode.Append())
.option(Constants.SINK, SerializationUtils.objectToString(sink));
.option(Constants.SINK_SERIALIZATION, SerializationUtils.objectToString(sink))
.option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTable));
}

public static DataFrameWriter<Row> inject(
DataFrameWriter<Row> dataset, SeaTunnelSink<?, ?, ?, ?> sink) {
DataFrameWriter<Row> dataset,
SeaTunnelSink<?, ?, ?, ?> sink,
CatalogTable catalogTable) {
return dataset.format(SPARK_SINK_CLASS_NAME)
.option(Constants.SINK, SerializationUtils.objectToString(sink));
.option(Constants.SINK_SERIALIZATION, SerializationUtils.objectToString(sink))
.option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTable));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.seatunnel.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;

import org.apache.spark.sql.catalyst.InternalRow;
Expand All @@ -44,10 +45,14 @@ public class SparkDataSourceWriter<StateT, CommitInfoT, AggregatedCommitInfoT>
@Nullable protected final SinkAggregatedCommitter<CommitInfoT, AggregatedCommitInfoT>
sinkAggregatedCommitter;

protected final CatalogTable catalogTable;

public SparkDataSourceWriter(
SeaTunnelSink<SeaTunnelRow, StateT, CommitInfoT, AggregatedCommitInfoT> sink)
SeaTunnelSink<SeaTunnelRow, StateT, CommitInfoT, AggregatedCommitInfoT> sink,
CatalogTable catalogTable)
throws IOException {
this.sink = sink;
this.catalogTable = catalogTable;
this.sinkAggregatedCommitter = sink.createAggregatedCommitter().orElse(null);
if (sinkAggregatedCommitter != null) {
sinkAggregatedCommitter.init();
Expand All @@ -56,7 +61,7 @@ public SparkDataSourceWriter(

@Override
public DataWriterFactory<InternalRow> createWriterFactory() {
return new SparkDataWriterFactory<>(sink);
return new SparkDataWriterFactory<>(sink, catalogTable);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.seatunnel.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.sink.SinkCommitter;
import org.apache.seatunnel.api.sink.SinkWriter;
import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;

import org.apache.spark.sql.catalyst.InternalRow;
Expand All @@ -32,9 +33,12 @@
public class SparkDataWriterFactory<CommitInfoT, StateT> implements DataWriterFactory<InternalRow> {

private final SeaTunnelSink<SeaTunnelRow, StateT, CommitInfoT, ?> sink;
private final CatalogTable catalogTable;

SparkDataWriterFactory(SeaTunnelSink<SeaTunnelRow, StateT, CommitInfoT, ?> sink) {
SparkDataWriterFactory(
SeaTunnelSink<SeaTunnelRow, StateT, CommitInfoT, ?> sink, CatalogTable catalogTable) {
this.sink = sink;
this.catalogTable = catalogTable;
}

@Override
Expand All @@ -53,6 +57,7 @@ public DataWriter<InternalRow> createDataWriter(int partitionId, long taskId, lo
} catch (IOException e) {
throw new RuntimeException("Failed to create SinkCommitter.", e);
}
return new SparkDataWriter<>(writer, committer, sink.getConsumedType(), epochId);
return new SparkDataWriter<>(
writer, committer, catalogTable.getSeaTunnelRowType(), epochId);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.seatunnel.translation.spark.sink.writer;

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.spark.sql.catalyst.InternalRow;
Expand All @@ -32,9 +33,10 @@ public class SparkStreamWriter<StateT, CommitInfoT, AggregatedCommitInfoT>
implements StreamWriter {

public SparkStreamWriter(
SeaTunnelSink<SeaTunnelRow, StateT, CommitInfoT, AggregatedCommitInfoT> sink)
SeaTunnelSink<SeaTunnelRow, StateT, CommitInfoT, AggregatedCommitInfoT> sink,
CatalogTable catalogTable)
throws IOException {
super(sink);
super(sink, catalogTable);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import org.apache.seatunnel.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.sink.SinkAggregatedCommitter;
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;
Expand All @@ -44,16 +45,20 @@ public class SeaTunnelBatchWrite<StateT, CommitInfoT, AggregatedCommitInfoT>

private final SinkAggregatedCommitter<CommitInfoT, AggregatedCommitInfoT> aggregatedCommitter;

private final CatalogTable catalogTable;

public SeaTunnelBatchWrite(
SeaTunnelSink<SeaTunnelRow, StateT, CommitInfoT, AggregatedCommitInfoT> sink)
SeaTunnelSink<SeaTunnelRow, StateT, CommitInfoT, AggregatedCommitInfoT> sink,
CatalogTable catalogTable)
throws IOException {
this.sink = sink;
this.catalogTable = catalogTable;
this.aggregatedCommitter = sink.createAggregatedCommitter().orElse(null);
}

@Override
public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
return new SeaTunnelSparkDataWriterFactory<>(sink);
return new SeaTunnelSparkDataWriterFactory<>(sink, catalogTable);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.seatunnel.translation.spark.sink;

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.common.Constants;
import org.apache.seatunnel.common.utils.SerializationUtils;
Expand Down Expand Up @@ -45,18 +46,27 @@ public class SeaTunnelSinkTable implements Table, SupportsWrite {

private final SeaTunnelSink<SeaTunnelRow, ?, ?, ?> sink;

private final CatalogTable catalogTable;

public SeaTunnelSinkTable(Map<String, String> properties) {
this.properties = properties;
String sinkSerialization = properties.getOrDefault(Constants.SINK_SERIALIZATION, "");
if (StringUtils.isBlank(sinkSerialization)) {
throw new IllegalArgumentException("sink.serialization must be specified");
throw new IllegalArgumentException(Constants.SINK_SERIALIZATION + " must be specified");
}
this.sink = SerializationUtils.stringToObject(sinkSerialization);
String sinkCatalogTableSerialization =
properties.getOrDefault(SparkSinkInjector.SINK_CATALOG_TABLE, "");
if (StringUtils.isBlank(sinkCatalogTableSerialization)) {
throw new IllegalArgumentException(
SparkSinkInjector.SINK_CATALOG_TABLE + " must be specified");
}
this.catalogTable = SerializationUtils.stringToObject(sinkCatalogTableSerialization);
}

@Override
public WriteBuilder newWriteBuilder(LogicalWriteInfo info) {
return new SeaTunnelWriteBuilder<>(sink);
return new SeaTunnelWriteBuilder<>(sink, catalogTable);
}

@Override
Expand All @@ -66,7 +76,7 @@ public String name() {

@Override
public StructType schema() {
return (StructType) TypeConverterUtils.convert(sink.getConsumedType());
return (StructType) TypeConverterUtils.convert(catalogTable.getSeaTunnelRowType());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.seatunnel.translation.spark.sink;

import org.apache.seatunnel.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.common.Constants;
import org.apache.seatunnel.common.utils.SerializationUtils;

Expand All @@ -30,16 +31,24 @@ public class SparkSinkInjector {

private static final String SINK_NAME = SeaTunnelSink.class.getSimpleName();

public static final String SINK_CATALOG_TABLE = "sink.catalog.table";

public static DataStreamWriter<Row> inject(
DataStreamWriter<Row> dataset, SeaTunnelSink<?, ?, ?, ?> sink) {
DataStreamWriter<Row> dataset,
SeaTunnelSink<?, ?, ?, ?> sink,
CatalogTable catalogTable) {
return dataset.format(SINK_NAME)
.outputMode(OutputMode.Append())
.option(Constants.SINK_SERIALIZATION, SerializationUtils.objectToString(sink));
.option(Constants.SINK_SERIALIZATION, SerializationUtils.objectToString(sink))
.option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTable));
}

public static DataFrameWriter<Row> inject(
DataFrameWriter<Row> dataset, SeaTunnelSink<?, ?, ?, ?> sink) {
DataFrameWriter<Row> dataset,
SeaTunnelSink<?, ?, ?, ?> sink,
CatalogTable catalogTable) {
return dataset.format(SINK_NAME)
.option(Constants.SINK_SERIALIZATION, SerializationUtils.objectToString(sink));
.option(Constants.SINK_SERIALIZATION, SerializationUtils.objectToString(sink))
.option(SINK_CATALOG_TABLE, SerializationUtils.objectToString(catalogTable));
}
}
Loading