ruanhang1993 commented on a change in pull request #17991:
URL: https://github.com/apache/flink/pull/17991#discussion_r760931561



##########
File path: 
flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/connector/kafka/source/reader/KafkaRecordEmitter.java
##########
@@ -19,21 +19,62 @@
 package org.apache.flink.connector.kafka.source.reader;
 
 import org.apache.flink.api.connector.source.SourceOutput;
-import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.connector.base.source.reader.RecordEmitter;
+import 
org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema;
 import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitState;
+import org.apache.flink.util.Collector;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 /** The {@link RecordEmitter} implementation for {@link KafkaSourceReader}. */
 public class KafkaRecordEmitter<T>
-        implements RecordEmitter<Tuple3<T, Long, Long>, T, 
KafkaPartitionSplitState> {
+        implements RecordEmitter<ConsumerRecord<byte[], byte[]>, T, 
KafkaPartitionSplitState> {
+
+    private final KafkaRecordDeserializationSchema<T> deserializationSchema;
+    private final SimpleCollector<T> collector = new SimpleCollector<>();
+
+    public KafkaRecordEmitter(KafkaRecordDeserializationSchema<T> 
deserializationSchema) {
+        this.deserializationSchema = deserializationSchema;
+    }
 
     @Override
     public void emitRecord(
-            Tuple3<T, Long, Long> element,
+            ConsumerRecord<byte[], byte[]> consumerRecord,
             SourceOutput<T> output,
             KafkaPartitionSplitState splitState)
             throws Exception {
-        output.collect(element.f0, element.f2);
-        splitState.setCurrentOffset(element.f1 + 1);
+        try {
+            deserializationSchema.deserialize(consumerRecord, collector);

Review comment:
       When KafkaRecordDeserializationSchema.deserialize only output one 
record, everything looks good to me. But 
KafkaRecordDeserializationSchema.deserialize could output multiple records, I 
am afraid that the same problem will occur when exists object reusing among 
them. 
   I am not sure if we should consider this situation. Or we should add a 
warning for it ? 




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