adoroszlai commented on code in PR #7471:
URL: https://github.com/apache/ozone/pull/7471#discussion_r1862972717
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManagerHA.java:
##########
@@ -323,4 +335,75 @@ private void waitForLeaderToBeReady()
}, 1000, (int) ScmConfigKeys
.OZONE_SCM_HA_RATIS_LEADER_READY_WAIT_TIMEOUT_DEFAULT);
}
+
+ @Test
+ public void testSCMLeadershipMetric() throws IOException,
InterruptedException {
+ // GIVEN
+ int scmInstancesCount = 3;
+ conf = new OzoneConfiguration();
+ MiniOzoneCluster.Builder haMiniClusterBuilder =
MiniOzoneCluster.newHABuilder(conf)
+ .setSCMServiceId("scm-service-id")
+ .setOMServiceId("om-service-id")
+ .setNumOfActiveOMs(0)
+ .setNumOfStorageContainerManagers(scmInstancesCount)
+ .setNumOfActiveSCMs(1)
+ .setNumDatanodes(0);
+
+ // start single SCM instance without other Ozone services
+ // in order to initialize and bootstrap SCM instances only
+ cluster = (MiniOzoneHAClusterImpl) haMiniClusterBuilder.build();
+
+ List<StorageContainerManager> storageContainerManagersList =
+ ((MiniOzoneHAClusterImpl) cluster).getStorageContainerManagersList();
+
+ // stop the single SCM instance in order to imitate further simultaneous
start of SCMs
+ storageContainerManagersList.get(0).stop();
+ storageContainerManagersList.get(0).join();
+
+ // WHEN (imitate simultaneous start of the SCMs)
+ CountDownLatch scmInstancesCounter = new CountDownLatch(scmInstancesCount);
+ int retryCount = 0;
+ while (true) {
+ AtomicInteger failedSCMs = new AtomicInteger();
+ for (StorageContainerManager scm : storageContainerManagersList) {
+ CountDownLatch finalScmInstancesCounter = scmInstancesCounter;
+ new Thread(() -> {
+ try {
+ scm.start();
+ } catch (IOException e) {
+ failedSCMs.incrementAndGet();
+ } finally {
+ finalScmInstancesCounter.countDown();
+ }
+ }).start();
+ }
+ scmInstancesCounter.await();
+ if (failedSCMs.get() == 0) {
+ break;
+ } else {
+ for (StorageContainerManager scm : storageContainerManagersList) {
+ scm.stop();
+ scm.join();
+ LOG.info("Stopping StorageContainerManager server at {}",
+ scm.getClientRpcAddress());
+ }
+ ++retryCount;
+ LOG.info("SCMs port conflicts, retried {} times",
+ retryCount);
+ failedSCMs.set(0);
+ scmInstancesCounter = new CountDownLatch(scmInstancesCount);
+ }
+ }
+
+ // THEN expect at least one SCM node (leader) will have
'scmha_metrics_scmha_leader_state' metric set to 1
Review Comment:
Is more than one leader OK? If not, we can assert that the sum of metrics
is 1.
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManagerHA.java:
##########
@@ -72,6 +77,8 @@
@Timeout(300)
public class TestStorageContainerManagerHA {
+ private static final Logger LOG =
LoggerFactory.getLogger(TestMiniOzoneCluster.class);
Review Comment:
```suggestion
private static final Logger LOG =
LoggerFactory.getLogger(TestStorageContainerManagerHA.class);
```
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManagerHA.java:
##########
@@ -323,4 +335,75 @@ private void waitForLeaderToBeReady()
}, 1000, (int) ScmConfigKeys
.OZONE_SCM_HA_RATIS_LEADER_READY_WAIT_TIMEOUT_DEFAULT);
}
+
+ @Test
+ public void testSCMLeadershipMetric() throws IOException,
InterruptedException {
+ // GIVEN
+ int scmInstancesCount = 3;
+ conf = new OzoneConfiguration();
+ MiniOzoneCluster.Builder haMiniClusterBuilder =
MiniOzoneCluster.newHABuilder(conf)
+ .setSCMServiceId("scm-service-id")
+ .setOMServiceId("om-service-id")
+ .setNumOfActiveOMs(0)
+ .setNumOfStorageContainerManagers(scmInstancesCount)
+ .setNumOfActiveSCMs(1)
+ .setNumDatanodes(0);
+
+ // start single SCM instance without other Ozone services
+ // in order to initialize and bootstrap SCM instances only
+ cluster = (MiniOzoneHAClusterImpl) haMiniClusterBuilder.build();
+
+ List<StorageContainerManager> storageContainerManagersList =
+ ((MiniOzoneHAClusterImpl) cluster).getStorageContainerManagersList();
Review Comment:
We can avoid the need to cast by moving `setNumDatanodes` off the chain.
```suggestion
MiniOzoneHAClusterImpl.Builder builder =
MiniOzoneCluster.newHABuilder(conf)
.setSCMServiceId("scm-service-id")
.setOMServiceId("om-service-id")
.setNumOfActiveOMs(0)
.setNumOfStorageContainerManagers(scmInstancesCount)
.setNumOfActiveSCMs(1);
builder.setNumDatanodes(0);
// start single SCM instance without other Ozone services
// in order to initialize and bootstrap SCM instances only
cluster = builder.build();
List<StorageContainerManager> storageContainerManagersList =
cluster.getStorageContainerManagersList();
```
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestStorageContainerManagerHA.java:
##########
@@ -323,4 +335,75 @@ private void waitForLeaderToBeReady()
}, 1000, (int) ScmConfigKeys
.OZONE_SCM_HA_RATIS_LEADER_READY_WAIT_TIMEOUT_DEFAULT);
}
+
+ @Test
+ public void testSCMLeadershipMetric() throws IOException,
InterruptedException {
+ // GIVEN
+ int scmInstancesCount = 3;
+ conf = new OzoneConfiguration();
+ MiniOzoneCluster.Builder haMiniClusterBuilder =
MiniOzoneCluster.newHABuilder(conf)
+ .setSCMServiceId("scm-service-id")
+ .setOMServiceId("om-service-id")
+ .setNumOfActiveOMs(0)
+ .setNumOfStorageContainerManagers(scmInstancesCount)
+ .setNumOfActiveSCMs(1)
+ .setNumDatanodes(0);
+
+ // start single SCM instance without other Ozone services
+ // in order to initialize and bootstrap SCM instances only
+ cluster = (MiniOzoneHAClusterImpl) haMiniClusterBuilder.build();
+
+ List<StorageContainerManager> storageContainerManagersList =
+ ((MiniOzoneHAClusterImpl) cluster).getStorageContainerManagersList();
+
+ // stop the single SCM instance in order to imitate further simultaneous
start of SCMs
+ storageContainerManagersList.get(0).stop();
+ storageContainerManagersList.get(0).join();
+
+ // WHEN (imitate simultaneous start of the SCMs)
+ CountDownLatch scmInstancesCounter = new CountDownLatch(scmInstancesCount);
+ int retryCount = 0;
+ while (true) {
+ AtomicInteger failedSCMs = new AtomicInteger();
Review Comment:
`scmInstancesCounter` can be defined inside the loop, just like
`failedSCMs`. Then we don't need `finalScmInstancesCounter` and reassignment
in `else`.
--
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]