Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -101,6 +101,7 @@
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
import org.apache.hadoop.hdds.scm.server.upgrade.SCMUpgradeFinalizer;
import org.apache.hadoop.hdds.security.SecurityConfig;
import org.apache.hadoop.hdds.utils.HddsServerUtil;
import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics;
Expand All @@ -116,6 +117,7 @@
import org.apache.hadoop.ozone.audit.AuditMessage;
import org.apache.hadoop.ozone.audit.Auditor;
import org.apache.hadoop.ozone.audit.SCMAction;
import org.apache.hadoop.ozone.upgrade.UpgradeFinalization;
import org.apache.hadoop.ozone.upgrade.UpgradeFinalization.StatusAndMessages;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
Expand Down Expand Up @@ -1153,13 +1155,16 @@ public HddsProtos.UpgradeStatus queryUpgradeStatus() throws IOException {
try {
getScm().checkAdminAccess(getRemoteUser(), true);

// Returning a placeholder for now.
HddsProtos.UpgradeStatus result = HddsProtos.UpgradeStatus.newBuilder()
.setScmFinalized(true)
.setNumDatanodesFinalized(10)
.setNumDatanodesTotal(10)
.setShouldFinalize(true)
.build();
UpgradeFinalization.Status scmUpgradeStatus = scm.getLayoutVersionManager().getUpgradeState();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This upgrade status enum is going to be removed in the new version manager. We will only use the proto facing version for compatibility with the older APIs. This call can be changed to scm.getLayoutVersionManager().needsFinalization() which easily maps to the same call in the new version manager when we switch. From here we can:

  • Remove the isScmFinalized helper method
  • Add a DatanodeFinalizationCounts#allNodesFinalized method as suggested here
  • Inline the shouldFinalize method to just a variable: shouldFinalize = scmFinalized && datanodeFinalizationCounts.allNodesFinalized()
  • Inline buildUpgradeStatus to invoke the builder directly in this method since the labelled builder setters are clearer than the unlabeled method parameters and it is now just a passthrough.

This minimizes the amount of upgrade specific logic in the server side translator.

SCMUpgradeFinalizer.DatanodeFinalizationCounts datanodeFinalizationCounts =
SCMUpgradeFinalizer.getNumFinalizedDatanodes(scm.getScmNodeManager());
int finalizedDatanodes = datanodeFinalizationCounts.getNumFinalizedDatanodes();
int healthyDatanodes = datanodeFinalizationCounts.getTotalHealthyDatanodes();

HddsProtos.UpgradeStatus result = buildUpgradeStatus(
scmUpgradeStatus,
finalizedDatanodes,
healthyDatanodes);
AUDIT.logReadSuccess(buildAuditMessageForSuccess(SCMAction.QUERY_UPGRADE_STATUS, null));
return result;
} catch (IOException ex) {
Expand All @@ -1168,6 +1173,29 @@ public HddsProtos.UpgradeStatus queryUpgradeStatus() throws IOException {
}
}

static HddsProtos.UpgradeStatus buildUpgradeStatus(
UpgradeFinalization.Status scmUpgradeStatus,
int finalizedDatanodes,
int healthyDatanodes) {
return HddsProtos.UpgradeStatus.newBuilder()
.setScmFinalized(isScmFinalized(scmUpgradeStatus))
.setNumDatanodesFinalized(finalizedDatanodes)
.setNumDatanodesTotal(healthyDatanodes)
.setShouldFinalize(
shouldFinalize(scmUpgradeStatus, finalizedDatanodes, healthyDatanodes))
.build();
}

static boolean isScmFinalized(UpgradeFinalization.Status scmUpgradeStatus) {
return UpgradeFinalization.isFinalized(scmUpgradeStatus)
|| UpgradeFinalization.isDone(scmUpgradeStatus);
}

static boolean shouldFinalize(UpgradeFinalization.Status scmUpgradeStatus,
int finalizedDatanodes, int healthyDatanodes) {
return isScmFinalized(scmUpgradeStatus) && finalizedDatanodes == healthyDatanodes;
}

@Override
public StartContainerBalancerResponseProto startContainerBalancer(
Optional<Double> threshold, Optional<Integer> iterations,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -121,44 +121,13 @@ private void waitForDatanodesToFinalize(SCMUpgradeFinalizationContext context)
// SCM is no longer the leader by throwing NotLeaderException.
context.getSCMContext().getTermOfLeader();

allDatanodesFinalized = true;
int totalHealthyNodes = 0;
int finalizedNodes = 0;
int unfinalizedNodes = 0;
DatanodeFinalizationCounts datanodeFinalizationCounts =
getNumFinalizedDatanodes(nodeManager);
int finalizedNodes = datanodeFinalizationCounts.getNumFinalizedDatanodes();
int totalHealthyNodes = datanodeFinalizationCounts.getTotalHealthyDatanodes();
int unfinalizedNodes = datanodeFinalizationCounts.getNumUnfinalizedDatanodes();

for (DatanodeDetails dn : nodeManager.getAllNodes()) {
try {
// Only check HEALTHY nodes. STALE/DEAD nodes will be told to
// finalize when they recover.
if (nodeManager.getNodeStatus(dn).isHealthy()) {
totalHealthyNodes++;
DatanodeInfo datanodeInfo = nodeManager.getDatanodeInfo(dn);
if (datanodeInfo == null) {
LOG.warn("Could not get DatanodeInfo for {}, skipping in " +
"finalization wait.", dn.getHostName());
continue;
}

LayoutVersionProto dnLayout = datanodeInfo.getLastKnownLayoutVersion();
int dnMlv = dnLayout.getMetadataLayoutVersion();
int dnSlv = dnLayout.getSoftwareLayoutVersion();

if (dnMlv < dnSlv) {
// Datanode has not yet finalized
allDatanodesFinalized = false;
unfinalizedNodes++;
LOG.debug("Datanode {} has not yet finalized: MLV={}, SLV={}",
dn.getHostName(), dnMlv, dnSlv);
} else {
finalizedNodes++;
}
}
} catch (NodeNotFoundException e) {
// Node was removed while we were iterating. This is OK, skip it.
LOG.debug("Node {} not found while waiting for finalization, " +
"skipping.", dn);
}
}
allDatanodesFinalized = unfinalizedNodes == 0;

if (!allDatanodesFinalized) {
LOG.info("Waiting for datanodes to finalize. Status: {}/{} healthy " +
Expand All @@ -177,4 +146,76 @@ private void waitForDatanodesToFinalize(SCMUpgradeFinalizationContext context)
}
}
}

public static DatanodeFinalizationCounts getNumFinalizedDatanodes(
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should be an instance method in NodeManager, not a static method that takes an instance as its first/only argument. DatanodeFinalizationCounts can be moved there as well, or moved to a standalone class.

NodeManager nodeManager) {
int finalizedNodes = 0;
int totalHealthyNodes = 0;
int unfinalizedNodes = 0;

for (DatanodeDetails dn : nodeManager.getAllNodes()) {
try {
// Only check HEALTHY nodes. STALE/DEAD nodes will be told to
// finalize when they recover.
if (!nodeManager.getNodeStatus(dn).isHealthy()) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This doesn't address the previous comment because it is only checking the health state returned by getNodeStatus, not the operational state. We can probably just loop over nodeManager.getNodes(NodeStatus.inServiceHealthy()) then not need to skip any entries in the loop.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I did wonder if we should let any decommission(ed / ing) and maintenance modes finalize if they are healthy (meaning they are still heartbeating). If they are heartbeating they should either finalize or stop heartbeating and go dead. They can be forced to finalize if they ever transition back to in_service but I don't think that needs a re-register so it would be another code path to worry about later.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I do think we should heartbeat to all nodes we can reach that they should finalize. We don't need to count all those nodes towards the HDDS finalization exit criteria, but maybe we should. It is simpler to reason that all live nodes have finalized, and we don't need to worry about decom/maintenance at all. I was considering a scenario where a decom node has just been shut down and might block finalization for the full heartbeat timeout, but that is only 10 minutes.

Along the same lines, I'm wondering if we should also count stale nodes towards the total count and wait for them to either finalize or go dead. If we exclude them from the count, we cannot say that all registered nodes have finalized because stale nodes will not have to re-register when they heartbeat again. If we include them, there could be one or a few slow nodes that periodically go stale and are also not processing the commands which would hold up OM.

Mostly I think we just need to make a decision on what states we count and why.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I was considering a scenario where a decom node has just been shut down and might block finalization for the full heartbeat timeout, but that is only 10 minutes.

Yea, I think we can live with that. People should not be upgrading clusters with nodes in strange states IMO, as it just adds to potential problems.

On stale nodes, there are:

stale : unfinalized
stale : finalized

The second we can just count as a finalized node and not worry about it. The second is a problem.

It can either go to dead and we can ignore it, or it can go back to healthy and may or may not finalize and go stale again. I guess if it is able to go stale -> healthy, it should must have heartbeat and hence picked up its finalize command, but it may not process it if the DN is in bad shape, and it may not heartbeat again.

I think our upgrade write path design could handle a node that slips through un-finalized? Or do we put some extra logic in to fence it out until it finalizes? For a stale node, all its Ratis pipelines get closed on transition to stale anyway, so it is effectively out of the write path at that point.

continue;
}
totalHealthyNodes++;
DatanodeInfo datanodeInfo = nodeManager.getDatanodeInfo(dn);
if (datanodeInfo == null) {
LOG.warn("Could not get DatanodeInfo for {}, skipping in " +
"finalization wait.", dn.getHostName());
continue;
}

LayoutVersionProto dnLayout = datanodeInfo.getLastKnownLayoutVersion();
int dnMlv = dnLayout.getMetadataLayoutVersion();
int dnSlv = dnLayout.getSoftwareLayoutVersion();

if (dnMlv < dnSlv) {
// Datanode has not yet finalized
unfinalizedNodes++;
LOG.debug("Datanode {} has not yet finalized: MLV={}, SLV={}",
dn.getHostName(), dnMlv, dnSlv);
} else {
finalizedNodes++;
}
} catch (NodeNotFoundException e) {
// Node was removed while we were iterating. This is OK, skip it.
LOG.debug("Node {} not found while waiting for finalization, " +
"skipping.", dn);
}
}

return new DatanodeFinalizationCounts(finalizedNodes, totalHealthyNodes, unfinalizedNodes);
}

