errose28 commented on code in PR #9779:
URL: https://github.com/apache/ozone/pull/9779#discussion_r2824958547


##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/upgrade/DataNodeUpgradeFinalizer.java:
##########
@@ -42,39 +37,10 @@ public DataNodeUpgradeFinalizer(HDDSLayoutVersionManager 
versionManager) {
   }
 
   @Override
-  public void preFinalizeUpgrade(DatanodeStateMachine dsm)
-      throws IOException {
-    if (!canFinalizeDataNode(dsm)) {
-      // DataNode is not yet ready to finalize.
-      // Reset the Finalization state.
-      getVersionManager().setUpgradeState(FINALIZATION_REQUIRED);
-      String msg = "Pre Finalization checks failed on the DataNode.";
-      logAndEmit(msg);
-      throw new UpgradeException(msg, PREFINALIZE_VALIDATION_FAILED);
-    }
+  public void preFinalizeUpgrade(DatanodeStateMachine dsm) {

Review Comment:
   We can remove this override, it is the same as the parent.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/SCMUpgradeFinalizer.java:
##########
@@ -115,89 +109,89 @@ public void 
postFinalizeUpgrade(SCMUpgradeFinalizationContext context)
         context.getFinalizationStateManager();
     if (!stateManager.crossedCheckpoint(
         FinalizationCheckpoint.FINALIZATION_COMPLETE)) {
-      createPipelinesAfterFinalization(context);
+      waitForDatanodesToFinalize(context);
       stateManager.removeFinalizingMark();
     }
   }
 
-  private void closePipelinesBeforeFinalization(PipelineManager 
pipelineManager)
-      throws IOException {
-    /*
-     * Before we can call finalize the feature, we need to make sure that
-     * all existing pipelines are closed and pipeline Manger would freeze
-     * all new pipeline creation.
-     */
-    String msg = "  Existing pipelines and containers will be closed " +
-        "during Upgrade.";
-    msg += "\n  New pipelines creation will remain frozen until Upgrade " +
-        "is finalized.";
-
-    // Pipeline creation should already be frozen when the finalization state
-    // manager set the checkpoint.
-    if (!pipelineManager.isPipelineCreationFrozen()) {
-      throw new SCMException("Error during finalization. Pipeline creation" +
-          "should have been frozen before closing existing pipelines.",
-          SCMException.ResultCodes.INTERNAL_ERROR);
-    }
-
-    for (Pipeline pipeline : pipelineManager.getPipelines()) {
-      if (pipeline.getPipelineState() != CLOSED) {
-        pipelineManager.closePipeline(pipeline.getId());
-      }
-    }
-
-    // We can not yet move all the existing data nodes to HEALTHY-READONLY
-    // state since the next heartbeat will move them back to HEALTHY state.
-    // This has to wait till postFinalizeUpgrade, when SCM MLV version is
-    // already upgraded as part of finalize processing.
-    // While in this state, it should be safe to do finalize processing for
-    // all new features. This will also update ondisk mlv version. Any
-    // disrupting upgrade can add a hook here to make sure that SCM is in a
-    // consistent state while finalizing the upgrade.
-
-    logAndEmit(msg);
-  }
+  /**
+   * Wait for all HEALTHY datanodes to complete finalization before finishing
+   * SCM finalization. This ensures that when the client receives a
+   * FINALIZATION_DONE status, all healthy datanodes have also finalized.
+   *
+   * A datanode is considered finalized when its metadata layout version (MLV)
+   * equals its software layout version (SLV), indicating it has completed
+   * processing all layout features.
+   *
+   * @param context The finalization context containing node manager reference
+   * @throws SCMException if waiting is interrupted or SCM loses leadership
+   * @throws NotLeaderException if SCM is no longer the leader
+   */
+  private void waitForDatanodesToFinalize(SCMUpgradeFinalizationContext 
context)

Review Comment:
   I don't think we need to block on the server side until all datadnodes 
finalize. Instead, I think this logic should be async in the heartbeat 
handling. When a pre-finalized datanode heartbeats to a finalized SCM, it 
should be instructed to finalize in the heartbeat response. This way we don't 
need any dedicated threads with interrupts/resumes on leader changes.
   
   When the client instructs SCM to finalize, it can get a response when SCM 
has finished finalizing even if the DNs have not. For now we can use the 
existing finalization complete status in the response, but in follow up changes 
we will change how the status API works to indicate each component's 
finalization status individually.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/FinalizationStateManagerImpl.java:
##########
@@ -72,27 +71,6 @@ private void publishCheckpoint(FinalizationCheckpoint 
checkpoint) {
     // Move the upgrade status according to this checkpoint. This is sent
     // back to the client if they query for the current upgrade status.
     versionManager.setUpgradeState(checkpoint.getStatus());
-
-    // Check whether this checkpoint change requires us to move node state.
-    // If this is necessary, it must be done before unfreezing pipeline
-    // creation to make sure nodes are not added to pipelines based on
-    // outdated layout information.
-    // This operation is not idempotent.
-    if (checkpoint == FinalizationCheckpoint.MLV_EQUALS_SLV) {
-      upgradeContext.getNodeManager().forceNodesToHealthyReadOnly();

Review Comment:
   We can actually delete `forceNodesToHealthyReadOnly` since this was the only 
caller outside of tests.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/FinalizationManager.java:
##########
@@ -55,11 +55,6 @@ void buildUpgradeContext(NodeManager nodeManager,
 
   void onLeaderReady();
 
-  static boolean shouldCreateNewPipelines(FinalizationCheckpoint checkpoint) {

Review Comment:
   `crossedCheckpoint` can also be removed from this interface since it is now 
unused.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/upgrade/TestHddsUpgradeUtils.java:
##########
@@ -128,17 +128,20 @@ public static void 
testPostUpgradeConditionsSCM(StorageContainerManager scm,
       fail("Timeout waiting for Upgrade to complete on SCM.");
     }
 
-    // SCM will not return from finalization until there is at least one
-    // RATIS 3 pipeline. For this to exist, all three of our datanodes must
-    // be in the HEALTHY state.
+    // SCM will not return from finalization until all HEALTHY datanodes
+    // have completed their finalization (MLV == SLV). This ensures datanodes
+    // are ready to serve requests even though containers may remain OPEN.
     testDataNodesStateOnSCM(scm, numDatanodes, HEALTHY, HEALTHY_READONLY);
 
     int countContainers = 0;
     for (ContainerInfo ci : scm.getContainerManager().getContainers()) {
       HddsProtos.LifeCycleState ciState = ci.getState();
       LOG.info("testPostUpgradeConditionsSCM: container state is {}",
           ciState.name());
-      assertTrue((ciState == HddsProtos.LifeCycleState.CLOSED) ||
+      // Containers can now be in any state since we no longer close pipelines

Review Comment:
   Can we remove this test on container states completely? The new upgrade flow 
should be independent of any container states. There's a few other places in 
this test where I think we can remove similar checks.



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to