AHeise commented on a change in pull request #16796:
URL: https://github.com/apache/flink/pull/16796#discussion_r688519236



##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicSink.java
##########
@@ -185,44 +190,57 @@ public SinkRuntimeProvider getSinkRuntimeProvider(Context 
context) {
         final SerializationSchema<RowData> valueSerialization =
                 createSerialization(context, valueEncodingFormat, 
valueProjection, null);
 
+        final KafkaSinkBuilder<RowData> sinkBuilder = KafkaSink.builder();
+        final List<LogicalType> physicalChildren = 
physicalDataType.getLogicalType().getChildren();
+        if (transactionalIdPrefix != null) {
+            sinkBuilder.setTransactionalIdPrefix(transactionalIdPrefix);
+        }
+        final KafkaSink<RowData> kafkaSink =
+                sinkBuilder
+                        .setDeliverGuarantee(deliveryGuarantee)
+                        .setBootstrapServers(
+                                
properties.get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG).toString())
+                        .setKafkaProducerConfig(properties)
+                        .setRecordSerializer(
+                                new DynamicKafkaRecordSerializationSchema(
+                                        topic,
+                                        partitioner,
+                                        keySerialization,
+                                        valueSerialization,
+                                        getFieldGetters(physicalChildren, 
keyProjection),
+                                        getFieldGetters(physicalChildren, 
valueProjection),
+                                        hasMetadata(),
+                                        getMetadataPositions(physicalChildren),
+                                        upsertMode))
+                        .build();
         if (flushMode.isEnabled() && upsertMode) {
-            BufferedUpsertSinkFunction buffedSinkFunction =
-                    new BufferedUpsertSinkFunction(
-                            createKafkaProducer(keySerialization, 
valueSerialization),
-                            physicalDataType,
-                            keyProjection,
-                            context.createTypeInformation(consumedDataType),
-                            flushMode);
-            return SinkFunctionProvider.of(buffedSinkFunction, parallelism);
-        } else {
-            final KafkaSinkBuilder<RowData> sinkBuilder = KafkaSink.builder();
-            final List<LogicalType> physicalChildren =
-                    physicalDataType.getLogicalType().getChildren();
-            if (transactionalIdPrefix != null) {
-                sinkBuilder.setTransactionalIdPrefix(transactionalIdPrefix);
-            }
-            return SinkProvider.of(
-                    sinkBuilder
-                            .setDeliverGuarantee(deliveryGuarantee)
-                            .setBootstrapServers(
-                                    properties
-                                            
.get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)
-                                            .toString())
-                            .setKafkaProducerConfig(properties)
-                            .setRecordSerializer(
-                                    new DynamicKafkaRecordSerializationSchema(
-                                            topic,
-                                            partitioner,
-                                            keySerialization,
-                                            valueSerialization,
-                                            getFieldGetters(physicalChildren, 
keyProjection),
-                                            getFieldGetters(physicalChildren, 
valueProjection),
-                                            hasMetadata(),
-                                            
getMetadataPositions(physicalChildren),
-                                            upsertMode))
-                            .build(),
-                    parallelism);
+            return (DataStreamSinkProvider)

Review comment:
       I guess it depends on the definition. Everything is quite whacky at this 
point. For example, if this was keeping track of a basketball game, you could 
miss some in-between scores (it could jump from 90-80 to 96-80 and you wouldn't 
know if it was 3x2 or 2x3 points). That's not exactly what I would be looking 
for in general and how I'd define EOS to begin with. However, if this was a 
log-compacted Kafka topic with heavily lagging consumer, then it is EOS.
   @twalthr any idea if folks would like to use upsert and really EOS on each 
record?




-- 
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]


Reply via email to