You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "devmadhuu (via GitHub)" <gi...@apache.org> on 2023/07/10 15:27:49 UTC

[GitHub] [ozone] devmadhuu commented on a diff in pull request #5037: HDDS-8627. Recon - API for Count of deletePending directories and amount of data mapped to such directories.

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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java:
##########
@@ -271,14 +327,15 @@ private void handlePutEvent(OMDBUpdateEvent<String, Object> event,
                               Collection<String> sizeRelatedTables,

Review Comment:
   We don't need to pass this argument as "getTablesToCalculateSize" is public method.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java:
##########
@@ -179,6 +200,40 @@ private Triple<Long, Long, Long> getTableSizeAndCount(
     return Triple.of(count, unReplicatedSize, replicatedSize);
   }
 
+  /**
+   * Fetches the size of a deleted directory identified by the given path.
+   * The size is obtained from the NSSummary table using the directory objectID.
+   * The path is expected to be in the format :-
+   * "volumeId/bucketId/parentId/dirName/dirObjectId".
+   *
+   * @param path             The path of the deleted directory.
+   * @return The size of the deleted directory.
+   * @throws IOException If an I/O error occurs while retrieving the size.
+   */
+  public long fetchSizeForDeletedDirectory(String path)
+      throws IOException {
+    if (path == null || path.isEmpty()) {
+      return 0L;
+    }
+    String[] parts = path.split("/");
+    String directoryObjectId = parts.length >= 6 ? parts[5] : "";

Review Comment:
   Can you add a comment why this special figure of 6 for length check ? Because as per key example, dirObjectId is at 4th index itself ?



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java:
##########
@@ -337,21 +407,34 @@ private void handleDeleteEvent(OMDBUpdateEvent<String, Object> event,
 
   private void handleSizeRelatedTableDeleteEvent(
       OMDBUpdateEvent<String, Object> event, String countKey,
-      String unReplicatedSizeKey, String replicatedSizeKey,
+      String unReplicatedSizeKey, String replicatedSizeKey, String tableName,
       HashMap<String, Long> objectCountMap,
       HashMap<String, Long> unreplicatedSizeCountMap,
-      HashMap<String, Long> replicatedSizeCountMap) {
+      HashMap<String, Long> replicatedSizeCountMap) throws IOException {
+
+    Boolean isFileTable = tableName.equals(OPEN_FILE_TABLE);
+    Boolean isKeyTable = tableName.equals(OPEN_KEY_TABLE);
+    Boolean isDeletedDirTable = tableName.equals(DELETED_DIR_TABLE);
+
     if (event.getValue() instanceof OmKeyInfo) {
       // Handle DELETE for OpenKeyTable & OpenFileTable
       OmKeyInfo omKeyInfo = (OmKeyInfo) event.getValue();
       objectCountMap.computeIfPresent(countKey,
           (k, count) -> count > 0 ? count - 1L : 0L);
-      unreplicatedSizeCountMap.computeIfPresent(unReplicatedSizeKey,
-          (k, size) -> size > omKeyInfo.getDataSize() ?
-              size - omKeyInfo.getDataSize() : 0L);
-      replicatedSizeCountMap.computeIfPresent(replicatedSizeKey,
-          (k, size) -> size > omKeyInfo.getReplicatedSize() ?
-              size - omKeyInfo.getReplicatedSize() : 0L);
+      if (isFileTable || isKeyTable) {

Review Comment:
   Try to reduce if else complexity as this method can grow in future for file and key tables also.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java:
##########
@@ -287,18 +344,30 @@ private void handlePutEvent(OMDBUpdateEvent<String, Object> event,
 
   private void handleSizeRelatedTablePutEvent(
       OMDBUpdateEvent<String, Object> event, String countKey,
-      String unReplicatedSizeKey, String replicatedSizeKey,
+      String unReplicatedSizeKey, String replicatedSizeKey, String tableName,
       HashMap<String, Long> objectCountMap,
       HashMap<String, Long> unreplicatedSizeCountMap,
-      HashMap<String, Long> replicatedSizeCountMap) {
+      HashMap<String, Long> replicatedSizeCountMap) throws IOException {
+
+    boolean isFileTable = tableName.equals(OPEN_FILE_TABLE);
+    boolean isKeyTable = tableName.equals(OPEN_KEY_TABLE);
+    boolean isDeletedDirTable = tableName.equals(DELETED_DIR_TABLE);
+
     // Handle PUT for OpenKeyTable & OpenFileTable
     if (event.getValue() instanceof OmKeyInfo) {
       OmKeyInfo omKeyInfo = (OmKeyInfo) event.getValue();
       objectCountMap.computeIfPresent(countKey, (k, count) -> count + 1L);
-      unreplicatedSizeCountMap.computeIfPresent(unReplicatedSizeKey,
-          (k, size) -> size + omKeyInfo.getDataSize());
-      replicatedSizeCountMap.computeIfPresent(replicatedSizeKey,
-          (k, size) -> size + omKeyInfo.getReplicatedSize());
+      if (isFileTable || isKeyTable) {

Review Comment:
   We can reduce if else complexity here, as we know that size of tables are meant based on files/keys which can be computed later at bucket or volume level as well or can be computed for file and key table as well , try to keep the code generic as much as possible.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java:
##########
@@ -287,18 +344,30 @@ private void handlePutEvent(OMDBUpdateEvent<String, Object> event,
 
   private void handleSizeRelatedTablePutEvent(
       OMDBUpdateEvent<String, Object> event, String countKey,
-      String unReplicatedSizeKey, String replicatedSizeKey,
+      String unReplicatedSizeKey, String replicatedSizeKey, String tableName,
       HashMap<String, Long> objectCountMap,
       HashMap<String, Long> unreplicatedSizeCountMap,
-      HashMap<String, Long> replicatedSizeCountMap) {
+      HashMap<String, Long> replicatedSizeCountMap) throws IOException {
+
+    boolean isFileTable = tableName.equals(OPEN_FILE_TABLE);

Review Comment:
   variable name is confusing as these are for openfile and openkey tables, though these might not be needed if we can control at the caller of this method itself.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OmTableInsightTask.java:
##########
@@ -139,28 +150,38 @@ public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
    * Returns a triple with the total count of records (left), total unreplicated
    * size (middle), and total replicated size (right) in the given iterator.
    * Increments count for each record and adds the dataSize if a record's value
-   * is an instance of OmKeyInfo. If the iterator is null, returns (0, 0, 0).
+   * is an instance of OmKeyInfo,RepeatedOmKeyInfo.
+   * If the iterator is null, returns (0, 0, 0).
    *
    * @param iterator The iterator over the table to be iterated.
    * @return A Triple with three Long values representing the count,
    *         unreplicated size and replicated size.
    * @throws IOException If an I/O error occurs during the iterator traversal.
    */
   private Triple<Long, Long, Long> getTableSizeAndCount(
-      TableIterator<String, ? extends Table.KeyValue<String, ?>> iterator)
+      TableIterator<String, ? extends Table.KeyValue<String, ?>> iterator,

Review Comment:
   @ArafatKhan2198  thanks for working on this patch, can you pls elaborate the purpose of this method since you added tableName as extra parameter now. Is this method only for counting records of tables which are having keys as OMKeyInfo and RepeatedOmKeyInfo only ?. Also if GlobalStatsTable will not have counts of OpenFile and OpenKey table ?



-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org