AHeise commented on a change in pull request #17870:
URL: https://github.com/apache/flink/pull/17870#discussion_r755789192
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertSink.java
##########
@@ -43,19 +45,19 @@
private final DataType physicalDataType;
private final int[] keyProjection;
private final SinkBufferFlushMode bufferFlushMode;
- private final Function<RowData, RowData> valueCopyFunction;
+ @Nullable private final TypeSerializer<RowData> typeSerializer;
ReducingUpsertSink(
Sink<RowData, ?, WriterState, ?> wrappedSink,
DataType physicalDataType,
int[] keyProjection,
SinkBufferFlushMode bufferFlushMode,
- Function<RowData, RowData> valueCopyFunction) {
Review comment:
Here we could also use `SerializableFunction` but your change is also
good (in general not a huge fan of nullable fields that imply different
behavior).
##########
File path:
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/SinkBufferFlushMode.java
##########
@@ -18,10 +18,11 @@
package org.apache.flink.streaming.connectors.kafka.table;
+import java.io.Serializable;
import java.util.Objects;
/** Sink buffer flush configuration. */
-public class SinkBufferFlushMode {
+public class SinkBufferFlushMode implements Serializable {
Review comment:
🙈
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectStreamSink.java
##########
@@ -30,17 +32,19 @@
@Internal
public class CollectStreamSink<T> extends DataStreamSink<T> {
- private final LegacySinkTransformation<T> transformation;
+ private final PhysicalTransformation<T> transformation;
+ @SuppressWarnings("unchecked")
public CollectStreamSink(DataStream<T> inputStream,
CollectSinkOperatorFactory<T> factory) {
super(inputStream, (CollectSinkOperator<T>) factory.getOperator());
this.transformation =
- new LegacySinkTransformation<>(
- inputStream.getTransformation(), "Collect Stream
Sink", factory, 1);
+ (PhysicalTransformation<T>)
Review comment:
Why is the cast necessary?
Can you try to make `public class LegacySinkTransformation<T> extends
PhysicalTransformation<T>`?
The second ctor should a factory method instead...
##########
File path:
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSinkITCase.java
##########
@@ -152,6 +154,43 @@ public void invoke(
assertTimestampResults(timestamps, rows);
}
+ @Test
+ public void testUnifiedSinksAreUsableWithDataStreamSinkProvider()
+ throws ExecutionException, InterruptedException {
+ final StreamTableEnvironment tableEnv =
StreamTableEnvironment.create(env);
+ final SharedReference<List<RowData>> fetched = sharedObjects.add(new
ArrayList<>());
+ final List<Row> rows = Arrays.asList(Row.of(1), Row.of(2));
+
+ final TableDescriptor sourceDescriptor =
+ TableFactoryHarness.newBuilder()
+ .schema(Schema.newBuilder().column("a", INT()).build())
+ .source(new TimestampTestSource(rows))
+ .sink(
+ new TableFactoryHarness.SinkBase() {
+ @Override
+ public DataStreamSinkProvider
getSinkRuntimeProvider(
+ DynamicTableSink.Context context) {
+ return dataStream ->
+ dataStream.sinkTo(
+ TestSink.newBuilder()
+ .setWriter(
+ new
RecordWriter(fetched))
+
.setCommittableSerializer(
+
TestSink
+
.StringCommittableSerializer
+
.INSTANCE)
+ .build());
+ }
+ })
Review comment:
Could you please extract that to a method local class to ease the
formatting?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]