From 4bf07d6f6be7ff5a235f0d0ffda262d958ec93ab Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Thu, 26 Feb 2026 14:42:01 +0530 Subject: [PATCH 1/3] display retry error msg when scm's are unavailable --- .../proxy/SCMFailoverProxyProviderBase.java | 36 ++++++++++++++ .../hdds/scm/TestFailoverWithSCMHA.java | 48 +++++++++++++++++++ 2 files changed, 84 insertions(+) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMFailoverProxyProviderBase.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMFailoverProxyProviderBase.java index 272db7a04ae4..0ef03da6000f 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMFailoverProxyProviderBase.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMFailoverProxyProviderBase.java @@ -333,6 +333,9 @@ public RetryAction shouldRetry(Exception e, int retry, } } + // Print user-facing retry messages for all attempts + printRetryMessage(e, failover); + if (SCMHAUtils.checkRetriableWithNoFailoverException(e)) { setUpdatedLeaderNodeID(); } else { @@ -347,4 +350,37 @@ public RetryAction shouldRetry(Exception e, int retry, public synchronized void setUpdatedLeaderNodeID() { this.updatedLeaderNodeID = getCurrentProxySCMNodeId(); } + + /** + * Print user-facing retry message to stderr. + * Shows connection attempts and failover progress. + * + * @param exception the exception that triggered the retry + * @param failoverCount the number of failover attempts made so far + */ + private void printRetryMessage(Exception exception, int failoverCount) { + Throwable cause = exception.getCause(); + String exceptionType = cause != null ? + cause.getClass().getSimpleName() : exception.getClass().getSimpleName(); + + // Extract concise error message + String errorMsg; + if (cause != null && cause.getMessage() != null) { + String fullMsg = cause.getMessage(); + int colonIndex = fullMsg.indexOf(':'); + errorMsg = colonIndex > 0 && colonIndex < 100 ? + fullMsg.substring(0, colonIndex) : fullMsg; + } else { + errorMsg = exception.getMessage(); + } + + System.err.printf("%s: %s, while invoking %s over %s. " + + "Retrying after %d failover attempt(s). Sleeping for %dms.%n", + exceptionType, + errorMsg, + protocolClass.getSimpleName(), + getCurrentProxySCMNodeId(), + failoverCount, + getRetryInterval()); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java index a4318a2cc698..a211bdd3b8c7 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java @@ -25,7 +25,10 @@ import com.google.protobuf.ByteString; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.TimeoutException; +import java.util.stream.Stream; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient; @@ -45,11 +48,15 @@ import org.apache.hadoop.hdds.tracing.TracingUtil; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl; +import org.apache.hadoop.ozone.admin.OzoneAdmin; import org.apache.ozone.test.GenericTestUtils; import org.apache.ozone.test.GenericTestUtils.LogCapturer; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; /** @@ -214,6 +221,47 @@ public void testContainerBalancerPersistsConfigurationInAllSCMs() } } + public static Stream ozoneAdminScmCommands() { + return Stream.of( + Arguments.of((Object) new String[]{"datanode", "list"}), + Arguments.of((Object) new String[]{"pipeline", "list"}), + Arguments.of((Object) new String[]{"scm", "roles"}), + Arguments.of((Object) new String[]{"container", "list"}), + Arguments.of((Object) new String[]{"safemode", "status"}) + ); + } + + /** + * Verifies that when SCMs are unavailable, the CLI shows retry messages + * on stderr before eventually failing for all SCM-querying commands. + */ + @ParameterizedTest + @MethodSource("ozoneAdminScmCommands") + public void testRetryMessageShownWhenScmUnavailable(String[] args) throws Exception { + SCMClientConfig scmClientConfig = conf.getObject(SCMClientConfig.class); + scmClientConfig.setRetryCount(2); + scmClientConfig.setRetryInterval(50); + conf.setFromObject(scmClientConfig); + + Map configOverrides = new HashMap<>(); + cluster.getConf().forEach(entry -> + configOverrides.put(entry.getKey(), entry.getValue())); + + cluster.shutdown(); + cluster = null; + + OzoneAdmin ozoneAdmin = new OzoneAdmin(); + ozoneAdmin.setConfigurationOverrides(configOverrides); + + try (GenericTestUtils.PrintStreamCapturer err = GenericTestUtils.captureErr()) { + ozoneAdmin.execute(args); + String stderrOutput = err.get(); + + // Retry message format: "Retrying after N failover attempt(s). Sleeping for Xms." + assertThat(stderrOutput).contains("Retrying after", "failover attempt(s)", "Sleeping for 50ms"); + } + } + static StorageContainerManager getLeader(MiniOzoneHAClusterImpl impl) { for (StorageContainerManager scm : impl.getStorageContainerManagers()) { if (scm.checkLeader()) { From 6e26c66c745d8fb02fb312a6c943a25c251cdfe5 Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Mon, 2 Mar 2026 10:10:36 +0530 Subject: [PATCH 2/3] check for retry action before printing msg and update test case --- .../proxy/SCMFailoverProxyProviderBase.java | 27 +++++++++----- .../hdds/scm/TestFailoverWithSCMHA.java | 37 ++++++++----------- 2 files changed, 33 insertions(+), 31 deletions(-) diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMFailoverProxyProviderBase.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMFailoverProxyProviderBase.java index 0ef03da6000f..05e06e57e1b3 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMFailoverProxyProviderBase.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/proxy/SCMFailoverProxyProviderBase.java @@ -39,6 +39,7 @@ import org.apache.hadoop.io.retry.FailoverProxyProvider; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; +import org.apache.hadoop.io.retry.RetryPolicy.RetryAction.RetryDecision; import org.apache.hadoop.ipc_.ProtobufRpcEngine; import org.apache.hadoop.ipc_.RPC; import org.apache.hadoop.net.NetUtils; @@ -333,16 +334,20 @@ public RetryAction shouldRetry(Exception e, int retry, } } - // Print user-facing retry messages for all attempts - printRetryMessage(e, failover); + RetryPolicy.RetryAction retryAction = SCMHAUtils.getRetryAction( + failover, retry, e, maxRetryCount, getRetryInterval()); + + if (retryAction.action == RetryDecision.RETRY + || retryAction.action == RetryDecision.FAILOVER_AND_RETRY) { + printRetryMessage(e, failover, retryAction.delayMillis); + } if (SCMHAUtils.checkRetriableWithNoFailoverException(e)) { setUpdatedLeaderNodeID(); } else { performFailoverToAssignedLeader(null, e); } - return SCMHAUtils.getRetryAction(failover, retry, e, maxRetryCount, - getRetryInterval()); + return retryAction; } }; } @@ -354,14 +359,16 @@ public synchronized void setUpdatedLeaderNodeID() { /** * Print user-facing retry message to stderr. * Shows connection attempts and failover progress. + * Only called when a retry will actually occur. * * @param exception the exception that triggered the retry * @param failoverCount the number of failover attempts made so far + * @param delayMillis the delay before the next retry attempt */ - private void printRetryMessage(Exception exception, int failoverCount) { + private void printRetryMessage(Exception exception, int failoverCount, + long delayMillis) { Throwable cause = exception.getCause(); - String exceptionType = cause != null ? - cause.getClass().getSimpleName() : exception.getClass().getSimpleName(); + String exceptionType = (cause != null ? cause : exception).getClass().getSimpleName(); // Extract concise error message String errorMsg; @@ -375,12 +382,12 @@ private void printRetryMessage(Exception exception, int failoverCount) { } System.err.printf("%s: %s, while invoking %s over %s. " + - "Retrying after %d failover attempt(s). Sleeping for %dms.%n", + "Retrying in %dms after %d failover attempt(s).%n", exceptionType, errorMsg, protocolClass.getSimpleName(), getCurrentProxySCMNodeId(), - failoverCount, - getRetryInterval()); + delayMillis, + failoverCount); } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java index a211bdd3b8c7..59d9fe1047ab 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java @@ -28,7 +28,6 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeoutException; -import java.util.stream.Stream; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient; @@ -54,9 +53,6 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.event.Level; /** @@ -221,23 +217,20 @@ public void testContainerBalancerPersistsConfigurationInAllSCMs() } } - public static Stream ozoneAdminScmCommands() { - return Stream.of( - Arguments.of((Object) new String[]{"datanode", "list"}), - Arguments.of((Object) new String[]{"pipeline", "list"}), - Arguments.of((Object) new String[]{"scm", "roles"}), - Arguments.of((Object) new String[]{"container", "list"}), - Arguments.of((Object) new String[]{"safemode", "status"}) - ); - } + private static final String[][] OZONE_ADMIN_SCM_COMMANDS = { + {"datanode", "list"}, + {"pipeline", "list"}, + {"scm", "roles"}, + {"container", "list"}, + {"safemode", "status"} + }; /** * Verifies that when SCMs are unavailable, the CLI shows retry messages * on stderr before eventually failing for all SCM-querying commands. */ - @ParameterizedTest - @MethodSource("ozoneAdminScmCommands") - public void testRetryMessageShownWhenScmUnavailable(String[] args) throws Exception { + @Test + public void testRetryMessageShownWhenScmUnavailable() throws Exception { SCMClientConfig scmClientConfig = conf.getObject(SCMClientConfig.class); scmClientConfig.setRetryCount(2); scmClientConfig.setRetryInterval(50); @@ -253,12 +246,14 @@ public void testRetryMessageShownWhenScmUnavailable(String[] args) throws Except OzoneAdmin ozoneAdmin = new OzoneAdmin(); ozoneAdmin.setConfigurationOverrides(configOverrides); - try (GenericTestUtils.PrintStreamCapturer err = GenericTestUtils.captureErr()) { - ozoneAdmin.execute(args); - String stderrOutput = err.get(); + for (String[] args : OZONE_ADMIN_SCM_COMMANDS) { + try (GenericTestUtils.PrintStreamCapturer err = GenericTestUtils.captureErr()) { + ozoneAdmin.execute(args); + String stderrOutput = err.get(); - // Retry message format: "Retrying after N failover attempt(s). Sleeping for Xms." - assertThat(stderrOutput).contains("Retrying after", "failover attempt(s)", "Sleeping for 50ms"); + // Retry message format: "... Retrying in Xms after N failover attempt(s)." + assertThat(stderrOutput.toLowerCase()).contains("retrying in", "failover attempt(s)"); + } } } From 738a5a7ac855c789202da3e77f171884ec13ff7c Mon Sep 17 00:00:00 2001 From: Gargi Jaiswal Date: Mon, 2 Mar 2026 10:47:06 +0530 Subject: [PATCH 3/3] fix pmd error --- .../hadoop/hdds/scm/TestFailoverWithSCMHA.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java index 59d9fe1047ab..ac80c4bd689b 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestFailoverWithSCMHA.java @@ -69,6 +69,14 @@ public class TestFailoverWithSCMHA { private static final long SNAPSHOT_THRESHOLD = 5; + private static final String[][] OZONE_ADMIN_SCM_COMMANDS = { + {"datanode", "list"}, + {"pipeline", "list"}, + {"scm", "roles"}, + {"container", "list"}, + {"safemode", "status"} + }; + @BeforeEach public void init() throws Exception { conf = new OzoneConfiguration(); @@ -217,14 +225,6 @@ public void testContainerBalancerPersistsConfigurationInAllSCMs() } } - private static final String[][] OZONE_ADMIN_SCM_COMMANDS = { - {"datanode", "list"}, - {"pipeline", "list"}, - {"scm", "roles"}, - {"container", "list"}, - {"safemode", "status"} - }; - /** * Verifies that when SCMs are unavailable, the CLI shows retry messages * on stderr before eventually failing for all SCM-querying commands.