|
@@ -25,8 +25,10 @@ import com.google.common.base.Preconditions;
|
|
|
import com.google.protobuf.BlockingService;
|
|
|
|
|
|
import java.security.KeyPair;
|
|
|
+import java.util.Collection;
|
|
|
import java.util.Objects;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
|
|
import org.apache.hadoop.hdds.HddsUtils;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
@@ -50,6 +52,7 @@ import org.apache.hadoop.ipc.Client;
|
|
|
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
|
|
import org.apache.hadoop.ipc.RPC;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
|
+import org.apache.hadoop.ozone.OzoneIllegalArgumentException;
|
|
|
import org.apache.hadoop.ozone.OzoneSecurityUtil;
|
|
|
import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
|
|
|
import org.apache.hadoop.ozone.security.OzoneSecurityException;
|
|
@@ -144,7 +147,6 @@ 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.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.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_DEFAULT;
|
|
@@ -161,6 +163,12 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys
|
|
|
.OZONE_OM_METRICS_SAVE_INTERVAL;
|
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys
|
|
|
.OZONE_OM_METRICS_SAVE_INTERVAL_DEFAULT;
|
|
|
+
|
|
|
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_NODE_ID_KEY;
|
|
|
+import static org.apache.hadoop.ozone.om.OMConfigKeys
|
|
|
+ .OZONE_OM_RATIS_PORT_DEFAULT;
|
|
|
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_SERVICE_IDS_KEY;
|
|
|
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_RATIS_PORT_KEY;
|
|
|
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_AUTH_METHOD;
|
|
|
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TOKEN_ERROR_OTHER;
|
|
|
import static org.apache.hadoop.ozone.protocol.proto
|
|
@@ -200,6 +208,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
private RPC.Server omRpcServer;
|
|
|
private InetSocketAddress omRpcAddress;
|
|
|
private String omId;
|
|
|
+ private OMNodeDetails omNodeDetails;
|
|
|
+ private List<OMNodeDetails> peerNodes;
|
|
|
+ private boolean isRatisEnabled;
|
|
|
private OzoneManagerRatisServer omRatisServer;
|
|
|
private OzoneManagerRatisClient omRatisClient;
|
|
|
private final OMMetadataManager metadataManager;
|
|
@@ -229,7 +240,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
private final S3SecretManager s3SecretManager;
|
|
|
private volatile boolean isOmRpcServerRunning = false;
|
|
|
|
|
|
- private OzoneManager(OzoneConfiguration conf) throws IOException {
|
|
|
+ private OzoneManager(OzoneConfiguration conf) throws IOException,
|
|
|
+ AuthenticationException {
|
|
|
super(OzoneVersionInfo.OZONE_VERSION_INFO);
|
|
|
Preconditions.checkNotNull(conf);
|
|
|
configuration = conf;
|
|
@@ -240,6 +252,14 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
ResultCodes.OM_NOT_INITIALIZED);
|
|
|
}
|
|
|
|
|
|
+ // Load HA related configurations
|
|
|
+ loadOMHAConfigs(configuration);
|
|
|
+
|
|
|
+ // Authenticate KSM if security is enabled
|
|
|
+ if (securityEnabled) {
|
|
|
+ loginOMUser(configuration);
|
|
|
+ }
|
|
|
+
|
|
|
if (!testSecureOmFlag || !isOzoneSecurityEnabled()) {
|
|
|
scmContainerClient = getScmContainerClient(configuration);
|
|
|
// verifies that the SCM info in the OM Version file is correct.
|
|
@@ -256,12 +276,15 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
scmBlockClient = null;
|
|
|
}
|
|
|
|
|
|
-
|
|
|
RPC.setProtocolEngine(configuration, OzoneManagerProtocolPB.class,
|
|
|
ProtobufRpcEngine.class);
|
|
|
|
|
|
+ startRatisServer();
|
|
|
+ startRatisClient();
|
|
|
+
|
|
|
+ InetSocketAddress omNodeRpcAddr = omNodeDetails.getRpcAddress();
|
|
|
+ omRpcAddressTxt = new Text(omNodeDetails.getRpcAddressString());
|
|
|
|
|
|
- omRpcAddressTxt = new Text(OmUtils.getOmRpcAddress(configuration));
|
|
|
secConfig = new SecurityConfig(configuration);
|
|
|
if (secConfig.isBlockTokenEnabled()) {
|
|
|
blockTokenMgr = createBlockTokenSecretManager(configuration);
|
|
@@ -269,7 +292,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
if(secConfig.isSecurityEnabled()){
|
|
|
delegationTokenMgr = createDelegationTokenSecretManager(configuration);
|
|
|
}
|
|
|
- InetSocketAddress omNodeRpcAddr = getOmAddress(configuration);
|
|
|
+
|
|
|
omRpcServer = getRpcServer(conf);
|
|
|
omRpcAddress = updateRPCListenAddress(configuration,
|
|
|
OZONE_OM_ADDRESS_KEY, omNodeRpcAddr, omRpcServer);
|
|
@@ -297,7 +320,157 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
accessAuthorizer = null;
|
|
|
}
|
|
|
omMetaDir = OmUtils.getOmDbDir(configuration);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Inspects and loads OM node configurations.
|
|
|
+ *
|
|
|
+ * If {@link OMConfigKeys#OZONE_OM_SERVICE_IDS_KEY} is configured with
|
|
|
+ * multiple ids and/ or if {@link OMConfigKeys#OZONE_OM_NODE_ID_KEY} is not
|
|
|
+ * specifically configured , this method determines the omServiceId
|
|
|
+ * and omNodeId by matching the node's address with the configured
|
|
|
+ * addresses. When a match is found, it sets the omServicId and omNodeId from
|
|
|
+ * the corresponding configuration key. This method also finds the OM peers
|
|
|
+ * nodes belonging to the same OM service.
|
|
|
+ *
|
|
|
+ * @param conf
|
|
|
+ */
|
|
|
+ private void loadOMHAConfigs(Configuration conf) {
|
|
|
+ InetSocketAddress localRpcAddress = null;
|
|
|
+ String localOMServiceId = null;
|
|
|
+ String localOMNodeId = null;
|
|
|
+ int localRatisPort = 0;
|
|
|
+ Collection<String> omServiceIds = conf.getTrimmedStringCollection(
|
|
|
+ OZONE_OM_SERVICE_IDS_KEY);
|
|
|
+
|
|
|
+ String knownOMNodeId = conf.get(OZONE_OM_NODE_ID_KEY);
|
|
|
+ int found = 0;
|
|
|
+ boolean isOMAddressSet = false;
|
|
|
+
|
|
|
+ for (String serviceId : OmUtils.emptyAsSingletonNull(omServiceIds)) {
|
|
|
+ Collection<String> omNodeIds = OmUtils.getOMNodeIds(conf, serviceId);
|
|
|
+
|
|
|
+ List<OMNodeDetails> peerNodesList = new ArrayList<>();
|
|
|
+ boolean isPeer = false;
|
|
|
+ for (String nodeId : OmUtils.emptyAsSingletonNull(omNodeIds)) {
|
|
|
+ if (knownOMNodeId != null && !knownOMNodeId.equals(nodeId)) {
|
|
|
+ isPeer = true;
|
|
|
+ } else {
|
|
|
+ isPeer = false;
|
|
|
+ }
|
|
|
+ String rpcAddrKey = OmUtils.addKeySuffixes(OZONE_OM_ADDRESS_KEY,
|
|
|
+ serviceId, nodeId);
|
|
|
+ String rpcAddrStr = conf.get(rpcAddrKey);
|
|
|
+ if (rpcAddrStr == null) {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+
|
|
|
+ // If OM address is set for any node id, we will not fallback to the
|
|
|
+ // default
|
|
|
+ isOMAddressSet = true;
|
|
|
+
|
|
|
+ String ratisPortKey = OmUtils.addKeySuffixes(OZONE_OM_RATIS_PORT_KEY,
|
|
|
+ serviceId, nodeId);
|
|
|
+ int ratisPort = conf.getInt(ratisPortKey, OZONE_OM_RATIS_PORT_DEFAULT);
|
|
|
|
|
|
+ InetSocketAddress addr = null;
|
|
|
+ try {
|
|
|
+ addr = NetUtils.createSocketAddr(rpcAddrStr);
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.warn("Exception in creating socket address " + addr, e);
|
|
|
+ continue;
|
|
|
+ }
|
|
|
+ if (!addr.isUnresolved()) {
|
|
|
+ if (!isPeer && OmUtils.isAddressLocal(addr)) {
|
|
|
+ localRpcAddress = addr;
|
|
|
+ localOMServiceId = serviceId;
|
|
|
+ localOMNodeId = nodeId;
|
|
|
+ localRatisPort = ratisPort;
|
|
|
+ found++;
|
|
|
+ } else {
|
|
|
+ // This OMNode belongs to same OM service as the current OMNode.
|
|
|
+ // Add it to peerNodes list.
|
|
|
+ OMNodeDetails peerNodeInfo = new OMNodeDetails.Builder()
|
|
|
+ .setOMServiceId(serviceId)
|
|
|
+ .setOMNodeId(nodeId)
|
|
|
+ .setRpcAddress(addr)
|
|
|
+ .setRatisPort(ratisPort)
|
|
|
+ .build();
|
|
|
+ peerNodesList.add(peerNodeInfo);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (found == 1) {
|
|
|
+ LOG.debug("Found one matching OM address with service ID: {} and node" +
|
|
|
+ " ID: {}", localOMServiceId, localOMNodeId);
|
|
|
+
|
|
|
+ setOMNodeDetails(localOMServiceId, localOMNodeId, localRpcAddress,
|
|
|
+ localRatisPort);
|
|
|
+ this.peerNodes = peerNodesList;
|
|
|
+
|
|
|
+ LOG.info("Found matching OM address with OMServiceId: {}, " +
|
|
|
+ "OMNodeId: {}, RPC Address: {} and Ratis port: {}",
|
|
|
+ localOMServiceId, localOMNodeId,
|
|
|
+ NetUtils.getHostPortString(localRpcAddress), localRatisPort);
|
|
|
+ return;
|
|
|
+ } else if (found > 1) {
|
|
|
+ String msg = "Configuration has multiple " + OZONE_OM_ADDRESS_KEY +
|
|
|
+ " addresses that match local node's address. Please configure the" +
|
|
|
+ " system with " + OZONE_OM_SERVICE_IDS_KEY + " and " +
|
|
|
+ OZONE_OM_ADDRESS_KEY;
|
|
|
+ throw new OzoneIllegalArgumentException(msg);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ if (!isOMAddressSet) {
|
|
|
+ // No OM address is set. Fallback to default
|
|
|
+ InetSocketAddress omAddress = OmUtils.getOmAddress(conf);
|
|
|
+ int ratisPort = conf.getInt(OZONE_OM_RATIS_PORT_KEY,
|
|
|
+ OZONE_OM_RATIS_PORT_DEFAULT);
|
|
|
+
|
|
|
+ LOG.info("Configuration either no {} set. Falling back to the default " +
|
|
|
+ "OM address {}", OZONE_OM_ADDRESS_KEY, omAddress);
|
|
|
+
|
|
|
+ setOMNodeDetails(null, null, omAddress, ratisPort);
|
|
|
+
|
|
|
+ } else {
|
|
|
+ String msg = "Configuration has no " + OZONE_OM_ADDRESS_KEY + " " +
|
|
|
+ "address that matches local node's address. Please configure the " +
|
|
|
+ "system with " + OZONE_OM_ADDRESS_KEY;
|
|
|
+ LOG.info(msg);
|
|
|
+ throw new OzoneIllegalArgumentException(msg);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Builds and sets OMNodeDetails object.
|
|
|
+ */
|
|
|
+ private void setOMNodeDetails(String serviceId, String nodeId,
|
|
|
+ InetSocketAddress rpcAddress, int ratisPort) {
|
|
|
+
|
|
|
+ if (serviceId == null) {
|
|
|
+ // If no serviceId is set, take the default serviceID om-service
|
|
|
+ serviceId = OzoneConsts.OM_SERVICE_ID_DEFAULT;
|
|
|
+ LOG.info("OM Service ID is not set. Setting it to the default ID: {}",
|
|
|
+ serviceId);
|
|
|
+ }
|
|
|
+ if (nodeId == null) {
|
|
|
+ // If no nodeId is set, take the omId from omStorage as the nodeID
|
|
|
+ nodeId = omId;
|
|
|
+ LOG.info("OM Node ID is not set. Setting it to the OmStorage's " +
|
|
|
+ "OmID: {}", nodeId);
|
|
|
+ }
|
|
|
+
|
|
|
+ this.omNodeDetails = new OMNodeDetails.Builder()
|
|
|
+ .setOMServiceId(serviceId)
|
|
|
+ .setOMNodeId(nodeId)
|
|
|
+ .setRpcAddress(rpcAddress)
|
|
|
+ .setRatisPort(ratisPort)
|
|
|
+ .build();
|
|
|
+
|
|
|
+ // Set this nodes OZONE_OM_ADDRESS_KEY to the discovered address.
|
|
|
+ configuration.set(OZONE_OM_ADDRESS_KEY,
|
|
|
+ NetUtils.getHostPortString(rpcAddress));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -479,7 +652,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
* @param conf
|
|
|
* @throws IOException, AuthenticationException
|
|
|
*/
|
|
|
- private static void loginOMUser(OzoneConfiguration conf)
|
|
|
+ private void loginOMUser(OzoneConfiguration conf)
|
|
|
throws IOException, AuthenticationException {
|
|
|
|
|
|
if (SecurityUtil.getAuthenticationMethod(conf).equals(
|
|
@@ -491,7 +664,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
|
|
|
UserGroupInformation.setConfiguration(conf);
|
|
|
|
|
|
- InetSocketAddress socAddr = getOmAddress(conf);
|
|
|
+ InetSocketAddress socAddr = OmUtils.getOmAddress(conf);
|
|
|
SecurityUtil.login(conf, OZONE_OM_KERBEROS_KEYTAB_FILE_KEY,
|
|
|
OZONE_OM_KERBEROS_PRINCIPAL_KEY, socAddr.getHostName());
|
|
|
} else {
|
|
@@ -660,10 +833,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
}
|
|
|
|
|
|
securityEnabled = OzoneSecurityUtil.isSecurityEnabled(conf);
|
|
|
- // Authenticate KSM if security is enabled
|
|
|
- if (securityEnabled) {
|
|
|
- loginOMUser(conf);
|
|
|
- }
|
|
|
+
|
|
|
switch (startOpt) {
|
|
|
case INIT:
|
|
|
if (printBanner) {
|
|
@@ -792,6 +962,16 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
return omStorage;
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ public OzoneManagerRatisServer getOmRatisServer() {
|
|
|
+ return omRatisServer;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public InetSocketAddress getOmRpcServerAddr() {
|
|
|
+ return omRpcAddress;
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
public LifeCycle.State getOmRatisServerState() {
|
|
|
if (omRatisServer == null) {
|
|
@@ -866,7 +1046,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
LOG.info(buildRpcServerStartMessage("OzoneManager RPC server",
|
|
|
omRpcAddress));
|
|
|
|
|
|
-
|
|
|
DefaultMetricsSystem.initialize("OzoneManager");
|
|
|
|
|
|
metadataManager.start(configuration);
|
|
@@ -894,6 +1073,10 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
omRpcServer = getRpcServer(configuration);
|
|
|
omRpcServer.start();
|
|
|
isOmRpcServerRunning = true;
|
|
|
+
|
|
|
+ startRatisServer();
|
|
|
+ startRatisClient();
|
|
|
+
|
|
|
try {
|
|
|
httpServer = new OzoneManagerHttpServer(configuration, this);
|
|
|
httpServer.start();
|
|
@@ -919,40 +1102,65 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
return omRpcServer;
|
|
|
}
|
|
|
|
|
|
- InetSocketAddress omNodeRpcAddr = getOmAddress(configuration);
|
|
|
+ InetSocketAddress omNodeRpcAddr = OmUtils.getOmAddress(configuration);
|
|
|
+
|
|
|
+ final int handlerCount = conf.getInt(OZONE_OM_HANDLER_COUNT_KEY,
|
|
|
+ OZONE_OM_HANDLER_COUNT_DEFAULT);
|
|
|
+ RPC.setProtocolEngine(configuration, OzoneManagerProtocolPB.class,
|
|
|
+ ProtobufRpcEngine.class);
|
|
|
+
|
|
|
+ BlockingService omService = newReflectiveBlockingService(
|
|
|
+ new OzoneManagerProtocolServerSideTranslatorPB(this, omRatisClient,
|
|
|
+ isRatisEnabled));
|
|
|
+ return startRpcServer(configuration, omNodeRpcAddr,
|
|
|
+ OzoneManagerProtocolPB.class, omService,
|
|
|
+ handlerCount);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Creates an instance of ratis server.
|
|
|
+ */
|
|
|
+ private void startRatisServer() throws IOException {
|
|
|
// This is a temporary check. Once fully implemented, all OM state change
|
|
|
- // should go through Ratis - either standalone (for non-HA) or replicated
|
|
|
+ // should go through Ratis - be it standalone (for non-HA) or replicated
|
|
|
// (for HA).
|
|
|
- boolean omRatisEnabled = configuration.getBoolean(
|
|
|
+ isRatisEnabled = configuration.getBoolean(
|
|
|
OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY,
|
|
|
OMConfigKeys.OZONE_OM_RATIS_ENABLE_DEFAULT);
|
|
|
- if (omRatisEnabled) {
|
|
|
- omRatisServer = OzoneManagerRatisServer.newOMRatisServer(this, omId,
|
|
|
- omNodeRpcAddr.getAddress(), configuration);
|
|
|
+ if (isRatisEnabled) {
|
|
|
+ if (omRatisServer == null) {
|
|
|
+ omRatisServer = OzoneManagerRatisServer.newOMRatisServer(
|
|
|
+ configuration, this, omNodeDetails, peerNodes);
|
|
|
+ }
|
|
|
omRatisServer.start();
|
|
|
|
|
|
LOG.info("OzoneManager Ratis server started at port {}",
|
|
|
omRatisServer.getServerPort());
|
|
|
+ } else {
|
|
|
+ omRatisServer = null;
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- omRatisClient = OzoneManagerRatisClient.newOzoneManagerRatisClient(
|
|
|
- omId, omRatisServer.getRaftGroup(), configuration);
|
|
|
+ /**
|
|
|
+ * Creates an instance of ratis client.
|
|
|
+ */
|
|
|
+ private void startRatisClient() throws IOException {
|
|
|
+ // This is a temporary check. Once fully implemented, all OM state change
|
|
|
+ // should go through Ratis - be it standalone (for non-HA) or replicated
|
|
|
+ // (for HA).
|
|
|
+ isRatisEnabled = configuration.getBoolean(
|
|
|
+ OMConfigKeys.OZONE_OM_RATIS_ENABLE_KEY,
|
|
|
+ OMConfigKeys.OZONE_OM_RATIS_ENABLE_DEFAULT);
|
|
|
+ if (isRatisEnabled) {
|
|
|
+ if (omRatisClient == null) {
|
|
|
+ omRatisClient = OzoneManagerRatisClient.newOzoneManagerRatisClient(
|
|
|
+ omNodeDetails.getOMNodeId(), omRatisServer.getRaftGroup(),
|
|
|
+ configuration);
|
|
|
+ }
|
|
|
omRatisClient.connect();
|
|
|
} else {
|
|
|
- omRatisServer = null;
|
|
|
omRatisClient = null;
|
|
|
}
|
|
|
-
|
|
|
- final int handlerCount = conf.getInt(OZONE_OM_HANDLER_COUNT_KEY,
|
|
|
- OZONE_OM_HANDLER_COUNT_DEFAULT);
|
|
|
- RPC.setProtocolEngine(configuration, OzoneManagerProtocolPB.class,
|
|
|
- ProtobufRpcEngine.class);
|
|
|
-
|
|
|
- BlockingService omService = newReflectiveBlockingService(
|
|
|
- new OzoneManagerProtocolServerSideTranslatorPB(this, omRatisClient,
|
|
|
- omRatisEnabled));
|
|
|
- return startRpcServer(configuration, omNodeRpcAddr,
|
|
|
- OzoneManagerProtocolPB.class, omService,
|
|
|
- handlerCount);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -970,6 +1178,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
if (omRatisServer != null) {
|
|
|
omRatisServer.stop();
|
|
|
}
|
|
|
+ if (omRatisClient != null) {
|
|
|
+ omRatisClient.close();
|
|
|
+ }
|
|
|
isOmRpcServerRunning = false;
|
|
|
keyManager.stop();
|
|
|
stopSecretManager();
|
|
@@ -2188,4 +2399,12 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
public static void setTestSecureOmFlag(boolean testSecureOmFlag) {
|
|
|
OzoneManager.testSecureOmFlag = testSecureOmFlag;
|
|
|
}
|
|
|
+
|
|
|
+ public String getOMNodId() {
|
|
|
+ return omNodeDetails.getOMNodeId();
|
|
|
+ }
|
|
|
+
|
|
|
+ public String getOMServiceId() {
|
|
|
+ return omNodeDetails.getOMServiceId();
|
|
|
+ }
|
|
|
}
|