Skip to content

Commit

Permalink
HDDS-11742. don't reset SCM leadership metric to null if the latter a…
Browse files Browse the repository at this point in the history
…lready defined at raft server init time
  • Loading branch information
Slava Tutrinov committed Nov 28, 2024
1 parent 9b26156 commit d4edc6f
Show file tree
Hide file tree
Showing 3 changed files with 52 additions and 16 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1595,8 +1595,22 @@ public void start() throws IOException {

setStartTime();

// At this point leader is not known
scmHAMetricsUpdate(null);
AtomicBoolean leaderIsDefined = new AtomicBoolean(false);
if (SCMHAUtils.isSCMHAEnabled(configuration)) {
getScmHAManager().getRatisServer().getRatisRoles().stream()
.filter(peerState -> peerState.contains("LEADER"))
.findFirst()
.ifPresent(peerState -> {
leaderIsDefined.set(true);
String[] peerInfo = peerState.split(":");
scmHAMetricsUpdate(peerInfo[3]);
});
}

if (!leaderIsDefined.get()) {
// At this point leader is not known
scmHAMetricsUpdate(null);
}

if (scmCertificateClient != null) {
// In case root CA certificate is rotated during this SCM is offline
Expand Down Expand Up @@ -2298,7 +2312,6 @@ public void scmHAMetricsUpdate(String leaderId) {
// unregister, in case metrics already exist
// so that the metric tags will get updated.
SCMHAMetrics.unRegister();

scmHAMetrics = SCMHAMetrics.create(getScmId(), leaderId);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster {
private CertificateClient caClient;
private final Set<AutoCloseable> clients = ConcurrentHashMap.newKeySet();
private SecretKeyClient secretKeyClient;
private static MockedStatic mockDNStatic = Mockito.mockStatic(HddsDatanodeService.class);
private static volatile MockedStatic mockDNStatic;

/**
* Creates a new MiniOzoneCluster with Recon.
Expand Down Expand Up @@ -571,6 +571,16 @@ private void setSecretKeyClient(SecretKeyClient client) {
this.secretKeyClient = client;
}

public static void mockDatanode() {
if (mockDNStatic == null) {
synchronized (MiniOzoneClusterImpl.class) {
if (mockDNStatic == null) {
mockDNStatic = Mockito.mockStatic(HddsDatanodeService.class);
}
}
}
}

private static void stopDatanodes(
Collection<HddsDatanodeService> hddsDatanodes) {
if (!hddsDatanodes.isEmpty()) {
Expand Down Expand Up @@ -867,6 +877,7 @@ protected Gateway createS3G() {
*/
protected List<HddsDatanodeService> createHddsDatanodes()
throws IOException {
mockDatanode();
List<HddsDatanodeService> hddsDatanodes = new ArrayList<>();

// Override default datanode initial and current version if necessary
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.Function;
Expand Down Expand Up @@ -540,7 +541,7 @@ protected OMHAService createOMService() throws IOException,
}

/**
* Start OM service with multiple OMs.
* Start SCM service with multiple SCMs.
*/
protected SCMHAService createSCMService()
throws IOException, AuthenticationException {
Expand All @@ -553,6 +554,7 @@ protected SCMHAService createSCMService()

int retryCount = 0;

CountDownLatch scmStartCounter = new CountDownLatch(numOfSCMs);
while (true) {
try {
initSCMHAConfig();
Expand Down Expand Up @@ -584,19 +586,29 @@ protected SCMHAService createSCMService()
}
scmList.add(scm);

if (i <= numOfActiveSCMs) {
scm.start();
activeSCMs.add(scm);
LOG.info("Started SCM RPC server at {}",
scm.getClientRpcAddress());
} else {
inactiveSCMs.add(scm);
LOG.info("Intialized SCM at {}. This SCM is currently "
+ "inactive (not running).", scm.getClientRpcAddress());
}
int scmCounter = i;
new Thread(() -> {
if (scmCounter <= numOfActiveSCMs) {
try {
scm.start();
} catch (IOException e) {
scmStartCounter.countDown();
throw new RuntimeException(e);
}
activeSCMs.add(scm);
LOG.info("Started SCM RPC server at {}",
scm.getClientRpcAddress());
} else {
inactiveSCMs.add(scm);
LOG.info("Initialized SCM at {}. This SCM is currently "
+ "inactive (not running).", scm.getClientRpcAddress());
}
scmStartCounter.countDown();
}).start();
}
scmStartCounter.await();
break;
} catch (BindException e) {
} catch (RuntimeException | InterruptedException e) {
for (StorageContainerManager scm : scmList) {
scm.stop();
scm.join();
Expand Down

0 comments on commit d4edc6f

Please sign in to comment.