|
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient
|
|
import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
|
|
import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.io.Text;
|
|
|
|
+import org.apache.hadoop.io.retry.RetryPolicy;
|
|
import org.apache.hadoop.ipc.Client;
|
|
import org.apache.hadoop.ipc.Client;
|
|
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
|
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
@@ -112,6 +113,7 @@ import org.apache.hadoop.util.JvmPauseMonitor;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
import org.apache.hadoop.util.ShutdownHookManager;
|
|
import org.apache.hadoop.util.ShutdownHookManager;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
|
+import org.apache.hadoop.utils.RetriableTask;
|
|
import org.apache.ratis.util.LifeCycle;
|
|
import org.apache.ratis.util.LifeCycle;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.Logger;
|
|
import org.slf4j.LoggerFactory;
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -140,6 +142,7 @@ import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForClients;
|
|
import static org.apache.hadoop.hdds.HddsUtils.isHddsEnabled;
|
|
import static org.apache.hadoop.hdds.HddsUtils.isHddsEnabled;
|
|
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
|
|
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
|
|
import static org.apache.hadoop.hdds.server.ServerUtils.updateRPCListenAddress;
|
|
import static org.apache.hadoop.hdds.server.ServerUtils.updateRPCListenAddress;
|
|
|
|
+import static org.apache.hadoop.io.retry.RetryPolicies.retryUpToMaximumCountWithFixedSleep;
|
|
import static org.apache.hadoop.ozone.OmUtils.getOmAddress;
|
|
import static org.apache.hadoop.ozone.OmUtils.getOmAddress;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
|
|
@@ -695,8 +698,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
StorageState state = omStorage.getState();
|
|
StorageState state = omStorage.getState();
|
|
if (state != StorageState.INITIALIZED) {
|
|
if (state != StorageState.INITIALIZED) {
|
|
try {
|
|
try {
|
|
- ScmBlockLocationProtocol scmBlockClient = getScmBlockClient(conf);
|
|
|
|
- ScmInfo scmInfo = scmBlockClient.getScmInfo();
|
|
|
|
|
|
+ ScmInfo scmInfo = getScmInfo(conf);
|
|
String clusterId = scmInfo.getClusterId();
|
|
String clusterId = scmInfo.getClusterId();
|
|
String scmId = scmInfo.getScmId();
|
|
String scmId = scmInfo.getScmId();
|
|
if (clusterId == null || clusterId.isEmpty()) {
|
|
if (clusterId == null || clusterId.isEmpty()) {
|
|
@@ -726,6 +728,22 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private static ScmInfo getScmInfo(OzoneConfiguration conf)
|
|
|
|
+ throws IOException {
|
|
|
|
+ try {
|
|
|
|
+ RetryPolicy retryPolicy = retryUpToMaximumCountWithFixedSleep(
|
|
|
|
+ 10, 5, TimeUnit.SECONDS);
|
|
|
|
+ RetriableTask<ScmInfo> retriable = new RetriableTask<>(
|
|
|
|
+ retryPolicy, "OM#getScmInfo",
|
|
|
|
+ () -> getScmBlockClient(conf).getScmInfo());
|
|
|
|
+ return retriable.call();
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ throw e;
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ throw new IOException("Failed to get SCM info", e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Parses the command line options for OM initialization.
|
|
* Parses the command line options for OM initialization.
|
|
*
|
|
*
|