sumitagrawl commented on code in PR #3947:
URL: https://github.com/apache/ozone/pull/3947#discussion_r1031598787


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMContainerMetrics.java:
##########
@@ -91,6 +92,9 @@ public void getMetrics(MetricsCollector collector, boolean 
all) {
         .addGauge(Interns.info("DeletedContainers",
             "Number of containers in deleted state"),
             stateCount.get(DELETED.toString()))
+        .addGauge(Interns.info("UnknownContainers",

Review Comment:
   can avoid metrics for unknown state, as there is no possiblity of such state.



##########
hadoop-ozone/dev-support/intellij/ozone-site.xml:
##########
@@ -71,4 +71,24 @@
     <name>datanode.replication.port</name>
     <value>0</value>
   </property>
+  <property>
+    <name>ozone.scm.ratis.enable</name>
+    <value>false</value>
+  </property>
+  <property>
+    <name>hdds.container.report.interval</name>
+    <value>60s</value>
+  </property>
+  <property>

Review Comment:
   we can have the config with default value and respective dev-config can be 
changed as required while test.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java:
##########
@@ -396,22 +445,90 @@ private void initializeSCMDB() {
       }
     } catch (IOException e) {
       LOG.error("Exception encountered while getting SCM DB.");
+    } finally {
+      isSyncDataFromSCMRunning.compareAndSet(true, false);
     }
   }
 
   public void updateReconSCMDBWithNewSnapshot() throws IOException {
-    DBCheckpoint dbSnapshot = scmServiceProvider.getSCMDBSnapshot();
-    if (dbSnapshot != null && dbSnapshot.getCheckpointLocation() != null) {
-      LOG.info("Got new checkpoint from SCM : " +
-          dbSnapshot.getCheckpointLocation());
+    if (isSyncDataFromSCMRunning.compareAndSet(false, true)) {
+      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.");
+      }
+    } else {
+      LOG.warn("SCM DB sync is already running.");
+    }
+  }
+
+  public boolean syncWithSCMContainerInfo()
+      throws IOException {
+    if (isSyncDataFromSCMRunning.compareAndSet(false, true)) {
       try {
-        initializeNewRdbStore(dbSnapshot.getCheckpointLocation().toFile());
+        List<ContainerInfo> containers = containerManager.getContainers();
+
+        long containerCount = scmServiceProvider.getContainerCount(
+            HddsProtos.LifeCycleState.CLOSED);
+        // Assumption of size of 1 container info object here is 1 MB
+        long containersMetaDataTotalRpcRespSizeMB =
+            CONTAINER_METADATA_SIZE * containerCount;
+        double numOfApiCalls =

Review Comment:
   Can get actual hadoop RPC size from conf with default value



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java:
##########
@@ -396,22 +445,90 @@ private void initializeSCMDB() {
       }
     } catch (IOException e) {
       LOG.error("Exception encountered while getting SCM DB.");
+    } finally {
+      isSyncDataFromSCMRunning.compareAndSet(true, false);
     }
   }
 
   public void updateReconSCMDBWithNewSnapshot() throws IOException {
-    DBCheckpoint dbSnapshot = scmServiceProvider.getSCMDBSnapshot();
-    if (dbSnapshot != null && dbSnapshot.getCheckpointLocation() != null) {
-      LOG.info("Got new checkpoint from SCM : " +
-          dbSnapshot.getCheckpointLocation());
+    if (isSyncDataFromSCMRunning.compareAndSet(false, true)) {
+      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.");
+      }
+    } else {
+      LOG.warn("SCM DB sync is already running.");
+    }
+  }
+
+  public boolean syncWithSCMContainerInfo()
+      throws IOException {
+    if (isSyncDataFromSCMRunning.compareAndSet(false, true)) {
       try {
-        initializeNewRdbStore(dbSnapshot.getCheckpointLocation().toFile());
+        List<ContainerInfo> containers = containerManager.getContainers();
+
+        long containerCount = scmServiceProvider.getContainerCount(
+            HddsProtos.LifeCycleState.CLOSED);
+        // Assumption of size of 1 container info object here is 1 MB
+        long containersMetaDataTotalRpcRespSizeMB =
+            CONTAINER_METADATA_SIZE * containerCount;
+        double numOfApiCalls =
+            Double.valueOf(containersMetaDataTotalRpcRespSizeMB) /
+                Double.valueOf(DEFAULT_RPC_SIZE);
+        long totalApiCalls = Math.round(Math.ceil(numOfApiCalls));
+        long containerCountPerCall = containersMetaDataTotalRpcRespSizeMB <=
+            DEFAULT_RPC_SIZE ? containerCount : DEFAULT_RPC_SIZE;
+        if (containerCount > 0) {
+          for (int startContainerId = 1; startContainerId <= totalApiCalls;
+               startContainerId++) {
+            List<ContainerInfo> listOfContainers = scmServiceProvider.
+                getListOfContainers(Long.valueOf(startContainerId).longValue(),
+                    Long.valueOf(containerCountPerCall).intValue(),
+                    HddsProtos.LifeCycleState.CLOSED);
+            if (null != listOfContainers && listOfContainers.size() > 0) {
+              LOG.info("Got list of containers from SCM : " +
+                  listOfContainers.size());
+              listOfContainers.forEach(containerInfo -> {
+                long containerID = containerInfo.getContainerID();
+                boolean isContainerPresentAtRecon =
+                    containers.remove(containerID);
+                if (!isContainerPresentAtRecon) {
+                  try {
+                    ContainerWithPipeline containerWithPipeline =
+                        scmServiceProvider.getContainerWithPipeline(
+                            containerID);
+                    containerWithPipeline.getContainerInfo().setState(
+                        HddsProtos.LifeCycleState.UNKNOWN);

Review Comment:
   Unknown state of container can be removed, its health report to 
differentiate if Unknown or missing if required.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java:
##########
@@ -396,22 +445,90 @@ private void initializeSCMDB() {
       }
     } catch (IOException e) {
       LOG.error("Exception encountered while getting SCM DB.");
+    } finally {
+      isSyncDataFromSCMRunning.compareAndSet(true, false);
     }
   }
 
   public void updateReconSCMDBWithNewSnapshot() throws IOException {
-    DBCheckpoint dbSnapshot = scmServiceProvider.getSCMDBSnapshot();
-    if (dbSnapshot != null && dbSnapshot.getCheckpointLocation() != null) {
-      LOG.info("Got new checkpoint from SCM : " +
-          dbSnapshot.getCheckpointLocation());
+    if (isSyncDataFromSCMRunning.compareAndSet(false, true)) {
+      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.");
+      }
+    } else {
+      LOG.warn("SCM DB sync is already running.");
+    }
+  }
+
+  public boolean syncWithSCMContainerInfo()
+      throws IOException {
+    if (isSyncDataFromSCMRunning.compareAndSet(false, true)) {
       try {
-        initializeNewRdbStore(dbSnapshot.getCheckpointLocation().toFile());
+        List<ContainerInfo> containers = containerManager.getContainers();
+
+        long containerCount = scmServiceProvider.getContainerCount(
+            HddsProtos.LifeCycleState.CLOSED);
+        // Assumption of size of 1 container info object here is 1 MB
+        long containersMetaDataTotalRpcRespSizeMB =
+            CONTAINER_METADATA_SIZE * containerCount;
+        double numOfApiCalls =
+            Double.valueOf(containersMetaDataTotalRpcRespSizeMB) /
+                Double.valueOf(DEFAULT_RPC_SIZE);
+        long totalApiCalls = Math.round(Math.ceil(numOfApiCalls));
+        long containerCountPerCall = containersMetaDataTotalRpcRespSizeMB <=
+            DEFAULT_RPC_SIZE ? containerCount : DEFAULT_RPC_SIZE;
+        if (containerCount > 0) {
+          for (int startContainerId = 1; startContainerId <= totalApiCalls;

Review Comment:
   startContainerId should start with 0 index excluding, to get next set of 
list, and next startIndex should be last queried containerId, this need fix at 
SCM and recon.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java:
##########
@@ -396,22 +445,90 @@ private void initializeSCMDB() {
       }
     } catch (IOException e) {
       LOG.error("Exception encountered while getting SCM DB.");
+    } finally {
+      isSyncDataFromSCMRunning.compareAndSet(true, false);
     }
   }
 
   public void updateReconSCMDBWithNewSnapshot() throws IOException {
-    DBCheckpoint dbSnapshot = scmServiceProvider.getSCMDBSnapshot();
-    if (dbSnapshot != null && dbSnapshot.getCheckpointLocation() != null) {
-      LOG.info("Got new checkpoint from SCM : " +
-          dbSnapshot.getCheckpointLocation());
+    if (isSyncDataFromSCMRunning.compareAndSet(false, true)) {
+      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.");
+      }
+    } else {
+      LOG.warn("SCM DB sync is already running.");
+    }
+  }
+
+  public boolean syncWithSCMContainerInfo()
+      throws IOException {
+    if (isSyncDataFromSCMRunning.compareAndSet(false, true)) {
       try {
-        initializeNewRdbStore(dbSnapshot.getCheckpointLocation().toFile());
+        List<ContainerInfo> containers = containerManager.getContainers();
+
+        long containerCount = scmServiceProvider.getContainerCount(
+            HddsProtos.LifeCycleState.CLOSED);
+        // Assumption of size of 1 container info object here is 1 MB
+        long containersMetaDataTotalRpcRespSizeMB =
+            CONTAINER_METADATA_SIZE * containerCount;
+        double numOfApiCalls =
+            Double.valueOf(containersMetaDataTotalRpcRespSizeMB) /
+                Double.valueOf(DEFAULT_RPC_SIZE);
+        long totalApiCalls = Math.round(Math.ceil(numOfApiCalls));
+        long containerCountPerCall = containersMetaDataTotalRpcRespSizeMB <=
+            DEFAULT_RPC_SIZE ? containerCount : DEFAULT_RPC_SIZE;
+        if (containerCount > 0) {
+          for (int startContainerId = 1; startContainerId <= totalApiCalls;
+               startContainerId++) {
+            List<ContainerInfo> listOfContainers = scmServiceProvider.
+                getListOfContainers(Long.valueOf(startContainerId).longValue(),
+                    Long.valueOf(containerCountPerCall).intValue(),
+                    HddsProtos.LifeCycleState.CLOSED);
+            if (null != listOfContainers && listOfContainers.size() > 0) {
+              LOG.info("Got list of containers from SCM : " +
+                  listOfContainers.size());
+              listOfContainers.forEach(containerInfo -> {
+                long containerID = containerInfo.getContainerID();
+                boolean isContainerPresentAtRecon =

Review Comment:
   Avoid removal of container, as replica information will be lost as 
registered by DN in periodic call.



-- 
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]

Reply via email to