devmadhuu commented on code in PR #4107:
URL: https://github.com/apache/ozone/pull/4107#discussion_r1052949796


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.tasks;
+
+
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.scm.ReconScmTask;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.hadoop.ozone.recon.schema.tables.daos.ContainerCountBySizeDao;
+import org.hadoop.ozone.recon.schema.tables.daos.ReconTaskStatusDao;
+import org.hadoop.ozone.recon.schema.tables.pojos.ContainerCountBySize;
+import org.jooq.DSLContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.hadoop.ozone.recon.schema.tables.ContainerCountBySizeTable.CONTAINER_COUNT_BY_SIZE;
+
+
+/**
+ * Class that scans the list of containers and keeps track of container sizes
+ * binned into ranges (1KB, 2Kb..,4MB,.., 1TB,..1PB) to the Recon
+ * containerSize DB.
+ */
+public class ContainerSizeCountTask extends ReconScmTask {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContainerSizeCountTask.class);
+
+  private StorageContainerServiceProvider scmClient;
+  private ContainerManager containerManager;
+  private final long interval;
+  private ContainerCountBySizeDao containerCountBySizeDao;
+  private DSLContext dslContext;
+
+
+  public ContainerSizeCountTask(
+      ContainerManager containerManager,
+      StorageContainerServiceProvider scmClient,
+      ReconTaskStatusDao reconTaskStatusDao,
+      ReconTaskConfig reconTaskConfig,
+      ContainerCountBySizeDao containerCountBySizeDao,
+      DSLContext dslContext) {
+    super(reconTaskStatusDao);
+    this.scmClient = scmClient;
+    this.containerManager = containerManager;
+    this.containerCountBySizeDao = containerCountBySizeDao;
+    this.dslContext = dslContext;
+    interval = reconTaskConfig.getContainerSizeCountTaskInterval().toMillis();
+  }
+
+
+  @Override
+  protected synchronized void run() {
+    try {
+      while (canRun()) {
+        wait(interval);
+
+        final List<ContainerInfo> containers = 
containerManager.getContainers();
+        Map<ContainerSizeCountKey, Long> containerSizeCountMap =
+            new HashMap<>();
+        for (int i = 0; i < containers.size(); i++) {
+          handlePutKeyEvent(containers.get(i), containerSizeCountMap);
+        }
+        // Truncate table before inserting new rows
+        int execute = dslContext.delete(CONTAINER_COUNT_BY_SIZE).execute();
+        LOG.info("Deleted {} records from {}", execute,
+            CONTAINER_COUNT_BY_SIZE);
+
+        writeCountsToDB(true, containerSizeCountMap);
+
+        LOG.info("Completed a 'reprocess' run of ContainerSizeCountTask.");
+      }
+    } catch (Throwable t) {
+      LOG.error("Exception in Container Size Distribution task Thread.", t);
+      if (t instanceof InterruptedException) {
+        Thread.currentThread().interrupt();
+      }
+    }
+  }
+
+  /**
+   * Populate DB with the counts of container sizes calculated
+   * using the dao.
+   */
+  private void writeCountsToDB(boolean isDbTruncated,

Review Comment:
   Do we need this "isDBTruncated" flag if it is always true ?



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.tasks;
+
+
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.scm.ReconScmTask;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.hadoop.ozone.recon.schema.tables.daos.ContainerCountBySizeDao;
+import org.hadoop.ozone.recon.schema.tables.daos.ReconTaskStatusDao;
+import org.hadoop.ozone.recon.schema.tables.pojos.ContainerCountBySize;
+import org.jooq.DSLContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.hadoop.ozone.recon.schema.tables.ContainerCountBySizeTable.CONTAINER_COUNT_BY_SIZE;
+
+
+/**
+ * Class that scans the list of containers and keeps track of container sizes
+ * binned into ranges (1KB, 2Kb..,4MB,.., 1TB,..1PB) to the Recon
+ * containerSize DB.
+ */
+public class ContainerSizeCountTask extends ReconScmTask {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContainerSizeCountTask.class);
+
+  private StorageContainerServiceProvider scmClient;
+  private ContainerManager containerManager;
+  private final long interval;
+  private ContainerCountBySizeDao containerCountBySizeDao;
+  private DSLContext dslContext;
+
+
+  public ContainerSizeCountTask(
+      ContainerManager containerManager,
+      StorageContainerServiceProvider scmClient,
+      ReconTaskStatusDao reconTaskStatusDao,
+      ReconTaskConfig reconTaskConfig,
+      ContainerCountBySizeDao containerCountBySizeDao,
+      DSLContext dslContext) {
+    super(reconTaskStatusDao);
+    this.scmClient = scmClient;
+    this.containerManager = containerManager;
+    this.containerCountBySizeDao = containerCountBySizeDao;
+    this.dslContext = dslContext;
+    interval = reconTaskConfig.getContainerSizeCountTaskInterval().toMillis();
+  }
+
+
+  @Override
+  protected synchronized void run() {
+    try {
+      while (canRun()) {
+        wait(interval);
+
+        final List<ContainerInfo> containers = 
containerManager.getContainers();
+        Map<ContainerSizeCountKey, Long> containerSizeCountMap =
+            new HashMap<>();
+        for (int i = 0; i < containers.size(); i++) {
+          handlePutKeyEvent(containers.get(i), containerSizeCountMap);
+        }
+        // Truncate table before inserting new rows
+        int execute = dslContext.delete(CONTAINER_COUNT_BY_SIZE).execute();
+        LOG.info("Deleted {} records from {}", execute,
+            CONTAINER_COUNT_BY_SIZE);
+
+        writeCountsToDB(true, containerSizeCountMap);
+
+        LOG.info("Completed a 'reprocess' run of ContainerSizeCountTask.");
+      }
+    } catch (Throwable t) {
+      LOG.error("Exception in Container Size Distribution task Thread.", t);
+      if (t instanceof InterruptedException) {
+        Thread.currentThread().interrupt();
+      }
+    }
+  }
+
+  /**
+   * Populate DB with the counts of container sizes calculated
+   * using the dao.
+   */
+  private void writeCountsToDB(boolean isDbTruncated,
+                               Map<ContainerSizeCountKey, Long>
+                                   containerSizeCountMap) {
+    List<ContainerCountBySize> insertToDb = new ArrayList<>();
+
+    containerSizeCountMap.keySet().forEach((ContainerSizeCountKey key) -> {
+      ContainerCountBySize newRecord = new ContainerCountBySize();
+      newRecord.setContainerSize(key.containerSizeUpperBound);
+      newRecord.setCount(containerSizeCountMap.get(key));
+      if (!isDbTruncated) {
+        insertToDb.add(newRecord);
+      }
+    });
+    containerCountBySizeDao.insert(insertToDb);
+  }
+
+  @Override
+  public String getTaskName() {
+    return "ContainerSizeCountTask";
+  }
+
+  /**
+   * Calculate and update the count of containers being tracked by
+   * containerSizeCountMap.
+   * Used by reprocess() and process().
+   *
+   * @param containerInfo OmKey being updated for count
+   */
+  private void handlePutKeyEvent(ContainerInfo containerInfo,

Review Comment:
   Try to give better name, as there is no event we are capturing with SCM DB 
sync, here we are taking data from Recon container cache.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskConfig.java:
##########
@@ -66,4 +66,22 @@ public void setMissingContainerTaskInterval(Duration 
interval) {
     this.missingContainerTaskInterval = interval.toMillis();
   }
 
+  @Config(key = "containercounttask.interval",
+      type = ConfigType.TIME,
+      defaultValue = "300s",
+      tags = { ConfigTag.RECON, ConfigTag.OZONE },
+      description = "The time interval of the periodic check for " +
+          "container size distribution in the cluster as reported "
+  )
+  private long containerSizeCountTaskInterval =
+      Duration.ofMinutes(1).toMillis();

Review Comment:
   Why setting 1 minute here when we are defining the config value default as 
300s ?



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.tasks;
+
+
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.scm.ReconScmTask;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.hadoop.ozone.recon.schema.tables.daos.ContainerCountBySizeDao;
+import org.hadoop.ozone.recon.schema.tables.daos.ReconTaskStatusDao;
+import org.hadoop.ozone.recon.schema.tables.pojos.ContainerCountBySize;
+import org.jooq.DSLContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.hadoop.ozone.recon.schema.tables.ContainerCountBySizeTable.CONTAINER_COUNT_BY_SIZE;
+
+
+/**
+ * Class that scans the list of containers and keeps track of container sizes
+ * binned into ranges (1KB, 2Kb..,4MB,.., 1TB,..1PB) to the Recon
+ * containerSize DB.
+ */
+public class ContainerSizeCountTask extends ReconScmTask {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContainerSizeCountTask.class);
+
+  private StorageContainerServiceProvider scmClient;
+  private ContainerManager containerManager;
+  private final long interval;
+  private ContainerCountBySizeDao containerCountBySizeDao;
+  private DSLContext dslContext;
+
+
+  public ContainerSizeCountTask(
+      ContainerManager containerManager,
+      StorageContainerServiceProvider scmClient,
+      ReconTaskStatusDao reconTaskStatusDao,
+      ReconTaskConfig reconTaskConfig,
+      ContainerCountBySizeDao containerCountBySizeDao,
+      DSLContext dslContext) {
+    super(reconTaskStatusDao);
+    this.scmClient = scmClient;
+    this.containerManager = containerManager;
+    this.containerCountBySizeDao = containerCountBySizeDao;
+    this.dslContext = dslContext;
+    interval = reconTaskConfig.getContainerSizeCountTaskInterval().toMillis();
+  }
+
+
+  @Override
+  protected synchronized void run() {
+    try {
+      while (canRun()) {
+        wait(interval);
+
+        final List<ContainerInfo> containers = 
containerManager.getContainers();
+        Map<ContainerSizeCountKey, Long> containerSizeCountMap =
+            new HashMap<>();
+        for (int i = 0; i < containers.size(); i++) {
+          handlePutKeyEvent(containers.get(i), containerSizeCountMap);
+        }
+        // Truncate table before inserting new rows
+        int execute = dslContext.delete(CONTAINER_COUNT_BY_SIZE).execute();
+        LOG.info("Deleted {} records from {}", execute,
+            CONTAINER_COUNT_BY_SIZE);
+
+        writeCountsToDB(true, containerSizeCountMap);
+
+        LOG.info("Completed a 'reprocess' run of ContainerSizeCountTask.");
+      }
+    } catch (Throwable t) {
+      LOG.error("Exception in Container Size Distribution task Thread.", t);
+      if (t instanceof InterruptedException) {
+        Thread.currentThread().interrupt();
+      }
+    }
+  }
+
+  /**
+   * Populate DB with the counts of container sizes calculated
+   * using the dao.
+   */
+  private void writeCountsToDB(boolean isDbTruncated,
+                               Map<ContainerSizeCountKey, Long>
+                                   containerSizeCountMap) {
+    List<ContainerCountBySize> insertToDb = new ArrayList<>();
+
+    containerSizeCountMap.keySet().forEach((ContainerSizeCountKey key) -> {
+      ContainerCountBySize newRecord = new ContainerCountBySize();
+      newRecord.setContainerSize(key.containerSizeUpperBound);
+      newRecord.setCount(containerSizeCountMap.get(key));
+      if (!isDbTruncated) {
+        insertToDb.add(newRecord);
+      }
+    });
+    containerCountBySizeDao.insert(insertToDb);
+  }
+
+  @Override
+  public String getTaskName() {
+    return "ContainerSizeCountTask";
+  }
+
+  /**
+   * Calculate and update the count of containers being tracked by
+   * containerSizeCountMap.
+   * Used by reprocess() and process().

Review Comment:
   Not sure if this comment is true. Don't see any reprocess or process method.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerSizeCountTask.java:
##########
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.tasks;
+
+
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.scm.ReconScmTask;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.hadoop.ozone.recon.schema.tables.daos.ContainerCountBySizeDao;
+import org.hadoop.ozone.recon.schema.tables.daos.ReconTaskStatusDao;
+import org.hadoop.ozone.recon.schema.tables.pojos.ContainerCountBySize;
+import org.jooq.DSLContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.hadoop.ozone.recon.schema.tables.ContainerCountBySizeTable.CONTAINER_COUNT_BY_SIZE;
+
+
+/**
+ * Class that scans the list of containers and keeps track of container sizes
+ * binned into ranges (1KB, 2Kb..,4MB,.., 1TB,..1PB) to the Recon
+ * containerSize DB.
+ */
+public class ContainerSizeCountTask extends ReconScmTask {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContainerSizeCountTask.class);
+
+  private StorageContainerServiceProvider scmClient;
+  private ContainerManager containerManager;
+  private final long interval;
+  private ContainerCountBySizeDao containerCountBySizeDao;
+  private DSLContext dslContext;
+
+
+  public ContainerSizeCountTask(
+      ContainerManager containerManager,
+      StorageContainerServiceProvider scmClient,
+      ReconTaskStatusDao reconTaskStatusDao,
+      ReconTaskConfig reconTaskConfig,
+      ContainerCountBySizeDao containerCountBySizeDao,
+      DSLContext dslContext) {
+    super(reconTaskStatusDao);
+    this.scmClient = scmClient;
+    this.containerManager = containerManager;
+    this.containerCountBySizeDao = containerCountBySizeDao;
+    this.dslContext = dslContext;
+    interval = reconTaskConfig.getContainerSizeCountTaskInterval().toMillis();
+  }
+
+
+  @Override
+  protected synchronized void run() {
+    try {
+      while (canRun()) {
+        wait(interval);
+
+        final List<ContainerInfo> containers = 
containerManager.getContainers();
+        Map<ContainerSizeCountKey, Long> containerSizeCountMap =
+            new HashMap<>();
+        for (int i = 0; i < containers.size(); i++) {
+          handlePutKeyEvent(containers.get(i), containerSizeCountMap);
+        }
+        // Truncate table before inserting new rows
+        int execute = dslContext.delete(CONTAINER_COUNT_BY_SIZE).execute();
+        LOG.info("Deleted {} records from {}", execute,
+            CONTAINER_COUNT_BY_SIZE);
+
+        writeCountsToDB(true, containerSizeCountMap);
+
+        LOG.info("Completed a 'reprocess' run of ContainerSizeCountTask.");

Review Comment:
   Correct this comment.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskConfig.java:
##########
@@ -66,4 +66,22 @@ public void setMissingContainerTaskInterval(Duration 
interval) {
     this.missingContainerTaskInterval = interval.toMillis();
   }
 
+  @Config(key = "containercounttask.interval",
+      type = ConfigType.TIME,
+      defaultValue = "300s",
+      tags = { ConfigTag.RECON, ConfigTag.OZONE },
+      description = "The time interval of the periodic check for " +
+          "container size distribution in the cluster as reported "
+  )
+  private long containerSizeCountTaskInterval =
+      Duration.ofMinutes(1).toMillis();
+
+  public Duration getContainerSizeCountTaskInterval() {
+    return Duration.ofMillis(missingContainerTaskInterval);
+  }
+
+  public void setContainerSizeCountTaskInterval(Duration interval) {
+    this.missingContainerTaskInterval = interval.toMillis();

Review Comment:
   Why we are getting and setting missingContainerTaskInterval ? This is 
different task. 



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