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


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

Review Comment:
   > Because there is instantToHeartbeatMap cache in HoodieHeartBeatClient 
which may return directly instead of update heart beat time.
   
   It's okay if the instant heartbeat timer is already there? BTW, this is 
legacy logic and I do not change it in this PR.



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