|
@@ -27,6 +27,7 @@ import com.google.common.cache.CacheBuilder;
|
|
import com.google.common.cache.RemovalListener;
|
|
import com.google.common.cache.RemovalListener;
|
|
import com.google.common.cache.RemovalNotification;
|
|
import com.google.common.cache.RemovalNotification;
|
|
import com.google.protobuf.BlockingService;
|
|
import com.google.protobuf.BlockingService;
|
|
|
|
+import java.util.Objects;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.hdds.HddsUtils;
|
|
import org.apache.hadoop.hdds.HddsUtils;
|
|
@@ -58,6 +59,8 @@ import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
|
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
|
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
|
|
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
|
|
|
|
+import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStore;
|
|
|
|
+import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStoreRDBImpl;
|
|
import org.apache.hadoop.hdds.scm.node.DeadNodeHandler;
|
|
import org.apache.hadoop.hdds.scm.node.DeadNodeHandler;
|
|
import org.apache.hadoop.hdds.scm.node.NewNodeHandler;
|
|
import org.apache.hadoop.hdds.scm.node.NewNodeHandler;
|
|
import org.apache.hadoop.hdds.scm.node.NonHealthyToHealthyNodeHandler;
|
|
import org.apache.hadoop.hdds.scm.node.NonHealthyToHealthyNodeHandler;
|
|
@@ -125,8 +128,7 @@ import static org.apache.hadoop.util.ExitUtil.terminate;
|
|
* and returns a pipeline.
|
|
* and returns a pipeline.
|
|
*
|
|
*
|
|
* <p>A client once it gets a pipeline (a list of datanodes) will connect to
|
|
* <p>A client once it gets a pipeline (a list of datanodes) will connect to
|
|
- * the datanodes and
|
|
|
|
- * create a container, which then can be used to store data.
|
|
|
|
|
|
+ * the datanodes and create a container, which then can be used to store data.
|
|
*/
|
|
*/
|
|
@InterfaceAudience.LimitedPrivate({"HDFS", "CBLOCK", "OZONE", "HBASE"})
|
|
@InterfaceAudience.LimitedPrivate({"HDFS", "CBLOCK", "OZONE", "HBASE"})
|
|
public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
@@ -158,16 +160,18 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
private final SCMDatanodeProtocolServer datanodeProtocolServer;
|
|
private final SCMDatanodeProtocolServer datanodeProtocolServer;
|
|
private final SCMBlockProtocolServer blockProtocolServer;
|
|
private final SCMBlockProtocolServer blockProtocolServer;
|
|
private final SCMClientProtocolServer clientProtocolServer;
|
|
private final SCMClientProtocolServer clientProtocolServer;
|
|
- private final SCMSecurityProtocolServer securityProtocolServer;
|
|
|
|
|
|
+ private SCMSecurityProtocolServer securityProtocolServer;
|
|
|
|
|
|
/*
|
|
/*
|
|
* State Managers of SCM.
|
|
* State Managers of SCM.
|
|
*/
|
|
*/
|
|
- private final NodeManager scmNodeManager;
|
|
|
|
- private final PipelineManager pipelineManager;
|
|
|
|
- private final ContainerManager containerManager;
|
|
|
|
- private final BlockManager scmBlockManager;
|
|
|
|
- private final SCMStorage scmStorage;
|
|
|
|
|
|
+ private NodeManager scmNodeManager;
|
|
|
|
+ private PipelineManager pipelineManager;
|
|
|
|
+ private ContainerManager containerManager;
|
|
|
|
+ private BlockManager scmBlockManager;
|
|
|
|
+ private final SCMStorageConfig scmStorageConfig;
|
|
|
|
+
|
|
|
|
+ private SCMMetadataStore scmMetadataStore;
|
|
|
|
|
|
private final EventQueue eventQueue;
|
|
private final EventQueue eventQueue;
|
|
/*
|
|
/*
|
|
@@ -188,13 +192,13 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
*/
|
|
*/
|
|
private Cache<String, ContainerStat> containerReportCache;
|
|
private Cache<String, ContainerStat> containerReportCache;
|
|
|
|
|
|
- private final ReplicationManager replicationManager;
|
|
|
|
|
|
+ private ReplicationManager replicationManager;
|
|
|
|
|
|
private final LeaseManager<Long> commandWatcherLeaseManager;
|
|
private final LeaseManager<Long> commandWatcherLeaseManager;
|
|
|
|
|
|
private final ReplicationActivityStatus replicationStatus;
|
|
private final ReplicationActivityStatus replicationStatus;
|
|
- private final SCMChillModeManager scmChillModeManager;
|
|
|
|
- private final CertificateServer certificateServer;
|
|
|
|
|
|
+ private SCMChillModeManager scmChillModeManager;
|
|
|
|
+ private CertificateServer certificateServer;
|
|
|
|
|
|
private JvmPauseMonitor jvmPauseMonitor;
|
|
private JvmPauseMonitor jvmPauseMonitor;
|
|
private final OzoneConfiguration configuration;
|
|
private final OzoneConfiguration configuration;
|
|
@@ -206,29 +210,54 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
*
|
|
*
|
|
* @param conf configuration
|
|
* @param conf configuration
|
|
*/
|
|
*/
|
|
- private StorageContainerManager(OzoneConfiguration conf)
|
|
|
|
|
|
+ public StorageContainerManager(OzoneConfiguration conf)
|
|
throws IOException, AuthenticationException {
|
|
throws IOException, AuthenticationException {
|
|
|
|
+ // default empty configurator means default managers will be used.
|
|
|
|
+ this(conf, new SCMConfigurator());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * This constructor offers finer control over how SCM comes up.
|
|
|
|
+ * To use this, user needs to create a SCMConfigurator and set various
|
|
|
|
+ * managers that user wants SCM to use, if a value is missing then SCM will
|
|
|
|
+ * use the default value for that manager.
|
|
|
|
+ *
|
|
|
|
+ * @param conf - Configuration
|
|
|
|
+ * @param configurator - configurator
|
|
|
|
+ */
|
|
|
|
+ public StorageContainerManager(OzoneConfiguration conf,
|
|
|
|
+ SCMConfigurator configurator)
|
|
|
|
+ throws IOException, AuthenticationException {
|
|
super(HddsVersionInfo.HDDS_VERSION_INFO);
|
|
super(HddsVersionInfo.HDDS_VERSION_INFO);
|
|
|
|
|
|
|
|
+ Objects.requireNonNull(configurator, "configurator cannot not be null");
|
|
|
|
+ Objects.requireNonNull(conf, "configuration cannot not be null");
|
|
|
|
+
|
|
configuration = conf;
|
|
configuration = conf;
|
|
StorageContainerManager.initMetrics();
|
|
StorageContainerManager.initMetrics();
|
|
initContainerReportCache(conf);
|
|
initContainerReportCache(conf);
|
|
- scmStorage = new SCMStorage(conf);
|
|
|
|
- if (scmStorage.getState() != StorageState.INITIALIZED) {
|
|
|
|
- throw new SCMException("SCM not initialized.", ResultCodes
|
|
|
|
- .SCM_NOT_INITIALIZED);
|
|
|
|
|
|
+ /**
|
|
|
|
+ * It is assumed the scm --init command creates the SCM Storage Config.
|
|
|
|
+ */
|
|
|
|
+ scmStorageConfig = new SCMStorageConfig(conf);
|
|
|
|
+ if (scmStorageConfig.getState() != StorageState.INITIALIZED) {
|
|
|
|
+ LOG.error("Please make sure you have run \'ozone scm --init\' " +
|
|
|
|
+ "command to generate all the required metadata.");
|
|
|
|
+ throw new SCMException("SCM not initialized due to storage config " +
|
|
|
|
+ "failure.", ResultCodes.SCM_NOT_INITIALIZED);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Important : This initialization sequence is assumed by some of our tests.
|
|
|
|
+ * The testSecureOzoneCluster assumes that security checks have to be
|
|
|
|
+ * passed before any artifacts like SCM DB is created. So please don't
|
|
|
|
+ * add any other initialization above the Security checks please.
|
|
|
|
+ */
|
|
|
|
+
|
|
// Authenticate SCM if security is enabled
|
|
// Authenticate SCM if security is enabled
|
|
if (OzoneSecurityUtil.isSecurityEnabled(conf)) {
|
|
if (OzoneSecurityUtil.isSecurityEnabled(conf)) {
|
|
- loginAsSCMUser(conf);
|
|
|
|
- certificateServer = initializeCertificateServer(
|
|
|
|
- getScmStorage().getClusterID(), getScmStorage().getScmId());
|
|
|
|
- // TODO: Support Intermediary CAs in future.
|
|
|
|
- certificateServer.init(new SecurityConfig(conf),
|
|
|
|
- CertificateServer.CAType.SELF_SIGNED_CA);
|
|
|
|
- securityProtocolServer = new SCMSecurityProtocolServer(conf,
|
|
|
|
- certificateServer);
|
|
|
|
|
|
+ initializeCAnSecurityProtocol(conf, configurator);
|
|
} else {
|
|
} else {
|
|
// if no Security, we do not create a Certificate Server at all.
|
|
// if no Security, we do not create a Certificate Server at all.
|
|
// This allows user to boot SCM without security temporarily
|
|
// This allows user to boot SCM without security temporarily
|
|
@@ -237,16 +266,16 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
securityProtocolServer = null;
|
|
securityProtocolServer = null;
|
|
}
|
|
}
|
|
|
|
|
|
- eventQueue = new EventQueue();
|
|
|
|
-
|
|
|
|
- scmNodeManager = new SCMNodeManager(
|
|
|
|
- conf, scmStorage.getClusterID(), this, eventQueue);
|
|
|
|
- pipelineManager = new SCMPipelineManager(conf, scmNodeManager, eventQueue);
|
|
|
|
- containerManager = new SCMContainerManager(
|
|
|
|
- conf, scmNodeManager, pipelineManager, eventQueue);
|
|
|
|
- scmBlockManager = new BlockManagerImpl(conf, scmNodeManager,
|
|
|
|
- pipelineManager, containerManager, eventQueue);
|
|
|
|
|
|
+ // Creates the SCM DBs or opens them if it exists.
|
|
|
|
+ initalizeMetadataStore(conf, configurator);
|
|
|
|
|
|
|
|
+ eventQueue = new EventQueue();
|
|
|
|
+ long watcherTimeout =
|
|
|
|
+ conf.getTimeDuration(ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT,
|
|
|
|
+ HDDS_SCM_WATCHER_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
|
|
|
|
+ commandWatcherLeaseManager = new LeaseManager<>("CommandWatcher",
|
|
|
|
+ watcherTimeout);
|
|
|
|
+ initalizeSystemManagers(conf, configurator);
|
|
replicationStatus = new ReplicationActivityStatus();
|
|
replicationStatus = new ReplicationActivityStatus();
|
|
|
|
|
|
CloseContainerEventHandler closeContainerHandler =
|
|
CloseContainerEventHandler closeContainerHandler =
|
|
@@ -280,12 +309,6 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
PipelineActionHandler pipelineActionHandler =
|
|
PipelineActionHandler pipelineActionHandler =
|
|
new PipelineActionHandler(pipelineManager, conf);
|
|
new PipelineActionHandler(pipelineManager, conf);
|
|
|
|
|
|
- long watcherTimeout =
|
|
|
|
- conf.getTimeDuration(ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT,
|
|
|
|
- HDDS_SCM_WATCHER_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
|
|
|
|
-
|
|
|
|
- commandWatcherLeaseManager = new LeaseManager<>("CommandWatcher",
|
|
|
|
- watcherTimeout);
|
|
|
|
|
|
|
|
RetriableDatanodeEventWatcher retriableDatanodeEventWatcher =
|
|
RetriableDatanodeEventWatcher retriableDatanodeEventWatcher =
|
|
new RetriableDatanodeEventWatcher<>(
|
|
new RetriableDatanodeEventWatcher<>(
|
|
@@ -294,13 +317,6 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
commandWatcherLeaseManager);
|
|
commandWatcherLeaseManager);
|
|
retriableDatanodeEventWatcher.start(eventQueue);
|
|
retriableDatanodeEventWatcher.start(eventQueue);
|
|
|
|
|
|
- //TODO: support configurable containerPlacement policy
|
|
|
|
- ContainerPlacementPolicy containerPlacementPolicy =
|
|
|
|
- new SCMContainerPlacementCapacity(scmNodeManager, conf);
|
|
|
|
-
|
|
|
|
- replicationManager = new ReplicationManager(containerPlacementPolicy,
|
|
|
|
- containerManager, eventQueue, commandWatcherLeaseManager);
|
|
|
|
-
|
|
|
|
scmAdminUsernames = conf.getTrimmedStringCollection(OzoneConfigKeys
|
|
scmAdminUsernames = conf.getTrimmedStringCollection(OzoneConfigKeys
|
|
.OZONE_ADMINISTRATORS);
|
|
.OZONE_ADMINISTRATORS);
|
|
scmUsername = UserGroupInformation.getCurrentUser().getUserName();
|
|
scmUsername = UserGroupInformation.getCurrentUser().getUserName();
|
|
@@ -342,13 +358,120 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
replicationStatus.getChillModeStatusListener());
|
|
replicationStatus.getChillModeStatusListener());
|
|
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS,
|
|
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS,
|
|
(BlockManagerImpl) scmBlockManager);
|
|
(BlockManagerImpl) scmBlockManager);
|
|
- scmChillModeManager = new SCMChillModeManager(conf,
|
|
|
|
- containerManager.getContainers(), pipelineManager, eventQueue);
|
|
|
|
-
|
|
|
|
eventQueue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
|
eventQueue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
|
scmChillModeManager);
|
|
scmChillModeManager);
|
|
registerMXBean();
|
|
registerMXBean();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * This function initializes the following managers. If the configurator
|
|
|
|
+ * specifies a value, we will use it, else we will use the default value.
|
|
|
|
+ *
|
|
|
|
+ * Node Manager
|
|
|
|
+ * Pipeline Manager
|
|
|
|
+ * Container Manager
|
|
|
|
+ * Block Manager
|
|
|
|
+ * Replication Manager
|
|
|
|
+ * Chill Mode Manager
|
|
|
|
+ *
|
|
|
|
+ * @param conf - Ozone Configuration.
|
|
|
|
+ * @param configurator - A customizer which allows different managers to be
|
|
|
|
+ * used if needed.
|
|
|
|
+ * @throws IOException - on Failure.
|
|
|
|
+ */
|
|
|
|
+ private void initalizeSystemManagers(OzoneConfiguration conf,
|
|
|
|
+ SCMConfigurator configurator)
|
|
|
|
+ throws IOException {
|
|
|
|
+ if(configurator.getScmNodeManager() != null) {
|
|
|
|
+ scmNodeManager = configurator.getScmNodeManager();
|
|
|
|
+ } else {
|
|
|
|
+ scmNodeManager = new SCMNodeManager(
|
|
|
|
+ conf, scmStorageConfig.getClusterID(), this, eventQueue);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ //TODO: support configurable containerPlacement policy
|
|
|
|
+ ContainerPlacementPolicy containerPlacementPolicy =
|
|
|
|
+ new SCMContainerPlacementCapacity(scmNodeManager, conf);
|
|
|
|
+
|
|
|
|
+ if (configurator.getPipelineManager() != null) {
|
|
|
|
+ pipelineManager = configurator.getPipelineManager();
|
|
|
|
+ } else {
|
|
|
|
+ pipelineManager =
|
|
|
|
+ new SCMPipelineManager(conf, scmNodeManager, eventQueue);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if(configurator.getContainerManager() != null) {
|
|
|
|
+ containerManager = configurator.getContainerManager();
|
|
|
|
+ } else {
|
|
|
|
+ containerManager = new SCMContainerManager(
|
|
|
|
+ conf, scmNodeManager, pipelineManager, eventQueue);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ if(configurator.getScmBlockManager() != null) {
|
|
|
|
+ scmBlockManager = configurator.getScmBlockManager();
|
|
|
|
+ } else {
|
|
|
|
+ scmBlockManager = new BlockManagerImpl(conf, this);
|
|
|
|
+ }
|
|
|
|
+ if (configurator.getReplicationManager() != null) {
|
|
|
|
+ replicationManager = configurator.getReplicationManager();
|
|
|
|
+ } else {
|
|
|
|
+ replicationManager = new ReplicationManager(containerPlacementPolicy,
|
|
|
|
+ containerManager, eventQueue, commandWatcherLeaseManager);
|
|
|
|
+ }
|
|
|
|
+ if(configurator.getScmChillModeManager() != null) {
|
|
|
|
+ scmChillModeManager = configurator.getScmChillModeManager();
|
|
|
|
+ } else {
|
|
|
|
+ scmChillModeManager = new SCMChillModeManager(conf,
|
|
|
|
+ containerManager.getContainers(), pipelineManager, eventQueue);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * If security is enabled we need to have the Security Protocol and a
|
|
|
|
+ * default CA. This function initializes those values based on the
|
|
|
|
+ * configurator.
|
|
|
|
+ *
|
|
|
|
+ * @param conf - Config
|
|
|
|
+ * @param configurator - configurator
|
|
|
|
+ * @throws IOException - on Failure
|
|
|
|
+ * @throws AuthenticationException - on Failure
|
|
|
|
+ */
|
|
|
|
+ private void initializeCAnSecurityProtocol(OzoneConfiguration conf,
|
|
|
|
+ SCMConfigurator configurator)
|
|
|
|
+ throws IOException, AuthenticationException {
|
|
|
|
+ loginAsSCMUser(conf);
|
|
|
|
+ if(configurator.getCertificateServer() != null) {
|
|
|
|
+ this.certificateServer = configurator.getCertificateServer();
|
|
|
|
+ } else {
|
|
|
|
+ certificateServer = initializeCertificateServer(
|
|
|
|
+ getScmStorageConfig().getClusterID(),
|
|
|
|
+ getScmStorageConfig().getScmId());
|
|
|
|
+ }
|
|
|
|
+ // TODO: Support Intermediary CAs in future.
|
|
|
|
+ certificateServer.init(new SecurityConfig(conf),
|
|
|
|
+ CertificateServer.CAType.SELF_SIGNED_CA);
|
|
|
|
+ securityProtocolServer = new SCMSecurityProtocolServer(conf,
|
|
|
|
+ certificateServer);
|
|
|
|
+ }
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Init the metadata store based on the configurator.
|
|
|
|
+ * @param conf - Config
|
|
|
|
+ * @param configurator - configurator
|
|
|
|
+ * @throws IOException - on Failure
|
|
|
|
+ */
|
|
|
|
+ private void initalizeMetadataStore(OzoneConfiguration conf,
|
|
|
|
+ SCMConfigurator configurator)
|
|
|
|
+ throws IOException {
|
|
|
|
+ if(configurator.getMetadataStore() != null) {
|
|
|
|
+ scmMetadataStore = configurator.getMetadataStore();
|
|
|
|
+ } else {
|
|
|
|
+ scmMetadataStore = new SCMMetadataStoreRDBImpl(conf);
|
|
|
|
+ if (scmMetadataStore == null) {
|
|
|
|
+ throw new SCMException("Unable to initialize metadata store",
|
|
|
|
+ ResultCodes.SCM_NOT_INITIALIZED);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -393,7 +516,6 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
// So it is easy to use different Certificate Servers if needed.
|
|
// So it is easy to use different Certificate Servers if needed.
|
|
String subject = "scm@" + InetAddress.getLocalHost().getHostName();
|
|
String subject = "scm@" + InetAddress.getLocalHost().getHostName();
|
|
return new DefaultCAServer(subject, clusterID, scmID);
|
|
return new DefaultCAServer(subject, clusterID, scmID);
|
|
-
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -562,21 +684,21 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
* @throws IOException if init fails due to I/O error
|
|
* @throws IOException if init fails due to I/O error
|
|
*/
|
|
*/
|
|
public static boolean scmInit(OzoneConfiguration conf) throws IOException {
|
|
public static boolean scmInit(OzoneConfiguration conf) throws IOException {
|
|
- SCMStorage scmStorage = new SCMStorage(conf);
|
|
|
|
- StorageState state = scmStorage.getState();
|
|
|
|
|
|
+ SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
|
|
|
|
+ StorageState state = scmStorageConfig.getState();
|
|
if (state != StorageState.INITIALIZED) {
|
|
if (state != StorageState.INITIALIZED) {
|
|
try {
|
|
try {
|
|
String clusterId = StartupOption.INIT.getClusterId();
|
|
String clusterId = StartupOption.INIT.getClusterId();
|
|
if (clusterId != null && !clusterId.isEmpty()) {
|
|
if (clusterId != null && !clusterId.isEmpty()) {
|
|
- scmStorage.setClusterId(clusterId);
|
|
|
|
|
|
+ scmStorageConfig.setClusterId(clusterId);
|
|
}
|
|
}
|
|
- scmStorage.initialize();
|
|
|
|
|
|
+ scmStorageConfig.initialize();
|
|
System.out.println(
|
|
System.out.println(
|
|
"SCM initialization succeeded."
|
|
"SCM initialization succeeded."
|
|
+ "Current cluster id for sd="
|
|
+ "Current cluster id for sd="
|
|
- + scmStorage.getStorageDir()
|
|
|
|
|
|
+ + scmStorageConfig.getStorageDir()
|
|
+ ";cid="
|
|
+ ";cid="
|
|
- + scmStorage.getClusterID());
|
|
|
|
|
|
+ + scmStorageConfig.getClusterID());
|
|
return true;
|
|
return true;
|
|
} catch (IOException ioe) {
|
|
} catch (IOException ioe) {
|
|
LOG.error("Could not initialize SCM version file", ioe);
|
|
LOG.error("Could not initialize SCM version file", ioe);
|
|
@@ -586,9 +708,9 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
System.out.println(
|
|
System.out.println(
|
|
"SCM already initialized. Reusing existing"
|
|
"SCM already initialized. Reusing existing"
|
|
+ " cluster id for sd="
|
|
+ " cluster id for sd="
|
|
- + scmStorage.getStorageDir()
|
|
|
|
|
|
+ + scmStorageConfig.getStorageDir()
|
|
+ ";cid="
|
|
+ ";cid="
|
|
- + scmStorage.getClusterID());
|
|
|
|
|
|
+ + scmStorageConfig.getClusterID());
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -649,8 +771,8 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
return metrics == null ? SCMMetrics.create() : metrics;
|
|
return metrics == null ? SCMMetrics.create() : metrics;
|
|
}
|
|
}
|
|
|
|
|
|
- public SCMStorage getScmStorage() {
|
|
|
|
- return scmStorage;
|
|
|
|
|
|
+ public SCMStorageConfig getScmStorageConfig() {
|
|
|
|
+ return scmStorageConfig;
|
|
}
|
|
}
|
|
|
|
|
|
public SCMDatanodeProtocolServer getDatanodeProtocolServer() {
|
|
public SCMDatanodeProtocolServer getDatanodeProtocolServer() {
|
|
@@ -878,6 +1000,12 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
}
|
|
}
|
|
IOUtils.cleanupWithLogger(LOG, containerManager);
|
|
IOUtils.cleanupWithLogger(LOG, containerManager);
|
|
IOUtils.cleanupWithLogger(LOG, pipelineManager);
|
|
IOUtils.cleanupWithLogger(LOG, pipelineManager);
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ scmMetadataStore.stop();
|
|
|
|
+ } catch (Exception ex) {
|
|
|
|
+ LOG.error("SCM Metadata store stop failed", ex);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1049,6 +1177,14 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
|
|
return nodeStateCount;
|
|
return nodeStateCount;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Returns the SCM metadata Store.
|
|
|
|
+ * @return SCMMetadataStore
|
|
|
|
+ */
|
|
|
|
+ public SCMMetadataStore getScmMetadataStore() {
|
|
|
|
+ return scmMetadataStore;
|
|
|
|
+ }
|
|
/**
|
|
/**
|
|
* Startup options.
|
|
* Startup options.
|
|
*/
|
|
*/
|