/**
* Class to store the number finalized, unfinalized and healthy datanodes.
*/
public static final class DatanodeFinalizationCounts {
private final int numFinalizedDatanodes;
private final int totalHealthyDatanodes;
private final int numUnfinalizedDatanodes;
Comment on lines +197 to +199
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we actually need three fields here? I think we can just track total healthy in-service nodes and number of finalized nodes. The number of unfinalized nodes can be derived from that. These are the only two fields the upgrade status API uses.

The finalization wait code can be changed to check that the healthy node count equals the finalized count. We could also wrap that in an allNodesFinalized method inside DatanodeFinalizationCount.

getNumUnfinalizedDatanodes has no test coverage, but I think it should be removed. getTotalHealthyDatanodes also has no test coverage, which should be added.


public DatanodeFinalizationCounts(int numFinalizedDatanodes,
int totalHealthyDatanodes,
int numUnfinalizedDatanodes) {
this.numFinalizedDatanodes = numFinalizedDatanodes;
this.totalHealthyDatanodes = totalHealthyDatanodes;
this.numUnfinalizedDatanodes = numUnfinalizedDatanodes;
}

public int getNumFinalizedDatanodes() {
return numFinalizedDatanodes;
}

public int getTotalHealthyDatanodes() {
return totalHealthyDatanodes;
}

public int getNumUnfinalizedDatanodes() {
return numUnfinalizedDatanodes;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,7 @@
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.NodeReportFromDatanode;
import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.hdds.scm.server.upgrade.SCMUpgradeFinalizer;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.hdds.upgrade.HDDSLayoutVersionManager;
Expand Down Expand Up @@ -737,6 +738,71 @@ public void testProcessLayoutVersion() throws IOException {
testProcessLayoutVersionReportHigherMlv();
}

@Test
Comment thread
dombizita marked this conversation as resolved.
public void testDatanodeFinalizedCounterTracksLayoutVersionReports()
throws IOException, AuthenticationException {
try (SCMNodeManager nodeManager = createNodeManager(getConf())) {
DatanodeDetails node =
HddsTestUtils.createRandomDatanodeAndRegister(nodeManager);
assertEquals(1, SCMUpgradeFinalizer.getNumFinalizedDatanodes(nodeManager)
.getNumFinalizedDatanodes(),
"Initial datanode should be counted as finalized");

int softwareVersion =
nodeManager.getLayoutVersionManager().getSoftwareLayoutVersion();
int metadataVersion =
nodeManager.getLayoutVersionManager().getMetadataLayoutVersion();
nodeManager.processLayoutVersionReport(node,
LayoutVersionProto.newBuilder()
.setMetadataLayoutVersion(metadataVersion - 1)
.setSoftwareLayoutVersion(softwareVersion)
.build());
assertEquals(0, SCMUpgradeFinalizer.getNumFinalizedDatanodes(nodeManager)
.getNumFinalizedDatanodes(),
"Lower metadata layout version should decrement finalized count");

nodeManager.processLayoutVersionReport(node,
LayoutVersionProto.newBuilder()
.setMetadataLayoutVersion(metadataVersion)
.setSoftwareLayoutVersion(softwareVersion)
.build());
assertEquals(1, SCMUpgradeFinalizer.getNumFinalizedDatanodes(nodeManager)
.getNumFinalizedDatanodes(),
"Restored metadata layout version should restore finalized count");
}
}

@Test
public void testDatanodeFinalizedCounterTracksRegistrationAndRemoveNode()
throws IOException, AuthenticationException, NodeNotFoundException {
try (SCMNodeManager nodeManager = createNodeManager(getConf())) {
DatanodeDetails finalizedNode =
registerWithCapacity(nodeManager, CORRECT_LAYOUT_PROTO, success);
assertEquals(1, SCMUpgradeFinalizer.getNumFinalizedDatanodes(nodeManager)
.getNumFinalizedDatanodes(),
"Finalized registration should increment finalized count");

DatanodeDetails nonFinalizedNode =
registerWithCapacity(nodeManager, SMALLER_MLV_LAYOUT_PROTO, success);
assertEquals(1, SCMUpgradeFinalizer.getNumFinalizedDatanodes(nodeManager)
.getNumFinalizedDatanodes(),
"Non-finalized registration should not increment finalized count");

nonFinalizedNode.setPersistedOpState(
HddsProtos.NodeOperationalState.DECOMMISSIONED);
Comment on lines +791 to +792
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should have test coverage for all ineligible health and op states to make the intent clear. It should be fast to enumerate all of them.

nodeManager.removeNode(nonFinalizedNode);
assertEquals(1, SCMUpgradeFinalizer.getNumFinalizedDatanodes(nodeManager)
.getNumFinalizedDatanodes(),
"Removing a non-finalized node should not change finalized count");

finalizedNode.setPersistedOpState(
HddsProtos.NodeOperationalState.DECOMMISSIONED);
nodeManager.removeNode(finalizedNode);
assertEquals(0, SCMUpgradeFinalizer.getNumFinalizedDatanodes(nodeManager).getNumFinalizedDatanodes(),
"Removing a finalized node should decrement finalized count");
}
}

// Currently invoked by testProcessLayoutVersion.
public void testProcessLayoutVersionReportHigherMlv()
throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_READONLY_ADMINISTRATORS;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.mock;
Expand All @@ -45,6 +46,7 @@
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB;
import org.apache.hadoop.hdds.utils.ProtocolMessageMetrics;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.upgrade.UpgradeFinalization;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.junit.jupiter.api.AfterEach;
Expand Down Expand Up @@ -138,6 +140,80 @@ public void testScmListContainer() throws Exception {
HddsProtos.ReplicationFactor.THREE).getContainerInfoList().size());
}

