errose28 commented on code in PR #9955:
URL: https://github.com/apache/ozone/pull/9955#discussion_r2978235197
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -760,19 +760,14 @@ protected void
sendFinalizeToDatanodeIfNeeded(DatanodeDetails datanodeDetails,
datanodeDetails.getHostName(), dnSlv, scmSlv);
}
- if (FinalizationManager.shouldTellDatanodesToFinalize(
- scmContext.getFinalizationCheckpoint())) {
- // Because we have crossed the MLV_EQUALS_SLV checkpoint, SCM metadata
- // layout version will not change. We can now compare it to the
- // datanodes' metadata layout versions to tell them to finalize.
+ if
(FinalizationManager.shouldTellDatanodesToFinalize(scmLayoutVersionManager)) {
+ // Because the finalizationManager / versionManager says finalization is
not needed
+ // it means any DN that is reporting a metadata layout version less than
the SCM's metadata layout version
+ // can be finalized.
int scmMlv = scmLayoutVersionManager.getMetadataLayoutVersion();
- // If the datanode mlv < scm mlv, it can not be allowed to be part of
- // any pipeline. However it can be allowed to join the cluster
if (dnMlv < scmMlv) {
- LOG.warn("Data node {} can not be used in any pipeline in the " +
- "cluster. " + "DataNode MetadataLayoutVersion = {}, SCM " +
- "MetadataLayoutVersion = {}",
+ LOG.warn("Data node {} has a MetadataLayoutVersion = {}, SCM
MetadataLayoutVersion = {}. Sending finalize",
Review Comment:
Just adding a note here that we may want to change the log level here later.
If SCM sends all the finalize commands out as soon as it finalizes, then
warning may make sense since it means the DN did not process the original
command sent in the last heartbeat. If we instead wait for DNs to heartbeat to
send out the finalize commands, then this would be the place they are expected
to be sent out, and even INFO may be too high a level if there are many
datanodes.
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java:
##########
@@ -760,19 +760,14 @@ protected void
sendFinalizeToDatanodeIfNeeded(DatanodeDetails datanodeDetails,
datanodeDetails.getHostName(), dnSlv, scmSlv);
}
- if (FinalizationManager.shouldTellDatanodesToFinalize(
- scmContext.getFinalizationCheckpoint())) {
- // Because we have crossed the MLV_EQUALS_SLV checkpoint, SCM metadata
- // layout version will not change. We can now compare it to the
- // datanodes' metadata layout versions to tell them to finalize.
+ if
(FinalizationManager.shouldTellDatanodesToFinalize(scmLayoutVersionManager)) {
Review Comment:
Without the checkpoints I don't think we need the wrapper method anymore.
```suggestion
if (!scmLayoutVersionManager.needsFinalization()) {
```
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/upgrade/TestScmHAFinalization.java:
##########
@@ -127,118 +111,19 @@ public void shutdown() {
}
}
- /**
- * Argument supplier for parameterized tests.
- */
- public static Stream<Arguments> injectionPointsToTest() {
- // Do not test from BEFORE_PRE_FINALIZE_UPGRADE injection point.
- // Finalization will not have started so there will be no persisted state
- // to resume from.
- return Stream.of(
- Arguments.of(UpgradeTestInjectionPoints.AFTER_PRE_FINALIZE_UPGRADE),
- Arguments.of(UpgradeTestInjectionPoints.AFTER_COMPLETE_FINALIZATION),
- Arguments.of(UpgradeTestInjectionPoints.AFTER_POST_FINALIZE_UPGRADE)
- );
- }
-
- @ParameterizedTest
- @MethodSource(METHOD_SOURCE)
- public void testFinalizationWithLeaderChange(
- UpgradeTestInjectionPoints haltingPoint) throws Exception {
-
- CountDownLatch pauseLatch = new CountDownLatch(1);
- CountDownLatch unpauseLatch = new CountDownLatch(1);
- init(new OzoneConfiguration(),
- UpgradeTestUtils.newPausingFinalizationExecutor(haltingPoint,
- pauseLatch, unpauseLatch, LOG), 0);
- pauseLatch.await();
-
- // Stop the leader, forcing a leader change in the middle of finalization.
- // This will cause the initial client call for finalization
- // to be interrupted.
- StorageContainerManager oldLeaderScm = cluster.getActiveSCM();
- LOG.info("Stopping current SCM leader {} to initiate a leader change.",
- oldLeaderScm.getSCMNodeId());
- cluster.shutdownStorageContainerManager(oldLeaderScm);
-
- // Wait for the remaining two SCMs to elect a new leader.
- cluster.waitForClusterToBeReady();
-
- // While finalization is paused, check its state on the remaining SCMs.
- checkMidFinalizationConditions(haltingPoint,
- cluster.getStorageContainerManagersList());
-
- // Restart actually creates a new SCM.
- // Since this SCM will be a follower, the implementation of its upgrade
- // finalization executor does not matter for this test.
- cluster.restartStorageContainerManager(oldLeaderScm, true);
-
- // Make sure the original SCM leader is not the leader anymore.
- StorageContainerManager newLeaderScm = cluster.getActiveSCM();
- assertNotEquals(newLeaderScm.getSCMNodeId(),
- oldLeaderScm.getSCMNodeId());
-
- // Resume finalization from the new leader.
- unpauseLatch.countDown();
-
- // Client should complete exceptionally since the original SCM it
- // requested to was restarted.
- finalizationFuture.get();
- TestHddsUpgradeUtils.waitForFinalizationFromClient(scmClient, CLIENT_ID);
- // Make sure old leader has caught up and all SCMs have finalized.
- waitForScmsToFinalize(cluster.getStorageContainerManagersList());
-
- TestHddsUpgradeUtils.testPostUpgradeConditionsSCM(
- cluster.getStorageContainerManagersList(), 0, NUM_DATANODES);
- TestHddsUpgradeUtils.testPostUpgradeConditionsDataNodes(
- cluster.getHddsDatanodes(), 0, CLOSED);
- }
-
- @ParameterizedTest
- @MethodSource(METHOD_SOURCE)
- @Flaky("HDDS-8714")
- public void testFinalizationWithRestart(
- UpgradeTestInjectionPoints haltingPoint) throws Exception {
- CountDownLatch terminateLatch = new CountDownLatch(1);
- init(new OzoneConfiguration(),
- UpgradeTestUtils.newTerminatingFinalizationExecutor(haltingPoint,
- terminateLatch, LOG),
- 0);
- terminateLatch.await();
-
- // Once upgrade finalization is stopped at the halting point, restart all
- // SCMs.
- LOG.info("Restarting all SCMs during upgrade finalization.");
- // Restarting an SCM from mini ozone actually replaces the SCM with a new
- // instance. We will use the normal upgrade finalization executor for
- // these new instances, since the last one aborted at the halting point.
- cluster.getSCMConfigurator()
- .setUpgradeFinalizationExecutor(
- new DefaultUpgradeFinalizationExecutor<>());
- List<StorageContainerManager> originalSCMs =
- cluster.getStorageContainerManagers();
-
- for (StorageContainerManager scm: originalSCMs) {
- cluster.restartStorageContainerManager(scm, false);
- }
-
- checkMidFinalizationConditions(haltingPoint,
- cluster.getStorageContainerManagersList());
-
- // After all SCMs were restarted, finalization should resume
- // automatically once a leader is elected.
- cluster.waitForClusterToBeReady();
-
+ @Test
+ public void testFinalizationWithLeaderChange() throws Exception {
Review Comment:
Since we can't test leader change in the middle of finalization since it is
one ratis request now, we should update the test name to reflect this.
```suggestion
public void testFinalization() throws Exception {
```
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/SCMUpgradeFinalizer.java:
##########
@@ -111,18 +93,9 @@ void replicatedFinalizationSteps(HDDSLayoutFeature lf,
}
@Override
- public void postFinalizeUpgrade(SCMUpgradeFinalizationContext context)
- throws IOException {
- // If we reached this phase of finalization, all layout features should
- // be finalized.
- logCheckpointCrossed(FinalizationCheckpoint.MLV_EQUALS_SLV);
- FinalizationStateManager stateManager =
- context.getFinalizationStateManager();
- if (!stateManager.crossedCheckpoint(
- FinalizationCheckpoint.FINALIZATION_COMPLETE)) {
- waitForDatanodesToFinalize(context);
- stateManager.removeFinalizingMark();
- }
+ public void postFinalizeUpgrade(SCMUpgradeFinalizationContext context)
throws IOException {
+ waitForDatanodesToFinalize(context);
+ getVersionManager().setUpgradeState(FINALIZATION_DONE);
Review Comment:
```suggestion
super(context)
```
Then when the DN finalization is made async if we decide not to handle it in
this method, the override can be removed.
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/FinalizationStateManagerImpl.java:
##########
@@ -59,152 +57,51 @@ protected FinalizationStateManagerImpl(Builder builder)
throws IOException {
this.upgradeFinalizer = builder.upgradeFinalizer;
this.versionManager = this.upgradeFinalizer.getVersionManager();
this.checkpointLock = new ReentrantReadWriteLock();
- initialize();
- }
-
- private void initialize() throws IOException {
- this.hasFinalizingMark =
- finalizationStore.isExist(OzoneConsts.FINALIZING_KEY);
- }
-
- 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());
- // Set the checkpoint in the SCM context so other components can read it.
- upgradeContext.getSCMContext().setFinalizationCheckpoint(checkpoint);
}
@Override
public void setUpgradeContext(SCMUpgradeFinalizationContext context) {
this.upgradeContext = context;
- FinalizationCheckpoint checkpoint = getFinalizationCheckpoint();
- upgradeContext.getSCMContext().setFinalizationCheckpoint(checkpoint);
- // Set the version manager's upgrade status (sent back to the client to
- // identify upgrade progress) based on the current checkpoint.
- versionManager.setUpgradeState(checkpoint.getStatus());
- }
-
- @Override
- public void addFinalizingMark() throws IOException {
- checkpointLock.writeLock().lock();
- try {
- hasFinalizingMark = true;
- } finally {
- checkpointLock.writeLock().unlock();
- }
- transactionBuffer.addToBuffer(finalizationStore,
- OzoneConsts.FINALIZING_KEY, "");
- publishCheckpoint(FinalizationCheckpoint.FINALIZATION_STARTED);
}
@Override
public void finalizeLayoutFeatures(Integer toVersion) throws IOException {
- int startLayoutVersion = versionManager.getMetadataLayoutVersion() + 1;
- for (int version = startLayoutVersion; version <= toVersion; version++) {
- finalizeLayoutFeatureLocal(version);
+ for (LayoutFeature feature : versionManager.unfinalizedFeatures()) {
+ finalizeLayoutFeatureLocal((HDDSLayoutFeature) feature);
}
}
/**
* A version of finalizeLayoutFeature without the {@link Replicate}
* annotation that can be called by followers to finalize from a snapshot.
*/
- private void finalizeLayoutFeatureLocal(Integer layoutVersion)
+ private void finalizeLayoutFeatureLocal(HDDSLayoutFeature layoutFeature)
throws IOException {
checkpointLock.writeLock().lock();
try {
// The VERSION file is the source of truth for the current layout
// version. This is updated in the replicated finalization steps.
// Layout version will be written to the DB as well so followers can
// finalize from a snapshot.
- if (versionManager.getMetadataLayoutVersion() >= layoutVersion) {
+ if (versionManager.getMetadataLayoutVersion() >=
layoutFeature.layoutVersion()) {
LOG.warn("Attempting to finalize layout feature for layout version {},
but " +
"current metadata layout version is {}. Skipping finalization for
this layout version.",
- layoutVersion, versionManager.getMetadataLayoutVersion());
+ layoutFeature.layoutVersion(),
versionManager.getMetadataLayoutVersion());
} else {
- HDDSLayoutFeature feature =
- (HDDSLayoutFeature) versionManager.getFeature(layoutVersion);
- upgradeFinalizer.replicatedFinalizationSteps(feature, upgradeContext);
+ upgradeFinalizer.replicatedFinalizationSteps(layoutFeature,
upgradeContext);
}
} finally {
checkpointLock.writeLock().unlock();
}
- if (!versionManager.needsFinalization()) {
- publishCheckpoint(FinalizationCheckpoint.MLV_EQUALS_SLV);
+ if (!versionManager.needsFinalization() &&
!upgradeContext.getSCMContext().isLeader()) {
+ // Only the followers complete finalize here, the leader must wait until
the DNs
+ // have finalized before making finalization done, otherwise a polling
client could
+ // be told it is complete too early.
Review Comment:
The upgrade status is already set in `SCMUpgradeFinalizer#
postFinalizeUpgrade`, we should only set it in one place. The SCM should
consider itself finalized when its software and apparent versions match. The
response to the endpoint that OM will use to poll can be different based on
whether all datanodes have finalized or not. The leader may change at any point
while the datanodes are finalizing.
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/upgrade/FinalizationStateManagerImpl.java:
##########
@@ -47,9 +48,6 @@ public class FinalizationStateManagerImpl implements
FinalizationStateManager {
// Ensures that we are not in the process of updating checkpoint state as
// we read it to determine the current checkpoint.
private final ReadWriteLock checkpointLock;
Review Comment:
We removed the method using the read lock and the checkpoint, so it could at
least be renamed and downgraded to a regular lock. We may even just be able to
use `synchronized` on `finalizeLayoutFeatureLocal` and `reinitialize`. Another
option is to remove coordination there entirely since the methods should be
triggered by Ratis and should not be called in parallel.
--
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]