Skip to content
Merged
Show file tree
Hide file tree
Changes from 6 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();
Comment thread
errose28 marked this conversation as resolved.
Outdated
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,30 @@ 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 UpgradeFinalization.Status.FINALIZATION_REQUIRED.equals(scmUpgradeStatus)
Comment thread
errose28 marked this conversation as resolved.
Outdated
&& 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(
Comment thread
errose28 marked this conversation as resolved.
Outdated
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()) {
Comment thread
errose28 marked this conversation as resolved.
Outdated
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 thread
errose28 marked this conversation as resolved.
Outdated

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 thread
errose28 marked this conversation as resolved.
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,78 @@ public void testScmListContainer() throws Exception {
HddsProtos.ReplicationFactor.THREE).getContainerInfoList().size());
}

@Test
Comment thread
errose28 marked this conversation as resolved.
Outdated
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());
assertTrue(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());
assertFalse(doneStatus.getShouldFinalize());
assertTrue(finalizedStatus.getScmFinalized());
assertFalse(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());
assertFalse(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