@Test
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

With the simplifications mentioned above I don't think we need all these tests, it looks like the AI went overboard here. These aren't actually testing the queryUpgradeStatus method either. Looking at the other methods in this class it looks like we usually only cover them with integration tests. So in this PR I would switch all test usage of queryUpgradeFinalizationProgress to queryUpgradeStatus for test coverage.

public void testBuildUpgradeStatusMapsFinalizationRequired() {
HddsProtos.UpgradeStatus status = SCMClientProtocolServer.buildUpgradeStatus(
UpgradeFinalization.Status.FINALIZATION_REQUIRED, 1, 2);

assertFalse(status.getScmFinalized());
assertFalse(status.getShouldFinalize());
assertEquals(1, status.getNumDatanodesFinalized());
assertEquals(2, status.getNumDatanodesTotal());
}

@Test
public void testBuildUpgradeStatusMapsFinalizationRequiredAllNodesFinalized() {
HddsProtos.UpgradeStatus status = SCMClientProtocolServer.buildUpgradeStatus(
UpgradeFinalization.Status.FINALIZATION_REQUIRED, 3, 3);

assertFalse(status.getScmFinalized());
assertEquals(SCMClientProtocolServer.shouldFinalize(
UpgradeFinalization.Status.FINALIZATION_REQUIRED, 3, 3),
status.getShouldFinalize());
assertEquals(3, status.getNumDatanodesFinalized());
assertEquals(3, status.getNumDatanodesTotal());
}

