Skip to content
Merged
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 @@ -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;
Expand Down Expand Up @@ -333,18 +334,60 @@ public RetryAction shouldRetry(Exception e, int retry,
}
}

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;
}
};
}

public synchronized void setUpdatedLeaderNodeID() {
this.updatedLeaderNodeID = getCurrentProxySCMNodeId();
}

/**
* 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,
long delayMillis) {
Throwable cause = exception.getCause();
String exceptionType = (cause != null ? cause : 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 in %dms after %d failover attempt(s).%n",
exceptionType,
errorMsg,
protocolClass.getSimpleName(),
getCurrentProxySCMNodeId(),
delayMillis,
failoverCount);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,8 @@

import com.google.protobuf.ByteString;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
Expand All @@ -45,6 +47,7 @@
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;
Expand All @@ -66,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();
Expand Down Expand Up @@ -214,6 +225,38 @@ public void testContainerBalancerPersistsConfigurationInAllSCMs()
}
}

/**
* Verifies that when SCMs are unavailable, the CLI shows retry messages
* on stderr before eventually failing for all SCM-querying commands.
*/
@Test
public void testRetryMessageShownWhenScmUnavailable() throws Exception {
SCMClientConfig scmClientConfig = conf.getObject(SCMClientConfig.class);
scmClientConfig.setRetryCount(2);
scmClientConfig.setRetryInterval(50);
conf.setFromObject(scmClientConfig);

Map<String, String> configOverrides = new HashMap<>();
cluster.getConf().forEach(entry ->
configOverrides.put(entry.getKey(), entry.getValue()));

cluster.shutdown();
cluster = null;

OzoneAdmin ozoneAdmin = new OzoneAdmin();
ozoneAdmin.setConfigurationOverrides(configOverrides);

for (String[] args : OZONE_ADMIN_SCM_COMMANDS) {
try (GenericTestUtils.PrintStreamCapturer err = GenericTestUtils.captureErr()) {
ozoneAdmin.execute(args);
String stderrOutput = err.get();

// Retry message format: "... Retrying in Xms after N failover attempt(s)."
assertThat(stderrOutput.toLowerCase()).contains("retrying in", "failover attempt(s)");
}
}
}

static StorageContainerManager getLeader(MiniOzoneHAClusterImpl impl) {
for (StorageContainerManager scm : impl.getStorageContainerManagers()) {
if (scm.checkLeader()) {
Expand Down