|
@@ -30,7 +30,7 @@ import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
|
|
import org.apache.hadoop.hdds.scm.node.states.NodeAlreadyExistsException;
|
|
|
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
|
|
|
-import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
|
|
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
|
|
|
import org.apache.hadoop.hdds.scm.VersionInfo;
|
|
|
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
|
|
|
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
|
|
@@ -94,13 +94,12 @@ public class SCMNodeManager implements NodeManager {
|
|
|
LoggerFactory.getLogger(SCMNodeManager.class);
|
|
|
|
|
|
private final NodeStateManager nodeStateManager;
|
|
|
- private final String clusterID;
|
|
|
private final VersionInfo version;
|
|
|
private final CommandQueue commandQueue;
|
|
|
private final SCMNodeMetrics metrics;
|
|
|
// Node manager MXBean
|
|
|
private ObjectName nmInfoBean;
|
|
|
- private final StorageContainerManager scmManager;
|
|
|
+ private final SCMStorageConfig scmStorageConfig;
|
|
|
private final NetworkTopology clusterMap;
|
|
|
private final DNSToSwitchMapping dnsToSwitchMapping;
|
|
|
private final boolean useHostname;
|
|
@@ -108,18 +107,17 @@ public class SCMNodeManager implements NodeManager {
|
|
|
/**
|
|
|
* Constructs SCM machine Manager.
|
|
|
*/
|
|
|
- public SCMNodeManager(OzoneConfiguration conf, String clusterID,
|
|
|
- StorageContainerManager scmManager, EventPublisher eventPublisher)
|
|
|
- throws IOException {
|
|
|
+ public SCMNodeManager(OzoneConfiguration conf,
|
|
|
+ SCMStorageConfig scmStorageConfig, EventPublisher eventPublisher,
|
|
|
+ NetworkTopology networkTopology) {
|
|
|
this.nodeStateManager = new NodeStateManager(conf, eventPublisher);
|
|
|
- this.clusterID = clusterID;
|
|
|
this.version = VersionInfo.getLatestVersion();
|
|
|
this.commandQueue = new CommandQueue();
|
|
|
- this.scmManager = scmManager;
|
|
|
+ this.scmStorageConfig = scmStorageConfig;
|
|
|
LOG.info("Entering startup safe mode.");
|
|
|
registerMXBean();
|
|
|
this.metrics = SCMNodeMetrics.create(this);
|
|
|
- this.clusterMap = scmManager.getClusterMap();
|
|
|
+ this.clusterMap = networkTopology;
|
|
|
Class<? extends DNSToSwitchMapping> dnsToSwitchMappingClass =
|
|
|
conf.getClass(DFSConfigKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
TableMapping.class, DNSToSwitchMapping.class);
|
|
@@ -221,9 +219,8 @@ public class SCMNodeManager implements NodeManager {
|
|
|
return VersionResponse.newBuilder()
|
|
|
.setVersion(this.version.getVersion())
|
|
|
.addValue(OzoneConsts.SCM_ID,
|
|
|
- this.scmManager.getScmStorageConfig().getScmId())
|
|
|
- .addValue(OzoneConsts.CLUSTER_ID, this.scmManager.getScmStorageConfig()
|
|
|
- .getClusterID())
|
|
|
+ this.scmStorageConfig.getScmId())
|
|
|
+ .addValue(OzoneConsts.CLUSTER_ID, this.scmStorageConfig.getClusterID())
|
|
|
.build();
|
|
|
}
|
|
|
|
|
@@ -274,7 +271,7 @@ public class SCMNodeManager implements NodeManager {
|
|
|
|
|
|
return RegisteredCommand.newBuilder().setErrorCode(ErrorCode.success)
|
|
|
.setDatanodeUUID(datanodeDetails.getUuidString())
|
|
|
- .setClusterID(this.clusterID)
|
|
|
+ .setClusterID(this.scmStorageConfig.getClusterID())
|
|
|
.setHostname(datanodeDetails.getHostName())
|
|
|
.setIpAddress(datanodeDetails.getIpAddress())
|
|
|
.build();
|