@Test
public void testBuildUpgradeStatusMapsFinalizationInProgress() {
HddsProtos.UpgradeStatus status = SCMClientProtocolServer.buildUpgradeStatus(
UpgradeFinalization.Status.FINALIZATION_IN_PROGRESS, 1, 2);

assertFalse(status.getScmFinalized());
assertFalse(status.getShouldFinalize());
}

@Test
public void testBuildUpgradeStatusMapsStartingFinalization() {
HddsProtos.UpgradeStatus status = SCMClientProtocolServer.buildUpgradeStatus(
UpgradeFinalization.Status.STARTING_FINALIZATION, 1, 2);

assertFalse(status.getScmFinalized());
assertFalse(status.getShouldFinalize());
assertEquals(1, status.getNumDatanodesFinalized());
assertEquals(2, status.getNumDatanodesTotal());
}

@Test
public void testBuildUpgradeStatusMapsCompletedStatesToFinalized() {
HddsProtos.UpgradeStatus doneStatus =
SCMClientProtocolServer.buildUpgradeStatus(
UpgradeFinalization.Status.FINALIZATION_DONE, 2, 2);
HddsProtos.UpgradeStatus finalizedStatus =
SCMClientProtocolServer.buildUpgradeStatus(
UpgradeFinalization.Status.ALREADY_FINALIZED, 2, 2);

assertTrue(doneStatus.getScmFinalized());
assertTrue(doneStatus.getShouldFinalize());
assertTrue(finalizedStatus.getScmFinalized());
assertTrue(finalizedStatus.getShouldFinalize());
}

