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


##########
hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/TimelineService.java:
##########
@@ -111,9 +111,6 @@ public static class Config implements Serializable {
     @Parameter(names = {"--enable-marker-requests", "-em"}, description = 
"Enable handling of marker-related requests")
     public boolean enableMarkerRequests = false;
 
-    @Parameter(names = {"--enable-instant-state-requests"}, description = 
"Enable handling of instant state requests")
-    public boolean enableInstantStateRequests = false;
-

Review Comment:
   also remove `instantStateForceRefreshRequestNumber`? 



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java:
##########
@@ -254,35 +219,6 @@ protected String lastPendingInstant() {
    * @return The instant time
    */
   protected String instantToWrite(boolean hasData) {

Review Comment:
   argument `hasData` is not needed anymore.



##########
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);
+    WriteMetadataEvent[] eventBuffer = getOrCreateBootstrapBuffer(event);
+    eventBuffer[event.getTaskID()] = event;
     if (Arrays.stream(eventBuffer).allMatch(evt -> evt != null && 
evt.isBootstrap())) {
-      // start to initialize the instant.
-      final String instant = Arrays.stream(eventBuffer)
-          .filter(evt -> evt.getWriteStatuses().size() > 0)
-          .findFirst().map(WriteMetadataEvent::getInstantTime)
-          .orElse(WriteMetadataEvent.BOOTSTRAP_INSTANT);
-
-      // if currentInstant is pending && bootstrap event instant is empty
-      // reuse currentInstant, reject bootstrap
-      if 
(this.metaClient.reloadActiveTimeline().filterInflightsAndRequested().containsInstant(this.instant)
-              && instant.equals(WriteMetadataEvent.BOOTSTRAP_INSTANT)
-              && this.tableState.operationType == WriteOperationType.INSERT) {
-        LOG.warn("Reuse current pending Instant {} with {} operationType, "
-                + "ignoring empty bootstrap event.", this.instant, 
WriteOperationType.INSERT.value());
-        reset();
-
-        // send commit act event to unblock write tasks
-        sendCommitAckEvents(-1L);
-        return;
-      }
+      // start to recommit the instant.

Review Comment:
   For partial failover, this condition will not be satisfied, will the event 
buffer be committed later? e.g., during next ckp complete



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/common/AbstractStreamWriteFunction.java:
##########
@@ -206,17 +186,8 @@ private void restoreWriteMetadata() throws Exception {
         // re-commit the inflight instant
         this.eventGateway.sendEventToCoordinator(event);
         LOG.info("Send uncommitted write metadata event to coordinator, 
task[{}].", taskID);
-        eventSent = true;
       }
     }
-    if (!eventSent) {
-      sendBootstrapEvent();
-    }
-  }
-
-  protected void sendBootstrapEvent() {
-    
this.eventGateway.sendEventToCoordinator(WriteMetadataEvent.emptyBootstrap(taskID));

Review Comment:
   `emptyBootstrap` can be also removed in `WriteMetadataEvent`



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