cshuo commented on code in PR #18040:
URL: https://github.com/apache/hudi/pull/18040#discussion_r2749414100


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/reader/HoodieRecordEmitter.java:
##########
@@ -18,19 +18,78 @@
 
 package org.apache.hudi.source.reader;
 
+import org.apache.flink.api.common.eventtime.Watermark;
 import org.apache.flink.api.connector.source.SourceOutput;
 import org.apache.flink.connector.base.source.reader.RecordEmitter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.source.split.HoodieSourceSplit;
 
 /**
  * Default Hoodie record emitter.
- * @param <T>
+ *
+ * <p>This emitter handles watermark emission based on split information.
+ *
+ * @param <T> The type of records to emit
  */
 public class HoodieRecordEmitter<T> implements 
RecordEmitter<HoodieRecordWithPosition<T>, T, HoodieSourceSplit> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieRecordEmitter.class);
+  private HoodieSourceSplit lastSplit = null;
+  private long watermark = Long.MIN_VALUE;
 
   @Override
   public void emitRecord(HoodieRecordWithPosition<T> record, SourceOutput<T> 
output, HoodieSourceSplit split) throws Exception {
+    if (lastSplit == null || !split.splitId().equals(lastSplit.splitId())) {

Review Comment:
   If we're going to support watermark alignment later, a split may be paused 
and the reader will switch to another split, so we cannot simply determine the 
end of split by checking the swtich of split id.
   
   A possible solution is emitting a sentinel record to mark the end of each 
split, then the record emitter can emit a watermark once receiving the sentinel 
record. And I think it's also necessary to provide a config option to enable 
this feature.



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