@Test
public void testBuildUpgradeStatusFromVersionManagerState() {
HddsProtos.UpgradeStatus needsFinalization =
SCMClientProtocolServer.buildUpgradeStatus(UpgradeFinalization.Status.FINALIZATION_REQUIRED, 1, 3);
assertFalse(needsFinalization.getScmFinalized());
assertFalse(needsFinalization.getShouldFinalize());
assertEquals(1, needsFinalization.getNumDatanodesFinalized());
assertEquals(3, needsFinalization.getNumDatanodesTotal());

HddsProtos.UpgradeStatus alreadyFinalized =
SCMClientProtocolServer.buildUpgradeStatus(UpgradeFinalization.Status.ALREADY_FINALIZED, 3, 3);
assertTrue(alreadyFinalized.getScmFinalized());
assertTrue(alreadyFinalized.getShouldFinalize());
}

private StorageContainerManager mockStorageContainerManager() {
List<ContainerInfo> infos = new ArrayList<>();
for (int i = 0; i < 10; i++) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,8 +39,7 @@ public class StatusSubCommand extends ScmSubcommand {
public void execute(ScmClient client) throws IOException {
HddsProtos.UpgradeStatus status = client.queryUpgradeStatus();

// Temporary output to validate the command is working.
out().println("Update status:");
out().println("Upgrade status:");
out().println(" SCM Finalized: " + status.getScmFinalized());
out().println(" Datanodes finalized: " + status.getNumDatanodesFinalized());
out().println(" Total Datanodes: " + status.getNumDatanodesTotal());
Expand Down
Loading
Loading