voonhous commented on code in PR #13285:
URL: https://github.com/apache/hudi/pull/13285#discussion_r2085739932


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java:
##########
@@ -108,7 +105,7 @@ public BootstrapOperator(Configuration conf) {
 
   @Override
   public void snapshotState(StateSnapshotContext context) throws Exception {
-    lastInstantTime = this.ckpMetadata.lastPendingInstant();
+    lastInstantTime = 
StreamerUtil.getLastCompletedInstant(StreamerUtil.createMetaClient(this.conf));

Review Comment:
   Will this incur large FS scans for every checkpoint? This might cause rate 
limit/high fs pressure for jobs with high parallelism. 
   
   is it possible for us to use a correspondent and delegate the fsView like 
operation to the coodinator?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java:
##########
@@ -397,75 +408,66 @@ private void startInstant() {
     this.instant = this.writeClient.startCommit(tableState.commitAction, 
this.metaClient);
     
this.metaClient.getActiveTimeline().transitionRequestedToInflight(tableState.commitAction,
 this.instant);
     this.writeClient.setWriteTimer(tableState.commitAction);
-    this.ckpMetadata.startInstant(this.instant);
     LOG.info("Create instant [{}] for table [{}] with type [{}]", this.instant,
         this.conf.getString(FlinkOptions.TABLE_NAME), 
conf.getString(FlinkOptions.TABLE_TYPE));
+    return this.instant;
   }
 
   /**
-   * Initializes the instant.
-   *
-   * <p>Recommits the last inflight instant if the write metadata checkpoint 
successfully
+   * Recommits the last inflight instant if the write metadata checkpoint 
successfully
    * but was not committed due to some rare cases.
-   *
-   * <p>Starts a new instant, a writer can not flush data buffer
-   * until it finds a new inflight instant on the timeline.
    */
-  private void initInstant(String instant) {
+  private void recommitInstant(long checkpointId, String instant, 
WriteMetadataEvent[] bootstrapBuffer) {
     HoodieTimeline completedTimeline = 
this.metaClient.getActiveTimeline().filterCompletedInstants();
     if (instant.equals(WriteMetadataEvent.BOOTSTRAP_INSTANT) || 
completedTimeline.containsInstant(instant)) {
       // the last instant committed successfully
-      reset();
+      // no-op
     } else {
       LOG.info("Recommit instant {}", instant);
-      // Recommit should start heartbeat for lazy failed writes clean policy 
to avoid aborting for heartbeat expired.
+      // Recommit should start heartbeat for lazy failed writes clean policy 
to avoid aborting for heartbeat expired;
+      // The following up checkpoints would recommit the instant.
       if (writeClient.getConfig().getFailedWritesCleanPolicy().isLazy()) {
         writeClient.getHeartbeatClient().start(instant);
       }
-      commitInstant(instant);
-    }
-    // stop the heartbeat for old instant
-    if (writeClient.getConfig().getFailedWritesCleanPolicy().isLazy() && 
!WriteMetadataEvent.BOOTSTRAP_INSTANT.equals(this.instant)) {
-      writeClient.getHeartbeatClient().stop(this.instant);
+      commitInstant(checkpointId, instant, bootstrapBuffer);
     }
-    // starts a new instant
-    startInstant();
     // upgrade downgrade
     this.writeClient.upgradeDowngrade(this.instant, this.metaClient);
   }
 
   private void handleBootstrapEvent(WriteMetadataEvent event) {
-    this.eventBuffer[event.getTaskID()] = event;
+    cleanLegacyEvent(event);

Review Comment:
   Just curious, if all `sendBootstrapEvent` calls are removed, no 
`bootstrapEvent`s should be sent, in what scenarious will there be 
`bootstrapEvent`s  to clean?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java:
##########
@@ -226,6 +197,7 @@ private void reloadWriteMetaState() throws Exception {
     this.writeMetadataState.clear();
     WriteMetadataEvent event = WriteMetadataEvent.builder()
         .taskID(taskID)
+        .checkpointId(checkpointId)
         .instantTime(currentInstant)
         .writeStatus(new ArrayList<>(writeStatuses))
         .bootstrap(true)

Review Comment:
   `bootstrapEvent`s are used to signal coordinator that it can start a new 
instant. 
   Since we are removing this signalling mechanism, is this attribute still 
needed? 
   
   Or is this left behind for migration purposes?



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