avijayanhwx commented on a change in pull request #2942:
URL: https://github.com/apache/ozone/pull/2942#discussion_r778339496
##########
File path:
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
##########
@@ -365,4 +365,6 @@ StatusAndMessages queryUpgradeFinalizationProgress(
* commands operating on {@code containerID}.
*/
Token<?> getContainerToken(ContainerID containerID) throws IOException;
+
+ long getSCMContainersCount() throws IOException;
Review comment:
nit. getSCMContainersCount --> getContainerCount
##########
File path:
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java
##########
@@ -307,6 +327,109 @@ private void initializePipelinesFromScm() {
}
}
+ private void initializeSCMDB() {
+ try {
+ long scmContainersCount = scmServiceProvider.getSCMContainersCount();
+ long reconContainerCount = containerManager.getContainers().size();
+ long threshold = ozoneConfiguration.getInt(
+ ReconServerConfigKeys.OZONE_RECON_SCM_CONTAINER_THRESHOLD,
+ ReconServerConfigKeys.OZONE_RECON_SCM_CONTAINER_THRESHOLD_DEFAULT);
+
+ if(Math.abs(scmContainersCount - reconContainerCount) > threshold) {
+ updateReconSCMDBWithNewSnapshot();
+ LOG.info("Update Recon DB with SCM DB");
Review comment:
nit. Log container counts here?
##########
File path:
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java
##########
@@ -307,6 +327,109 @@ private void initializePipelinesFromScm() {
}
}
+ private void initializeSCMDB() {
+ try {
+ long scmContainersCount = scmServiceProvider.getSCMContainersCount();
+ long reconContainerCount = containerManager.getContainers().size();
+ long threshold = ozoneConfiguration.getInt(
+ ReconServerConfigKeys.OZONE_RECON_SCM_CONTAINER_THRESHOLD,
+ ReconServerConfigKeys.OZONE_RECON_SCM_CONTAINER_THRESHOLD_DEFAULT);
+
+ if(Math.abs(scmContainersCount - reconContainerCount) > threshold) {
+ updateReconSCMDBWithNewSnapshot();
+ LOG.info("Update Recon DB with SCM DB");
+ } else {
+ initializePipelinesFromScm();
+ }
+ } catch (IOException e) {
+ LOG.error("Exception encountered while getting SCM DB.");
+ }
+ }
+
+ public void updateReconSCMDBWithNewSnapshot() throws IOException {
+ DBCheckpoint dbSnapshot = scmServiceProvider.getSCMDBSnapshot();
+ if (dbSnapshot != null && dbSnapshot.getCheckpointLocation() != null) {
+ LOG.info("Got new checkpoint from SCM : " +
+ dbSnapshot.getCheckpointLocation());
+ try {
+ initializeNewRdbStore(dbSnapshot.getCheckpointLocation().toFile());
+ } catch (IOException e) {
+ LOG.error("Unable to refresh Recon SCM DB Snapshot. ", e);
+ }
+ } else {
+ LOG.error("Null snapshot location got from SCM.");
+ }
+ }
+
+ private void deleteOldSCMDB() throws IOException {
+ if (dbStore != null) {
+ File oldDBLocation = dbStore.getDbLocation();
+ if (oldDBLocation.exists()) {
+ LOG.info("Cleaning up old SCM snapshot db at {}.",
+ oldDBLocation.getAbsolutePath());
+ FileUtils.deleteDirectory(oldDBLocation);
+ }
+ }
+ }
+
+ private void initializeNewRdbStore(File dbFile) throws IOException {
+ try {
+ DBStore newStore = createDBAndAddSCMTablesAndCodecs(
+ dbFile, new ReconSCMDBDefinition());
+ Table<UUID, DatanodeDetails> nodeTable =
+ ReconSCMDBDefinition.NODES.getTable(dbStore);
+ Table<UUID, DatanodeDetails> newNodeTable =
+ ReconSCMDBDefinition.NODES.getTable(newStore);
+ TableIterator<UUID, ? extends KeyValue<UUID,
+ DatanodeDetails>> iterator = nodeTable.iterator();
+ while (iterator.hasNext()) {
+ KeyValue<UUID, DatanodeDetails> keyValue = iterator.next();
+ newNodeTable.put(keyValue.getKey(), keyValue.getValue());
+ }
+ deleteOldSCMDB();
Review comment:
The deletion of the old SCM DB should be the last step, after we have
successfully re-init the different SCM managers.
##########
File path: hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
##########
@@ -74,6 +74,7 @@ message ScmContainerLocationRequest {
optional ContainerBalancerStatusRequestProto containerBalancerStatusRequest
= 35;
optional FinalizeScmUpgradeRequestProto finalizeScmUpgradeRequest = 36;
optional QueryUpgradeFinalizationProgressRequestProto
queryUpgradeFinalizationProgressRequest = 37;
+ optional GetSCMContainersCountRequestProto getSCMContainersCountRequest = 38;
Review comment:
nit. GetSCMContainersCountRequestProto --> GetContainerCountRequestProto
##########
File path:
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/StorageContainerServiceProvider.java
##########
@@ -66,4 +67,7 @@ ContainerWithPipeline getContainerWithPipeline(long
containerId)
*/
List<HddsProtos.Node> getNodes() throws IOException;
+ long getSCMContainersCount() throws IOException;
+
+ DBCheckpoint getSCMDBSnapshot();
Review comment:
Please add Javadoc here.
##########
File path:
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/StorageContainerServiceProviderImpl.java
##########
@@ -38,12 +61,53 @@
public class StorageContainerServiceProviderImpl
implements StorageContainerServiceProvider {
+ private static final Logger LOG =
+ LoggerFactory.getLogger(StorageContainerServiceProviderImpl.class);
private StorageContainerLocationProtocol scmClient;
+ private final OzoneConfiguration configuration;
+ private String scmDBSnapshotUrl;
+ private File scmSnapshotDBParentDir;
+ private URLConnectionFactory connectionFactory;
+ private ReconUtils reconUtils;
@Inject
public StorageContainerServiceProviderImpl(
- StorageContainerLocationProtocol scmClient) {
+ StorageContainerLocationProtocol scmClient,
+ ReconUtils reconUtils,
+ OzoneConfiguration configuration) {
+
+ int connectionTimeout = (int) configuration.getTimeDuration(
+ OZONE_RECON_SCM_CONNECTION_TIMEOUT,
+ OZONE_RECON_SCM_CONNECTION_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
+ int connectionRequestTimeout = (int) configuration.getTimeDuration(
+ OZONE_RECON_SCM_CONNECTION_REQUEST_TIMEOUT,
+ OZONE_RECON_SCM_CONNECTION_REQUEST_TIMEOUT_DEFAULT,
+ TimeUnit.MILLISECONDS);
+ connectionFactory =
+ URLConnectionFactory.newDefaultURLConnectionFactory(connectionTimeout,
+ connectionRequestTimeout, configuration);
+
+ String scmHttpAddress = configuration.get(ScmConfigKeys
+ .OZONE_SCM_HTTP_ADDRESS_KEY);
+
+ String scmHttpsAddress = configuration.get(ScmConfigKeys
+ .OZONE_SCM_HTTPS_ADDRESS_KEY);
+
+ HttpConfig.Policy policy = HttpConfig.getHttpPolicy(configuration);
+
+ scmSnapshotDBParentDir = ReconUtils.getReconScmDbDir(configuration);
+
+ scmDBSnapshotUrl = "http://" + scmHttpAddress +
+ OZONE_OM_DB_CHECKPOINT_HTTP_ENDPOINT;
+
+ if (policy.isHttpsEnabled()) {
+ scmDBSnapshotUrl = "https://" + scmHttpsAddress +
+ OZONE_OM_DB_CHECKPOINT_HTTP_ENDPOINT;
Review comment:
nit. Rename OZONE_OM_DB_CHECKPOINT_HTTP_ENDPOINT to
OZONE_DB_CHECKPOINT_HTTP_ENDPOINT?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]