You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/09/09 23:06:28 UTC

[GitHub] [ozone] xBis7 opened a new pull request, #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

xBis7 opened a new pull request, #3746:
URL: https://github.com/apache/ozone/pull/3746

   ## What changes were proposed in this pull request?
   
   In the previous PR [HDDS-5504](https://github.com/apache/ozone/pull/3309), we refactored `NSSummaryEndpoint` and `NSSummaryTask` so that we can extract the bucket specific code to new separated classes. So far, Recon has only been supporting FSO buckets. All the existing bucket related code exists in `handlers/FSOBucketHandler` and `tasks/NSSummaryTaskWithFSO`. In order to add support for Legacy buckets, we created two new classes `handlers/LegacyBucketHandler` and `tasks/NSSummaryTaskWithLegacy`.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-7121
   
   ## How was this patch tested?
   
   This patch was tested with new unit tests. We added two new test classes, `TestNSSummaryEndpointWithLegacy` and `TestNSSummaryTaskWithLegacy`. These two classes are as similar as possible to the corresponding FSO classes. We tried to keep the setups exactly the same with `TestNSSummaryEndpointWithFSO` and `TestNSSummaryTaskWithFSO` as if they were handling FSO related code. 


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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r979138926


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/LegacyBucketHandler.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.api.handlers;
+
+import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+
+/**
+ * Class for handling Legacy buckets.
+ */
+public class LegacyBucketHandler extends BucketHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      LegacyBucketHandler.class);
+
+  private final String vol;
+  private final String bucket;
+  private final OmBucketInfo omBucketInfo;
+
+  public LegacyBucketHandler(
+      ReconNamespaceSummaryManager reconNamespaceSummaryManager,
+      ReconOMMetadataManager omMetadataManager,
+      OzoneStorageContainerManager reconSCM,
+      OmBucketInfo bucketInfo) {
+    super(reconNamespaceSummaryManager, omMetadataManager,
+        reconSCM);
+    this.omBucketInfo = bucketInfo;
+    this.vol = omBucketInfo.getVolumeName();
+    this.bucket = omBucketInfo.getBucketName();
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  @Override
+  public EntityType determineKeyPath(String keyName)
+      throws IOException {
+
+    String filename = OzoneFSUtils.removeTrailingSlashIfNeeded(keyName);
+    // For example, /vol1/buck1/a/b/c/d/e/file1.txt
+    // Look in the KeyTable for the key path,
+    // if the first one we seek to is the same as the seek key,
+    // it is a key;
+    // if it is the seekKey with a trailing slash, it is a directory
+    // else it is unknown
+    String key = OM_KEY_PREFIX + vol +
+        OM_KEY_PREFIX + bucket +
+        OM_KEY_PREFIX + filename;
+
+    Table<String, OmKeyInfo> keyTable = getKeyTable();
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+        iterator = keyTable.iterator();
+
+    iterator.seek(key);
+    if (iterator.hasNext()) {
+      Table.KeyValue<String, OmKeyInfo> kv = iterator.next();
+      String dbKey = kv.getKey();
+      if (dbKey.equals(key)) {
+        return EntityType.KEY;
+      }
+      if (dbKey.equals(key + OM_KEY_PREFIX)) {
+        return EntityType.DIRECTORY;
+      }
+    }
+    return EntityType.UNKNOWN;
+  }
+
+  // KeyTable's key is in the format of "vol/bucket/keyName"
+  // Make use of RocksDB's order to seek to the prefix and avoid full iteration
+  @Override
+  public long calculateDUUnderObject(long parentId)
+      throws IOException {
+    Table<String, OmKeyInfo> keyTable = getKeyTable();
+
+    long totalDU = 0L;
+    TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+        iterator = keyTable.iterator();
+
+    String seekPrefix = OM_KEY_PREFIX +
+        vol +
+        OM_KEY_PREFIX +
+        bucket +
+        OM_KEY_PREFIX;
+
+    NSSummary nsSummary = getReconNamespaceSummaryManager()
+        .getNSSummary(parentId);
+    // empty bucket
+    if (nsSummary == null) {
+      return 0;
+    }
+
+    if (omBucketInfo.getObjectID() != parentId) {
+      String dirName = nsSummary.getDirName();
+      seekPrefix += dirName;
+    }
+
+    String[] seekKeys = seekPrefix.split(OM_KEY_PREFIX);
+    iterator.seek(seekPrefix);
+    // handle direct keys
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmKeyInfo> kv = iterator.next();
+      String dbKey = kv.getKey();
+      // since the RocksDB is ordered, seek until the prefix isn't matched
+      if (!dbKey.startsWith(seekPrefix)) {
+        break;
+      }
+
+      String[] keys = dbKey.split(OM_KEY_PREFIX);
+
+      // iteration moved to the next level
+      // and not handling direct keys
+      if (keys.length - seekKeys.length > 1) {
+        continue;
+      }
+
+      OmKeyInfo keyInfo = kv.getValue();
+      if (keyInfo != null) {
+        // skip directory markers, just include directKeys
+        if (keyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+          continue;
+        }
+        totalDU += getKeySizeWithReplication(keyInfo);
+      }
+    }
+
+    // handle nested keys (DFS)
+    Set<Long> subDirIds = nsSummary.getChildDir();
+    for (long subDirId: subDirIds) {
+      totalDU += calculateDUUnderObject(subDirId);
+    }
+    return totalDU;
+  }
+
+  /**
+   * This method handles disk usage of direct keys.
+   * @param parentId parent directory/bucket
+   * @param withReplica if withReplica is enabled, set sizeWithReplica
+   * for each direct key's DU
+   * @param listFile if listFile is enabled, append key DU as a subpath
+   * @param duData the current DU data
+   * @param normalizedPath the normalized path request
+   * @return the total DU of all direct keys
+   * @throws IOException IOE
+   */
+  @Override
+  public long handleDirectKeys(long parentId, boolean withReplica,
+                               boolean listFile,
+                               List<DUResponse.DiskUsage> duData,
+                               String normalizedPath) throws IOException {
+
+    Table<String, OmKeyInfo> keyTable = getKeyTable();
+    long keyDataSizeWithReplica = 0L;
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+        iterator = keyTable.iterator();
+
+    String seekPrefix = OM_KEY_PREFIX +
+        vol +
+        OM_KEY_PREFIX +
+        bucket +
+        OM_KEY_PREFIX;
+
+    NSSummary nsSummary = getReconNamespaceSummaryManager()
+        .getNSSummary(parentId);
+    // empty bucket
+    if (nsSummary == null) {
+      return 0;
+    }
+
+    if (omBucketInfo.getObjectID() != parentId) {
+      String dirName = nsSummary.getDirName();
+      seekPrefix += dirName;
+    }
+    String[] seekKeys = seekPrefix.split(OM_KEY_PREFIX);
+    iterator.seek(seekPrefix);
+
+    while (iterator.hasNext()) {
+      Table.KeyValue<String, OmKeyInfo> kv = iterator.next();
+      String dbKey = kv.getKey();
+
+      if (!dbKey.startsWith(seekPrefix)) {
+        break;
+      }
+
+      String[] keys = dbKey.split(OM_KEY_PREFIX);
+
+      // iteration moved to the next level
+      // and not handling direct keys
+      if (keys.length - seekKeys.length > 1) {
+        continue;
+      }
+
+      OmKeyInfo keyInfo = kv.getValue();
+      if (keyInfo != null) {
+        // skip directory markers, just include directKeys
+        if (keyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+          continue;
+        }
+        DUResponse.DiskUsage diskUsage = new DUResponse.DiskUsage();
+        String subpath = buildSubpath(normalizedPath,
+            keyInfo.getFileName());
+        diskUsage.setSubpath(subpath);
+        diskUsage.setKey(true);
+        diskUsage.setSize(keyInfo.getDataSize());
+
+        if (withReplica) {
+          long keyDU = getKeySizeWithReplication(keyInfo);
+          keyDataSizeWithReplica += keyDU;
+          diskUsage.setSizeWithReplica(keyDU);
+        }
+        // list the key as a subpath
+        if (listFile) {
+          duData.add(diskUsage);
+        }
+      }
+    }
+
+    return keyDataSizeWithReplica;
+  }
+
+  /**
+   * Given a valid path request for a directory,
+   * return the directory object ID.
+   * @param names parsed path request in a list of names
+   * @return directory object ID
+   */
+  @Override
+  public long getDirObjectId(String[] names) throws IOException {
+    return getDirObjectId(names, names.length);
+  }
+
+  /**
+   * Given a valid path request and a cutoff length where should be iterated
+   * up to.
+   * return the directory object ID for the object at the cutoff length
+   * @param names parsed path request in a list of names
+   * @param cutoff cannot be larger than the names' length. If equals,
+   *               return the directory object id for the whole path
+   * @return directory object ID
+   */
+  @Override
+  public long getDirObjectId(String[] names, int cutoff) throws IOException {
+    long dirObjectId = getBucketObjectId(names);
+    StringBuilder bld = new StringBuilder();
+    for (int i = 0; i < cutoff; ++i) {
+      bld.append(OM_KEY_PREFIX)
+          .append(names[i]);
+    }
+    bld.append(OM_KEY_PREFIX);
+    String dirKey = bld.toString();
+    OmKeyInfo dirInfo = getKeyTable().getSkipCache(dirKey);
+
+    try {
+      dirObjectId = dirInfo.getObjectID();
+    } catch (NullPointerException e) {

Review Comment:
   why is the exception being swallowed?  it will cause the bucketid to be returned.



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r979139397


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java:
##########
@@ -183,8 +183,17 @@ public static BucketHandler getBucketHandler(
                 OzoneStorageContainerManager reconSCM,
                 OmBucketInfo bucketInfo) throws IOException {
 
-    return new FSOBucketHandler(reconNamespaceSummaryManager,
+    if (bucketInfo.getBucketLayout()

Review Comment:
    This version of getBucketHandler will throw an NPE if bucketInfo is null.  The other version just returns [null](https://github.com/apache/ozone/blob/39c0c25c07e359a1e5f0968a2143db1dc9da3c19/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java#L210).  They should do the same thing, probably both throw.
   



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999367755


##########
hadoop-ozone/dist/src/main/smoketest/recon/recon-nssummary.robot:
##########
@@ -131,7 +132,8 @@ Check Recon Namespace Summary Key
     Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/file1   KEY
 
 Check Recon Namespace Summary Directory
-    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2   DIRECTORY
+    Run Keyword If    '${BUCKET_LAYOUT}' == 'LEGACY'                    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2/   DIRECTORY

Review Comment:
   @smengcl  @dombizita The trailing slash is necessary. Without it, we get `PATH_NOT_FOUND` and entity `UNKNOWN`.
   
   > the command after this line 135 will only be executed when '${BUCKET_LAYOUT}' == 'LEGACY'.
   If bucket layout is FSO, it will run the command line below instead, with the subtle difference of the trailing slash in the directory path.
   
   You are right. That's how it works.
   
   We give `${BUCKET_LAYOUT}` a default value of FSO and [here](https://github.com/xBis7/ozone/blob/HDDS-7121/hadoop-ozone/dist/src/main/compose/ozone-legacy/test.sh#L31) we execute the smoketest with value LEGACY.
   



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


[GitHub] [ozone] xBis7 commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1254029928

   @dombizita @smengcl Another solution to this, would be to have a common NSSummary table for both bucket layouts. Instead of calling both tasks, we could add `process` and `reprocess` methods to `NSSummaryTask` and call that class in place of the subclasses. `NSSummaryTask.process` will call both `NSSummaryTaskWithFSO.process` and `NSSummaryTaskWithLegacy.process` and we will do the same for `reprocess`. `NSSummaryTask` will also manage common setups or operations for NSSummary table, such as clearing it. What do you think about this approach?


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999982173


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -44,166 +49,97 @@
  * For dirTable, we need to fetch the parent object (bucket or directory),
  * add the current directory's objectID to the parent object's childDir field.
  *
+ * For keyTable, the parent object is not available. Get the parent object,
+ * add it to the current object and reuse the existing methods for FSO.
+ * Only processing entries that belong to Legacy buckets. If the entry
+ * refers to a directory then build directory info object from it.
+ *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
   private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private final ReconOMMetadataManager reconOMMetadataManager;
+  private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
+  private final OzoneConfiguration ozoneConfiguration;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager,
+                       OzoneConfiguration
+                             ozoneConfiguration) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.ozoneConfiguration = ozoneConfiguration;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager,
+        reconOMMetadataManager, ozoneConfiguration);
   }
 
-  public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
-    return reconNamespaceSummaryManager;
-  }
-
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
-  }
-
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    boolean success;
+    success = nsSummaryTaskWithFSO.processWithFSO(events);
+    if (success) {
+      success = nsSummaryTaskWithLegacy.processWithLegacy(events);
     }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-
-    long dataSize = keyInfo.getDataSize();
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    // decrement count, data size, and bucket count
-    // even if there's no direct key, we still keep the entry because
-    // we still need children dir IDs info
-    nsSummary.setNumOfFiles(numOfFile - 1);
-    nsSummary.setSizeOfFiles(sizeOfFile - dataSize);
-    --fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+    return new ImmutablePair<>(getTaskName(), success);
   }
 
-  protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    Collection<Callable<Boolean>> tasks = new ArrayList<>();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.clearNSSummaryTable();
+    } catch (IOException ioEx) {
+      LOG.error("Unable to clear NSSummary table in Recon DB. ",
+          ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
     }
 
-    // Just in case the OmDirectoryInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+    tasks.add(() -> nsSummaryTaskWithFSO
+        .reprocessWithFSO(omMetadataManager));
+    tasks.add(() -> nsSummaryTaskWithLegacy
+        .reprocessWithLegacy(reconOMMetadataManager));
+
+    List<Future<Boolean>> results;
+    ExecutorService executorService = Executors
+        .newFixedThreadPool(2);
+    try {
+      results = executorService.invokeAll(tasks);

Review Comment:
   There should be no issues. We clear the NSSummary table only once and then we invoke the tasks and let them run asynchronously.



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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r1000895186


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/nssummary/NSSummaryAdmin.java:
##########
@@ -108,6 +110,60 @@ public boolean isFileSystemOptimizedBucket(String path) throws IOException {
     }
   }
 
+  public boolean isOBSBucket(String path) throws IOException {
+    OFSPath ofsPath = new OFSPath(path);
+
+    boolean enableFileSystemPaths = getOzoneConfig()
+        .get(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");

Review Comment:
   Thanks for the change @xBis7 !
   
   Though my bad I just realized we should probably straight use `.getBoolean()` with the default value, like this:
   
   https://github.com/apache/ozone/blob/c0de6bcdebee6a4271fc73ea6542902e0eeee28f/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L207-L209
   
   Otherwise when the config key is not explicitly set the `.get()` result could be `null`.



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r981037801


##########
hadoop-ozone/dist/src/main/smoketest/recon/recon-nssummary.robot:
##########
@@ -14,7 +14,7 @@
 # limitations under the License.
 
 *** Settings ***
-Documentation       Smoke test for Recon Namespace Summary Endpoint for FSO buckets.
+Documentation       Smoke test for Recon Namespace Summary Endpoint for FSO and Legacy buckets.

Review Comment:
   @dombizita Apart from the Namespace Summary for a directory there is no difference between FSO and LEGACY. In order to avoid duplication, we added the `BUCKET_LAYOUT` as a parameter and gave it the default value of FSO. The idea is to add an extra command to `test.sh` to also run the test with a parameter `-v BUCKET_LAYOUT:LEGACY`.



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999367755


##########
hadoop-ozone/dist/src/main/smoketest/recon/recon-nssummary.robot:
##########
@@ -131,7 +132,8 @@ Check Recon Namespace Summary Key
     Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/file1   KEY
 
 Check Recon Namespace Summary Directory
-    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2   DIRECTORY
+    Run Keyword If    '${BUCKET_LAYOUT}' == 'LEGACY'                    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2/   DIRECTORY

Review Comment:
   The trailing slash is necessary. Without it, we get `PATH_NOT_FOUND` and entity `UNKNOWN`.
   
   > the command after this line 135 will only be executed when '${BUCKET_LAYOUT}' == 'LEGACY'.
   If bucket layout is FSO, it will run the command line below instead, with the subtle difference of the trailing slash in the directory path.
   
   You are right. That's how it works.
   
   We give `${BUCKET_LAYOUT}` a default value of FSO and [here](https://github.com/xBis7/ozone/blob/HDDS-7121/hadoop-ozone/dist/src/main/compose/ozone-legacy/test.sh#L31) we execute the smoketest with value LEGACY.
   



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999380802


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java:
##########
@@ -183,8 +185,22 @@ public static BucketHandler getBucketHandler(
                 OzoneStorageContainerManager reconSCM,
                 OmBucketInfo bucketInfo) throws IOException {
 
-    return new FSOBucketHandler(reconNamespaceSummaryManager,
-              omMetadataManager, reconSCM, bucketInfo);
+    // If bucketInfo is null then entity type is UNKNOWN
+    if (Objects.isNull(bucketInfo)) {
+      return null;
+    } else {
+      if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.FILE_SYSTEM_OPTIMIZED)) {
+        return new FSOBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.LEGACY)) {
+        return new LegacyBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else {
+        throw new RuntimeException("Unsupported bucket layout.");

Review Comment:
   > Could it crash the Recon thread / Recon process?
   
   Probably... This exception will come from hitting an endpoint in `NSSummaryEndpoint`. We could wrap it with a try catch and return a `Response.serverError()` in the catch block. What do you think?



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


[GitHub] [ozone] GeorgeJahad commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1248704754

   Also because the two tasks are writing to the same table from separate threads, it would be could to clarify why no synchronization is needed.


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984672750


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -45,165 +41,71 @@
  * add the current directory's objectID to the parent object's childDir field.
  *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
-  private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private ReconOMMetadataManager reconOMMetadataManager;
+  private NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
   }
 
   public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
     return reconNamespaceSummaryManager;
   }
 
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  public ReconOMMetadataManager getReconOMMetadataManager() {
+    return reconOMMetadataManager;
   }
 
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {

Review Comment:
   okay, I suppose this will be under a new class, `TestNSSummaryTask`.



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984041570


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java:
##########
@@ -183,8 +183,17 @@ public static BucketHandler getBucketHandler(
                 OzoneStorageContainerManager reconSCM,
                 OmBucketInfo bucketInfo) throws IOException {
 
-    return new FSOBucketHandler(reconNamespaceSummaryManager,
+    if (bucketInfo.getBucketLayout()

Review Comment:
   It's usually good practice to keep the methods consistent.  If the null check is required, it would be better to put it in the one that is called by this one, so they both have it.



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


[GitHub] [ozone] dombizita commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
dombizita commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1255387336

   If we have two separate tables for `NSSummary`s based on the bucket layout (legacy and FSO) we can iterate through both tables right after each other if there is a task that needs information from both types. I think this approach would be nice to avoid issues as we already separated the classes based on the bucket layout.
   But as I looked into it for both bucket types we would maintain a table with the same columns, as we add long value (ID) and a `NSSummary` in both cases. Based on this it would be better to avoid having two same tables.
   Your approach sounds good to me, if I understand correctly the `NSSummaryTask.process` would call eg. first `NSSummaryTaskWithFSO.process` and after the `NSSummaryTaskWithLegacy.process`, right? 


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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984035691


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/DirectoryEntityHandler.java:
##########
@@ -89,7 +91,15 @@ public DUResponse getDuResponse(
     for (long subdirObjectId: subdirs) {
       NSSummary subdirNSSummary =
               getReconNamespaceSummaryManager().getNSSummary(subdirObjectId);
-      String subdirName = subdirNSSummary.getDirName();
+      // for the subdirName we need the filename, not the key name
+      Path subdirPath = Paths.get(subdirNSSummary.getDirName());
+      Path subdirFileName = subdirPath.getFileName();
+      String subdirName;
+      if (subdirFileName != null) {
+        subdirName = subdirFileName.toString();
+      } else {
+        throw new NullPointerException("Subdirectory file name is null.");
+      }

Review Comment:
   So then why test for null and then do an explicit throw?  won't the throw happen even if you don't check for it?



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984119386


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -45,165 +41,71 @@
  * add the current directory's objectID to the parent object's childDir field.
  *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
-  private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private ReconOMMetadataManager reconOMMetadataManager;
+  private NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
   }
 
   public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
     return reconNamespaceSummaryManager;
   }
 
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  public ReconOMMetadataManager getReconOMMetadataManager() {
+    return reconOMMetadataManager;
   }
 
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    boolean success;
+    success = nsSummaryTaskWithFSO.processWithFSO(events);
+    if (success) {
+      success = nsSummaryTaskWithLegacy.processWithLegacy(events);
     }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-
-    long dataSize = keyInfo.getDataSize();
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    // decrement count, data size, and bucket count
-    // even if there's no direct key, we still keep the entry because
-    // we still need children dir IDs info
-    nsSummary.setNumOfFiles(numOfFile - 1);
-    nsSummary.setSizeOfFiles(sizeOfFile - dataSize);
-    --fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+    return new ImmutablePair<>(getTaskName(), success);
   }
 
-  protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    boolean success;
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.clearNSSummaryTable();
+    } catch (IOException ioEx) {
+      LOG.error("Unable to clear NSSummary table in Recon DB. ",
+          ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
     }
-
-    // Just in case the OmDirectoryInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+    success = nsSummaryTaskWithFSO.reprocessWithFSO(omMetadataManager);

Review Comment:
   same as above



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r981038815


##########
hadoop-ozone/dist/src/main/smoketest/recon/recon-nssummary.robot:
##########
@@ -131,7 +132,8 @@ Check Recon Namespace Summary Key
     Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/file1   KEY
 
 Check Recon Namespace Summary Directory
-    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2   DIRECTORY
+    Run Keyword If    '${BUCKET_LAYOUT}' == 'LEGACY'                    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2/   DIRECTORY

Review Comment:
   After running the test with the parameter `-v BUCKET_LAYOUT:LEGACY`, here it will execute the part for LEGACY.



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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r1000895186


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/nssummary/NSSummaryAdmin.java:
##########
@@ -108,6 +110,60 @@ public boolean isFileSystemOptimizedBucket(String path) throws IOException {
     }
   }
 
+  public boolean isOBSBucket(String path) throws IOException {
+    OFSPath ofsPath = new OFSPath(path);
+
+    boolean enableFileSystemPaths = getOzoneConfig()
+        .get(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");

Review Comment:
   Thanks for the change @xBis7 !
   
   Though my bad I just realized we should probably straight use `.getBoolean()` with the default value, like this:
   
   https://github.com/apache/ozone/blob/c0de6bcdebee6a4271fc73ea6542902e0eeee28f/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L207-L209



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


[GitHub] [ozone] smengcl commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1290997052

   Merged. Thanks @xBis7 for the work! Thanks @GeorgeJahad and @dombizita for the review.


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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999877695


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/OMMetadataManagerTestUtils.java:
##########
@@ -256,6 +255,28 @@ public static void writeKeyToOm(OMMetadataManager omMetadataManager,
                     .build());
   }
 
+  @SuppressWarnings("checkstyle:parameternumber")
+  /**
+   * Write a directory as key on OM instance.
+   * We don't need to set size.
+   * @throws IOException
+   */

Review Comment:
   Sounds good. Thanks!



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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r1000895186


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/nssummary/NSSummaryAdmin.java:
##########
@@ -108,6 +110,60 @@ public boolean isFileSystemOptimizedBucket(String path) throws IOException {
     }
   }
 
+  public boolean isOBSBucket(String path) throws IOException {
+    OFSPath ofsPath = new OFSPath(path);
+
+    boolean enableFileSystemPaths = getOzoneConfig()
+        .get(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");

Review Comment:
   Thanks for the change @xBis7 !
   
   Though my bad I just realized we should probably straight use `.getBoolean()` with the default value, like this:
   
   https://github.com/apache/ozone/blob/c0de6bcdebee6a4271fc73ea6542902e0eeee28f/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L207-L209
   
   ~~Otherwise when the config key is not explicitly set the `.get()` result could be `null`.~~ Looks like the default value is populated into the map so at least we won't get NPE. But using `.getBoolean()` could still be the cleaner solution so we won't have to worry about the letter case.



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999459701


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/OMMetadataManagerTestUtils.java:
##########
@@ -256,6 +255,28 @@ public static void writeKeyToOm(OMMetadataManager omMetadataManager,
                     .build());
   }
 
+  @SuppressWarnings("checkstyle:parameternumber")
+  /**
+   * Write a directory as key on OM instance.
+   * We don't need to set size.
+   * @throws IOException
+   */

Review Comment:
   The annotation was placed before the javadoc, because that is also done above in the method `writeKeyToOm`. I will move that annotation as well.



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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r1000013576


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -44,166 +49,97 @@
  * For dirTable, we need to fetch the parent object (bucket or directory),
  * add the current directory's objectID to the parent object's childDir field.
  *
+ * For keyTable, the parent object is not available. Get the parent object,
+ * add it to the current object and reuse the existing methods for FSO.
+ * Only processing entries that belong to Legacy buckets. If the entry
+ * refers to a directory then build directory info object from it.
+ *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
   private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private final ReconOMMetadataManager reconOMMetadataManager;
+  private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
+  private final OzoneConfiguration ozoneConfiguration;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager,
+                       OzoneConfiguration
+                             ozoneConfiguration) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.ozoneConfiguration = ozoneConfiguration;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager,
+        reconOMMetadataManager, ozoneConfiguration);
   }
 
-  public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
-    return reconNamespaceSummaryManager;
-  }
-
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
-  }
-
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    boolean success;
+    success = nsSummaryTaskWithFSO.processWithFSO(events);
+    if (success) {
+      success = nsSummaryTaskWithLegacy.processWithLegacy(events);
     }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-
-    long dataSize = keyInfo.getDataSize();
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    // decrement count, data size, and bucket count
-    // even if there's no direct key, we still keep the entry because
-    // we still need children dir IDs info
-    nsSummary.setNumOfFiles(numOfFile - 1);
-    nsSummary.setSizeOfFiles(sizeOfFile - dataSize);
-    --fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+    return new ImmutablePair<>(getTaskName(), success);
   }
 
-  protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    Collection<Callable<Boolean>> tasks = new ArrayList<>();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.clearNSSummaryTable();
+    } catch (IOException ioEx) {
+      LOG.error("Unable to clear NSSummary table in Recon DB. ",
+          ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
     }
 
-    // Just in case the OmDirectoryInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+    tasks.add(() -> nsSummaryTaskWithFSO
+        .reprocessWithFSO(omMetadataManager));
+    tasks.add(() -> nsSummaryTaskWithLegacy
+        .reprocessWithLegacy(reconOMMetadataManager));
+
+    List<Future<Boolean>> results;
+    ExecutorService executorService = Executors
+        .newFixedThreadPool(2);
+    try {
+      results = executorService.invokeAll(tasks);

Review Comment:
   Sounds good. Thanks!



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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r1000013971


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java:
##########
@@ -183,8 +185,22 @@ public static BucketHandler getBucketHandler(
                 OzoneStorageContainerManager reconSCM,
                 OmBucketInfo bucketInfo) throws IOException {
 
-    return new FSOBucketHandler(reconNamespaceSummaryManager,
-              omMetadataManager, reconSCM, bucketInfo);
+    // If bucketInfo is null then entity type is UNKNOWN
+    if (Objects.isNull(bucketInfo)) {
+      return null;
+    } else {
+      if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.FILE_SYSTEM_OPTIMIZED)) {
+        return new FSOBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.LEGACY)) {
+        return new LegacyBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else {
+        throw new RuntimeException("Unsupported bucket layout.");

Review Comment:
   Okay let's just do `LOG.error` then. No need to throw unchecked exception here.



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984119772


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -45,165 +41,71 @@
  * add the current directory's objectID to the parent object's childDir field.
  *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
-  private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private ReconOMMetadataManager reconOMMetadataManager;
+  private NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
   }
 
   public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
     return reconNamespaceSummaryManager;
   }
 
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  public ReconOMMetadataManager getReconOMMetadataManager() {
+    return reconOMMetadataManager;
   }
 
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {

Review Comment:
   Also, please sure the test coverage results



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -45,165 +41,71 @@
  * add the current directory's objectID to the parent object's childDir field.
  *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
-  private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private ReconOMMetadataManager reconOMMetadataManager;
+  private NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
   }
 
   public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
     return reconNamespaceSummaryManager;
   }
 
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  public ReconOMMetadataManager getReconOMMetadataManager() {
+    return reconOMMetadataManager;
   }
 
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {

Review Comment:
   Also, please share the test coverage results



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


[GitHub] [ozone] xBis7 commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1258020743

   @dombizita I've made the changes so that `NSSummaryTask` gets called in place of `NSSummaryTaskWithFSO` and `NSSummaryTaskWithLegacy`, like we discussed. Let me know how it looks. I've also modified the tests to go only over each subclass and not `NSSummaryTask`. If we decide to move forward with this approach I will cleanup the code and add a new class `TestNSSummaryTask` that will test the case of having buckets of different layouts. 


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


[GitHub] [ozone] xBis7 commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1284099287

   @smengcl thanks for the review. I 've addressed all your comments except [this one](https://github.com/apache/ozone/pull/3746#discussion_r998884262). Do you want to do something about the RunTimeException or do you want to leave prolong and crash the thread?


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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999818846


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java:
##########
@@ -183,8 +185,22 @@ public static BucketHandler getBucketHandler(
                 OzoneStorageContainerManager reconSCM,
                 OmBucketInfo bucketInfo) throws IOException {
 
-    return new FSOBucketHandler(reconNamespaceSummaryManager,
-              omMetadataManager, reconSCM, bucketInfo);
+    // If bucketInfo is null then entity type is UNKNOWN
+    if (Objects.isNull(bucketInfo)) {
+      return null;
+    } else {
+      if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.FILE_SYSTEM_OPTIMIZED)) {
+        return new FSOBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.LEGACY)) {
+        return new LegacyBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else {
+        throw new RuntimeException("Unsupported bucket layout.");

Review Comment:
   Yup let's return an error instead of potentially crashing Recon



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


[GitHub] [ozone] xBis7 commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1253892551

   Hi @dombizita, thanks for the feedback. With two separate tables, what happens when Recon gets statistics for both bucket layouts? Do we iterate both tables and sum the results?


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r979891871


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java:
##########
@@ -183,8 +183,17 @@ public static BucketHandler getBucketHandler(
                 OzoneStorageContainerManager reconSCM,
                 OmBucketInfo bucketInfo) throws IOException {
 
-    return new FSOBucketHandler(reconNamespaceSummaryManager,
+    if (bucketInfo.getBucketLayout()

Review Comment:
   We need the other version to return null, so that we know that the BucketHandler doesn't exist and later return `EntityType.UNKNOWN`. See [here](https://github.com/xBis7/ozone/blob/HDDS-7121/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/EntityHandler.java#L149-L158).



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984118556


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java:
##########
@@ -0,0 +1,1286 @@
+/*
+ * 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.api;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconTestInjector;
+import org.apache.hadoop.ozone.recon.api.handlers.BucketHandler;
+import org.apache.hadoop.ozone.recon.api.handlers.EntityHandler;
+import org.apache.hadoop.ozone.recon.api.types.NamespaceSummaryResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl;
+import org.apache.hadoop.ozone.recon.spi.impl.StorageContainerServiceProviderImpl;
+import org.apache.hadoop.ozone.recon.tasks.NSSummaryTaskWithLegacy;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import javax.ws.rs.core.Response;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.HashSet;
+
+import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_DB_DIRS;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeDirToOm;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProvider;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test for NSSummary REST APIs with Legacy.
+ * We tested on a mini file system with the following setting:
+ *                vol
+ *             /       \
+ *        bucket1      bucket2
+ *        /    \         /    \
+ *     file1    dir1    file4  file5
+ *           /   \   \
+ *        dir2  dir3  dir4
+ *         /     \      \
+ *       file2   file3  file6
+ *  ----------------------------------------
+ *                  vol2
+ *              /         \
+ *      bucket3          bucket4
+ *      /      \           /
+ *   file8     dir5      file11
+ *            /    \
+ *        file9    file10
+ * This is a test for the Rest APIs only. We have tested NSSummaryTask before,
+ * so there is no need to test process() on DB's updates
+ */
+public class TestNSSummaryEndpointWithLegacy {
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private ReconOMMetadataManager reconOMMetadataManager;
+  private NSSummaryEndpoint nsSummaryEndpoint;
+
+  private static final String TEST_PATH_UTILITY =
+      "/vol1/buck1/a/b/c/d/e/file1.txt";
+  private static final String PARENT_DIR = "vol1/buck1/a/b/c/d/e";
+  private static final String[] TEST_NAMES =
+      new String[]{"vol1", "buck1", "a", "b", "c", "d", "e", "file1.txt"};
+  private static final String TEST_KEY_NAMES = "a/b/c/d/e/file1.txt";
+
+  // Object names
+  private static final String VOL = "vol";
+  private static final String VOL_TWO = "vol2";
+  private static final String BUCKET_ONE = "bucket1";
+  private static final String BUCKET_TWO = "bucket2";
+  private static final String BUCKET_THREE = "bucket3";
+  private static final String BUCKET_FOUR = "bucket4";
+  private static final String KEY_ONE = "file1";
+  private static final String KEY_TWO = "dir1/dir2/file2";
+  private static final String KEY_THREE = "dir1/dir3/file3";
+  private static final String KEY_FOUR = "file4";
+  private static final String KEY_FIVE = "file5";
+  private static final String KEY_SIX = "dir1/dir4/file6";
+  private static final String KEY_SEVEN = "dir1/file7";
+  private static final String KEY_EIGHT = "file8";
+  private static final String KEY_NINE = "dir5/file9";
+  private static final String KEY_TEN = "dir5/file10";
+  private static final String KEY_ELEVEN = "file11";
+  private static final String MULTI_BLOCK_KEY = "dir1/file7";
+  private static final String MULTI_BLOCK_FILE = "file7";
+
+  private static final String FILE_ONE = "file1";
+  private static final String FILE_TWO = "file2";
+  private static final String FILE_THREE = "file3";
+  private static final String FILE_FOUR = "file4";
+  private static final String FILE_FIVE = "file5";
+  private static final String FILE_SIX = "file6";
+  private static final String FILE_SEVEN = "file7";
+  private static final String FILE_EIGHT = "file8";
+  private static final String FILE_NINE = "file9";
+  private static final String FILE_TEN = "file10";
+  private static final String FILE_ELEVEN = "file11";
+
+  private static final String DIR_ONE = "dir1";
+  private static final String DIR_TWO = "dir2";
+  private static final String DIR_THREE = "dir3";
+  private static final String DIR_FOUR = "dir4";
+  private static final String DIR_FIVE = "dir5";
+  // objects IDs
+  private static final long VOL_OBJECT_ID = 0L;
+  private static final long BUCKET_ONE_OBJECT_ID = 1L;
+  private static final long BUCKET_TWO_OBJECT_ID = 2L;
+  private static final long KEY_ONE_OBJECT_ID = 3L;
+  private static final long DIR_ONE_OBJECT_ID = 4L;
+  private static final long KEY_TWO_OBJECT_ID = 5L;
+  private static final long KEY_FOUR_OBJECT_ID = 6L;
+  private static final long DIR_TWO_OBJECT_ID = 7L;
+  private static final long KEY_THREE_OBJECT_ID = 8L;
+  private static final long KEY_FIVE_OBJECT_ID = 9L;
+  private static final long KEY_SIX_OBJECT_ID = 10L;
+  private static final long DIR_THREE_OBJECT_ID = 11L;
+  private static final long DIR_FOUR_OBJECT_ID = 12L;
+  private static final long MULTI_BLOCK_KEY_OBJECT_ID = 13L;
+  private static final long KEY_SEVEN_OBJECT_ID = 13L;
+  private static final long VOL_TWO_OBJECT_ID = 14L;
+  private static final long BUCKET_THREE_OBJECT_ID = 15L;
+  private static final long BUCKET_FOUR_OBJECT_ID = 16L;
+  private static final long KEY_EIGHT_OBJECT_ID = 17L;
+  private static final long DIR_FIVE_OBJECT_ID = 18L;
+  private static final long KEY_NINE_OBJECT_ID = 19L;
+  private static final long KEY_TEN_OBJECT_ID = 20L;
+  private static final long KEY_ELEVEN_OBJECT_ID = 21L;
+
+  // container IDs
+  private static final long CONTAINER_ONE_ID = 1L;
+  private static final long CONTAINER_TWO_ID = 2L;
+  private static final long CONTAINER_THREE_ID = 3L;
+  private static final long CONTAINER_FOUR_ID = 4L;
+  private static final long CONTAINER_FIVE_ID = 5L;
+  private static final long CONTAINER_SIX_ID = 6L;
+
+  // replication factors
+  private static final int CONTAINER_ONE_REPLICA_COUNT  = 3;
+  private static final int CONTAINER_TWO_REPLICA_COUNT  = 2;
+  private static final int CONTAINER_THREE_REPLICA_COUNT  = 4;
+  private static final int CONTAINER_FOUR_REPLICA_COUNT  = 5;
+  private static final int CONTAINER_FIVE_REPLICA_COUNT  = 2;
+  private static final int CONTAINER_SIX_REPLICA_COUNT  = 3;
+
+  // block lengths
+  private static final long BLOCK_ONE_LENGTH = 1000L;
+  private static final long BLOCK_TWO_LENGTH = 2000L;
+  private static final long BLOCK_THREE_LENGTH = 3000L;
+  private static final long BLOCK_FOUR_LENGTH = 4000L;
+  private static final long BLOCK_FIVE_LENGTH = 5000L;
+  private static final long BLOCK_SIX_LENGTH = 6000L;
+
+  // data size in bytes
+  private static final long KEY_ONE_SIZE = 500L; // bin 0
+  private static final long KEY_TWO_SIZE = OzoneConsts.KB + 1; // bin 1
+  private static final long KEY_THREE_SIZE = 4 * OzoneConsts.KB + 1; // bin 3
+  private static final long KEY_FOUR_SIZE = 2 * OzoneConsts.KB + 1; // bin 2
+  private static final long KEY_FIVE_SIZE = 100L; // bin 0
+  private static final long KEY_SIX_SIZE = 2 * OzoneConsts.KB + 1; // bin 2
+  private static final long KEY_EIGHT_SIZE = OzoneConsts.KB + 1; // bin 1
+  private static final long KEY_NINE_SIZE = 2 * OzoneConsts.KB + 1; // bin 2
+  private static final long KEY_TEN_SIZE = 2 * OzoneConsts.KB + 1; // bin 2
+  private static final long KEY_ELEVEN_SIZE = OzoneConsts.KB + 1; // bin 1
+  private static final long LOCATION_INFO_GROUP_ONE_SIZE
+          = CONTAINER_ONE_REPLICA_COUNT * BLOCK_ONE_LENGTH
+          + CONTAINER_TWO_REPLICA_COUNT * BLOCK_TWO_LENGTH
+          + CONTAINER_THREE_REPLICA_COUNT * BLOCK_THREE_LENGTH;
+
+  private static final long MULTI_BLOCK_KEY_SIZE_WITH_REPLICA
+          = LOCATION_INFO_GROUP_ONE_SIZE;
+
+  private static final long LOCATION_INFO_GROUP_TWO_SIZE
+      = CONTAINER_FOUR_REPLICA_COUNT * BLOCK_FOUR_LENGTH
+      + CONTAINER_FIVE_REPLICA_COUNT * BLOCK_FIVE_LENGTH
+      + CONTAINER_SIX_REPLICA_COUNT * BLOCK_SIX_LENGTH;
+
+  private static final long FILE1_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+  private static final long FILE2_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_TWO_SIZE;
+  private static final long FILE3_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+  private static final long FILE4_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_TWO_SIZE;
+  private static final long FILE5_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+  private static final long FILE6_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_TWO_SIZE;
+  private static final long FILE7_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+  private static final long FILE8_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_TWO_SIZE;
+  private static final long FILE9_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+  private static final long FILE10_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_TWO_SIZE;
+  private static final long FILE11_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_ROOT
+      = FILE1_SIZE_WITH_REPLICA
+      + FILE2_SIZE_WITH_REPLICA
+      + FILE3_SIZE_WITH_REPLICA
+      + FILE4_SIZE_WITH_REPLICA
+      + FILE5_SIZE_WITH_REPLICA
+      + FILE6_SIZE_WITH_REPLICA
+      + FILE7_SIZE_WITH_REPLICA
+      + FILE8_SIZE_WITH_REPLICA
+      + FILE9_SIZE_WITH_REPLICA
+      + FILE10_SIZE_WITH_REPLICA
+      + FILE11_SIZE_WITH_REPLICA;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_VOL
+      = FILE1_SIZE_WITH_REPLICA
+      + FILE2_SIZE_WITH_REPLICA
+      + FILE3_SIZE_WITH_REPLICA
+      + FILE4_SIZE_WITH_REPLICA
+      + FILE5_SIZE_WITH_REPLICA
+      + FILE6_SIZE_WITH_REPLICA
+      + FILE7_SIZE_WITH_REPLICA;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_BUCKET1
+      = FILE1_SIZE_WITH_REPLICA
+      + FILE2_SIZE_WITH_REPLICA
+      + FILE3_SIZE_WITH_REPLICA
+      + FILE6_SIZE_WITH_REPLICA
+      + FILE7_SIZE_WITH_REPLICA;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_DIR1
+      = FILE2_SIZE_WITH_REPLICA
+      + FILE3_SIZE_WITH_REPLICA
+      + FILE6_SIZE_WITH_REPLICA
+      + FILE7_SIZE_WITH_REPLICA;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_DIR2
+      = FILE2_SIZE_WITH_REPLICA;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_KEY
+      = FILE4_SIZE_WITH_REPLICA;
+
+  // quota in bytes
+  private static final long ROOT_QUOTA = 2 * (2 * OzoneConsts.MB);
+  private static final long VOL_QUOTA = 2 * OzoneConsts.MB;
+  private static final long VOL_TWO_QUOTA = 2 * OzoneConsts.MB;
+  private static final long BUCKET_ONE_QUOTA = OzoneConsts.MB;
+  private static final long BUCKET_TWO_QUOTA = OzoneConsts.MB;
+  private static final long BUCKET_THREE_QUOTA = OzoneConsts.MB;
+  private static final long BUCKET_FOUR_QUOTA = OzoneConsts.MB;
+
+  // mock client's path requests
+  private static final String TEST_USER = "TestUser";
+  private static final String ROOT_PATH = "/";
+  private static final String VOL_PATH = "/vol";
+  private static final String VOL_TWO_PATH = "/vol2";
+  private static final String BUCKET_ONE_PATH = "/vol/bucket1";
+  private static final String BUCKET_TWO_PATH = "/vol/bucket2";
+  private static final String DIR_ONE_PATH = "/vol/bucket1/dir1";
+  private static final String DIR_TWO_PATH = "/vol/bucket1/dir1/dir2";
+  private static final String DIR_THREE_PATH = "/vol/bucket1/dir1/dir3";
+  private static final String DIR_FOUR_PATH = "/vol/bucket1/dir1/dir4";
+  private static final String KEY_PATH = "/vol/bucket2/file4";
+  private static final String MULTI_BLOCK_KEY_PATH = "/vol/bucket1/dir1/file7";
+  private static final String INVALID_PATH = "/vol/path/not/found";
+
+  // some expected answers
+  private static final long ROOT_DATA_SIZE = KEY_ONE_SIZE + KEY_TWO_SIZE +
+      KEY_THREE_SIZE + KEY_FOUR_SIZE + KEY_FIVE_SIZE + KEY_SIX_SIZE +
+      KEY_EIGHT_SIZE + KEY_NINE_SIZE + KEY_TEN_SIZE + KEY_ELEVEN_SIZE;
+  private static final long VOL_DATA_SIZE = KEY_ONE_SIZE + KEY_TWO_SIZE +
+      KEY_THREE_SIZE + KEY_FOUR_SIZE + KEY_FIVE_SIZE + KEY_SIX_SIZE;
+
+  private static final long VOL_TWO_DATA_SIZE =
+      KEY_EIGHT_SIZE + KEY_NINE_SIZE + KEY_TEN_SIZE + KEY_ELEVEN_SIZE;
+
+  private static final long BUCKET_ONE_DATA_SIZE = KEY_ONE_SIZE + KEY_TWO_SIZE +
+      KEY_THREE_SIZE + KEY_SIX_SIZE;
+
+  private static final long BUCKET_TWO_DATA_SIZE =
+      KEY_FOUR_SIZE + KEY_FIVE_SIZE;
+
+  private static final long DIR_ONE_DATA_SIZE = KEY_TWO_SIZE +
+      KEY_THREE_SIZE + KEY_SIX_SIZE;
+
+  @Before
+  public void setUp() throws Exception {
+    OMMetadataManager omMetadataManager = initializeNewOmMetadataManager(
+        temporaryFolder.newFolder());
+    OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
+        getMockOzoneManagerServiceProvider();
+    reconOMMetadataManager = getTestReconOmMetadataManager(omMetadataManager, 
+        temporaryFolder.newFolder());
+
+    ReconTestInjector reconTestInjector =
+        new ReconTestInjector.Builder(temporaryFolder)
+            .withReconOm(reconOMMetadataManager)
+            .withOmServiceProvider(ozoneManagerServiceProvider)
+            .withReconSqlDb()
+            .withContainerDB()
+            .addBinding(OzoneStorageContainerManager.class,
+                getMockReconSCM())
+            .addBinding(StorageContainerServiceProvider.class,
+                mock(StorageContainerServiceProviderImpl.class))
+            .addBinding(NSSummaryEndpoint.class)
+            .build();
+    ReconNamespaceSummaryManager reconNamespaceSummaryManager =
+        reconTestInjector.getInstance(ReconNamespaceSummaryManager.class);
+    nsSummaryEndpoint = reconTestInjector.getInstance(NSSummaryEndpoint.class);
+
+    // populate OM DB and reprocess into Recon RocksDB
+    populateOMDB();
+    NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = 
+        new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, 
+                                    reconOMMetadataManager);
+    nsSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager);
+  }
+
+  @Test
+  public void testUtility() {
+    String[] names = EntityHandler.parseRequestPath(TEST_PATH_UTILITY);
+    Assert.assertArrayEquals(TEST_NAMES, names);
+    String keyName = BucketHandler.getKeyName(names);
+    Assert.assertEquals(TEST_KEY_NAMES, keyName);
+    String subpath = BucketHandler.buildSubpath(PARENT_DIR, "file1.txt");
+    Assert.assertEquals(TEST_PATH_UTILITY, subpath);
+  }
+
+  @Test
+  public void testGetBasicInfoRoot() throws Exception {
+    // Test root basics
+    Response rootResponse = nsSummaryEndpoint.getBasicInfo(ROOT_PATH);
+    NamespaceSummaryResponse rootResponseObj =
+        (NamespaceSummaryResponse) rootResponse.getEntity();
+    Assert.assertEquals(EntityType.ROOT, rootResponseObj.getEntityType());
+    Assert.assertEquals(2, rootResponseObj.getNumVolume());
+    Assert.assertEquals(4, rootResponseObj.getNumBucket());
+    Assert.assertEquals(5, rootResponseObj.getNumTotalDir());
+    Assert.assertEquals(10, rootResponseObj.getNumTotalKey());
+  }
+
+  @Test
+  public void testGetBasicInfoVol() throws Exception {
+    // Test volume basics
+    Response volResponse = nsSummaryEndpoint.getBasicInfo(VOL_PATH);
+    NamespaceSummaryResponse volResponseObj =
+        (NamespaceSummaryResponse) volResponse.getEntity();
+    Assert.assertEquals(EntityType.VOLUME, volResponseObj.getEntityType());
+    Assert.assertEquals(2, volResponseObj.getNumBucket());
+    Assert.assertEquals(4, volResponseObj.getNumTotalDir());
+    Assert.assertEquals(6, volResponseObj.getNumTotalKey());
+  }
+
+  @Test
+  public void testGetBasicInfoBucketOne() throws Exception {
+    // Test bucket 1's basics
+    Response bucketOneResponse =
+        nsSummaryEndpoint.getBasicInfo(BUCKET_ONE_PATH);
+    NamespaceSummaryResponse bucketOneObj =
+        (NamespaceSummaryResponse) bucketOneResponse.getEntity();
+    Assert.assertEquals(EntityType.BUCKET, bucketOneObj.getEntityType());
+    Assert.assertEquals(4, bucketOneObj.getNumTotalDir());
+    Assert.assertEquals(4, bucketOneObj.getNumTotalKey());
+  }
+
+  @Test
+  public void testGetBasicInfoBucketTwo() throws Exception {
+    // Test bucket 2's basics
+    Response bucketTwoResponse =
+        nsSummaryEndpoint.getBasicInfo(BUCKET_TWO_PATH);
+    NamespaceSummaryResponse bucketTwoObj =
+        (NamespaceSummaryResponse) bucketTwoResponse.getEntity();
+    Assert.assertEquals(EntityType.BUCKET, bucketTwoObj.getEntityType());
+    Assert.assertEquals(0, bucketTwoObj.getNumTotalDir());
+    Assert.assertEquals(2, bucketTwoObj.getNumTotalKey());
+  }
+
+  @Test
+  public void testGetBasicInfoDir() throws Exception {
+    // Test intermediate directory basics
+    Response dirOneResponse = nsSummaryEndpoint.getBasicInfo(DIR_ONE_PATH);
+    NamespaceSummaryResponse dirOneObj =
+        (NamespaceSummaryResponse) dirOneResponse.getEntity();
+    Assert.assertEquals(EntityType.DIRECTORY, dirOneObj.getEntityType());
+    Assert.assertEquals(3, dirOneObj.getNumTotalDir());
+    Assert.assertEquals(3, dirOneObj.getNumTotalKey());
+  }
+
+  @Test
+  public void testGetBasicInfoNoPath() throws Exception {
+    // Test invalid path
+    Response invalidResponse = nsSummaryEndpoint.getBasicInfo(INVALID_PATH);
+    NamespaceSummaryResponse invalidObj =
+        (NamespaceSummaryResponse) invalidResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.PATH_NOT_FOUND,
+        invalidObj.getStatus());
+  }
+
+  @Test
+  public void testGetBasicInfoKey() throws Exception {
+    // Test key
+    Response keyResponse = nsSummaryEndpoint.getBasicInfo(KEY_PATH);
+    NamespaceSummaryResponse keyResObj =
+        (NamespaceSummaryResponse) keyResponse.getEntity();
+    Assert.assertEquals(EntityType.KEY, keyResObj.getEntityType());
+  }
+
+  @Test
+  public void testDiskUsageRoot() throws Exception {
+    // root level DU
+    Response rootResponse = nsSummaryEndpoint.getDiskUsage(ROOT_PATH,
+        false, false);
+    DUResponse duRootRes = (DUResponse) rootResponse.getEntity();
+    Assert.assertEquals(2, duRootRes.getCount());
+    List<DUResponse.DiskUsage> duRootData = duRootRes.getDuData();
+    // sort based on subpath
+    Collections.sort(duRootData,
+        Comparator.comparing(DUResponse.DiskUsage::getSubpath));
+    DUResponse.DiskUsage duVol1 = duRootData.get(0);
+    DUResponse.DiskUsage duVol2 = duRootData.get(1);
+    Assert.assertEquals(VOL_PATH, duVol1.getSubpath());
+    Assert.assertEquals(VOL_TWO_PATH, duVol2.getSubpath());
+    Assert.assertEquals(VOL_DATA_SIZE, duVol1.getSize());
+    Assert.assertEquals(VOL_TWO_DATA_SIZE, duVol2.getSize());
+  }
+  @Test
+  public void testDiskUsageVolume() throws Exception {
+    // volume level DU
+    Response volResponse = nsSummaryEndpoint.getDiskUsage(VOL_PATH,
+        false, false);
+    DUResponse duVolRes = (DUResponse) volResponse.getEntity();
+    Assert.assertEquals(2, duVolRes.getCount());
+    List<DUResponse.DiskUsage> duData = duVolRes.getDuData();
+    // sort based on subpath
+    Collections.sort(duData,
+        Comparator.comparing(DUResponse.DiskUsage::getSubpath));
+    DUResponse.DiskUsage duBucket1 = duData.get(0);
+    DUResponse.DiskUsage duBucket2 = duData.get(1);
+    Assert.assertEquals(BUCKET_ONE_PATH, duBucket1.getSubpath());
+    Assert.assertEquals(BUCKET_TWO_PATH, duBucket2.getSubpath());
+    Assert.assertEquals(BUCKET_ONE_DATA_SIZE, duBucket1.getSize());
+    Assert.assertEquals(BUCKET_TWO_DATA_SIZE, duBucket2.getSize());
+
+  }
+  @Test
+  public void testDiskUsageBucket() throws Exception {
+    // bucket level DU
+    Response bucketResponse = nsSummaryEndpoint.getDiskUsage(BUCKET_ONE_PATH,
+        false, false);
+    DUResponse duBucketResponse = (DUResponse) bucketResponse.getEntity();
+    Assert.assertEquals(1, duBucketResponse.getCount());
+    DUResponse.DiskUsage duDir1 = duBucketResponse.getDuData().get(0);
+    Assert.assertEquals(DIR_ONE_PATH, duDir1.getSubpath());
+    Assert.assertEquals(DIR_ONE_DATA_SIZE, duDir1.getSize());
+
+  }
+  @Test
+  public void testDiskUsageDir() throws Exception {
+    // dir level DU
+    Response dirResponse = nsSummaryEndpoint.getDiskUsage(DIR_ONE_PATH,
+        false, false);
+    DUResponse duDirReponse = (DUResponse) dirResponse.getEntity();
+    Assert.assertEquals(3, duDirReponse.getCount());
+    List<DUResponse.DiskUsage> duSubDir = duDirReponse.getDuData();
+    Collections.sort(duSubDir,
+        Comparator.comparing(DUResponse.DiskUsage::getSubpath));
+    DUResponse.DiskUsage duDir2 = duSubDir.get(0);
+    DUResponse.DiskUsage duDir3 = duSubDir.get(1);
+    DUResponse.DiskUsage duDir4 = duSubDir.get(2);
+    Assert.assertEquals(DIR_TWO_PATH, duDir2.getSubpath());
+    Assert.assertEquals(KEY_TWO_SIZE, duDir2.getSize());
+
+    Assert.assertEquals(DIR_THREE_PATH, duDir3.getSubpath());
+    Assert.assertEquals(KEY_THREE_SIZE, duDir3.getSize());
+
+    Assert.assertEquals(DIR_FOUR_PATH, duDir4.getSubpath());
+    Assert.assertEquals(KEY_SIX_SIZE, duDir4.getSize());
+
+  }
+  @Test
+  public void testDiskUsageKey() throws Exception {
+    // key level DU
+    Response keyResponse = nsSummaryEndpoint.getDiskUsage(KEY_PATH,
+        false, false);
+    DUResponse keyObj = (DUResponse) keyResponse.getEntity();
+    Assert.assertEquals(0, keyObj.getCount());
+    Assert.assertEquals(KEY_FOUR_SIZE, keyObj.getSize());
+
+  }
+  @Test
+  public void testDiskUsageUnknown() throws Exception {
+    // invalid path check
+    Response invalidResponse = nsSummaryEndpoint.getDiskUsage(INVALID_PATH,
+        false, false);
+    DUResponse invalidObj = (DUResponse) invalidResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.PATH_NOT_FOUND,
+        invalidObj.getStatus());
+  }
+
+  @Test
+  public void testDiskUsageWithReplication() throws Exception {
+    setUpMultiBlockKey();
+    Response keyResponse = nsSummaryEndpoint.getDiskUsage(MULTI_BLOCK_KEY_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) keyResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_KEY_SIZE_WITH_REPLICA,
+        replicaDUResponse.getSizeWithReplica());
+  }
+
+  @Test
+  public void testDataSizeUnderRootWithReplication() throws IOException {
+    setUpMultiBlockReplicatedKeys();
+    //   withReplica is true
+    Response rootResponse = nsSummaryEndpoint.getDiskUsage(ROOT_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) rootResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_ROOT,
+        replicaDUResponse.getSizeWithReplica());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_VOL,
+        replicaDUResponse.getDuData().get(0).getSizeWithReplica());
+
+  }
+
+  @Test
+  public void testDataSizeUnderVolWithReplication() throws IOException {
+    setUpMultiBlockReplicatedKeys();
+    Response volResponse = nsSummaryEndpoint.getDiskUsage(VOL_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) volResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_VOL,
+        replicaDUResponse.getSizeWithReplica());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_BUCKET1,
+        replicaDUResponse.getDuData().get(0).getSizeWithReplica());
+  }
+
+  @Test
+  public void testDataSizeUnderBucketWithReplication() throws IOException {
+    setUpMultiBlockReplicatedKeys();
+    Response bucketResponse = nsSummaryEndpoint.getDiskUsage(BUCKET_ONE_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) bucketResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_BUCKET1,
+        replicaDUResponse.getSizeWithReplica());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_DIR1,
+        replicaDUResponse.getDuData().get(0).getSizeWithReplica());
+  }
+
+  /**
+   * When calculating DU under dir1
+   * there are 3 keys, file2, file3, file6.
+   * There is one direct key, file7.
+   * @throws IOException
+   */
+  @Test
+  public void testDataSizeUnderDirWithReplication() throws IOException {
+    setUpMultiBlockReplicatedKeys();
+    Response dir1Response = nsSummaryEndpoint.getDiskUsage(DIR_ONE_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) dir1Response.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_DIR1,
+        replicaDUResponse.getSizeWithReplica());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_DIR2,
+        replicaDUResponse.getDuData().get(0).getSizeWithReplica());
+  }
+
+  @Test
+  public void testDataSizeUnderKeyWithReplication() throws IOException {
+    setUpMultiBlockReplicatedKeys();
+    Response keyResponse = nsSummaryEndpoint.getDiskUsage(KEY_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) keyResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_KEY,
+        replicaDUResponse.getSizeWithReplica());
+  }
+
+  @Test
+  public void testQuotaUsage() throws Exception {
+    // root level quota usage
+    Response rootResponse = nsSummaryEndpoint.getQuotaUsage(ROOT_PATH);
+    QuotaUsageResponse quRootRes =
+        (QuotaUsageResponse) rootResponse.getEntity();
+    Assert.assertEquals(ROOT_QUOTA, quRootRes.getQuota());
+    Assert.assertEquals(ROOT_DATA_SIZE, quRootRes.getQuotaUsed());
+
+    // volume level quota usage
+    Response volResponse = nsSummaryEndpoint.getQuotaUsage(VOL_PATH);
+    QuotaUsageResponse quVolRes = (QuotaUsageResponse) volResponse.getEntity();
+    Assert.assertEquals(VOL_QUOTA, quVolRes.getQuota());
+    Assert.assertEquals(VOL_DATA_SIZE, quVolRes.getQuotaUsed());
+
+    // bucket level quota usage
+    Response bucketRes = nsSummaryEndpoint.getQuotaUsage(BUCKET_ONE_PATH);
+    QuotaUsageResponse quBucketRes = (QuotaUsageResponse) bucketRes.getEntity();
+    Assert.assertEquals(BUCKET_ONE_QUOTA, quBucketRes.getQuota());
+    Assert.assertEquals(BUCKET_ONE_DATA_SIZE, quBucketRes.getQuotaUsed());
+
+    Response bucketRes2 = nsSummaryEndpoint.getQuotaUsage(BUCKET_TWO_PATH);
+    QuotaUsageResponse quBucketRes2 =
+        (QuotaUsageResponse) bucketRes2.getEntity();
+    Assert.assertEquals(BUCKET_TWO_QUOTA, quBucketRes2.getQuota());
+    Assert.assertEquals(BUCKET_TWO_DATA_SIZE, quBucketRes2.getQuotaUsed());
+
+    // other level not applicable
+    Response naResponse1 = nsSummaryEndpoint.getQuotaUsage(DIR_ONE_PATH);
+    QuotaUsageResponse quotaUsageResponse1 =
+        (QuotaUsageResponse) naResponse1.getEntity();
+    Assert.assertEquals(ResponseStatus.TYPE_NOT_APPLICABLE,
+        quotaUsageResponse1.getResponseCode());
+
+    Response naResponse2 = nsSummaryEndpoint.getQuotaUsage(KEY_PATH);
+    QuotaUsageResponse quotaUsageResponse2 =
+        (QuotaUsageResponse) naResponse2.getEntity();
+    Assert.assertEquals(ResponseStatus.TYPE_NOT_APPLICABLE,
+        quotaUsageResponse2.getResponseCode());
+
+    // invalid path request
+    Response invalidRes = nsSummaryEndpoint.getQuotaUsage(INVALID_PATH);
+    QuotaUsageResponse invalidResObj =
+        (QuotaUsageResponse) invalidRes.getEntity();
+    Assert.assertEquals(ResponseStatus.PATH_NOT_FOUND,
+        invalidResObj.getResponseCode());
+  }
+
+
+  @Test
+  public void testFileSizeDist() throws Exception {
+    checkFileSizeDist(ROOT_PATH, 2, 3, 4, 1);
+    checkFileSizeDist(VOL_PATH, 2, 1, 2, 1);
+    checkFileSizeDist(BUCKET_ONE_PATH, 1, 1, 1, 1);
+    checkFileSizeDist(DIR_ONE_PATH, 0, 1, 1, 1);
+  }
+
+  public void checkFileSizeDist(String path, int bin0,
+      int bin1, int bin2, int bin3) throws Exception {
+    Response res = nsSummaryEndpoint.getFileSizeDistribution(path);
+    FileSizeDistributionResponse fileSizeDistResObj =
+            (FileSizeDistributionResponse) res.getEntity();
+    int[] fileSizeDist = fileSizeDistResObj.getFileSizeDist();
+    Assert.assertEquals(bin0, fileSizeDist[0]);
+    Assert.assertEquals(bin1, fileSizeDist[1]);
+    Assert.assertEquals(bin2, fileSizeDist[2]);
+    Assert.assertEquals(bin3, fileSizeDist[3]);
+    for (int i = 4; i < ReconConstants.NUM_OF_BINS; ++i) {
+      Assert.assertEquals(0, fileSizeDist[i]);
+    }
+  }
+
+  /**
+   * Write directories and keys info into OM DB.
+   * @throws Exception
+   */
+  @SuppressWarnings("checkstyle:MethodLength")
+  private void populateOMDB() throws Exception {
+    // write all directories
+    writeDirToOm(reconOMMetadataManager,
+          (DIR_ONE + OM_KEY_PREFIX),
+          BUCKET_ONE,
+          VOL,
+          DIR_ONE,
+          DIR_ONE_OBJECT_ID,
+          BUCKET_ONE_OBJECT_ID,
+          BUCKET_ONE_OBJECT_ID,
+          VOL_OBJECT_ID,
+          getBucketLayout());
+    writeDirToOm(reconOMMetadataManager,
+          (DIR_ONE + OM_KEY_PREFIX + DIR_TWO + OM_KEY_PREFIX),
+          BUCKET_ONE,
+          VOL,
+          DIR_TWO,
+          DIR_TWO_OBJECT_ID,
+          DIR_ONE_OBJECT_ID,
+          BUCKET_ONE_OBJECT_ID,
+          VOL_OBJECT_ID,
+          getBucketLayout());
+    writeDirToOm(reconOMMetadataManager,
+          (DIR_ONE + OM_KEY_PREFIX + DIR_THREE + OM_KEY_PREFIX),
+          BUCKET_ONE,
+          VOL,
+          DIR_THREE,
+          DIR_THREE_OBJECT_ID,
+          DIR_ONE_OBJECT_ID,
+          BUCKET_ONE_OBJECT_ID,
+          VOL_OBJECT_ID,
+          getBucketLayout());
+    writeDirToOm(reconOMMetadataManager,
+          (DIR_ONE + OM_KEY_PREFIX + DIR_FOUR + OM_KEY_PREFIX),
+          BUCKET_ONE,
+          VOL,
+          DIR_FOUR,
+          DIR_FOUR_OBJECT_ID,
+          DIR_ONE_OBJECT_ID,
+          BUCKET_ONE_OBJECT_ID,
+          VOL_OBJECT_ID,
+          getBucketLayout());
+    writeDirToOm(reconOMMetadataManager,
+          (DIR_FIVE + OM_KEY_PREFIX),
+          BUCKET_THREE,
+          VOL_TWO,
+          DIR_FIVE,
+          DIR_FIVE_OBJECT_ID,
+          BUCKET_THREE_OBJECT_ID,
+          BUCKET_THREE_OBJECT_ID,
+          VOL_TWO_OBJECT_ID,
+          getBucketLayout());
+
+    // write all keys
+    writeKeyToOm(reconOMMetadataManager,
+          KEY_ONE,
+          BUCKET_ONE,
+          VOL,
+          FILE_ONE,
+          KEY_ONE_OBJECT_ID,
+          BUCKET_ONE_OBJECT_ID,
+          BUCKET_ONE_OBJECT_ID,
+          VOL_OBJECT_ID,
+          KEY_ONE_SIZE,
+          getBucketLayout());
+    writeKeyToOm(reconOMMetadataManager,
+          KEY_TWO,
+          BUCKET_ONE,
+          VOL,
+          FILE_TWO,
+          KEY_TWO_OBJECT_ID,
+          DIR_TWO_OBJECT_ID,
+          BUCKET_ONE_OBJECT_ID,
+          VOL_OBJECT_ID,
+          KEY_TWO_SIZE,
+          getBucketLayout());
+    writeKeyToOm(reconOMMetadataManager,
+          KEY_THREE,
+          BUCKET_ONE,
+          VOL,
+          FILE_THREE,
+          KEY_THREE_OBJECT_ID,
+          DIR_THREE_OBJECT_ID,
+          BUCKET_ONE_OBJECT_ID,
+          VOL_OBJECT_ID,
+          KEY_THREE_SIZE,
+          getBucketLayout());
+    writeKeyToOm(reconOMMetadataManager,
+          KEY_FOUR,
+          BUCKET_TWO,
+          VOL,
+          FILE_FOUR,
+          KEY_FOUR_OBJECT_ID,
+          BUCKET_TWO_OBJECT_ID,
+          BUCKET_TWO_OBJECT_ID,
+          VOL_OBJECT_ID,
+          KEY_FOUR_SIZE,
+          getBucketLayout());
+    writeKeyToOm(reconOMMetadataManager,
+          KEY_FIVE,
+          BUCKET_TWO,
+          VOL,
+          FILE_FIVE,
+          KEY_FIVE_OBJECT_ID,
+          BUCKET_TWO_OBJECT_ID,
+          BUCKET_TWO_OBJECT_ID,
+          VOL_OBJECT_ID,
+          KEY_FIVE_SIZE,
+          getBucketLayout());
+    writeKeyToOm(reconOMMetadataManager,
+          KEY_SIX,
+          BUCKET_ONE,
+          VOL,
+          FILE_SIX,
+          KEY_SIX_OBJECT_ID,
+          DIR_FOUR_OBJECT_ID,
+          BUCKET_ONE_OBJECT_ID,
+          VOL_OBJECT_ID,
+          KEY_SIX_SIZE,
+          getBucketLayout());
+    writeKeyToOm(reconOMMetadataManager,
+          KEY_EIGHT,
+          BUCKET_THREE,
+          VOL_TWO,
+          FILE_EIGHT,
+          KEY_EIGHT_OBJECT_ID,
+          BUCKET_THREE_OBJECT_ID,
+          BUCKET_THREE_OBJECT_ID,
+          VOL_TWO_OBJECT_ID,
+          KEY_EIGHT_SIZE,
+          getBucketLayout());
+    writeKeyToOm(reconOMMetadataManager,
+          KEY_NINE,
+          BUCKET_THREE,
+          VOL_TWO,
+          FILE_NINE,
+          KEY_NINE_OBJECT_ID,
+          DIR_FIVE_OBJECT_ID,
+          BUCKET_THREE_OBJECT_ID,
+          VOL_TWO_OBJECT_ID,
+          KEY_NINE_SIZE,
+          getBucketLayout());
+    writeKeyToOm(reconOMMetadataManager,
+          KEY_TEN,
+          BUCKET_THREE,
+          VOL_TWO,
+          FILE_TEN,
+          KEY_TEN_OBJECT_ID,
+          DIR_FIVE_OBJECT_ID,
+          BUCKET_THREE_OBJECT_ID,
+          VOL_TWO_OBJECT_ID,
+          KEY_TEN_SIZE,
+          getBucketLayout());
+    writeKeyToOm(reconOMMetadataManager,
+          KEY_ELEVEN,
+          BUCKET_FOUR,
+          VOL_TWO,
+          FILE_ELEVEN,
+          KEY_ELEVEN_OBJECT_ID,
+          BUCKET_FOUR_OBJECT_ID,
+          BUCKET_FOUR_OBJECT_ID,
+          VOL_TWO_OBJECT_ID,
+          KEY_ELEVEN_SIZE,
+          getBucketLayout());
+  }
+
+  /**
+   * Create a new OM Metadata manager instance with one user, one vol, and two
+   * buckets.
+   * @throws IOException ioEx
+   */
+  private static OMMetadataManager initializeNewOmMetadataManager(
+      File omDbDir)
+      throws IOException {
+    OzoneConfiguration omConfiguration = new OzoneConfiguration();
+    omConfiguration.set(OZONE_OM_DB_DIRS,
+        omDbDir.getAbsolutePath());
+    OMMetadataManager omMetadataManager = new OmMetadataManagerImpl(
+        omConfiguration);
+
+    String volumeKey = omMetadataManager.getVolumeKey(VOL);
+    OmVolumeArgs args =
+        OmVolumeArgs.newBuilder()
+            .setObjectID(VOL_OBJECT_ID)
+            .setVolume(VOL)
+            .setAdminName(TEST_USER)
+            .setOwnerName(TEST_USER)
+            .setQuotaInBytes(VOL_QUOTA)
+            .build();
+
+    String volume2Key = omMetadataManager.getVolumeKey(VOL_TWO);
+    OmVolumeArgs args2 =
+        OmVolumeArgs.newBuilder()
+            .setObjectID(VOL_TWO_OBJECT_ID)
+            .setVolume(VOL_TWO)
+            .setAdminName(TEST_USER)
+            .setOwnerName(TEST_USER)
+            .setQuotaInBytes(VOL_TWO_QUOTA)
+            .build();
+
+    omMetadataManager.getVolumeTable().put(volumeKey, args);
+    omMetadataManager.getVolumeTable().put(volume2Key, args2);
+
+    OmBucketInfo bucketInfo = OmBucketInfo.newBuilder()
+        .setVolumeName(VOL)
+        .setBucketName(BUCKET_ONE)
+        .setObjectID(BUCKET_ONE_OBJECT_ID)
+        .setQuotaInBytes(BUCKET_ONE_QUOTA)
+        .setBucketLayout(getBucketLayout())
+        .build();
+
+    OmBucketInfo bucketInfo2 = OmBucketInfo.newBuilder()
+        .setVolumeName(VOL)
+        .setBucketName(BUCKET_TWO)
+        .setObjectID(BUCKET_TWO_OBJECT_ID)
+        .setQuotaInBytes(BUCKET_TWO_QUOTA)
+        .setBucketLayout(getBucketLayout())
+        .build();
+
+    OmBucketInfo bucketInfo3 = OmBucketInfo.newBuilder()
+        .setVolumeName(VOL_TWO)
+        .setBucketName(BUCKET_THREE)
+        .setObjectID(BUCKET_THREE_OBJECT_ID)
+        .setQuotaInBytes(BUCKET_THREE_QUOTA)
+        .setBucketLayout(getBucketLayout())
+        .build();
+
+    OmBucketInfo bucketInfo4 = OmBucketInfo.newBuilder()
+        .setVolumeName(VOL_TWO)
+        .setBucketName(BUCKET_FOUR)
+        .setObjectID(BUCKET_FOUR_OBJECT_ID)
+        .setQuotaInBytes(BUCKET_FOUR_QUOTA)
+        .setBucketLayout(getBucketLayout())
+        .build();
+
+    String bucketKey = omMetadataManager.getBucketKey(
+        bucketInfo.getVolumeName(), bucketInfo.getBucketName());
+    String bucketKey2 = omMetadataManager.getBucketKey(
+        bucketInfo2.getVolumeName(), bucketInfo2.getBucketName());
+    String bucketKey3 = omMetadataManager.getBucketKey(
+        bucketInfo3.getVolumeName(), bucketInfo3.getBucketName());
+    String bucketKey4 = omMetadataManager.getBucketKey(
+        bucketInfo4.getVolumeName(), bucketInfo4.getBucketName());
+
+    omMetadataManager.getBucketTable().put(bucketKey, bucketInfo);
+    omMetadataManager.getBucketTable().put(bucketKey2, bucketInfo2);
+    omMetadataManager.getBucketTable().put(bucketKey3, bucketInfo3);
+    omMetadataManager.getBucketTable().put(bucketKey4, bucketInfo4);
+
+    return omMetadataManager;
+  }
+
+  private void setUpMultiBlockKey() throws IOException {
+    OmKeyLocationInfoGroup locationInfoGroup =
+        getLocationInfoGroup1();
+
+    // add the multi-block key to Recon's OM
+    writeKeyToOm(reconOMMetadataManager,
+        MULTI_BLOCK_KEY,
+        BUCKET_ONE,
+        VOL,
+        MULTI_BLOCK_FILE,
+        MULTI_BLOCK_KEY_OBJECT_ID,
+        DIR_ONE_OBJECT_ID,
+        BUCKET_ONE_OBJECT_ID,
+        VOL_OBJECT_ID,
+        Collections.singletonList(locationInfoGroup),
+        getBucketLayout());
+  }
+
+  private OmKeyLocationInfoGroup getLocationInfoGroup1() {
+    List<OmKeyLocationInfo> locationInfoList = new ArrayList<>();
+    BlockID block1 = new BlockID(CONTAINER_ONE_ID, 0L);
+    BlockID block2 = new BlockID(CONTAINER_TWO_ID, 0L);
+    BlockID block3 = new BlockID(CONTAINER_THREE_ID, 0L);
+
+    OmKeyLocationInfo location1 = new OmKeyLocationInfo.Builder()
+        .setBlockID(block1)
+        .setLength(BLOCK_ONE_LENGTH)
+        .build();
+    OmKeyLocationInfo location2 = new OmKeyLocationInfo.Builder()
+        .setBlockID(block2)
+        .setLength(BLOCK_TWO_LENGTH)
+        .build();
+    OmKeyLocationInfo location3 = new OmKeyLocationInfo.Builder()
+        .setBlockID(block3)
+        .setLength(BLOCK_THREE_LENGTH)
+        .build();
+    locationInfoList.add(location1);
+    locationInfoList.add(location2);
+    locationInfoList.add(location3);
+
+    return new OmKeyLocationInfoGroup(0L, locationInfoList);
+  }
+
+  /**
+   * Testing the following case.
+   *                     vol
+   *               /             \
+   *        bucket1               bucket2
+   *        /    \                /     \
+   *     file1      dir1        file4  file5
+   *           /   \   \     \
+   *        dir2  dir3  dir4  file7
+   *         /     \      \
+   *       file2   file3  file6
+   *  ----------------------------------------
+   *                  vol2
+   *              /         \
+   *      bucket3          bucket4
+   *      /      \           /
+   *   file8     dir5      file11
+   *            /    \
+   *        file9    file10
+   * Write these keys to OM and
+   * replicate them.
+   */
+  private OmKeyLocationInfoGroup getLocationInfoGroup2() {
+    List<OmKeyLocationInfo> locationInfoList = new ArrayList<>();
+    BlockID block4 = new BlockID(CONTAINER_FOUR_ID, 0L);
+    BlockID block5 = new BlockID(CONTAINER_FIVE_ID, 0L);
+    BlockID block6 = new BlockID(CONTAINER_SIX_ID, 0L);
+
+    OmKeyLocationInfo location4 = new OmKeyLocationInfo.Builder()
+        .setBlockID(block4)
+        .setLength(BLOCK_FOUR_LENGTH)
+        .build();
+    OmKeyLocationInfo location5 = new OmKeyLocationInfo.Builder()
+        .setBlockID(block5)
+        .setLength(BLOCK_FIVE_LENGTH)
+        .build();
+    OmKeyLocationInfo location6 = new OmKeyLocationInfo.Builder()
+        .setBlockID(block6)
+        .setLength(BLOCK_SIX_LENGTH)
+        .build();
+    locationInfoList.add(location4);
+    locationInfoList.add(location5);
+    locationInfoList.add(location6);
+    return new OmKeyLocationInfoGroup(0L, locationInfoList);
+
+  }
+
+  @SuppressWarnings("checkstyle:MethodLength")
+  private void setUpMultiBlockReplicatedKeys() throws IOException {
+    OmKeyLocationInfoGroup locationInfoGroup1 =
+        getLocationInfoGroup1();
+    OmKeyLocationInfoGroup locationInfoGroup2 =
+        getLocationInfoGroup2();
+
+    //vol/bucket1/file1
+    writeKeyToOm(reconOMMetadataManager,
+        KEY_ONE,
+        BUCKET_ONE,
+        VOL,
+        FILE_ONE,
+        KEY_ONE_OBJECT_ID,
+        BUCKET_ONE_OBJECT_ID,

Review Comment:
   The parent object id for legacy records is 0, (which is why we added the setKeyParentId() method.) Let's set them back to 0 in the test code as well, just to be sure that setKeyParentId() is really working.
   



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r988410895


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTaskDbEventHandler {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  private OzoneConfiguration ozoneConfiguration;
+
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager,
+                                 OzoneConfiguration
+                                     ozoneConfiguration) {
+    super(reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.ozoneConfiguration = ozoneConfiguration;
+  }
+
+  public boolean processWithLegacy(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+          WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnKeyTable = table.equals(KEY_TABLE);
+      if (!updateOnKeyTable) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+            (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+        OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+        OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+        // KeyTable entries belong to both Legacy and OBS buckets.
+        // Check bucket layout and if it's OBS
+        // continue to the next iteration.
+        // Check just for the current KeyInfo.
+        String volumeName = updatedKeyInfo.getVolumeName();
+        String bucketName = updatedKeyInfo.getBucketName();
+        String bucketDBKey = getReconOMMetadataManager()
+            .getBucketKey(volumeName, bucketName);
+        // Get bucket info from bucket table
+        OmBucketInfo omBucketInfo = getReconOMMetadataManager()
+            .getBucketTable().getSkipCache(bucketDBKey);
+        // True if FileSystemPaths enabled
+        boolean enableFileSystemPaths =
+            ozoneConfiguration.get(OMConfigKeys
+                .OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");
+
+        if (omBucketInfo.getBucketLayout()
+            .isObjectStore(enableFileSystemPaths)) {
+          continue;
+        }
+
+        setKeyParentID(updatedKeyInfo);
+
+        if (!updatedKeyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+          switch (action) {
+          case PUT:
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              setKeyParentID(oldKeyInfo);
+              handleDeleteKeyEvent(oldKeyInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                  updatedKey);
+            }
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        } else {
+          OmDirectoryInfo updatedDirectoryInfo =
+              new OmDirectoryInfo.Builder()
+                  .setName(updatedKeyInfo.getKeyName())
+                  .setObjectID(updatedKeyInfo.getObjectID())
+                  .setParentObjectID(updatedKeyInfo.getParentObjectID())
+                  .build();
+
+          OmDirectoryInfo oldDirectoryInfo = null;
+
+          if (oldKeyInfo != null) {
+            oldDirectoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(oldKeyInfo.getKeyName())
+                    .setObjectID(oldKeyInfo.getObjectID())
+                    .setParentObjectID(oldKeyInfo.getParentObjectID())
+                    .build();
+          }
+
+          switch (action) {
+          case PUT:
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                  updatedKey);
+            }
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        }
+      } catch (IOException ioEx) {
+        LOG.error("Unable to process Namespace Summary data in Recon DB. ",
+            ioEx);
+        return false;
+      }
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return false;
+    }
+
+    LOG.info("Completed a process run of NSSummaryTaskWithLegacy");
+    return true;
+  }
+
+  public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) {
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    try {
+      Table<String, OmKeyInfo> keyTable =
+          omMetadataManager.getKeyTable(BUCKET_LAYOUT);
+
+      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+          keyTableIter = keyTable.iterator()) {
+
+        while (keyTableIter.hasNext()) {
+          Table.KeyValue<String, OmKeyInfo> kv = keyTableIter.next();
+          OmKeyInfo keyInfo = kv.getValue();
+
+          // KeyTable entries belong to both Legacy and OBS buckets.
+          // Check bucket layout and if it's OBS
+          // continue to the next iteration.
+          String volumeName = keyInfo.getVolumeName();
+          String bucketName = keyInfo.getBucketName();
+          String bucketDBKey = omMetadataManager
+              .getBucketKey(volumeName, bucketName);
+          // Get bucket info from bucket table
+          OmBucketInfo omBucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketDBKey);
+          // True if FileSystemPaths enabled
+          boolean enableFileSystemPaths =

Review Comment:
   same as above



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r988410702


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTaskDbEventHandler {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  private OzoneConfiguration ozoneConfiguration;
+
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager,
+                                 OzoneConfiguration
+                                     ozoneConfiguration) {
+    super(reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.ozoneConfiguration = ozoneConfiguration;
+  }
+
+  public boolean processWithLegacy(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+          WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnKeyTable = table.equals(KEY_TABLE);
+      if (!updateOnKeyTable) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+            (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+        OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+        OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+        // KeyTable entries belong to both Legacy and OBS buckets.
+        // Check bucket layout and if it's OBS
+        // continue to the next iteration.
+        // Check just for the current KeyInfo.
+        String volumeName = updatedKeyInfo.getVolumeName();
+        String bucketName = updatedKeyInfo.getBucketName();
+        String bucketDBKey = getReconOMMetadataManager()
+            .getBucketKey(volumeName, bucketName);
+        // Get bucket info from bucket table
+        OmBucketInfo omBucketInfo = getReconOMMetadataManager()
+            .getBucketTable().getSkipCache(bucketDBKey);
+        // True if FileSystemPaths enabled
+        boolean enableFileSystemPaths =

Review Comment:
   This can be set at the top of the method, outside of the while loop.



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


[GitHub] [ozone] xBis7 commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1274678299

   @dombizita @smengcl Could you take a look at this patch?


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


[GitHub] [ozone] dombizita commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
dombizita commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1252201152

   Hi @xBis7 and @GeorgeJahad, thanks for the patch and the problem description. I believe if we already separated the NSSummary handling for legacy and FSO buckets ([here](https://github.com/apache/ozone/pull/3309)) we can maintain two separate NSSummary tables, one for each bucket layout. This would help with the table clear and if one of them fails and there is a retry. Let me know what you think of this idea. cc @smengcl 


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999358413


##########
hadoop-ozone/dist/src/main/compose/ozone-legacy/README.md:
##########
@@ -0,0 +1,21 @@
+<!---

Review Comment:
   You are right, just `legacy` might give the wrong impression.



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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r1000905248


##########
hadoop-ozone/dist/src/main/compose/ozone-legacy/README.md:
##########
@@ -0,0 +1,21 @@
+<!---

Review Comment:
   Thanks!



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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r1000895186


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/nssummary/NSSummaryAdmin.java:
##########
@@ -108,6 +110,60 @@ public boolean isFileSystemOptimizedBucket(String path) throws IOException {
     }
   }
 
+  public boolean isOBSBucket(String path) throws IOException {
+    OFSPath ofsPath = new OFSPath(path);
+
+    boolean enableFileSystemPaths = getOzoneConfig()
+        .get(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");

Review Comment:
   Thanks for the change @xBis7 !
   
   Though my bad I just realized we should probably straight use `.getBoolean()` with the default value, like this:
   
   https://github.com/apache/ozone/blob/c0de6bcdebee6a4271fc73ea6542902e0eeee28f/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L207-L209
   
   ~~Otherwise when the config key is not explicitly set the `.get()` result could be `null`.~~
   Looks like the default value is populated into the config map, so at least we won't get NPE. But using `.getBoolean()` could still be the cleaner solution so we won't have to worry about the letter case.



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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r998861609


##########
hadoop-ozone/dist/src/main/compose/ozone-legacy/README.md:
##########
@@ -0,0 +1,21 @@
+<!---
+  Licensed 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
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+
+# For Legacy Bucket Operations
+
+For Legacy buckets, `ozone.om.enable.filesystem.paths` has to be true otherwise a
+Legacy buckets acts like an OBS bucket.

Review Comment:
   nit
   ```suggestion
   For Legacy buckets, set `ozone.om.enable.filesystem.paths` to `true` for them to behave like FSO buckets, otherwise
   Legacy buckets act like OBS buckets.
   ```



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskDbEventHandler.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.utils.db.RDBBatchOperation;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Class for holding all NSSummaryTask methods
+ * related to DB operations so that they can commonly be
+ * used in NSSummaryTaskWithFSO and NSSummaryTaskWithLegacy.
+ */
+public class NSSummaryTaskDbEventHandler {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskDbEventHandler.class);
+
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private ReconOMMetadataManager reconOMMetadataManager;
+
+  public NSSummaryTaskDbEventHandler(ReconNamespaceSummaryManager
+                               reconNamespaceSummaryManager,
+                            ReconOMMetadataManager
+                                reconOMMetadataManager) {

Review Comment:
   nit: fix the indentation?



##########
hadoop-ozone/dist/src/main/compose/ozone-legacy/README.md:
##########
@@ -0,0 +1,21 @@
+<!---
+  Licensed 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
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  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. See accompanying LICENSE file.
+-->
+
+# For Legacy Bucket Operations
+
+For Legacy buckets, `ozone.om.enable.filesystem.paths` has to be true otherwise a
+Legacy buckets acts like an OBS bucket.
+
+This is the same as `compose/ozone` but for testing operations that need `filesystem.paths`

Review Comment:
   ```suggestion
   This is the same as `compose/ozone` but for testing operations that need `ozone.om.enable.filesystem.paths`
   ```



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/nssummary/NSSummaryAdmin.java:
##########
@@ -108,6 +110,60 @@ public boolean isFileSystemOptimizedBucket(String path) throws IOException {
     }
   }
 
+  public boolean isOBSBucket(String path) throws IOException {

Review Comment:
   nit: either rename this to `isObjectStoreBucket` like `isFileSystemOptimizedBucket` above, or add a javadoc?



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/nssummary/NSSummaryCLIUtils.java:
##########
@@ -152,12 +152,13 @@ public static void printWithUnderline(String str, boolean newLine) {
     }
   }
 
-  public static void printFSOReminder() {
+  public static void printBucketReminder() {
     printNewLines(1);
     System.out.println(
-        "[Warning] Namespace CLI is only designed for FSO mode.\n" +
-            "Bucket being accessed must be of type FILE_SYSTEM_OPTIMIZED" +
-            " bucket layout.");
+        "[Warning] Namespace CLI is not designed for OBS mode.\n" +

Review Comment:
   ```suggestion
           "[Warning] Namespace CLI is not designed for OBS bucket layout.\n" +
   ```



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTaskDbEventHandler {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  private OzoneConfiguration ozoneConfiguration;
+
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager,
+                                 OzoneConfiguration
+                                     ozoneConfiguration) {
+    super(reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.ozoneConfiguration = ozoneConfiguration;
+  }
+
+  public boolean processWithLegacy(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+    // true if FileSystemPaths enabled
+    boolean enableFileSystemPaths =
+        ozoneConfiguration.get(OMConfigKeys
+            .OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+          WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnKeyTable = table.equals(KEY_TABLE);
+      if (!updateOnKeyTable) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+            (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+        OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+        OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+        // KeyTable entries belong to both Legacy and OBS buckets.
+        // Check bucket layout and if it's OBS
+        // continue to the next iteration.
+        // Check just for the current KeyInfo.
+        String volumeName = updatedKeyInfo.getVolumeName();
+        String bucketName = updatedKeyInfo.getBucketName();
+        String bucketDBKey = getReconOMMetadataManager()
+            .getBucketKey(volumeName, bucketName);
+        // Get bucket info from bucket table
+        OmBucketInfo omBucketInfo = getReconOMMetadataManager()
+            .getBucketTable().getSkipCache(bucketDBKey);
+
+        if (omBucketInfo.getBucketLayout()
+            .isObjectStore(enableFileSystemPaths)) {
+          continue;
+        }
+
+        setKeyParentID(updatedKeyInfo);
+
+        if (!updatedKeyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+          switch (action) {
+          case PUT:
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              setKeyParentID(oldKeyInfo);
+              handleDeleteKeyEvent(oldKeyInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                  updatedKey);
+            }
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        } else {
+          OmDirectoryInfo updatedDirectoryInfo =
+              new OmDirectoryInfo.Builder()
+                  .setName(updatedKeyInfo.getKeyName())
+                  .setObjectID(updatedKeyInfo.getObjectID())
+                  .setParentObjectID(updatedKeyInfo.getParentObjectID())
+                  .build();
+
+          OmDirectoryInfo oldDirectoryInfo = null;
+
+          if (oldKeyInfo != null) {
+            oldDirectoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(oldKeyInfo.getKeyName())
+                    .setObjectID(oldKeyInfo.getObjectID())
+                    .setParentObjectID(oldKeyInfo.getParentObjectID())
+                    .build();
+          }
+
+          switch (action) {
+          case PUT:
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                  updatedKey);
+            }
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        }
+      } catch (IOException ioEx) {
+        LOG.error("Unable to process Namespace Summary data in Recon DB. ",
+            ioEx);
+        return false;
+      }
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return false;
+    }
+
+    LOG.info("Completed a process run of NSSummaryTaskWithLegacy");
+    return true;
+  }
+
+  public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) {
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+    // true if FileSystemPaths enabled
+    boolean enableFileSystemPaths =
+        ozoneConfiguration.get(OMConfigKeys
+            .OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");

Review Comment:
   here as well



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTaskDbEventHandler {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  private OzoneConfiguration ozoneConfiguration;
+
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager,
+                                 OzoneConfiguration
+                                     ozoneConfiguration) {
+    super(reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.ozoneConfiguration = ozoneConfiguration;
+  }
+
+  public boolean processWithLegacy(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+    // true if FileSystemPaths enabled
+    boolean enableFileSystemPaths =
+        ozoneConfiguration.get(OMConfigKeys
+            .OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");

Review Comment:
   but here we would want to read the config key only once in the constructor and store it to a local variable. then use that local variable here.
   
   also note: config value case sensitivity.



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/nssummary/NSSummaryAdmin.java:
##########
@@ -108,6 +110,60 @@ public boolean isFileSystemOptimizedBucket(String path) throws IOException {
     }
   }
 
+  public boolean isOBSBucket(String path) throws IOException {
+    OFSPath ofsPath = new OFSPath(path);
+
+    boolean enableFileSystemPaths = getOzoneConfig()
+        .get(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");

Review Comment:
   Just want to mentioned that usually, we only read the config once in constructor, which stores it in a local variable in this object. For instance:
   
   https://github.com/apache/ozone/blob/a8495053ced1a04bf612b1a074353b7dcec38355/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java#L505-L507
   
   But since this is client CLI I think it's fine.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java:
##########
@@ -183,8 +185,22 @@ public static BucketHandler getBucketHandler(
                 OzoneStorageContainerManager reconSCM,
                 OmBucketInfo bucketInfo) throws IOException {
 
-    return new FSOBucketHandler(reconNamespaceSummaryManager,
-              omMetadataManager, reconSCM, bucketInfo);
+    // If bucketInfo is null then entity type is UNKNOWN
+    if (Objects.isNull(bucketInfo)) {
+      return null;
+    } else {
+      if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.FILE_SYSTEM_OPTIMIZED)) {
+        return new FSOBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.LEGACY)) {
+        return new LegacyBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else {
+        throw new RuntimeException("Unsupported bucket layout.");

Review Comment:
   Also, how is the caller expected to handle this `RuntimeException` here just in case it is thrown? Could it crash the Recon thread / Recon process?



##########
hadoop-ozone/dist/src/main/compose/ozone-legacy/README.md:
##########
@@ -0,0 +1,21 @@
+<!---

Review Comment:
   Maybe rename the directory from `ozone-legacy` to `ozone-legacy-bucket` to be more specific? `-legacy` alone sounds too generic to me. What do you think?



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java:
##########
@@ -183,8 +185,22 @@ public static BucketHandler getBucketHandler(
                 OzoneStorageContainerManager reconSCM,
                 OmBucketInfo bucketInfo) throws IOException {
 
-    return new FSOBucketHandler(reconNamespaceSummaryManager,
-              omMetadataManager, reconSCM, bucketInfo);
+    // If bucketInfo is null then entity type is UNKNOWN
+    if (Objects.isNull(bucketInfo)) {
+      return null;
+    } else {
+      if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.FILE_SYSTEM_OPTIMIZED)) {
+        return new FSOBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.LEGACY)) {
+        return new LegacyBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else {
+        throw new RuntimeException("Unsupported bucket layout.");

Review Comment:
   nit
   
   ```suggestion
           throw new RuntimeException("Unsupported bucket layout: " + bucketInfo.getBucketLayout());
   ```



##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/nssummary/NSSummaryAdmin.java:
##########
@@ -108,6 +110,60 @@ public boolean isFileSystemOptimizedBucket(String path) throws IOException {
     }
   }
 
+  public boolean isOBSBucket(String path) throws IOException {
+    OFSPath ofsPath = new OFSPath(path);
+
+    boolean enableFileSystemPaths = getOzoneConfig()
+        .get(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");

Review Comment:
   Could case sensitivity break this? e.g. `TRUE` or `True` If so:
   
   ```suggestion
           .get(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS).toLowerCase().equals("true");
   ```



##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/OMMetadataManagerTestUtils.java:
##########
@@ -235,7 +235,6 @@ public static void writeKeyToOm(OMMetadataManager omMetadataManager,
                                   List<OmKeyLocationInfoGroup> locationVersions,
                                   BucketLayout bucketLayout)
           throws IOException {
-

Review Comment:
   nit: restore the newline



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -44,166 +49,97 @@
  * For dirTable, we need to fetch the parent object (bucket or directory),
  * add the current directory's objectID to the parent object's childDir field.
  *
+ * For keyTable, the parent object is not available. Get the parent object,
+ * add it to the current object and reuse the existing methods for FSO.
+ * Only processing entries that belong to Legacy buckets. If the entry
+ * refers to a directory then build directory info object from it.
+ *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
   private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private final ReconOMMetadataManager reconOMMetadataManager;
+  private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
+  private final OzoneConfiguration ozoneConfiguration;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager,
+                       OzoneConfiguration
+                             ozoneConfiguration) {

Review Comment:
   Use more consistent indentation?



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTaskDbEventHandler {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  private OzoneConfiguration ozoneConfiguration;
+
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager,
+                                 OzoneConfiguration
+                                     ozoneConfiguration) {

Review Comment:
   nit: indentation consistency?



##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/OMMetadataManagerTestUtils.java:
##########
@@ -256,6 +255,28 @@ public static void writeKeyToOm(OMMetadataManager omMetadataManager,
                     .build());
   }
 
+  @SuppressWarnings("checkstyle:parameternumber")
+  /**
+   * Write a directory as key on OM instance.
+   * We don't need to set size.
+   * @throws IOException
+   */

Review Comment:
   nit: put annotation after javadoc
   ```suggestion
     /**
      * Write a directory as key on OM instance.
      * We don't need to set size.
      * @throws IOException
      */
     @SuppressWarnings("checkstyle:parameternumber")
   ```



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -44,166 +49,97 @@
  * For dirTable, we need to fetch the parent object (bucket or directory),
  * add the current directory's objectID to the parent object's childDir field.
  *
+ * For keyTable, the parent object is not available. Get the parent object,
+ * add it to the current object and reuse the existing methods for FSO.
+ * Only processing entries that belong to Legacy buckets. If the entry
+ * refers to a directory then build directory info object from it.
+ *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
   private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private final ReconOMMetadataManager reconOMMetadataManager;
+  private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
+  private final OzoneConfiguration ozoneConfiguration;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager,
+                       OzoneConfiguration
+                             ozoneConfiguration) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.ozoneConfiguration = ozoneConfiguration;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager,
+        reconOMMetadataManager, ozoneConfiguration);
   }
 
-  public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
-    return reconNamespaceSummaryManager;
-  }
-
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
-  }
-
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    boolean success;
+    success = nsSummaryTaskWithFSO.processWithFSO(events);
+    if (success) {
+      success = nsSummaryTaskWithLegacy.processWithLegacy(events);
     }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-
-    long dataSize = keyInfo.getDataSize();
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    // decrement count, data size, and bucket count
-    // even if there's no direct key, we still keep the entry because
-    // we still need children dir IDs info
-    nsSummary.setNumOfFiles(numOfFile - 1);
-    nsSummary.setSizeOfFiles(sizeOfFile - dataSize);
-    --fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+    return new ImmutablePair<>(getTaskName(), success);
   }
 
-  protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    Collection<Callable<Boolean>> tasks = new ArrayList<>();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.clearNSSummaryTable();
+    } catch (IOException ioEx) {
+      LOG.error("Unable to clear NSSummary table in Recon DB. ",
+          ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
     }
 
-    // Just in case the OmDirectoryInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+    tasks.add(() -> nsSummaryTaskWithFSO
+        .reprocessWithFSO(omMetadataManager));
+    tasks.add(() -> nsSummaryTaskWithLegacy
+        .reprocessWithLegacy(reconOMMetadataManager));
+
+    List<Future<Boolean>> results;
+    ExecutorService executorService = Executors
+        .newFixedThreadPool(2);
+    try {
+      results = executorService.invokeAll(tasks);

Review Comment:
   Does this imply `reprocessWithFSO` and `reprocessWithLegacy` will be run in parallel?



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,309 @@
+/*
+ * 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.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTaskDbEventHandler {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  private OzoneConfiguration ozoneConfiguration;
+
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager,
+                                 OzoneConfiguration
+                                     ozoneConfiguration) {
+    super(reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.ozoneConfiguration = ozoneConfiguration;
+  }
+
+  public boolean processWithLegacy(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+    // true if FileSystemPaths enabled
+    boolean enableFileSystemPaths =
+        ozoneConfiguration.get(OMConfigKeys
+            .OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+          WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnKeyTable = table.equals(KEY_TABLE);
+      if (!updateOnKeyTable) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+            (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+        OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+        OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+        // KeyTable entries belong to both Legacy and OBS buckets.
+        // Check bucket layout and if it's OBS
+        // continue to the next iteration.
+        // Check just for the current KeyInfo.
+        String volumeName = updatedKeyInfo.getVolumeName();
+        String bucketName = updatedKeyInfo.getBucketName();
+        String bucketDBKey = getReconOMMetadataManager()
+            .getBucketKey(volumeName, bucketName);
+        // Get bucket info from bucket table
+        OmBucketInfo omBucketInfo = getReconOMMetadataManager()
+            .getBucketTable().getSkipCache(bucketDBKey);
+
+        if (omBucketInfo.getBucketLayout()
+            .isObjectStore(enableFileSystemPaths)) {
+          continue;
+        }
+
+        setKeyParentID(updatedKeyInfo);
+
+        if (!updatedKeyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+          switch (action) {
+          case PUT:
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              setKeyParentID(oldKeyInfo);
+              handleDeleteKeyEvent(oldKeyInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                  updatedKey);
+            }
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        } else {
+          OmDirectoryInfo updatedDirectoryInfo =
+              new OmDirectoryInfo.Builder()
+                  .setName(updatedKeyInfo.getKeyName())
+                  .setObjectID(updatedKeyInfo.getObjectID())
+                  .setParentObjectID(updatedKeyInfo.getParentObjectID())
+                  .build();
+
+          OmDirectoryInfo oldDirectoryInfo = null;
+
+          if (oldKeyInfo != null) {
+            oldDirectoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(oldKeyInfo.getKeyName())
+                    .setObjectID(oldKeyInfo.getObjectID())
+                    .setParentObjectID(oldKeyInfo.getParentObjectID())
+                    .build();
+          }
+
+          switch (action) {
+          case PUT:
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                  updatedKey);
+            }
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        }
+      } catch (IOException ioEx) {
+        LOG.error("Unable to process Namespace Summary data in Recon DB. ",
+            ioEx);
+        return false;
+      }
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return false;
+    }
+
+    LOG.info("Completed a process run of NSSummaryTaskWithLegacy");
+    return true;
+  }
+
+  public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) {
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+    // true if FileSystemPaths enabled
+    boolean enableFileSystemPaths =
+        ozoneConfiguration.get(OMConfigKeys
+            .OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");
+
+    try {
+      Table<String, OmKeyInfo> keyTable =
+          omMetadataManager.getKeyTable(BUCKET_LAYOUT);
+
+      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+          keyTableIter = keyTable.iterator()) {
+
+        while (keyTableIter.hasNext()) {
+          Table.KeyValue<String, OmKeyInfo> kv = keyTableIter.next();
+          OmKeyInfo keyInfo = kv.getValue();
+
+          // KeyTable entries belong to both Legacy and OBS buckets.
+          // Check bucket layout and if it's OBS
+          // continue to the next iteration.
+          String volumeName = keyInfo.getVolumeName();
+          String bucketName = keyInfo.getBucketName();
+          String bucketDBKey = omMetadataManager
+              .getBucketKey(volumeName, bucketName);
+          // Get bucket info from bucket table
+          OmBucketInfo omBucketInfo = omMetadataManager
+              .getBucketTable().getSkipCache(bucketDBKey);
+
+          if (omBucketInfo.getBucketLayout()
+              .isObjectStore(enableFileSystemPaths)) {
+            continue;
+          }
+
+          setKeyParentID(keyInfo);
+
+          if (keyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+            OmDirectoryInfo directoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(keyInfo.getKeyName())
+                    .setObjectID(keyInfo.getObjectID())
+                    .setParentObjectID(keyInfo.getParentObjectID())
+                    .build();
+            handlePutDirEvent(directoryInfo, nsSummaryMap);
+          } else {
+            handlePutKeyEvent(keyInfo, nsSummaryMap);
+          }
+        }
+      }
+    } catch (IOException ioEx) {
+      LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ",
+          ioEx);
+      return false;
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return false;
+    }
+    LOG.info("Completed a reprocess run of NSSummaryTaskWithLegacy");
+    return true;
+  }
+
+  /**
+   * KeyTable entries don't have the parentId set.
+   * In order to reuse the existing FSO methods that rely on
+   * the parentId, we have to set it explicitly.
+   * @param keyInfo
+   * @throws IOException
+   */
+  private void setKeyParentID(OmKeyInfo keyInfo) throws IOException {
+    String[] keyPath = keyInfo.getKeyName().split(OM_KEY_PREFIX);
+
+    //if (keyPath > 1) there is one or more directories

Review Comment:
   nit: rephrase this comment (make this line of comment more readable)



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -44,166 +49,97 @@
  * For dirTable, we need to fetch the parent object (bucket or directory),
  * add the current directory's objectID to the parent object's childDir field.
  *
+ * For keyTable, the parent object is not available. Get the parent object,
+ * add it to the current object and reuse the existing methods for FSO.
+ * Only processing entries that belong to Legacy buckets. If the entry
+ * refers to a directory then build directory info object from it.
+ *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
   private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private final ReconOMMetadataManager reconOMMetadataManager;
+  private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
+  private final OzoneConfiguration ozoneConfiguration;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager,
+                       OzoneConfiguration
+                             ozoneConfiguration) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.ozoneConfiguration = ozoneConfiguration;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager,
+        reconOMMetadataManager, ozoneConfiguration);
   }
 
-  public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
-    return reconNamespaceSummaryManager;
-  }
-
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
-  }
-
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    boolean success;
+    success = nsSummaryTaskWithFSO.processWithFSO(events);
+    if (success) {
+      success = nsSummaryTaskWithLegacy.processWithLegacy(events);
     }

Review Comment:
   more logging
   ```suggestion
       } else {
           LOG.error("processWithFSO failed");
       }
   ```



##########
hadoop-ozone/dist/src/main/smoketest/recon/recon-nssummary.robot:
##########
@@ -131,7 +132,8 @@ Check Recon Namespace Summary Key
     Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/file1   KEY
 
 Check Recon Namespace Summary Directory
-    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2   DIRECTORY
+    Run Keyword If    '${BUCKET_LAYOUT}' == 'LEGACY'                    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2/   DIRECTORY

Review Comment:
   @dombizita IIUC, the command after this line 135 will only be executed when `'${BUCKET_LAYOUT}' == 'LEGACY'`.
   
   If bucket layout is FSO, it will run the command line below instead, with the subtle difference of the trailing slash in the directory path.
   
   I haven't checked myself whether the trailing slash is required for `LEGACY` here with `Wait For Summary`. @xBis7 Any insights into this? 



##########
hadoop-ozone/dist/src/main/compose/ozone-legacy/docker-config:
##########
@@ -0,0 +1,51 @@
+# 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
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# 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.
+
+CORE-SITE.XML_fs.defaultFS=ofs://om
+CORE-SITE.XML_fs.trash.interval=1
+
+OZONE-SITE.XML_ozone.om.address=om
+OZONE-SITE.XML_ozone.om.enable.filesystem.paths=true

Review Comment:
   +1 this. @xBis7 do you want to add the extra config line here?



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


[GitHub] [ozone] smengcl merged pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl merged PR #3746:
URL: https://github.com/apache/ozone/pull/3746


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


[GitHub] [ozone] xBis7 commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1285911362

   @smengcl I pushed the latest changes for getting `FileSystemPaths` value from configuration.


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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999888466


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -44,166 +49,97 @@
  * For dirTable, we need to fetch the parent object (bucket or directory),
  * add the current directory's objectID to the parent object's childDir field.
  *
+ * For keyTable, the parent object is not available. Get the parent object,
+ * add it to the current object and reuse the existing methods for FSO.
+ * Only processing entries that belong to Legacy buckets. If the entry
+ * refers to a directory then build directory info object from it.
+ *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
   private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private final ReconOMMetadataManager reconOMMetadataManager;
+  private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
+  private final OzoneConfiguration ozoneConfiguration;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager,
+                       OzoneConfiguration
+                             ozoneConfiguration) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.ozoneConfiguration = ozoneConfiguration;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager,
+        reconOMMetadataManager, ozoneConfiguration);
   }
 
-  public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
-    return reconNamespaceSummaryManager;
-  }
-
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
-  }
-
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    boolean success;
+    success = nsSummaryTaskWithFSO.processWithFSO(events);
+    if (success) {
+      success = nsSummaryTaskWithLegacy.processWithLegacy(events);
     }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-
-    long dataSize = keyInfo.getDataSize();
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    // decrement count, data size, and bucket count
-    // even if there's no direct key, we still keep the entry because
-    // we still need children dir IDs info
-    nsSummary.setNumOfFiles(numOfFile - 1);
-    nsSummary.setSizeOfFiles(sizeOfFile - dataSize);
-    --fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+    return new ImmutablePair<>(getTaskName(), success);
   }
 
-  protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    Collection<Callable<Boolean>> tasks = new ArrayList<>();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.clearNSSummaryTable();
+    } catch (IOException ioEx) {
+      LOG.error("Unable to clear NSSummary table in Recon DB. ",
+          ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
     }
 
-    // Just in case the OmDirectoryInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+    tasks.add(() -> nsSummaryTaskWithFSO
+        .reprocessWithFSO(omMetadataManager));
+    tasks.add(() -> nsSummaryTaskWithLegacy
+        .reprocessWithLegacy(reconOMMetadataManager));
+
+    List<Future<Boolean>> results;
+    ExecutorService executorService = Executors
+        .newFixedThreadPool(2);
+    try {
+      results = executorService.invokeAll(tasks);

Review Comment:
   I see. As long as this won't cause similar issue we discussed much earlier (one thread clearing NSSummaryTable while another is writing to it) it should be fine. I see `clearNSSummaryTable()` being called a few line above invoke here.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -44,166 +49,97 @@
  * For dirTable, we need to fetch the parent object (bucket or directory),
  * add the current directory's objectID to the parent object's childDir field.
  *
+ * For keyTable, the parent object is not available. Get the parent object,
+ * add it to the current object and reuse the existing methods for FSO.
+ * Only processing entries that belong to Legacy buckets. If the entry
+ * refers to a directory then build directory info object from it.
+ *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
   private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private final ReconOMMetadataManager reconOMMetadataManager;
+  private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
+  private final OzoneConfiguration ozoneConfiguration;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager,
+                       OzoneConfiguration
+                             ozoneConfiguration) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.ozoneConfiguration = ozoneConfiguration;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager,
+        reconOMMetadataManager, ozoneConfiguration);
   }
 
-  public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
-    return reconNamespaceSummaryManager;
-  }
-
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
-  }
-
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    boolean success;
+    success = nsSummaryTaskWithFSO.processWithFSO(events);
+    if (success) {
+      success = nsSummaryTaskWithLegacy.processWithLegacy(events);
     }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-
-    long dataSize = keyInfo.getDataSize();
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    // decrement count, data size, and bucket count
-    // even if there's no direct key, we still keep the entry because
-    // we still need children dir IDs info
-    nsSummary.setNumOfFiles(numOfFile - 1);
-    nsSummary.setSizeOfFiles(sizeOfFile - dataSize);
-    --fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+    return new ImmutablePair<>(getTaskName(), success);
   }
 
-  protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    Collection<Callable<Boolean>> tasks = new ArrayList<>();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.clearNSSummaryTable();
+    } catch (IOException ioEx) {
+      LOG.error("Unable to clear NSSummary table in Recon DB. ",
+          ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
     }
 
-    // Just in case the OmDirectoryInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+    tasks.add(() -> nsSummaryTaskWithFSO
+        .reprocessWithFSO(omMetadataManager));
+    tasks.add(() -> nsSummaryTaskWithLegacy
+        .reprocessWithLegacy(reconOMMetadataManager));
+
+    List<Future<Boolean>> results;
+    ExecutorService executorService = Executors
+        .newFixedThreadPool(2);
+    try {
+      results = executorService.invokeAll(tasks);

Review Comment:
   I see. As long as this won't cause similar issue we discussed much earlier (one thread clearing NSSummaryTable while another is writing to it) it should be fine. I see `clearNSSummaryTable()` being called a few lines above invoke here.



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984117821


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,734 @@
+/**
+ * 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.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.RDBBatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconTestInjector;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.ClassRule;
+import org.junit.Assert;
+import org.junit.experimental.runners.Enclosed;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_DB_DIRS;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeDirToOm;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProvider;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager;
+
+/**
+ * Test for NSSummaryTaskWithLegacy.
+ */
+@RunWith(Enclosed.class)
+public final class TestNSSummaryTaskWithLegacy {
+
+  @ClassRule
+  public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder();
+
+  private static ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private static OMMetadataManager omMetadataManager;
+  private static ReconOMMetadataManager reconOMMetadataManager;
+  private static NSSummaryTaskWithLegacy nSSummaryTaskWithLegacy;
+
+  // Object names
+  private static final String VOL = "vol";
+  private static final String BUCKET_ONE = "bucket1";
+  private static final String BUCKET_TWO = "bucket2";
+  private static final String KEY_ONE = "file1";
+  private static final String KEY_TWO = "file2";
+  private static final String KEY_THREE = "dir1/dir2/file3";
+  private static final String KEY_FOUR = "file4";
+  private static final String KEY_FIVE = "file5";
+  private static final String FILE_ONE = "file1";
+  private static final String FILE_TWO = "file2";
+  private static final String FILE_THREE = "file3";
+  private static final String FILE_FOUR = "file4";
+  private static final String FILE_FIVE = "file5";
+  private static final String DIR_ONE = "dir1";
+  private static final String DIR_ONE_RENAME = "dir1_new";
+  private static final String DIR_TWO = "dir2";
+  private static final String DIR_THREE = "dir3";
+  private static final String DIR_FOUR = "dir4";
+  private static final String DIR_FIVE = "dir5";
+
+  private static final String TEST_USER = "TestUser";
+
+  private static final long VOL_OBJECT_ID = 0L;
+  private static final long BUCKET_ONE_OBJECT_ID = 1L;
+  private static final long BUCKET_TWO_OBJECT_ID = 2L;
+  private static final long KEY_ONE_OBJECT_ID = 3L;
+  private static final long DIR_ONE_OBJECT_ID = 4L;
+  private static final long KEY_TWO_OBJECT_ID = 5L;
+  private static final long KEY_FOUR_OBJECT_ID = 6L;
+  private static final long DIR_TWO_OBJECT_ID = 7L;
+  private static final long KEY_THREE_OBJECT_ID = 8L;
+  private static final long KEY_FIVE_OBJECT_ID = 9L;
+  private static final long DIR_THREE_OBJECT_ID = 10L;
+  private static final long DIR_FOUR_OBJECT_ID = 11L;
+  private static final long DIR_FIVE_OBJECT_ID = 12L;
+
+  private static final long KEY_ONE_SIZE = 500L;
+  private static final long KEY_TWO_OLD_SIZE = 1025L;
+  private static final long KEY_TWO_UPDATE_SIZE = 1023L;
+  private static final long KEY_THREE_SIZE =
+      ReconConstants.MAX_FILE_SIZE_UPPER_BOUND - 100L;
+  private static final long KEY_FOUR_SIZE = 2050L;
+  private static final long KEY_FIVE_SIZE = 100L;
+
+  private static Set<Long> bucketOneAns = new HashSet<>();
+  private static Set<Long> bucketTwoAns = new HashSet<>();
+  private static Set<Long> dirOneAns = new HashSet<>();
+
+  private TestNSSummaryTaskWithLegacy() {
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    initializeNewOmMetadataManager(TEMPORARY_FOLDER.newFolder());
+    OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
+        getMockOzoneManagerServiceProvider();
+    reconOMMetadataManager = getTestReconOmMetadataManager(omMetadataManager,
+        TEMPORARY_FOLDER.newFolder());
+
+    ReconTestInjector reconTestInjector =
+        new ReconTestInjector.Builder(TEMPORARY_FOLDER)
+            .withReconOm(reconOMMetadataManager)
+            .withOmServiceProvider(ozoneManagerServiceProvider)
+            .withReconSqlDb()
+            .withContainerDB()
+            .build();
+    reconNamespaceSummaryManager =
+        reconTestInjector.getInstance(ReconNamespaceSummaryManager.class);
+
+    NSSummary nonExistentSummary =
+        reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID);
+    Assert.assertNull(nonExistentSummary);
+
+    populateOMDB();
+
+    nSSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+  }
+
+  /**
+   * Nested class for testing NSSummaryTaskWithLegacy reprocess.
+   */
+  public static class TestReprocess {
+
+    private static NSSummary nsSummaryForBucket1;
+    private static NSSummary nsSummaryForBucket2;
+
+    @BeforeClass
+    public static void setUp() throws IOException {
+      // write a NSSummary prior to reprocess
+      // verify it got cleaned up after.
+      NSSummary staleNSSummary = new NSSummary();
+      RDBBatchOperation rdbBatchOperation = new RDBBatchOperation();
+      reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation, -1L,
+          staleNSSummary);
+      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
+
+      // Verify commit
+      Assert.assertNotNull(reconNamespaceSummaryManager.getNSSummary(-1L));
+
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.clearNSSummaryTable();
+
+      nSSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager);
+      Assert.assertNull(reconNamespaceSummaryManager.getNSSummary(-1L));
+
+      nsSummaryForBucket1 =
+          reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID);
+      nsSummaryForBucket2 =
+          reconNamespaceSummaryManager.getNSSummary(BUCKET_TWO_OBJECT_ID);
+      Assert.assertNotNull(nsSummaryForBucket1);
+      Assert.assertNotNull(nsSummaryForBucket2);
+    }
+
+    @Test
+    public void testReprocessNSSummaryNull() throws IOException {
+      Assert.assertNull(reconNamespaceSummaryManager.getNSSummary(-1L));
+    }
+
+    @Test
+    public void testReprocessGetFiles() {
+      Assert.assertEquals(1, nsSummaryForBucket1.getNumOfFiles());
+      Assert.assertEquals(2, nsSummaryForBucket2.getNumOfFiles());
+
+      Assert.assertEquals(KEY_ONE_SIZE, nsSummaryForBucket1.getSizeOfFiles());
+      Assert.assertEquals(KEY_TWO_OLD_SIZE + KEY_FOUR_SIZE,
+          nsSummaryForBucket2.getSizeOfFiles());
+    }
+
+    @Test
+    public void testReprocessFileBucketSize() {
+      int[] fileDistBucket1 = nsSummaryForBucket1.getFileSizeBucket();
+      int[] fileDistBucket2 = nsSummaryForBucket2.getFileSizeBucket();
+      Assert.assertEquals(ReconConstants.NUM_OF_BINS, fileDistBucket1.length);
+      Assert.assertEquals(ReconConstants.NUM_OF_BINS, fileDistBucket2.length);
+
+      Assert.assertEquals(1, fileDistBucket1[0]);
+      for (int i = 1; i < ReconConstants.NUM_OF_BINS; ++i) {
+        Assert.assertEquals(0, fileDistBucket1[i]);
+      }
+      Assert.assertEquals(1, fileDistBucket2[1]);
+      Assert.assertEquals(1, fileDistBucket2[2]);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        if (i == 1 || i == 2) {
+          continue;
+        }
+        Assert.assertEquals(0, fileDistBucket2[i]);
+      }
+    }
+
+    @Test
+    public void testReprocessBucketDirs() {
+      // Bucket one has one dir, bucket two has none.
+      Set<Long> childDirBucketOne = nsSummaryForBucket1.getChildDir();
+      Set<Long> childDirBucketTwo = nsSummaryForBucket2.getChildDir();
+      Assert.assertEquals(1, childDirBucketOne.size());
+      bucketOneAns.clear();
+      bucketOneAns.add(DIR_ONE_OBJECT_ID);
+      Assert.assertEquals(bucketOneAns, childDirBucketOne);
+      Assert.assertEquals(0, childDirBucketTwo.size());
+    }
+
+    @Test
+    public void testReprocessDirsUnderDir() throws Exception {
+
+      // Dir 1 has two dir: dir2 and dir3.
+      NSSummary nsSummaryInDir1 = reconNamespaceSummaryManager
+          .getNSSummary(DIR_ONE_OBJECT_ID);
+      Assert.assertNotNull(nsSummaryInDir1);
+      Set<Long> childDirForDirOne = nsSummaryInDir1.getChildDir();
+      Assert.assertEquals(2, childDirForDirOne.size());
+      dirOneAns.clear();
+      dirOneAns.add(DIR_TWO_OBJECT_ID);
+      dirOneAns.add(DIR_THREE_OBJECT_ID);
+      Assert.assertEquals(dirOneAns, childDirForDirOne);
+
+      NSSummary nsSummaryInDir2 = reconNamespaceSummaryManager
+          .getNSSummary(DIR_TWO_OBJECT_ID);
+      Assert.assertEquals(1, nsSummaryInDir2.getNumOfFiles());
+      Assert.assertEquals(KEY_THREE_SIZE, nsSummaryInDir2.getSizeOfFiles());
+
+      int[] fileDistForDir2 = nsSummaryInDir2.getFileSizeBucket();
+      Assert.assertEquals(ReconConstants.NUM_OF_BINS, fileDistForDir2.length);
+      Assert.assertEquals(1, fileDistForDir2[fileDistForDir2.length - 1]);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS - 1; ++i) {
+        Assert.assertEquals(0, fileDistForDir2[i]);
+      }
+      Assert.assertEquals(0, nsSummaryInDir2.getChildDir().size());
+
+      // bucket should have empty dirName
+      Assert.assertEquals(0, nsSummaryForBucket1.getDirName().length());
+      Assert.assertEquals(0, nsSummaryForBucket2.getDirName().length());
+      // check dirName is correctly written
+      Assert.assertEquals(DIR_ONE, nsSummaryInDir1.getDirName());
+      Assert.assertEquals(DIR_ONE + OM_KEY_PREFIX + DIR_TWO,
+          nsSummaryInDir2.getDirName());
+    }
+  }
+
+  /**
+   * Nested class for testing NSSummaryTaskWithLegacy process.
+   */
+  public static class TestProcess {
+
+    private static NSSummary nsSummaryForBucket1;
+    private static NSSummary nsSummaryForBucket2;
+
+    private static OMDBUpdateEvent keyEvent1;
+    private static OMDBUpdateEvent keyEvent2;
+    private static OMDBUpdateEvent keyEvent3;
+    private static OMDBUpdateEvent keyEvent4;
+    private static OMDBUpdateEvent keyEvent5;
+    private static OMDBUpdateEvent keyEvent6;
+    private static OMDBUpdateEvent keyEvent7;
+
+    @BeforeClass
+    public static void setUp() throws IOException {
+      nSSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager);
+      nSSummaryTaskWithLegacy.processWithLegacy(processEventBatch());
+
+      nsSummaryForBucket1 =
+          reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID);
+      Assert.assertNotNull(nsSummaryForBucket1);
+      nsSummaryForBucket2 =
+          reconNamespaceSummaryManager.getNSSummary(BUCKET_TWO_OBJECT_ID);
+      Assert.assertNotNull(nsSummaryForBucket2);
+    }
+
+    private static OMUpdateEventBatch processEventBatch() throws IOException {
+      // put file5 under bucket 2
+      String omPutKey =
+          OM_KEY_PREFIX + VOL +
+              OM_KEY_PREFIX + BUCKET_TWO +
+              OM_KEY_PREFIX + FILE_FIVE;
+      OmKeyInfo omPutKeyInfo = buildOmKeyInfo(VOL, BUCKET_TWO, KEY_FIVE,
+          FILE_FIVE, KEY_FIVE_OBJECT_ID, BUCKET_TWO_OBJECT_ID, KEY_FIVE_SIZE);
+      keyEvent1 = new OMDBUpdateEvent.
+          OMUpdateEventBuilder<String, OmKeyInfo>()
+          .setKey(omPutKey)
+          .setValue(omPutKeyInfo)
+          .setTable(omMetadataManager.getKeyTable(getBucketLayout())
+              .getName())
+          .setAction(OMDBUpdateEvent.OMDBUpdateAction.PUT)
+          .build();
+
+      // delete file 1 under bucket 1
+      String omDeleteKey =
+          OM_KEY_PREFIX + VOL +
+              OM_KEY_PREFIX + BUCKET_ONE +
+              OM_KEY_PREFIX + FILE_ONE;
+      OmKeyInfo omDeleteInfo = buildOmKeyInfo(
+          VOL, BUCKET_ONE, KEY_ONE,
+          FILE_ONE, KEY_ONE_OBJECT_ID, BUCKET_ONE_OBJECT_ID);
+      keyEvent2 = new OMDBUpdateEvent.
+          OMUpdateEventBuilder<String, OmKeyInfo>()
+          .setKey(omDeleteKey)
+          .setValue(omDeleteInfo)
+          .setTable(omMetadataManager.getKeyTable(getBucketLayout())
+              .getName())
+          .setAction(OMDBUpdateEvent.OMDBUpdateAction.DELETE)
+          .build();
+
+      // update file 2's size under bucket 2
+      String omUpdateKey =
+          OM_KEY_PREFIX + VOL +
+              OM_KEY_PREFIX + BUCKET_TWO +
+              OM_KEY_PREFIX + FILE_TWO;
+      OmKeyInfo omOldInfo = buildOmKeyInfo(
+          VOL, BUCKET_TWO, KEY_TWO, FILE_TWO,
+          KEY_TWO_OBJECT_ID, BUCKET_TWO_OBJECT_ID, KEY_TWO_OLD_SIZE);
+      OmKeyInfo omUpdateInfo = buildOmKeyInfo(
+          VOL, BUCKET_TWO, KEY_TWO, FILE_TWO,
+          KEY_TWO_OBJECT_ID, BUCKET_TWO_OBJECT_ID, KEY_TWO_UPDATE_SIZE);
+      keyEvent3 = new OMDBUpdateEvent.
+          OMUpdateEventBuilder<String, OmKeyInfo>()
+          .setKey(omUpdateKey)
+          .setValue(omUpdateInfo)
+          .setOldValue(omOldInfo)
+          .setTable(omMetadataManager.getKeyTable(getBucketLayout())
+              .getName())
+          .setAction(OMDBUpdateEvent.OMDBUpdateAction.UPDATE)
+          .build();
+
+      // add dir 4 under bucket 1
+      String omDirPutKey1 =
+          OM_KEY_PREFIX + VOL +
+              OM_KEY_PREFIX + BUCKET_ONE +
+              OM_KEY_PREFIX + DIR_FOUR + OM_KEY_PREFIX;
+      OmKeyInfo omDirPutValue1 = buildOmDirKeyInfo(VOL, BUCKET_ONE,
+          (DIR_FOUR + OM_KEY_PREFIX), DIR_FOUR,
+          DIR_FOUR_OBJECT_ID);
+      keyEvent4 = new OMDBUpdateEvent.
+          OMUpdateEventBuilder<String, OmKeyInfo>()
+          .setKey(omDirPutKey1)
+          .setValue(omDirPutValue1)
+          .setAction(OMDBUpdateEvent.OMDBUpdateAction.PUT)
+          .setTable(omMetadataManager.getKeyTable(getBucketLayout()).getName())
+          .build();
+
+      // add dir 5 under bucket 2
+      String omDirPutKey2 =
+          OM_KEY_PREFIX + VOL +
+              OM_KEY_PREFIX + BUCKET_TWO +
+              OM_KEY_PREFIX + DIR_FIVE + OM_KEY_PREFIX;
+      OmKeyInfo omDirPutValue2 = buildOmDirKeyInfo(VOL, BUCKET_TWO,
+          (DIR_FIVE + OM_KEY_PREFIX), DIR_FIVE,
+          DIR_FIVE_OBJECT_ID);
+      keyEvent5 = new OMDBUpdateEvent.
+          OMUpdateEventBuilder<String, OmKeyInfo>()
+          .setKey(omDirPutKey2)
+          .setValue(omDirPutValue2)
+          .setAction(OMDBUpdateEvent.OMDBUpdateAction.PUT)
+          .setTable(omMetadataManager.getKeyTable(getBucketLayout()).getName())
+          .build();
+
+      // delete dir 3 under dir 1
+      String omDirDeleteKey =
+          OM_KEY_PREFIX + VOL +
+              OM_KEY_PREFIX + BUCKET_ONE +
+              OM_KEY_PREFIX + DIR_ONE +
+              OM_KEY_PREFIX + DIR_THREE + OM_KEY_PREFIX;
+      OmKeyInfo omDirDeleteValue = buildOmKeyInfo(VOL, BUCKET_ONE,
+          (DIR_ONE + OM_KEY_PREFIX + DIR_THREE + OM_KEY_PREFIX),
+          DIR_THREE, DIR_THREE_OBJECT_ID, DIR_ONE_OBJECT_ID);
+      keyEvent6 = new OMDBUpdateEvent.
+          OMUpdateEventBuilder<String, OmKeyInfo>()
+          .setKey(omDirDeleteKey)
+          .setValue(omDirDeleteValue)
+          .setAction(OMDBUpdateEvent.OMDBUpdateAction.DELETE)
+          .setTable(omMetadataManager.getKeyTable(getBucketLayout()).getName())
+          .build();
+
+      // rename dir1
+      String omDirUpdateKey =
+          OM_KEY_PREFIX + VOL +
+              OM_KEY_PREFIX + BUCKET_ONE +
+              OM_KEY_PREFIX + DIR_ONE + OM_KEY_PREFIX;
+      OmKeyInfo omDirOldValue = buildOmDirKeyInfo(VOL, BUCKET_ONE,
+          (DIR_ONE + OM_KEY_PREFIX), DIR_ONE,
+          DIR_ONE_OBJECT_ID);
+      OmKeyInfo omDirUpdateValue = buildOmDirKeyInfo(VOL, BUCKET_ONE,
+          (DIR_ONE_RENAME + OM_KEY_PREFIX), DIR_ONE_RENAME,
+          DIR_ONE_OBJECT_ID);
+      keyEvent7 = new OMDBUpdateEvent.
+          OMUpdateEventBuilder<String, OmKeyInfo>()
+          .setKey(omDirUpdateKey)
+          .setValue(omDirUpdateValue)
+          .setOldValue(omDirOldValue)
+          .setAction(OMDBUpdateEvent.OMDBUpdateAction.UPDATE)
+          .setTable(omMetadataManager.getKeyTable(getBucketLayout()).getName())
+          .build();
+
+      OMUpdateEventBatch omUpdateEventBatch = new OMUpdateEventBatch(
+          new ArrayList<OMDBUpdateEvent>() {{
+              add(keyEvent1);
+              add(keyEvent2);
+              add(keyEvent3);
+              add(keyEvent4);
+              add(keyEvent5);
+              add(keyEvent6);
+              add(keyEvent7);
+              }});
+
+      return omUpdateEventBatch;
+    }
+
+    @Test
+    public void testProcessUpdateFileSize() throws IOException {
+      // file 1 is gone, so bucket 1 is empty now
+      Assert.assertNotNull(nsSummaryForBucket1);
+      Assert.assertEquals(0, nsSummaryForBucket1.getNumOfFiles());
+
+      Set<Long> childDirBucket1 = nsSummaryForBucket1.getChildDir();
+      // after put dir4, bucket1 now has two child dirs: dir1 and dir4
+      Assert.assertEquals(2, childDirBucket1.size());
+      bucketOneAns.clear();
+      bucketOneAns.add(DIR_ONE_OBJECT_ID);
+      bucketOneAns.add(DIR_FOUR_OBJECT_ID);
+      Assert.assertEquals(bucketOneAns, childDirBucket1);
+    }
+
+    @Test
+    public void testProcessBucket() throws IOException {
+      // file 5 is added under bucket 2, so bucket 2 has 3 keys now
+      // file 2 is updated with new datasize,
+      // so file size dist for bucket 2 should be updated
+      Assert.assertNotNull(nsSummaryForBucket2);
+      Assert.assertEquals(3, nsSummaryForBucket2.getNumOfFiles());
+      // key 4 + key 5 + updated key 2
+      Assert.assertEquals(KEY_FOUR_SIZE + KEY_FIVE_SIZE + KEY_TWO_UPDATE_SIZE,
+          nsSummaryForBucket2.getSizeOfFiles());
+
+      int[] fileSizeDist = nsSummaryForBucket2.getFileSizeBucket();
+      Assert.assertEquals(ReconConstants.NUM_OF_BINS, fileSizeDist.length);
+      // 1023L and 100L
+      Assert.assertEquals(2, fileSizeDist[0]);
+      // 2050L
+      Assert.assertEquals(1, fileSizeDist[2]);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        if (i == 0 || i == 2) {
+          continue;
+        }
+        Assert.assertEquals(0, fileSizeDist[i]);
+      }
+
+      // after put dir5, bucket 2 now has one dir
+      Set<Long> childDirBucket2 = nsSummaryForBucket2.getChildDir();
+      Assert.assertEquals(1, childDirBucket2.size());
+      bucketTwoAns.add(DIR_FIVE_OBJECT_ID);
+      Assert.assertEquals(bucketTwoAns, childDirBucket2);
+    }
+
+    @Test
+    public void testProcessDirDeleteRename() throws IOException {
+      // after delete dir 3, dir 1 now has only one dir: dir2
+      NSSummary nsSummaryForDir1 = reconNamespaceSummaryManager
+          .getNSSummary(DIR_ONE_OBJECT_ID);
+      Assert.assertNotNull(nsSummaryForDir1);
+      Set<Long> childDirForDir1 = nsSummaryForDir1.getChildDir();
+      Assert.assertEquals(1, childDirForDir1.size());
+      dirOneAns.clear();
+      dirOneAns.add(DIR_TWO_OBJECT_ID);
+      Assert.assertEquals(dirOneAns, childDirForDir1);
+
+      // after renaming dir1, check its new name
+      Assert.assertEquals(DIR_ONE_RENAME, nsSummaryForDir1.getDirName());
+    }
+  }
+
+  /**
+   * Build a key info for put/update action.
+   * @param volume volume name
+   * @param bucket bucket name
+   * @param key key name
+   * @param fileName file name
+   * @param objectID object ID
+   * @param parentObjectId parent object ID
+   * @param dataSize file size
+   * @return the KeyInfo
+   */
+  private static OmKeyInfo buildOmKeyInfo(String volume,
+                                          String bucket,
+                                          String key,
+                                          String fileName,
+                                          long objectID,
+                                          long parentObjectId,
+                                          long dataSize) {
+    return new OmKeyInfo.Builder()
+        .setBucketName(bucket)
+        .setVolumeName(volume)
+        .setKeyName(key)
+        .setFileName(fileName)
+        .setReplicationConfig(
+            StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+        .setObjectID(objectID)
+        .setParentObjectID(parentObjectId)
+        .setDataSize(dataSize)
+        .build();
+  }
+
+  /**
+   * Build a key info for delete action.
+   * @param volume volume name
+   * @param bucket bucket name
+   * @param key key name
+   * @param fileName file name
+   * @param objectID object ID
+   * @param parentObjectId parent object ID
+   * @return the KeyInfo
+   */
+  private static OmKeyInfo buildOmKeyInfo(String volume,
+                                          String bucket,
+                                          String key,
+                                          String fileName,
+                                          long objectID,
+                                          long parentObjectId) {
+    return new OmKeyInfo.Builder()
+        .setBucketName(bucket)
+        .setVolumeName(volume)
+        .setKeyName(key)
+        .setFileName(fileName)
+        .setReplicationConfig(
+            StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+        .setObjectID(objectID)
+        .setParentObjectID(parentObjectId)
+        .build();
+  }
+
+  /**
+   * Build a directory as key info for put/update action.
+   * We don't need to set size.
+   * @param volume volume name
+   * @param bucket bucket name
+   * @param key key name
+   * @param fileName file name
+   * @param objectID object ID
+   * @return the KeyInfo
+   */
+  private static OmKeyInfo buildOmDirKeyInfo(String volume,
+                                             String bucket,
+                                             String key,
+                                             String fileName,
+                                             long objectID) {
+    return new OmKeyInfo.Builder()
+        .setBucketName(bucket)
+        .setVolumeName(volume)
+        .setKeyName(key)
+        .setFileName(fileName)
+        .setReplicationConfig(
+            StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+        .setObjectID(objectID)
+        .build();
+  }
+
+  /**
+   * Populate OMDB with the following configs.
+   *              vol
+   *            /     \
+   *        bucket1   bucket2
+   *        /    \      /    \
+   *     file1  dir1  file2  file4
+   *            /   \
+   *         dir2   dir3
+   *          /
+   *        file3
+   *
+   * @throws IOException
+   */
+  private static void populateOMDB() throws IOException {
+    writeKeyToOm(reconOMMetadataManager,
+        KEY_ONE,
+        BUCKET_ONE,
+        VOL,
+        FILE_ONE,
+        KEY_ONE_OBJECT_ID,
+        BUCKET_ONE_OBJECT_ID,

Review Comment:
   The parent object id for legacy records is 0, (which is why we added the setKeyParentId() method.)  Let's set them back to 0 in the test code as well, just to be sure that setKeyParentId() is really working. 



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984120821


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.inject.Inject;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTaskUtils {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager) {
+    super(reconNamespaceSummaryManager, reconOMMetadataManager);
+  }
+
+  public boolean processWithLegacy(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+          WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnKeyTable = table.equals(KEY_TABLE);
+      if (!updateOnKeyTable) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+            (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+        OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+        OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+        setKeyParentID(updatedKeyInfo);
+
+        if (!updatedKeyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+          switch (action) {
+          case PUT:
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              setKeyParentID(oldKeyInfo);
+              handleDeleteKeyEvent(oldKeyInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                  updatedKey);
+            }
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        } else {
+          OmDirectoryInfo updatedDirectoryInfo =
+              new OmDirectoryInfo.Builder()
+                  .setName(updatedKeyInfo.getKeyName())
+                  .setObjectID(updatedKeyInfo.getObjectID())
+                  .setParentObjectID(updatedKeyInfo.getParentObjectID())
+                  .build();
+
+          OmDirectoryInfo oldDirectoryInfo = null;
+
+          if (oldKeyInfo != null) {
+            oldDirectoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(oldKeyInfo.getKeyName())
+                    .setObjectID(oldKeyInfo.getObjectID())
+                    .setParentObjectID(oldKeyInfo.getParentObjectID())
+                    .build();
+          }
+
+          switch (action) {
+          case PUT:
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                  updatedKey);
+            }
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        }
+      } catch (IOException ioEx) {
+        LOG.error("Unable to process Namespace Summary data in Recon DB. ",
+            ioEx);
+        return false;
+      }
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return false;
+    }
+
+    LOG.info("Completed a process run of NSSummaryTaskWithLegacy");
+    return true;
+  }
+
+  public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) {
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    try {
+      Table<String, OmKeyInfo> keyTable =
+          omMetadataManager.getKeyTable(BUCKET_LAYOUT);
+
+      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+          keyTableIter = keyTable.iterator()) {

Review Comment:
   It just occurred to me that since both the legacy and the obs keys are both in the keytable, and this task is just for legacy keys, it will have to explicitly skip obs keys.
   



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


[GitHub] [ozone] GeorgeJahad commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1248697782

   So there is a problem with clearing the NSSummary table during reprocess().
   
   Both the fso and legacy tasks write to the nssummary table. The reprocess() method for FSO clears the nssummary table [here](https://github.com/apache/ozone/blob/39c0c25c07e359a1e5f0968a2143db1dc9da3c19/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithFSO.java#L176) The Legacy task doesn't clear the table.
   
   But both tasks run in separate threads in [parallel](https://github.com/apache/ozone/blob/39c0c25c07e359a1e5f0968a2143db1dc9da3c19/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java#L172-L179)  There is no guarantee that FSO will run first.
   
   In addition failed tasks get rerun [here](https://github.com/apache/ozone/blob/39c0c25c07e359a1e5f0968a2143db1dc9da3c19/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ReconTaskControllerImpl.java#L120-L131)  If one fails and the other doesn't the delete won't work correctly.
   


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


[GitHub] [ozone] dombizita commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
dombizita commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r980944454


##########
hadoop-ozone/dist/src/main/smoketest/recon/recon-nssummary.robot:
##########
@@ -14,7 +14,7 @@
 # limitations under the License.
 
 *** Settings ***
-Documentation       Smoke test for Recon Namespace Summary Endpoint for FSO buckets.
+Documentation       Smoke test for Recon Namespace Summary Endpoint for FSO and Legacy buckets.

Review Comment:
   If you are also testing the `LEGACY` buckets here why did you added a `${BUCKET_LAYOUT}` variable below that is set to `FILE_SYSTEM_OPTIMIZED`? Will you add test cases in this robot test file for `LEGACY` too? 



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/LegacyBucketHandler.java:
##########
@@ -0,0 +1,318 @@
+/*
+ * 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
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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.api.handlers;
+
+import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+
+/**
+ * Class for handling Legacy buckets.
+ */
+public class LegacyBucketHandler extends BucketHandler {
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      LegacyBucketHandler.class);
+
+  private final String vol;
+  private final String bucket;
+  private final OmBucketInfo omBucketInfo;
+
+  public LegacyBucketHandler(
+      ReconNamespaceSummaryManager reconNamespaceSummaryManager,
+      ReconOMMetadataManager omMetadataManager,
+      OzoneStorageContainerManager reconSCM,
+      OmBucketInfo bucketInfo) {
+    super(reconNamespaceSummaryManager, omMetadataManager,
+        reconSCM);
+    this.omBucketInfo = bucketInfo;
+    this.vol = omBucketInfo.getVolumeName();
+    this.bucket = omBucketInfo.getBucketName();
+  }
+
+  /**
+   * Helper function to check if a path is a directory, key, or invalid.
+   * @param keyName key name
+   * @return DIRECTORY, KEY, or UNKNOWN
+   * @throws IOException
+   */
+  @Override
+  public EntityType determineKeyPath(String keyName)
+      throws IOException {
+
+    String filename = OzoneFSUtils.removeTrailingSlashIfNeeded(keyName);
+    // For example, /vol1/buck1/a/b/c/d/e/file1.txt
+    // Look in the KeyTable for the key path,
+    // if the first one we seek to is the same as the seek key,
+    // it is a key;
+    // if it is the seekKey with a trailing slash, it is a directory
+    // else it is unknown
+    String key = OM_KEY_PREFIX + vol +
+        OM_KEY_PREFIX + bucket +
+        OM_KEY_PREFIX + filename;
+
+    Table<String, OmKeyInfo> keyTable = getKeyTable();
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+        iterator = keyTable.iterator();
+
+    iterator.seek(key);
+    if (iterator.hasNext()) {
+      Table.KeyValue<String, OmKeyInfo> kv = iterator.next();
+      String dbKey = kv.getKey();
+      if (dbKey.equals(key)) {
+        return EntityType.KEY;
+      }
+      if (dbKey.equals(key + OM_KEY_PREFIX)) {
+        return EntityType.DIRECTORY;
+      }
+    }
+    return EntityType.UNKNOWN;
+  }
+
+  // KeyTable's key is in the format of "vol/bucket/keyName"
+  // Make use of RocksDB's order to seek to the prefix and avoid full iteration
+  @Override
+  public long calculateDUUnderObject(long parentId)

Review Comment:
   Could you expand this method's documentation here? 



##########
hadoop-ozone/dist/src/main/smoketest/recon/recon-nssummary.robot:
##########
@@ -131,7 +132,8 @@ Check Recon Namespace Summary Key
     Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/file1   KEY
 
 Check Recon Namespace Summary Directory
-    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2   DIRECTORY
+    Run Keyword If    '${BUCKET_LAYOUT}' == 'LEGACY'                    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2/   DIRECTORY

Review Comment:
   If the `${BUCKET_LAYOUT}` is set to `FILE_SYSTEM_OPTIMIZED` why are we checking if it is `LEGACY`? Is there a place where we change it? What kind of response do we expect here?



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984118375


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestNSSummaryEndpointWithLegacy.java:
##########
@@ -0,0 +1,1286 @@
+/*
+ * 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.api;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconTestInjector;
+import org.apache.hadoop.ozone.recon.api.handlers.BucketHandler;
+import org.apache.hadoop.ozone.recon.api.handlers.EntityHandler;
+import org.apache.hadoop.ozone.recon.api.types.NamespaceSummaryResponse;
+import org.apache.hadoop.ozone.recon.api.types.DUResponse;
+import org.apache.hadoop.ozone.recon.api.types.EntityType;
+import org.apache.hadoop.ozone.recon.api.types.FileSizeDistributionResponse;
+import org.apache.hadoop.ozone.recon.api.types.ResponseStatus;
+import org.apache.hadoop.ozone.recon.api.types.QuotaUsageResponse;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl;
+import org.apache.hadoop.ozone.recon.spi.impl.StorageContainerServiceProviderImpl;
+import org.apache.hadoop.ozone.recon.tasks.NSSummaryTaskWithLegacy;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import javax.ws.rs.core.Response;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Set;
+import java.util.HashSet;
+
+import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_DB_DIRS;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeDirToOm;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProvider;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Test for NSSummary REST APIs with Legacy.
+ * We tested on a mini file system with the following setting:
+ *                vol
+ *             /       \
+ *        bucket1      bucket2
+ *        /    \         /    \
+ *     file1    dir1    file4  file5
+ *           /   \   \
+ *        dir2  dir3  dir4
+ *         /     \      \
+ *       file2   file3  file6
+ *  ----------------------------------------
+ *                  vol2
+ *              /         \
+ *      bucket3          bucket4
+ *      /      \           /
+ *   file8     dir5      file11
+ *            /    \
+ *        file9    file10
+ * This is a test for the Rest APIs only. We have tested NSSummaryTask before,
+ * so there is no need to test process() on DB's updates
+ */
+public class TestNSSummaryEndpointWithLegacy {
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private ReconOMMetadataManager reconOMMetadataManager;
+  private NSSummaryEndpoint nsSummaryEndpoint;
+
+  private static final String TEST_PATH_UTILITY =
+      "/vol1/buck1/a/b/c/d/e/file1.txt";
+  private static final String PARENT_DIR = "vol1/buck1/a/b/c/d/e";
+  private static final String[] TEST_NAMES =
+      new String[]{"vol1", "buck1", "a", "b", "c", "d", "e", "file1.txt"};
+  private static final String TEST_KEY_NAMES = "a/b/c/d/e/file1.txt";
+
+  // Object names
+  private static final String VOL = "vol";
+  private static final String VOL_TWO = "vol2";
+  private static final String BUCKET_ONE = "bucket1";
+  private static final String BUCKET_TWO = "bucket2";
+  private static final String BUCKET_THREE = "bucket3";
+  private static final String BUCKET_FOUR = "bucket4";
+  private static final String KEY_ONE = "file1";
+  private static final String KEY_TWO = "dir1/dir2/file2";
+  private static final String KEY_THREE = "dir1/dir3/file3";
+  private static final String KEY_FOUR = "file4";
+  private static final String KEY_FIVE = "file5";
+  private static final String KEY_SIX = "dir1/dir4/file6";
+  private static final String KEY_SEVEN = "dir1/file7";
+  private static final String KEY_EIGHT = "file8";
+  private static final String KEY_NINE = "dir5/file9";
+  private static final String KEY_TEN = "dir5/file10";
+  private static final String KEY_ELEVEN = "file11";
+  private static final String MULTI_BLOCK_KEY = "dir1/file7";
+  private static final String MULTI_BLOCK_FILE = "file7";
+
+  private static final String FILE_ONE = "file1";
+  private static final String FILE_TWO = "file2";
+  private static final String FILE_THREE = "file3";
+  private static final String FILE_FOUR = "file4";
+  private static final String FILE_FIVE = "file5";
+  private static final String FILE_SIX = "file6";
+  private static final String FILE_SEVEN = "file7";
+  private static final String FILE_EIGHT = "file8";
+  private static final String FILE_NINE = "file9";
+  private static final String FILE_TEN = "file10";
+  private static final String FILE_ELEVEN = "file11";
+
+  private static final String DIR_ONE = "dir1";
+  private static final String DIR_TWO = "dir2";
+  private static final String DIR_THREE = "dir3";
+  private static final String DIR_FOUR = "dir4";
+  private static final String DIR_FIVE = "dir5";
+  // objects IDs
+  private static final long VOL_OBJECT_ID = 0L;
+  private static final long BUCKET_ONE_OBJECT_ID = 1L;
+  private static final long BUCKET_TWO_OBJECT_ID = 2L;
+  private static final long KEY_ONE_OBJECT_ID = 3L;
+  private static final long DIR_ONE_OBJECT_ID = 4L;
+  private static final long KEY_TWO_OBJECT_ID = 5L;
+  private static final long KEY_FOUR_OBJECT_ID = 6L;
+  private static final long DIR_TWO_OBJECT_ID = 7L;
+  private static final long KEY_THREE_OBJECT_ID = 8L;
+  private static final long KEY_FIVE_OBJECT_ID = 9L;
+  private static final long KEY_SIX_OBJECT_ID = 10L;
+  private static final long DIR_THREE_OBJECT_ID = 11L;
+  private static final long DIR_FOUR_OBJECT_ID = 12L;
+  private static final long MULTI_BLOCK_KEY_OBJECT_ID = 13L;
+  private static final long KEY_SEVEN_OBJECT_ID = 13L;
+  private static final long VOL_TWO_OBJECT_ID = 14L;
+  private static final long BUCKET_THREE_OBJECT_ID = 15L;
+  private static final long BUCKET_FOUR_OBJECT_ID = 16L;
+  private static final long KEY_EIGHT_OBJECT_ID = 17L;
+  private static final long DIR_FIVE_OBJECT_ID = 18L;
+  private static final long KEY_NINE_OBJECT_ID = 19L;
+  private static final long KEY_TEN_OBJECT_ID = 20L;
+  private static final long KEY_ELEVEN_OBJECT_ID = 21L;
+
+  // container IDs
+  private static final long CONTAINER_ONE_ID = 1L;
+  private static final long CONTAINER_TWO_ID = 2L;
+  private static final long CONTAINER_THREE_ID = 3L;
+  private static final long CONTAINER_FOUR_ID = 4L;
+  private static final long CONTAINER_FIVE_ID = 5L;
+  private static final long CONTAINER_SIX_ID = 6L;
+
+  // replication factors
+  private static final int CONTAINER_ONE_REPLICA_COUNT  = 3;
+  private static final int CONTAINER_TWO_REPLICA_COUNT  = 2;
+  private static final int CONTAINER_THREE_REPLICA_COUNT  = 4;
+  private static final int CONTAINER_FOUR_REPLICA_COUNT  = 5;
+  private static final int CONTAINER_FIVE_REPLICA_COUNT  = 2;
+  private static final int CONTAINER_SIX_REPLICA_COUNT  = 3;
+
+  // block lengths
+  private static final long BLOCK_ONE_LENGTH = 1000L;
+  private static final long BLOCK_TWO_LENGTH = 2000L;
+  private static final long BLOCK_THREE_LENGTH = 3000L;
+  private static final long BLOCK_FOUR_LENGTH = 4000L;
+  private static final long BLOCK_FIVE_LENGTH = 5000L;
+  private static final long BLOCK_SIX_LENGTH = 6000L;
+
+  // data size in bytes
+  private static final long KEY_ONE_SIZE = 500L; // bin 0
+  private static final long KEY_TWO_SIZE = OzoneConsts.KB + 1; // bin 1
+  private static final long KEY_THREE_SIZE = 4 * OzoneConsts.KB + 1; // bin 3
+  private static final long KEY_FOUR_SIZE = 2 * OzoneConsts.KB + 1; // bin 2
+  private static final long KEY_FIVE_SIZE = 100L; // bin 0
+  private static final long KEY_SIX_SIZE = 2 * OzoneConsts.KB + 1; // bin 2
+  private static final long KEY_EIGHT_SIZE = OzoneConsts.KB + 1; // bin 1
+  private static final long KEY_NINE_SIZE = 2 * OzoneConsts.KB + 1; // bin 2
+  private static final long KEY_TEN_SIZE = 2 * OzoneConsts.KB + 1; // bin 2
+  private static final long KEY_ELEVEN_SIZE = OzoneConsts.KB + 1; // bin 1
+  private static final long LOCATION_INFO_GROUP_ONE_SIZE
+          = CONTAINER_ONE_REPLICA_COUNT * BLOCK_ONE_LENGTH
+          + CONTAINER_TWO_REPLICA_COUNT * BLOCK_TWO_LENGTH
+          + CONTAINER_THREE_REPLICA_COUNT * BLOCK_THREE_LENGTH;
+
+  private static final long MULTI_BLOCK_KEY_SIZE_WITH_REPLICA
+          = LOCATION_INFO_GROUP_ONE_SIZE;
+
+  private static final long LOCATION_INFO_GROUP_TWO_SIZE
+      = CONTAINER_FOUR_REPLICA_COUNT * BLOCK_FOUR_LENGTH
+      + CONTAINER_FIVE_REPLICA_COUNT * BLOCK_FIVE_LENGTH
+      + CONTAINER_SIX_REPLICA_COUNT * BLOCK_SIX_LENGTH;
+
+  private static final long FILE1_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+  private static final long FILE2_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_TWO_SIZE;
+  private static final long FILE3_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+  private static final long FILE4_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_TWO_SIZE;
+  private static final long FILE5_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+  private static final long FILE6_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_TWO_SIZE;
+  private static final long FILE7_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+  private static final long FILE8_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_TWO_SIZE;
+  private static final long FILE9_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+  private static final long FILE10_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_TWO_SIZE;
+  private static final long FILE11_SIZE_WITH_REPLICA =
+      LOCATION_INFO_GROUP_ONE_SIZE;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_ROOT
+      = FILE1_SIZE_WITH_REPLICA
+      + FILE2_SIZE_WITH_REPLICA
+      + FILE3_SIZE_WITH_REPLICA
+      + FILE4_SIZE_WITH_REPLICA
+      + FILE5_SIZE_WITH_REPLICA
+      + FILE6_SIZE_WITH_REPLICA
+      + FILE7_SIZE_WITH_REPLICA
+      + FILE8_SIZE_WITH_REPLICA
+      + FILE9_SIZE_WITH_REPLICA
+      + FILE10_SIZE_WITH_REPLICA
+      + FILE11_SIZE_WITH_REPLICA;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_VOL
+      = FILE1_SIZE_WITH_REPLICA
+      + FILE2_SIZE_WITH_REPLICA
+      + FILE3_SIZE_WITH_REPLICA
+      + FILE4_SIZE_WITH_REPLICA
+      + FILE5_SIZE_WITH_REPLICA
+      + FILE6_SIZE_WITH_REPLICA
+      + FILE7_SIZE_WITH_REPLICA;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_BUCKET1
+      = FILE1_SIZE_WITH_REPLICA
+      + FILE2_SIZE_WITH_REPLICA
+      + FILE3_SIZE_WITH_REPLICA
+      + FILE6_SIZE_WITH_REPLICA
+      + FILE7_SIZE_WITH_REPLICA;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_DIR1
+      = FILE2_SIZE_WITH_REPLICA
+      + FILE3_SIZE_WITH_REPLICA
+      + FILE6_SIZE_WITH_REPLICA
+      + FILE7_SIZE_WITH_REPLICA;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_DIR2
+      = FILE2_SIZE_WITH_REPLICA;
+
+  private static final long
+      MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_KEY
+      = FILE4_SIZE_WITH_REPLICA;
+
+  // quota in bytes
+  private static final long ROOT_QUOTA = 2 * (2 * OzoneConsts.MB);
+  private static final long VOL_QUOTA = 2 * OzoneConsts.MB;
+  private static final long VOL_TWO_QUOTA = 2 * OzoneConsts.MB;
+  private static final long BUCKET_ONE_QUOTA = OzoneConsts.MB;
+  private static final long BUCKET_TWO_QUOTA = OzoneConsts.MB;
+  private static final long BUCKET_THREE_QUOTA = OzoneConsts.MB;
+  private static final long BUCKET_FOUR_QUOTA = OzoneConsts.MB;
+
+  // mock client's path requests
+  private static final String TEST_USER = "TestUser";
+  private static final String ROOT_PATH = "/";
+  private static final String VOL_PATH = "/vol";
+  private static final String VOL_TWO_PATH = "/vol2";
+  private static final String BUCKET_ONE_PATH = "/vol/bucket1";
+  private static final String BUCKET_TWO_PATH = "/vol/bucket2";
+  private static final String DIR_ONE_PATH = "/vol/bucket1/dir1";
+  private static final String DIR_TWO_PATH = "/vol/bucket1/dir1/dir2";
+  private static final String DIR_THREE_PATH = "/vol/bucket1/dir1/dir3";
+  private static final String DIR_FOUR_PATH = "/vol/bucket1/dir1/dir4";
+  private static final String KEY_PATH = "/vol/bucket2/file4";
+  private static final String MULTI_BLOCK_KEY_PATH = "/vol/bucket1/dir1/file7";
+  private static final String INVALID_PATH = "/vol/path/not/found";
+
+  // some expected answers
+  private static final long ROOT_DATA_SIZE = KEY_ONE_SIZE + KEY_TWO_SIZE +
+      KEY_THREE_SIZE + KEY_FOUR_SIZE + KEY_FIVE_SIZE + KEY_SIX_SIZE +
+      KEY_EIGHT_SIZE + KEY_NINE_SIZE + KEY_TEN_SIZE + KEY_ELEVEN_SIZE;
+  private static final long VOL_DATA_SIZE = KEY_ONE_SIZE + KEY_TWO_SIZE +
+      KEY_THREE_SIZE + KEY_FOUR_SIZE + KEY_FIVE_SIZE + KEY_SIX_SIZE;
+
+  private static final long VOL_TWO_DATA_SIZE =
+      KEY_EIGHT_SIZE + KEY_NINE_SIZE + KEY_TEN_SIZE + KEY_ELEVEN_SIZE;
+
+  private static final long BUCKET_ONE_DATA_SIZE = KEY_ONE_SIZE + KEY_TWO_SIZE +
+      KEY_THREE_SIZE + KEY_SIX_SIZE;
+
+  private static final long BUCKET_TWO_DATA_SIZE =
+      KEY_FOUR_SIZE + KEY_FIVE_SIZE;
+
+  private static final long DIR_ONE_DATA_SIZE = KEY_TWO_SIZE +
+      KEY_THREE_SIZE + KEY_SIX_SIZE;
+
+  @Before
+  public void setUp() throws Exception {
+    OMMetadataManager omMetadataManager = initializeNewOmMetadataManager(
+        temporaryFolder.newFolder());
+    OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
+        getMockOzoneManagerServiceProvider();
+    reconOMMetadataManager = getTestReconOmMetadataManager(omMetadataManager, 
+        temporaryFolder.newFolder());
+
+    ReconTestInjector reconTestInjector =
+        new ReconTestInjector.Builder(temporaryFolder)
+            .withReconOm(reconOMMetadataManager)
+            .withOmServiceProvider(ozoneManagerServiceProvider)
+            .withReconSqlDb()
+            .withContainerDB()
+            .addBinding(OzoneStorageContainerManager.class,
+                getMockReconSCM())
+            .addBinding(StorageContainerServiceProvider.class,
+                mock(StorageContainerServiceProviderImpl.class))
+            .addBinding(NSSummaryEndpoint.class)
+            .build();
+    ReconNamespaceSummaryManager reconNamespaceSummaryManager =
+        reconTestInjector.getInstance(ReconNamespaceSummaryManager.class);
+    nsSummaryEndpoint = reconTestInjector.getInstance(NSSummaryEndpoint.class);
+
+    // populate OM DB and reprocess into Recon RocksDB
+    populateOMDB();
+    NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy = 
+        new NSSummaryTaskWithLegacy(reconNamespaceSummaryManager, 
+                                    reconOMMetadataManager);
+    nsSummaryTaskWithLegacy.reprocessWithLegacy(reconOMMetadataManager);
+  }
+
+  @Test
+  public void testUtility() {
+    String[] names = EntityHandler.parseRequestPath(TEST_PATH_UTILITY);
+    Assert.assertArrayEquals(TEST_NAMES, names);
+    String keyName = BucketHandler.getKeyName(names);
+    Assert.assertEquals(TEST_KEY_NAMES, keyName);
+    String subpath = BucketHandler.buildSubpath(PARENT_DIR, "file1.txt");
+    Assert.assertEquals(TEST_PATH_UTILITY, subpath);
+  }
+
+  @Test
+  public void testGetBasicInfoRoot() throws Exception {
+    // Test root basics
+    Response rootResponse = nsSummaryEndpoint.getBasicInfo(ROOT_PATH);
+    NamespaceSummaryResponse rootResponseObj =
+        (NamespaceSummaryResponse) rootResponse.getEntity();
+    Assert.assertEquals(EntityType.ROOT, rootResponseObj.getEntityType());
+    Assert.assertEquals(2, rootResponseObj.getNumVolume());
+    Assert.assertEquals(4, rootResponseObj.getNumBucket());
+    Assert.assertEquals(5, rootResponseObj.getNumTotalDir());
+    Assert.assertEquals(10, rootResponseObj.getNumTotalKey());
+  }
+
+  @Test
+  public void testGetBasicInfoVol() throws Exception {
+    // Test volume basics
+    Response volResponse = nsSummaryEndpoint.getBasicInfo(VOL_PATH);
+    NamespaceSummaryResponse volResponseObj =
+        (NamespaceSummaryResponse) volResponse.getEntity();
+    Assert.assertEquals(EntityType.VOLUME, volResponseObj.getEntityType());
+    Assert.assertEquals(2, volResponseObj.getNumBucket());
+    Assert.assertEquals(4, volResponseObj.getNumTotalDir());
+    Assert.assertEquals(6, volResponseObj.getNumTotalKey());
+  }
+
+  @Test
+  public void testGetBasicInfoBucketOne() throws Exception {
+    // Test bucket 1's basics
+    Response bucketOneResponse =
+        nsSummaryEndpoint.getBasicInfo(BUCKET_ONE_PATH);
+    NamespaceSummaryResponse bucketOneObj =
+        (NamespaceSummaryResponse) bucketOneResponse.getEntity();
+    Assert.assertEquals(EntityType.BUCKET, bucketOneObj.getEntityType());
+    Assert.assertEquals(4, bucketOneObj.getNumTotalDir());
+    Assert.assertEquals(4, bucketOneObj.getNumTotalKey());
+  }
+
+  @Test
+  public void testGetBasicInfoBucketTwo() throws Exception {
+    // Test bucket 2's basics
+    Response bucketTwoResponse =
+        nsSummaryEndpoint.getBasicInfo(BUCKET_TWO_PATH);
+    NamespaceSummaryResponse bucketTwoObj =
+        (NamespaceSummaryResponse) bucketTwoResponse.getEntity();
+    Assert.assertEquals(EntityType.BUCKET, bucketTwoObj.getEntityType());
+    Assert.assertEquals(0, bucketTwoObj.getNumTotalDir());
+    Assert.assertEquals(2, bucketTwoObj.getNumTotalKey());
+  }
+
+  @Test
+  public void testGetBasicInfoDir() throws Exception {
+    // Test intermediate directory basics
+    Response dirOneResponse = nsSummaryEndpoint.getBasicInfo(DIR_ONE_PATH);
+    NamespaceSummaryResponse dirOneObj =
+        (NamespaceSummaryResponse) dirOneResponse.getEntity();
+    Assert.assertEquals(EntityType.DIRECTORY, dirOneObj.getEntityType());
+    Assert.assertEquals(3, dirOneObj.getNumTotalDir());
+    Assert.assertEquals(3, dirOneObj.getNumTotalKey());
+  }
+
+  @Test
+  public void testGetBasicInfoNoPath() throws Exception {
+    // Test invalid path
+    Response invalidResponse = nsSummaryEndpoint.getBasicInfo(INVALID_PATH);
+    NamespaceSummaryResponse invalidObj =
+        (NamespaceSummaryResponse) invalidResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.PATH_NOT_FOUND,
+        invalidObj.getStatus());
+  }
+
+  @Test
+  public void testGetBasicInfoKey() throws Exception {
+    // Test key
+    Response keyResponse = nsSummaryEndpoint.getBasicInfo(KEY_PATH);
+    NamespaceSummaryResponse keyResObj =
+        (NamespaceSummaryResponse) keyResponse.getEntity();
+    Assert.assertEquals(EntityType.KEY, keyResObj.getEntityType());
+  }
+
+  @Test
+  public void testDiskUsageRoot() throws Exception {
+    // root level DU
+    Response rootResponse = nsSummaryEndpoint.getDiskUsage(ROOT_PATH,
+        false, false);
+    DUResponse duRootRes = (DUResponse) rootResponse.getEntity();
+    Assert.assertEquals(2, duRootRes.getCount());
+    List<DUResponse.DiskUsage> duRootData = duRootRes.getDuData();
+    // sort based on subpath
+    Collections.sort(duRootData,
+        Comparator.comparing(DUResponse.DiskUsage::getSubpath));
+    DUResponse.DiskUsage duVol1 = duRootData.get(0);
+    DUResponse.DiskUsage duVol2 = duRootData.get(1);
+    Assert.assertEquals(VOL_PATH, duVol1.getSubpath());
+    Assert.assertEquals(VOL_TWO_PATH, duVol2.getSubpath());
+    Assert.assertEquals(VOL_DATA_SIZE, duVol1.getSize());
+    Assert.assertEquals(VOL_TWO_DATA_SIZE, duVol2.getSize());
+  }
+  @Test
+  public void testDiskUsageVolume() throws Exception {
+    // volume level DU
+    Response volResponse = nsSummaryEndpoint.getDiskUsage(VOL_PATH,
+        false, false);
+    DUResponse duVolRes = (DUResponse) volResponse.getEntity();
+    Assert.assertEquals(2, duVolRes.getCount());
+    List<DUResponse.DiskUsage> duData = duVolRes.getDuData();
+    // sort based on subpath
+    Collections.sort(duData,
+        Comparator.comparing(DUResponse.DiskUsage::getSubpath));
+    DUResponse.DiskUsage duBucket1 = duData.get(0);
+    DUResponse.DiskUsage duBucket2 = duData.get(1);
+    Assert.assertEquals(BUCKET_ONE_PATH, duBucket1.getSubpath());
+    Assert.assertEquals(BUCKET_TWO_PATH, duBucket2.getSubpath());
+    Assert.assertEquals(BUCKET_ONE_DATA_SIZE, duBucket1.getSize());
+    Assert.assertEquals(BUCKET_TWO_DATA_SIZE, duBucket2.getSize());
+
+  }
+  @Test
+  public void testDiskUsageBucket() throws Exception {
+    // bucket level DU
+    Response bucketResponse = nsSummaryEndpoint.getDiskUsage(BUCKET_ONE_PATH,
+        false, false);
+    DUResponse duBucketResponse = (DUResponse) bucketResponse.getEntity();
+    Assert.assertEquals(1, duBucketResponse.getCount());
+    DUResponse.DiskUsage duDir1 = duBucketResponse.getDuData().get(0);
+    Assert.assertEquals(DIR_ONE_PATH, duDir1.getSubpath());
+    Assert.assertEquals(DIR_ONE_DATA_SIZE, duDir1.getSize());
+
+  }
+  @Test
+  public void testDiskUsageDir() throws Exception {
+    // dir level DU
+    Response dirResponse = nsSummaryEndpoint.getDiskUsage(DIR_ONE_PATH,
+        false, false);
+    DUResponse duDirReponse = (DUResponse) dirResponse.getEntity();
+    Assert.assertEquals(3, duDirReponse.getCount());
+    List<DUResponse.DiskUsage> duSubDir = duDirReponse.getDuData();
+    Collections.sort(duSubDir,
+        Comparator.comparing(DUResponse.DiskUsage::getSubpath));
+    DUResponse.DiskUsage duDir2 = duSubDir.get(0);
+    DUResponse.DiskUsage duDir3 = duSubDir.get(1);
+    DUResponse.DiskUsage duDir4 = duSubDir.get(2);
+    Assert.assertEquals(DIR_TWO_PATH, duDir2.getSubpath());
+    Assert.assertEquals(KEY_TWO_SIZE, duDir2.getSize());
+
+    Assert.assertEquals(DIR_THREE_PATH, duDir3.getSubpath());
+    Assert.assertEquals(KEY_THREE_SIZE, duDir3.getSize());
+
+    Assert.assertEquals(DIR_FOUR_PATH, duDir4.getSubpath());
+    Assert.assertEquals(KEY_SIX_SIZE, duDir4.getSize());
+
+  }
+  @Test
+  public void testDiskUsageKey() throws Exception {
+    // key level DU
+    Response keyResponse = nsSummaryEndpoint.getDiskUsage(KEY_PATH,
+        false, false);
+    DUResponse keyObj = (DUResponse) keyResponse.getEntity();
+    Assert.assertEquals(0, keyObj.getCount());
+    Assert.assertEquals(KEY_FOUR_SIZE, keyObj.getSize());
+
+  }
+  @Test
+  public void testDiskUsageUnknown() throws Exception {
+    // invalid path check
+    Response invalidResponse = nsSummaryEndpoint.getDiskUsage(INVALID_PATH,
+        false, false);
+    DUResponse invalidObj = (DUResponse) invalidResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.PATH_NOT_FOUND,
+        invalidObj.getStatus());
+  }
+
+  @Test
+  public void testDiskUsageWithReplication() throws Exception {
+    setUpMultiBlockKey();
+    Response keyResponse = nsSummaryEndpoint.getDiskUsage(MULTI_BLOCK_KEY_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) keyResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_KEY_SIZE_WITH_REPLICA,
+        replicaDUResponse.getSizeWithReplica());
+  }
+
+  @Test
+  public void testDataSizeUnderRootWithReplication() throws IOException {
+    setUpMultiBlockReplicatedKeys();
+    //   withReplica is true
+    Response rootResponse = nsSummaryEndpoint.getDiskUsage(ROOT_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) rootResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_ROOT,
+        replicaDUResponse.getSizeWithReplica());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_VOL,
+        replicaDUResponse.getDuData().get(0).getSizeWithReplica());
+
+  }
+
+  @Test
+  public void testDataSizeUnderVolWithReplication() throws IOException {
+    setUpMultiBlockReplicatedKeys();
+    Response volResponse = nsSummaryEndpoint.getDiskUsage(VOL_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) volResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_VOL,
+        replicaDUResponse.getSizeWithReplica());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_BUCKET1,
+        replicaDUResponse.getDuData().get(0).getSizeWithReplica());
+  }
+
+  @Test
+  public void testDataSizeUnderBucketWithReplication() throws IOException {
+    setUpMultiBlockReplicatedKeys();
+    Response bucketResponse = nsSummaryEndpoint.getDiskUsage(BUCKET_ONE_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) bucketResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_BUCKET1,
+        replicaDUResponse.getSizeWithReplica());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_DIR1,
+        replicaDUResponse.getDuData().get(0).getSizeWithReplica());
+  }
+
+  /**
+   * When calculating DU under dir1
+   * there are 3 keys, file2, file3, file6.
+   * There is one direct key, file7.
+   * @throws IOException
+   */
+  @Test
+  public void testDataSizeUnderDirWithReplication() throws IOException {
+    setUpMultiBlockReplicatedKeys();
+    Response dir1Response = nsSummaryEndpoint.getDiskUsage(DIR_ONE_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) dir1Response.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_DIR1,
+        replicaDUResponse.getSizeWithReplica());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_DIR2,
+        replicaDUResponse.getDuData().get(0).getSizeWithReplica());
+  }
+
+  @Test
+  public void testDataSizeUnderKeyWithReplication() throws IOException {
+    setUpMultiBlockReplicatedKeys();
+    Response keyResponse = nsSummaryEndpoint.getDiskUsage(KEY_PATH,
+        false, true);
+    DUResponse replicaDUResponse = (DUResponse) keyResponse.getEntity();
+    Assert.assertEquals(ResponseStatus.OK, replicaDUResponse.getStatus());
+    Assert.assertEquals(MULTI_BLOCK_TOTAL_SIZE_WITH_REPLICA_UNDER_KEY,
+        replicaDUResponse.getSizeWithReplica());
+  }
+
+  @Test
+  public void testQuotaUsage() throws Exception {
+    // root level quota usage
+    Response rootResponse = nsSummaryEndpoint.getQuotaUsage(ROOT_PATH);
+    QuotaUsageResponse quRootRes =
+        (QuotaUsageResponse) rootResponse.getEntity();
+    Assert.assertEquals(ROOT_QUOTA, quRootRes.getQuota());
+    Assert.assertEquals(ROOT_DATA_SIZE, quRootRes.getQuotaUsed());
+
+    // volume level quota usage
+    Response volResponse = nsSummaryEndpoint.getQuotaUsage(VOL_PATH);
+    QuotaUsageResponse quVolRes = (QuotaUsageResponse) volResponse.getEntity();
+    Assert.assertEquals(VOL_QUOTA, quVolRes.getQuota());
+    Assert.assertEquals(VOL_DATA_SIZE, quVolRes.getQuotaUsed());
+
+    // bucket level quota usage
+    Response bucketRes = nsSummaryEndpoint.getQuotaUsage(BUCKET_ONE_PATH);
+    QuotaUsageResponse quBucketRes = (QuotaUsageResponse) bucketRes.getEntity();
+    Assert.assertEquals(BUCKET_ONE_QUOTA, quBucketRes.getQuota());
+    Assert.assertEquals(BUCKET_ONE_DATA_SIZE, quBucketRes.getQuotaUsed());
+
+    Response bucketRes2 = nsSummaryEndpoint.getQuotaUsage(BUCKET_TWO_PATH);
+    QuotaUsageResponse quBucketRes2 =
+        (QuotaUsageResponse) bucketRes2.getEntity();
+    Assert.assertEquals(BUCKET_TWO_QUOTA, quBucketRes2.getQuota());
+    Assert.assertEquals(BUCKET_TWO_DATA_SIZE, quBucketRes2.getQuotaUsed());
+
+    // other level not applicable
+    Response naResponse1 = nsSummaryEndpoint.getQuotaUsage(DIR_ONE_PATH);
+    QuotaUsageResponse quotaUsageResponse1 =
+        (QuotaUsageResponse) naResponse1.getEntity();
+    Assert.assertEquals(ResponseStatus.TYPE_NOT_APPLICABLE,
+        quotaUsageResponse1.getResponseCode());
+
+    Response naResponse2 = nsSummaryEndpoint.getQuotaUsage(KEY_PATH);
+    QuotaUsageResponse quotaUsageResponse2 =
+        (QuotaUsageResponse) naResponse2.getEntity();
+    Assert.assertEquals(ResponseStatus.TYPE_NOT_APPLICABLE,
+        quotaUsageResponse2.getResponseCode());
+
+    // invalid path request
+    Response invalidRes = nsSummaryEndpoint.getQuotaUsage(INVALID_PATH);
+    QuotaUsageResponse invalidResObj =
+        (QuotaUsageResponse) invalidRes.getEntity();
+    Assert.assertEquals(ResponseStatus.PATH_NOT_FOUND,
+        invalidResObj.getResponseCode());
+  }
+
+
+  @Test
+  public void testFileSizeDist() throws Exception {
+    checkFileSizeDist(ROOT_PATH, 2, 3, 4, 1);
+    checkFileSizeDist(VOL_PATH, 2, 1, 2, 1);
+    checkFileSizeDist(BUCKET_ONE_PATH, 1, 1, 1, 1);
+    checkFileSizeDist(DIR_ONE_PATH, 0, 1, 1, 1);
+  }
+
+  public void checkFileSizeDist(String path, int bin0,
+      int bin1, int bin2, int bin3) throws Exception {
+    Response res = nsSummaryEndpoint.getFileSizeDistribution(path);
+    FileSizeDistributionResponse fileSizeDistResObj =
+            (FileSizeDistributionResponse) res.getEntity();
+    int[] fileSizeDist = fileSizeDistResObj.getFileSizeDist();
+    Assert.assertEquals(bin0, fileSizeDist[0]);
+    Assert.assertEquals(bin1, fileSizeDist[1]);
+    Assert.assertEquals(bin2, fileSizeDist[2]);
+    Assert.assertEquals(bin3, fileSizeDist[3]);
+    for (int i = 4; i < ReconConstants.NUM_OF_BINS; ++i) {
+      Assert.assertEquals(0, fileSizeDist[i]);
+    }
+  }
+
+  /**
+   * Write directories and keys info into OM DB.
+   * @throws Exception
+   */
+  @SuppressWarnings("checkstyle:MethodLength")
+  private void populateOMDB() throws Exception {
+    // write all directories
+    writeDirToOm(reconOMMetadataManager,
+          (DIR_ONE + OM_KEY_PREFIX),
+          BUCKET_ONE,
+          VOL,
+          DIR_ONE,
+          DIR_ONE_OBJECT_ID,
+          BUCKET_ONE_OBJECT_ID,

Review Comment:
   The parent object id for legacy records is 0, (which is why we added the setKeyParentId() method.) Let's set them back to 0 in the test code as well, just to be sure that setKeyParentId() is really working.
   



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


[GitHub] [ozone] GeorgeJahad commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1270549079

   LGTM


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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984116315


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.inject.Inject;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTaskUtils {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager) {
+    super(reconNamespaceSummaryManager, reconOMMetadataManager);
+  }
+
+  public boolean processWithLegacy(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+          WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnKeyTable = table.equals(KEY_TABLE);
+      if (!updateOnKeyTable) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+            (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+        OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+        OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+        setKeyParentID(updatedKeyInfo);
+
+        if (!updatedKeyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+          switch (action) {
+          case PUT:
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              setKeyParentID(oldKeyInfo);
+              handleDeleteKeyEvent(oldKeyInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                  updatedKey);
+            }
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        } else {
+          OmDirectoryInfo updatedDirectoryInfo =
+              new OmDirectoryInfo.Builder()
+                  .setName(updatedKeyInfo.getKeyName())
+                  .setObjectID(updatedKeyInfo.getObjectID())
+                  .setParentObjectID(updatedKeyInfo.getParentObjectID())
+                  .build();
+
+          OmDirectoryInfo oldDirectoryInfo = null;
+
+          if (oldKeyInfo != null) {
+            oldDirectoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(oldKeyInfo.getKeyName())
+                    .setObjectID(oldKeyInfo.getObjectID())
+                    .setParentObjectID(oldKeyInfo.getParentObjectID())
+                    .build();
+          }
+
+          switch (action) {
+          case PUT:
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                  updatedKey);
+            }
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        }
+      } catch (IOException ioEx) {
+        LOG.error("Unable to process Namespace Summary data in Recon DB. ",
+            ioEx);
+        return false;
+      }
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return false;
+    }
+
+    LOG.info("Completed a process run of NSSummaryTaskWithLegacy");
+    return true;
+  }
+
+  public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) {
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    try {
+      Table<String, OmKeyInfo> keyTable =
+          omMetadataManager.getKeyTable(BUCKET_LAYOUT);
+
+      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+          keyTableIter = keyTable.iterator()) {
+
+        while (keyTableIter.hasNext()) {
+          Table.KeyValue<String, OmKeyInfo> kv = keyTableIter.next();
+          OmKeyInfo keyInfo = kv.getValue();
+
+          setKeyParentID(keyInfo);
+
+          if (keyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+            OmDirectoryInfo directoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(keyInfo.getKeyName())
+                    .setObjectID(keyInfo.getObjectID())
+                    .setParentObjectID(keyInfo.getParentObjectID())
+                    .build();
+            handlePutDirEvent(directoryInfo, nsSummaryMap);
+          } else {
+            handlePutKeyEvent(keyInfo, nsSummaryMap);
+          }
+        }
+      }
+    } catch (IOException ioEx) {
+      LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ",
+          ioEx);
+      return false;
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return false;
+    }
+    LOG.info("Completed a reprocess run of NSSummaryTaskWithLegacy");
+    return true;
+  }
+
+  private void setKeyParentID(OmKeyInfo keyInfo) throws IOException {

Review Comment:
   Add a comment explaining that legacy objects don't have the parentid set, so we have to do it explicitly using this method.
   
   



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984121232


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskUtils.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.utils.db.RDBBatchOperation;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * Class for holding all NSSummaryTask methods
+ * related to DB operations so that they can commonly be
+ * used in NSSummaryTaskWithFSO and NSSummaryTaskWithLegacy.
+ */
+public class NSSummaryTaskUtils {

Review Comment:
   NSSummaryTaskUtils is not a great name for a parent class.  How about NSSummaryDbEventHandler?



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r981037801


##########
hadoop-ozone/dist/src/main/smoketest/recon/recon-nssummary.robot:
##########
@@ -14,7 +14,7 @@
 # limitations under the License.
 
 *** Settings ***
-Documentation       Smoke test for Recon Namespace Summary Endpoint for FSO buckets.
+Documentation       Smoke test for Recon Namespace Summary Endpoint for FSO and Legacy buckets.

Review Comment:
   @dombizita Apart from the Namespace Summary for a directory there is no difference between FSO and LEGACY. In order to avoid duplication, we added the `BUCKET_LAYOUT` as a parameter and gave it the default value of FSO. The idea is to add an extra command to `run.sh` to also run the test with a parameter `-v BUCKET_LAYOUT:LEGACY`.



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r988416727


##########
hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java:
##########
@@ -0,0 +1,491 @@
+/**
+ * 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.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.utils.db.RDBBatchOperation;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconTestInjector;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.apache.hadoop.ozone.recon.spi.impl.OzoneManagerServiceProviderImpl;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.ClassRule;
+import org.junit.Assert;
+import org.junit.experimental.runners.Enclosed;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_DB_DIRS;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProvider;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager;
+
+/**
+ * Test for NSSummaryTask. Create one bucket of each layout
+ * and test process and reprocess. Currently, there is no
+ * support for OBS buckets. Check that the NSSummary
+ * for the OBS bucket is null.
+ */
+@RunWith(Enclosed.class)
+public final class TestNSSummaryTask {
+
+  @ClassRule
+  public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder();
+
+  private static ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private static OMMetadataManager omMetadataManager;
+  private static ReconOMMetadataManager reconOMMetadataManager;
+  private static NSSummaryTask nSSummaryTask;
+  private static OzoneConfiguration omConfiguration;
+
+  // Object names
+  private static final String VOL = "vol";
+  private static final String BUCKET_ONE = "bucket1";
+  private static final String BUCKET_TWO = "bucket2";
+  private static final String BUCKET_THREE = "bucket3";
+  private static final String KEY_ONE = "file1";
+  private static final String KEY_TWO = "file2";
+  private static final String KEY_THREE = "file3";
+  private static final String KEY_FIVE = "file5";
+  private static final String FILE_ONE = "file1";
+  private static final String FILE_TWO = "file2";
+  private static final String FILE_THREE = "file3";
+  private static final String FILE_FIVE = "file5";
+
+  private static final String TEST_USER = "TestUser";
+
+  private static final long PARENT_OBJECT_ID_ZERO = 0L;
+  private static final long VOL_OBJECT_ID = 0L;
+  private static final long BUCKET_ONE_OBJECT_ID = 1L;
+  private static final long BUCKET_TWO_OBJECT_ID = 2L;
+  private static final long BUCKET_THREE_OBJECT_ID = 4L;
+  private static final long KEY_ONE_OBJECT_ID = 3L;
+  private static final long KEY_TWO_OBJECT_ID = 5L;
+  private static final long KEY_THREE_OBJECT_ID = 8L;
+  private static final long KEY_FIVE_OBJECT_ID = 9L;
+
+  private static final long KEY_ONE_SIZE = 500L;
+  private static final long KEY_TWO_SIZE = 1025L;
+  private static final long KEY_THREE_SIZE =
+      ReconConstants.MAX_FILE_SIZE_UPPER_BOUND - 100L;
+  private static final long KEY_FIVE_SIZE = 100L;
+
+  private TestNSSummaryTask() {
+  }
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    initializeNewOmMetadataManager(TEMPORARY_FOLDER.newFolder());
+    OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
+        getMockOzoneManagerServiceProvider();
+    reconOMMetadataManager = getTestReconOmMetadataManager(omMetadataManager,
+        TEMPORARY_FOLDER.newFolder());
+
+    ReconTestInjector reconTestInjector =
+        new ReconTestInjector.Builder(TEMPORARY_FOLDER)
+            .withReconOm(reconOMMetadataManager)
+            .withOmServiceProvider(ozoneManagerServiceProvider)
+            .withReconSqlDb()
+            .withContainerDB()
+            .build();
+    reconNamespaceSummaryManager =
+        reconTestInjector.getInstance(ReconNamespaceSummaryManager.class);
+
+    NSSummary nonExistentSummary =
+        reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID);
+    Assert.assertNull(nonExistentSummary);
+
+    populateOMDB();
+
+    nSSummaryTask = new NSSummaryTask(reconNamespaceSummaryManager,
+        reconOMMetadataManager, omConfiguration);
+  }
+
+  /**
+   * Nested class for testing NSSummaryTaskWithLegacy reprocess.
+   */
+  public static class TestReprocess {
+
+    private static NSSummary nsSummaryForBucket1;
+    private static NSSummary nsSummaryForBucket2;
+    private static NSSummary nsSummaryForBucket3;
+
+    @BeforeClass
+    public static void setUp() throws IOException {
+      // write a NSSummary prior to reprocess
+      // verify it got cleaned up after.
+      NSSummary staleNSSummary = new NSSummary();
+      RDBBatchOperation rdbBatchOperation = new RDBBatchOperation();
+      reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation, -1L,
+          staleNSSummary);
+      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
+
+      // Verify commit
+      Assert.assertNotNull(reconNamespaceSummaryManager.getNSSummary(-1L));
+
+      nSSummaryTask.reprocess(reconOMMetadataManager);
+      Assert.assertNull(reconNamespaceSummaryManager.getNSSummary(-1L));
+
+      nsSummaryForBucket1 =
+          reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID);
+      nsSummaryForBucket2 =
+          reconNamespaceSummaryManager.getNSSummary(BUCKET_TWO_OBJECT_ID);
+      nsSummaryForBucket3 =
+      reconNamespaceSummaryManager.getNSSummary(BUCKET_THREE_OBJECT_ID);
+      Assert.assertNotNull(nsSummaryForBucket1);
+      Assert.assertNotNull(nsSummaryForBucket2);
+      Assert.assertNull(nsSummaryForBucket3);
+    }
+
+    @Test
+    public void testReprocessNSSummaryNull() throws IOException {
+      Assert.assertNull(reconNamespaceSummaryManager.getNSSummary(-1L));
+    }
+
+    @Test
+    public void testReprocessGetFiles() {
+      Assert.assertEquals(1, nsSummaryForBucket1.getNumOfFiles());
+      Assert.assertEquals(1, nsSummaryForBucket2.getNumOfFiles());
+
+      Assert.assertEquals(KEY_ONE_SIZE, nsSummaryForBucket1.getSizeOfFiles());
+      Assert.assertEquals(KEY_TWO_SIZE, nsSummaryForBucket2.getSizeOfFiles());
+    }
+
+    @Test
+    public void testReprocessFileBucketSize() {
+      int[] fileDistBucket1 = nsSummaryForBucket1.getFileSizeBucket();
+      int[] fileDistBucket2 = nsSummaryForBucket2.getFileSizeBucket();
+      Assert.assertEquals(ReconConstants.NUM_OF_BINS, fileDistBucket1.length);
+      Assert.assertEquals(ReconConstants.NUM_OF_BINS, fileDistBucket2.length);
+
+      Assert.assertEquals(1, fileDistBucket1[0]);
+      for (int i = 1; i < ReconConstants.NUM_OF_BINS; ++i) {
+        Assert.assertEquals(0, fileDistBucket1[i]);
+      }
+      Assert.assertEquals(1, fileDistBucket2[1]);
+      for (int i = 0; i < ReconConstants.NUM_OF_BINS; ++i) {
+        if (i == 1) {
+          continue;
+        }
+        Assert.assertEquals(0, fileDistBucket2[i]);
+      }
+    }
+
+  }
+
+  /**
+   * Nested class for testing NSSummaryTaskWithLegacy process.
+   */
+  public static class TestProcess {
+
+    private static NSSummary nsSummaryForBucket1;
+    private static NSSummary nsSummaryForBucket2;
+    private static NSSummary nsSummaryForBucket3;
+
+    private static OMDBUpdateEvent keyEvent1;
+    private static OMDBUpdateEvent keyEvent2;
+
+    @BeforeClass
+    public static void setUp() throws IOException {
+      nSSummaryTask.reprocess(reconOMMetadataManager);
+      nSSummaryTask.process(processEventBatch());
+
+      nsSummaryForBucket1 =
+          reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID);
+      Assert.assertNotNull(nsSummaryForBucket1);
+      nsSummaryForBucket2 =
+          reconNamespaceSummaryManager.getNSSummary(BUCKET_TWO_OBJECT_ID);
+      Assert.assertNotNull(nsSummaryForBucket2);
+      nsSummaryForBucket3 =
+          reconNamespaceSummaryManager.getNSSummary(BUCKET_THREE_OBJECT_ID);
+      Assert.assertNull(nsSummaryForBucket3);
+    }
+
+    private static OMUpdateEventBatch processEventBatch() throws IOException {
+      // put file5 under bucket 2
+      String omPutKey =
+          OM_KEY_PREFIX + VOL +
+              OM_KEY_PREFIX + BUCKET_TWO +
+              OM_KEY_PREFIX + FILE_FIVE;
+      OmKeyInfo omPutKeyInfo = buildOmKeyInfo(VOL, BUCKET_TWO, KEY_FIVE,
+          FILE_FIVE, KEY_FIVE_OBJECT_ID, BUCKET_TWO_OBJECT_ID, KEY_FIVE_SIZE);
+      keyEvent1 = new OMDBUpdateEvent.
+          OMUpdateEventBuilder<String, OmKeyInfo>()
+          .setKey(omPutKey)
+          .setValue(omPutKeyInfo)
+          .setTable(omMetadataManager.getKeyTable(getLegacyBucketLayout())
+              .getName())
+          .setAction(OMDBUpdateEvent.OMDBUpdateAction.PUT)
+          .build();
+
+      // delete file 1 under bucket 1
+      String omDeleteKey = BUCKET_ONE_OBJECT_ID + OM_KEY_PREFIX + FILE_ONE;
+      OmKeyInfo omDeleteInfo = buildOmKeyInfo(
+          VOL, BUCKET_ONE, KEY_ONE, FILE_ONE,
+          KEY_ONE_OBJECT_ID, BUCKET_ONE_OBJECT_ID);
+      keyEvent2 = new OMDBUpdateEvent.
+          OMUpdateEventBuilder<String, OmKeyInfo>()
+          .setKey(omDeleteKey)
+          .setValue(omDeleteInfo)
+          .setTable(omMetadataManager.getKeyTable(getFSOBucketLayout())
+              .getName())
+          .setAction(OMDBUpdateEvent.OMDBUpdateAction.DELETE)
+          .build();
+
+      OMUpdateEventBatch omUpdateEventBatch = new OMUpdateEventBatch(
+          new ArrayList<OMDBUpdateEvent>() {{
+              add(keyEvent1);
+              add(keyEvent2);
+              }});
+
+      return omUpdateEventBatch;
+    }
+
+    @Test
+    public void testProcessUpdateFileSize() throws IOException {
+      // file 1 is gone, so bucket 1 is empty now
+      Assert.assertNotNull(nsSummaryForBucket1);
+      Assert.assertEquals(0, nsSummaryForBucket1.getNumOfFiles());
+
+      Set<Long> childDirBucket1 = nsSummaryForBucket1.getChildDir();
+      Assert.assertEquals(0, childDirBucket1.size());
+    }
+
+    @Test
+    public void testProcessBucket() throws IOException {
+      // file 5 is added under bucket 2, so bucket 2 has 2 keys now
+      Assert.assertNotNull(nsSummaryForBucket2);
+      Assert.assertEquals(2, nsSummaryForBucket2.getNumOfFiles());
+      // key 2 + key 5
+      Assert.assertEquals(KEY_TWO_SIZE + KEY_FIVE_SIZE,
+          nsSummaryForBucket2.getSizeOfFiles());
+
+      int[] fileSizeDist = nsSummaryForBucket2.getFileSizeBucket();
+      Assert.assertEquals(ReconConstants.NUM_OF_BINS, fileSizeDist.length);
+      // 1025L
+      Assert.assertEquals(1, fileSizeDist[0]);
+      // 2050L
+      Assert.assertEquals(1, fileSizeDist[1]);
+      for (int i = 2; i < ReconConstants.NUM_OF_BINS; ++i) {
+        Assert.assertEquals(0, fileSizeDist[i]);
+      }
+    }
+  }
+
+  /**
+   * Build a key info for put/update action.
+   * @param volume         volume name
+   * @param bucket         bucket name
+   * @param key            key name
+   * @param fileName       file name
+   * @param objectID       object ID
+   * @param parentObjectId parent object ID
+   * @param dataSize       file size
+   * @return the KeyInfo
+   */
+  private static OmKeyInfo buildOmKeyInfo(String volume,
+                                          String bucket,
+                                          String key,
+                                          String fileName,
+                                          long objectID,
+                                          long parentObjectId,
+                                          long dataSize) {
+    return new OmKeyInfo.Builder()
+        .setBucketName(bucket)
+        .setVolumeName(volume)
+        .setKeyName(key)
+        .setFileName(fileName)
+        .setReplicationConfig(
+            StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+        .setObjectID(objectID)
+        .setParentObjectID(parentObjectId)
+        .setDataSize(dataSize)
+        .build();
+  }
+
+  /**
+   * Build a key info for delete action.
+   * @param volume         volume name
+   * @param bucket         bucket name
+   * @param key            key name
+   * @param fileName       file name
+   * @param objectID       object ID
+   * @param parentObjectId parent object ID
+   * @return the KeyInfo
+   */
+  private static OmKeyInfo buildOmKeyInfo(String volume,
+                                          String bucket,
+                                          String key,
+                                          String fileName,
+                                          long objectID,
+                                          long parentObjectId) {
+    return new OmKeyInfo.Builder()
+        .setBucketName(bucket)
+        .setVolumeName(volume)
+        .setKeyName(key)
+        .setFileName(fileName)
+        .setReplicationConfig(
+            StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+        .setObjectID(objectID)
+        .setParentObjectID(parentObjectId)
+        .build();
+  }
+
+  /**
+   * Populate OMDB with the following configs.
+   *        vol
+   *      /     \         \
+   * bucket1   bucket2    bucket3
+   * /          /           \
+   * file1    file2       file3
+   *
+   * @throws IOException
+   */
+  private static void populateOMDB() throws IOException {
+    // Bucket1 FSO layout
+    writeKeyToOm(reconOMMetadataManager,
+        KEY_ONE,
+        BUCKET_ONE,
+        VOL,
+        FILE_ONE,
+        KEY_ONE_OBJECT_ID,
+        BUCKET_ONE_OBJECT_ID,
+        BUCKET_ONE_OBJECT_ID,
+        VOL_OBJECT_ID,
+        KEY_ONE_SIZE,
+        getFSOBucketLayout());
+
+    // Bucket2 Legacy layout
+    writeKeyToOm(reconOMMetadataManager,
+        KEY_TWO,
+        BUCKET_TWO,
+        VOL,
+        FILE_TWO,
+        KEY_TWO_OBJECT_ID,
+        PARENT_OBJECT_ID_ZERO,
+        BUCKET_TWO_OBJECT_ID,
+        VOL_OBJECT_ID,
+          KEY_TWO_SIZE,
+        getLegacyBucketLayout());
+
+    // Bucket3 OBS layout
+    writeKeyToOm(reconOMMetadataManager,
+        KEY_THREE,
+        BUCKET_THREE,
+        VOL,
+        FILE_THREE,
+        KEY_THREE_OBJECT_ID,
+        PARENT_OBJECT_ID_ZERO,
+        BUCKET_THREE_OBJECT_ID,
+        VOL_OBJECT_ID,
+        KEY_THREE_SIZE,
+        getOBSBucketLayout());
+  }
+
+  /**
+   * Create a new OM Metadata manager instance with one user, one vol, and two
+   * buckets. Bucket1 will have FSO layout and bucket2 will have Legacy layout.

Review Comment:
   mention bucket 3 here as well



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r979909970


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/DirectoryEntityHandler.java:
##########
@@ -89,7 +91,15 @@ public DUResponse getDuResponse(
     for (long subdirObjectId: subdirs) {
       NSSummary subdirNSSummary =
               getReconNamespaceSummaryManager().getNSSummary(subdirObjectId);
-      String subdirName = subdirNSSummary.getDirName();
+      // for the subdirName we need the filename, not the key name
+      Path subdirPath = Paths.get(subdirNSSummary.getDirName());
+      Path subdirFileName = subdirPath.getFileName();
+      String subdirName;
+      if (subdirFileName != null) {
+        subdirName = subdirFileName.toString();
+      } else {
+        throw new NullPointerException("Subdirectory file name is null.");
+      }

Review Comment:
   My mistake, I updated the comment to be more descriptive of the issue. Eg. the key name is `dir1/dir2`, then we just need to add the subdir to the path, we need `dir2`. Otherwise, we will end up having `dir1` twice while this is an iterative process going over all directories and adding them one by one.



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


[GitHub] [ozone] xBis7 commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1258010995

   @GeorgeJahad I've made the changes based on your recommendations.


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


[GitHub] [ozone] xBis7 commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1256045242

   @dombizita `NSSummary` isn't bucket specific. We would have an issue if we were gathering info for volumes but we are dealing with buckets. 
   
   > NSSummaryTask.process would call eg. first NSSummaryTaskWithFSO.process and after the NSSummaryTaskWithLegacy.process, right?
   
   Yes, you are right. This way we can control the order by which we are calling any bucket specific task. `NSSummaryTask.process` will return `new ImmutablePair<>(getTaskName(), true)` after both bucket tasks have finished running and `getTaskName()` will be `NSSummaryTask`. Since `NSSummary` isn't specific to bucket layout and the entries for different layouts won't have an attribute that differentiates them, it seems more clean to have one table instead of two tables with entries of the same type.


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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r1000895186


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/nssummary/NSSummaryAdmin.java:
##########
@@ -108,6 +110,60 @@ public boolean isFileSystemOptimizedBucket(String path) throws IOException {
     }
   }
 
+  public boolean isOBSBucket(String path) throws IOException {
+    OFSPath ofsPath = new OFSPath(path);
+
+    boolean enableFileSystemPaths = getOzoneConfig()
+        .get(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");

Review Comment:
   Thanks for the change @xBis7 !
   
   Though my bad I just realized we should probably straight use `.getBoolean()` with the default value, like this:
   
   https://github.com/apache/ozone/blob/c0de6bcdebee6a4271fc73ea6542902e0eeee28f/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L207-L209
   
   ~~Otherwise when the config key is not explicitly set the `.get()` result could be `null`.~~
   Looks like the default value is populated into the config map, so at least we won't get NPE. But using `.getBoolean()` could still be the cleaner solution so we won't have to worry about the letter case.
   
   This applies here as well: https://github.com/apache/ozone/pull/3746#discussion_r998957719



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999477324


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/admin/nssummary/NSSummaryAdmin.java:
##########
@@ -108,6 +110,60 @@ public boolean isFileSystemOptimizedBucket(String path) throws IOException {
     }
   }
 
+  public boolean isOBSBucket(String path) throws IOException {
+    OFSPath ofsPath = new OFSPath(path);
+
+    boolean enableFileSystemPaths = getOzoneConfig()
+        .get(OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS).equals("true");

Review Comment:
   ok, I'll keep that in mind. Since it's CLI there is not constructor...



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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999883068


##########
hadoop-ozone/dist/src/main/smoketest/recon/recon-nssummary.robot:
##########
@@ -131,7 +132,8 @@ Check Recon Namespace Summary Key
     Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/file1   KEY
 
 Check Recon Namespace Summary Directory
-    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2   DIRECTORY
+    Run Keyword If    '${BUCKET_LAYOUT}' == 'LEGACY'                    Wait For Summary      ${SUMMARY_URL}?path=/${VOLUME}/${BUCKET}/dir1/dir2/   DIRECTORY

Review Comment:
   Thanks @xBis7 for the explanation. The link would now be [this](https://github.com/xBis7/ozone/blob/HDDS-7121/hadoop-ozone/dist/src/main/compose/ozone-legacy-bucket/test.sh#L31) since the dir is renamed.



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999390306


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -44,166 +49,97 @@
  * For dirTable, we need to fetch the parent object (bucket or directory),
  * add the current directory's objectID to the parent object's childDir field.
  *
+ * For keyTable, the parent object is not available. Get the parent object,
+ * add it to the current object and reuse the existing methods for FSO.
+ * Only processing entries that belong to Legacy buckets. If the entry
+ * refers to a directory then build directory info object from it.
+ *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
   private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private final ReconOMMetadataManager reconOMMetadataManager;
+  private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
+  private final OzoneConfiguration ozoneConfiguration;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager,
+                       OzoneConfiguration
+                             ozoneConfiguration) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.ozoneConfiguration = ozoneConfiguration;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager,
+        reconOMMetadataManager, ozoneConfiguration);
   }
 
-  public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
-    return reconNamespaceSummaryManager;
-  }
-
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
-  }
-
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    boolean success;
+    success = nsSummaryTaskWithFSO.processWithFSO(events);
+    if (success) {
+      success = nsSummaryTaskWithLegacy.processWithLegacy(events);
     }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-
-    long dataSize = keyInfo.getDataSize();
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    // decrement count, data size, and bucket count
-    // even if there's no direct key, we still keep the entry because
-    // we still need children dir IDs info
-    nsSummary.setNumOfFiles(numOfFile - 1);
-    nsSummary.setSizeOfFiles(sizeOfFile - dataSize);
-    --fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+    return new ImmutablePair<>(getTaskName(), success);
   }
 
-  protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    Collection<Callable<Boolean>> tasks = new ArrayList<>();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.clearNSSummaryTable();
+    } catch (IOException ioEx) {
+      LOG.error("Unable to clear NSSummary table in Recon DB. ",
+          ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
     }
 
-    // Just in case the OmDirectoryInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+    tasks.add(() -> nsSummaryTaskWithFSO
+        .reprocessWithFSO(omMetadataManager));
+    tasks.add(() -> nsSummaryTaskWithLegacy
+        .reprocessWithLegacy(reconOMMetadataManager));
+
+    List<Future<Boolean>> results;
+    ExecutorService executorService = Executors
+        .newFixedThreadPool(2);
+    try {
+      results = executorService.invokeAll(tasks);

Review Comment:
   I'm not sure if they will run in parallel but they will run asynchronously on different threads. We did this to optimize reprocess since it takes much longer than process to finish running.



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999978658


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/BucketHandler.java:
##########
@@ -183,8 +185,22 @@ public static BucketHandler getBucketHandler(
                 OzoneStorageContainerManager reconSCM,
                 OmBucketInfo bucketInfo) throws IOException {
 
-    return new FSOBucketHandler(reconNamespaceSummaryManager,
-              omMetadataManager, reconSCM, bucketInfo);
+    // If bucketInfo is null then entity type is UNKNOWN
+    if (Objects.isNull(bucketInfo)) {
+      return null;
+    } else {
+      if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.FILE_SYSTEM_OPTIMIZED)) {
+        return new FSOBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else if (bucketInfo.getBucketLayout()
+          .equals(BucketLayout.LEGACY)) {
+        return new LegacyBucketHandler(reconNamespaceSummaryManager,
+            omMetadataManager, reconSCM, bucketInfo);
+      } else {
+        throw new RuntimeException("Unsupported bucket layout.");

Review Comment:
   Log an error instead of throwing the exception or catch the exception and then log an error?



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


[GitHub] [ozone] smengcl commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
smengcl commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r999888466


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -44,166 +49,97 @@
  * For dirTable, we need to fetch the parent object (bucket or directory),
  * add the current directory's objectID to the parent object's childDir field.
  *
+ * For keyTable, the parent object is not available. Get the parent object,
+ * add it to the current object and reuse the existing methods for FSO.
+ * Only processing entries that belong to Legacy buckets. If the entry
+ * refers to a directory then build directory info object from it.
+ *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
   private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private final ReconOMMetadataManager reconOMMetadataManager;
+  private final NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private final NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
+  private final OzoneConfiguration ozoneConfiguration;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager,
+                       OzoneConfiguration
+                             ozoneConfiguration) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.ozoneConfiguration = ozoneConfiguration;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager,
+        reconOMMetadataManager, ozoneConfiguration);
   }
 
-  public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
-    return reconNamespaceSummaryManager;
-  }
-
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
-  }
-
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    boolean success;
+    success = nsSummaryTaskWithFSO.processWithFSO(events);
+    if (success) {
+      success = nsSummaryTaskWithLegacy.processWithLegacy(events);
     }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-
-    long dataSize = keyInfo.getDataSize();
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    // decrement count, data size, and bucket count
-    // even if there's no direct key, we still keep the entry because
-    // we still need children dir IDs info
-    nsSummary.setNumOfFiles(numOfFile - 1);
-    nsSummary.setSizeOfFiles(sizeOfFile - dataSize);
-    --fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+    return new ImmutablePair<>(getTaskName(), success);
   }
 
-  protected void handleDeleteDirEvent(OmDirectoryInfo directoryInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    Collection<Callable<Boolean>> tasks = new ArrayList<>();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.clearNSSummaryTable();
+    } catch (IOException ioEx) {
+      LOG.error("Unable to clear NSSummary table in Recon DB. ",
+          ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
     }
 
-    // Just in case the OmDirectoryInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+    tasks.add(() -> nsSummaryTaskWithFSO
+        .reprocessWithFSO(omMetadataManager));
+    tasks.add(() -> nsSummaryTaskWithLegacy
+        .reprocessWithLegacy(reconOMMetadataManager));
+
+    List<Future<Boolean>> results;
+    ExecutorService executorService = Executors
+        .newFixedThreadPool(2);
+    try {
+      results = executorService.invokeAll(tasks);

Review Comment:
   I see. As long as this won't cause similar issue we discussed much earlier (one thread clearing NSSummaryTable while another is writing to it) it should be fine. I see `clearNSSummaryTable()` being called right before invoke here.



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984671077


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/DirectoryEntityHandler.java:
##########
@@ -89,7 +91,15 @@ public DUResponse getDuResponse(
     for (long subdirObjectId: subdirs) {
       NSSummary subdirNSSummary =
               getReconNamespaceSummaryManager().getNSSummary(subdirObjectId);
-      String subdirName = subdirNSSummary.getDirName();
+      // for the subdirName we need the filename, not the key name
+      Path subdirPath = Paths.get(subdirNSSummary.getDirName());
+      Path subdirFileName = subdirPath.getFileName();
+      String subdirName;
+      if (subdirFileName != null) {
+        subdirName = subdirFileName.toString();
+      } else {
+        throw new NullPointerException("Subdirectory file name is null.");
+      }

Review Comment:
   I added the null check to get rid of a `findbugs` error but I also didn't want to consume the NPT exception. I can change the exception type to `IOException`.
   
   ```
   M D NP: Possible null pointer dereference in org.apache.hadoop.ozone.recon.api.handlers.DirectoryEntityHandler.getDuResponse(boolean, boolean) due to return value of called method  Dereferenced at DirectoryEntityHandler.java:[line 102]
   ```



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984678277


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.inject.Inject;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTaskUtils {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager) {
+    super(reconNamespaceSummaryManager, reconOMMetadataManager);
+  }
+
+  public boolean processWithLegacy(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+          WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnKeyTable = table.equals(KEY_TABLE);
+      if (!updateOnKeyTable) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+            (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+        OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+        OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+        setKeyParentID(updatedKeyInfo);
+
+        if (!updatedKeyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+          switch (action) {
+          case PUT:
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              setKeyParentID(oldKeyInfo);
+              handleDeleteKeyEvent(oldKeyInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                  updatedKey);
+            }
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        } else {
+          OmDirectoryInfo updatedDirectoryInfo =
+              new OmDirectoryInfo.Builder()
+                  .setName(updatedKeyInfo.getKeyName())
+                  .setObjectID(updatedKeyInfo.getObjectID())
+                  .setParentObjectID(updatedKeyInfo.getParentObjectID())
+                  .build();
+
+          OmDirectoryInfo oldDirectoryInfo = null;
+
+          if (oldKeyInfo != null) {
+            oldDirectoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(oldKeyInfo.getKeyName())
+                    .setObjectID(oldKeyInfo.getObjectID())
+                    .setParentObjectID(oldKeyInfo.getParentObjectID())
+                    .build();
+          }
+
+          switch (action) {
+          case PUT:
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                  updatedKey);
+            }
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        }
+      } catch (IOException ioEx) {
+        LOG.error("Unable to process Namespace Summary data in Recon DB. ",
+            ioEx);
+        return false;
+      }
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return false;
+    }
+
+    LOG.info("Completed a process run of NSSummaryTaskWithLegacy");
+    return true;
+  }
+
+  public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) {
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    try {
+      Table<String, OmKeyInfo> keyTable =
+          omMetadataManager.getKeyTable(BUCKET_LAYOUT);
+
+      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+          keyTableIter = keyTable.iterator()) {

Review Comment:
   The only way I see to do that, is for each OmKeyInfo, get the bucketName and using that, get the bucket info from the bucket table and then check the layout.



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


[GitHub] [ozone] xBis7 commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984678277


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.inject.Inject;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTaskUtils {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager) {
+    super(reconNamespaceSummaryManager, reconOMMetadataManager);
+  }
+
+  public boolean processWithLegacy(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+          WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnKeyTable = table.equals(KEY_TABLE);
+      if (!updateOnKeyTable) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+            (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+        OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+        OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+        setKeyParentID(updatedKeyInfo);
+
+        if (!updatedKeyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+          switch (action) {
+          case PUT:
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              setKeyParentID(oldKeyInfo);
+              handleDeleteKeyEvent(oldKeyInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                  updatedKey);
+            }
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        } else {
+          OmDirectoryInfo updatedDirectoryInfo =
+              new OmDirectoryInfo.Builder()
+                  .setName(updatedKeyInfo.getKeyName())
+                  .setObjectID(updatedKeyInfo.getObjectID())
+                  .setParentObjectID(updatedKeyInfo.getParentObjectID())
+                  .build();
+
+          OmDirectoryInfo oldDirectoryInfo = null;
+
+          if (oldKeyInfo != null) {
+            oldDirectoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(oldKeyInfo.getKeyName())
+                    .setObjectID(oldKeyInfo.getObjectID())
+                    .setParentObjectID(oldKeyInfo.getParentObjectID())
+                    .build();
+          }
+
+          switch (action) {
+          case PUT:
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                  updatedKey);
+            }
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        }
+      } catch (IOException ioEx) {
+        LOG.error("Unable to process Namespace Summary data in Recon DB. ",
+            ioEx);
+        return false;
+      }
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return false;
+    }
+
+    LOG.info("Completed a process run of NSSummaryTaskWithLegacy");
+    return true;
+  }
+
+  public boolean reprocessWithLegacy(OMMetadataManager omMetadataManager) {
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    try {
+      Table<String, OmKeyInfo> keyTable =
+          omMetadataManager.getKeyTable(BUCKET_LAYOUT);
+
+      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+          keyTableIter = keyTable.iterator()) {

Review Comment:
   The only way I see doing that, is for each OmKeyInfo, get the bucketName and using that, get the bucket info from the bucket table and then check the layout.



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r984119286


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java:
##########
@@ -45,165 +41,71 @@
  * add the current directory's objectID to the parent object's childDir field.
  *
  * Process() will write all OMDB updates to RocksDB.
- * The write logic is the same as above. For update action, we will treat it as
+ * Write logic is the same as above. For update action, we will treat it as
  * delete old value first, and write updated value then.
  */
-public abstract class NSSummaryTask implements ReconOmTask {
+public class NSSummaryTask implements ReconOmTask {
   private static final Logger LOG =
           LoggerFactory.getLogger(NSSummaryTask.class);
 
-  private final ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private ReconOMMetadataManager reconOMMetadataManager;
+  private NSSummaryTaskWithFSO nsSummaryTaskWithFSO;
+  private NSSummaryTaskWithLegacy nsSummaryTaskWithLegacy;
 
   @Inject
   public NSSummaryTask(ReconNamespaceSummaryManager
-                                 reconNamespaceSummaryManager) {
+                       reconNamespaceSummaryManager,
+                       ReconOMMetadataManager
+                           reconOMMetadataManager) {
     this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+    this.reconOMMetadataManager = reconOMMetadataManager;
+    this.nsSummaryTaskWithFSO = new NSSummaryTaskWithFSO(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
+    this.nsSummaryTaskWithLegacy = new NSSummaryTaskWithLegacy(
+        reconNamespaceSummaryManager, reconOMMetadataManager);
   }
 
   public ReconNamespaceSummaryManager getReconNamespaceSummaryManager() {
     return reconNamespaceSummaryManager;
   }
 
-  public abstract String getTaskName();
-
-  public abstract Pair<String, Boolean> process(OMUpdateEventBatch events);
-
-  public abstract Pair<String, Boolean> reprocess(
-      OMMetadataManager omMetadataManager);
-
-  protected void writeNSSummariesToDB(Map<Long, NSSummary> nsSummaryMap)
-      throws IOException {
-    try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
-      nsSummaryMap.keySet().forEach((Long key) -> {
-        try {
-          reconNamespaceSummaryManager.batchStoreNSSummaries(rdbBatchOperation,
-              key, nsSummaryMap.get(key));
-        } catch (IOException e) {
-          LOG.error("Unable to write Namespace Summary data in Recon DB.",
-              e);
-        }
-      });
-      reconNamespaceSummaryManager.commitBatchOperation(rdbBatchOperation);
-    }
-  }
-
-  protected void handlePutKeyEvent(OmKeyInfo keyInfo, Map<Long,
-      NSSummary> nsSummaryMap) throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    int numOfFile = nsSummary.getNumOfFiles();
-    long sizeOfFile = nsSummary.getSizeOfFiles();
-    int[] fileBucket = nsSummary.getFileSizeBucket();
-    nsSummary.setNumOfFiles(numOfFile + 1);
-    long dataSize = keyInfo.getDataSize();
-    nsSummary.setSizeOfFiles(sizeOfFile + dataSize);
-    int binIndex = ReconUtils.getBinIndex(dataSize);
-
-    ++fileBucket[binIndex];
-    nsSummary.setFileSizeBucket(fileBucket);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  public ReconOMMetadataManager getReconOMMetadataManager() {
+    return reconOMMetadataManager;
   }
 
-  protected void handlePutDirEvent(OmDirectoryInfo directoryInfo,
-                                 Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = directoryInfo.getParentObjectID();
-    long objectId = directoryInfo.getObjectID();
-    // write the dir name to the current directory
-    String dirName = directoryInfo.getName();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary curNSSummary = nsSummaryMap.get(objectId);
-    if (curNSSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      curNSSummary = reconNamespaceSummaryManager.getNSSummary(objectId);
-    }
-    if (curNSSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      curNSSummary = new NSSummary();
-    }
-    curNSSummary.setDirName(dirName);
-    nsSummaryMap.put(objectId, curNSSummary);
-
-    // Write the child dir list to the parent directory
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-    if (nsSummary == null) {
-      // If we don't have it locally and in the DB we create a new instance
-      // as this is a new ID
-      nsSummary = new NSSummary();
-    }
-    nsSummary.addChildDir(objectId);
-    nsSummaryMap.put(parentObjectId, nsSummary);
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
   }
 
-  protected void handleDeleteKeyEvent(OmKeyInfo keyInfo,
-                                    Map<Long, NSSummary> nsSummaryMap)
-          throws IOException {
-    long parentObjectId = keyInfo.getParentObjectID();
-    // Try to get the NSSummary from our local map that maps NSSummaries to IDs
-    NSSummary nsSummary = nsSummaryMap.get(parentObjectId);
-    if (nsSummary == null) {
-      // If we don't have it in this batch we try to get it from the DB
-      nsSummary = reconNamespaceSummaryManager.getNSSummary(parentObjectId);
-    }
-
-    // Just in case the OmKeyInfo isn't correctly written.
-    if (nsSummary == null) {
-      LOG.error("The namespace table is not correctly populated.");
-      return;
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {

Review Comment:
   Now that the two tasks have been merged we'll need to add a simple test.  one fso bucket and one legacy bucket, each with one key.



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


[GitHub] [ozone] xBis7 commented on pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
xBis7 commented on PR #3746:
URL: https://github.com/apache/ozone/pull/3746#issuecomment-1290871266

   @smengcl CI looks good. Can we merge this?


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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r979139000


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.inject.Inject;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTask {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private final ReconOMMetadataManager reconOMMetadataManager;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  @Inject
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager) {
+    super(reconNamespaceSummaryManager);
+    this.reconOMMetadataManager = reconOMMetadataManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTaskWithLegacy";
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+          WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnKeyTable = table.equals(KEY_TABLE);
+      if (!updateOnKeyTable) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+            (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+        OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+        OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+        setKeyParentID(updatedKeyInfo);
+
+        if (!updatedKeyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+          switch (action) {
+          case PUT:
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              setKeyParentID(oldKeyInfo);
+              handleDeleteKeyEvent(oldKeyInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                  updatedKey);
+            }
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        } else {
+          OmDirectoryInfo updatedDirectoryInfo =
+              new OmDirectoryInfo.Builder()
+                  .setName(updatedKeyInfo.getKeyName())
+                  .setObjectID(updatedKeyInfo.getObjectID())
+                  .setParentObjectID(updatedKeyInfo.getParentObjectID())
+                  .build();
+
+          OmDirectoryInfo oldDirectoryInfo = null;
+
+          if (oldKeyInfo != null) {
+            oldDirectoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(oldKeyInfo.getKeyName())
+                    .setObjectID(oldKeyInfo.getObjectID())
+                    .setParentObjectID(oldKeyInfo.getParentObjectID())
+                    .build();
+          }
+
+          switch (action) {
+          case PUT:
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                  updatedKey);
+            }
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        }
+      } catch (IOException ioEx) {
+        LOG.error("Unable to process Namespace Summary data in Recon DB. ",
+            ioEx);
+        return new ImmutablePair<>(getTaskName(), false);
+      }
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return new ImmutablePair<>(getTaskName(), false);
+    }
+
+    LOG.info("Completed a process run of NSSummaryTaskWithLegacy");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    try {
+      Table<String, OmKeyInfo> keyTable =
+          omMetadataManager.getKeyTable(BUCKET_LAYOUT);
+
+      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+          keyTableIter = keyTable.iterator()) {
+
+        while (keyTableIter.hasNext()) {
+          Table.KeyValue<String, OmKeyInfo> kv = keyTableIter.next();
+          OmKeyInfo keyInfo = kv.getValue();
+
+          setKeyParentID(keyInfo);
+
+          if (keyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+            OmDirectoryInfo directoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(keyInfo.getKeyName())
+                    .setObjectID(keyInfo.getObjectID())
+                    .setParentObjectID(keyInfo.getParentObjectID())
+                    .build();
+            handlePutDirEvent(directoryInfo, nsSummaryMap);
+          } else {
+            handlePutKeyEvent(keyInfo, nsSummaryMap);
+          }
+        }
+      }
+    } catch (IOException ioEx) {
+      LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ",
+          ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return new ImmutablePair<>(getTaskName(), false);
+    }
+    LOG.info("Completed a reprocess run of NSSummaryTaskWithLegacy");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  private void setKeyParentID(OmKeyInfo keyInfo) throws IOException {
+    String[] keyPath = keyInfo.getKeyName().split(OM_KEY_PREFIX);
+
+    //if (keyPath > 1) there is one or more directories
+    if (keyPath.length > 1) {
+      String[] dirs = Arrays.copyOf(keyPath, keyPath.length - 1);
+      String parentKeyName = String.join(OM_KEY_PREFIX, dirs);
+      parentKeyName += OM_KEY_PREFIX;
+      String fullParentKeyName =
+          reconOMMetadataManager.getOzoneKey(keyInfo.getVolumeName(),
+              keyInfo.getBucketName(), parentKeyName);
+      OmKeyInfo parentKeyInfo = reconOMMetadataManager
+          .getKeyTable(BUCKET_LAYOUT)
+          .get(fullParentKeyName);
+
+      try {
+        keyInfo.setParentObjectID(parentKeyInfo.getObjectID());
+      } catch (NullPointerException e) {

Review Comment:
   why is the npe being swallowed?



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTaskWithLegacy.java:
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+import org.apache.hadoop.ozone.recon.recovery.ReconOMMetadataManager;
+import org.apache.hadoop.ozone.recon.spi.ReconNamespaceSummaryManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.inject.Inject;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Class for handling Legacy specific tasks.
+ */
+public class NSSummaryTaskWithLegacy extends NSSummaryTask {
+
+  private static final BucketLayout BUCKET_LAYOUT = BucketLayout.LEGACY;
+
+  private final ReconOMMetadataManager reconOMMetadataManager;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NSSummaryTaskWithLegacy.class);
+
+  @Inject
+  public NSSummaryTaskWithLegacy(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager,
+                                 ReconOMMetadataManager
+                                 reconOMMetadataManager) {
+    super(reconNamespaceSummaryManager);
+    this.reconOMMetadataManager = reconOMMetadataManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTaskWithLegacy";
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+          WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnKeyTable = table.equals(KEY_TABLE);
+      if (!updateOnKeyTable) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+            (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+        OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+        OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+        setKeyParentID(updatedKeyInfo);
+
+        if (!updatedKeyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+          switch (action) {
+          case PUT:
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              setKeyParentID(oldKeyInfo);
+              handleDeleteKeyEvent(oldKeyInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                  updatedKey);
+            }
+            handlePutKeyEvent(updatedKeyInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        } else {
+          OmDirectoryInfo updatedDirectoryInfo =
+              new OmDirectoryInfo.Builder()
+                  .setName(updatedKeyInfo.getKeyName())
+                  .setObjectID(updatedKeyInfo.getObjectID())
+                  .setParentObjectID(updatedKeyInfo.getParentObjectID())
+                  .build();
+
+          OmDirectoryInfo oldDirectoryInfo = null;
+
+          if (oldKeyInfo != null) {
+            oldDirectoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(oldKeyInfo.getKeyName())
+                    .setObjectID(oldKeyInfo.getObjectID())
+                    .setParentObjectID(oldKeyInfo.getParentObjectID())
+                    .build();
+          }
+
+          switch (action) {
+          case PUT:
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case DELETE:
+            handleDeleteDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          case UPDATE:
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              handleDeleteDirEvent(oldDirectoryInfo, nsSummaryMap);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                  updatedKey);
+            }
+            handlePutDirEvent(updatedDirectoryInfo, nsSummaryMap);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                omdbUpdateEvent.getAction());
+          }
+        }
+      } catch (IOException ioEx) {
+        LOG.error("Unable to process Namespace Summary data in Recon DB. ",
+            ioEx);
+        return new ImmutablePair<>(getTaskName(), false);
+      }
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return new ImmutablePair<>(getTaskName(), false);
+    }
+
+    LOG.info("Completed a process run of NSSummaryTaskWithLegacy");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    Map<Long, NSSummary> nsSummaryMap = new HashMap<>();
+
+    try {
+      Table<String, OmKeyInfo> keyTable =
+          omMetadataManager.getKeyTable(BUCKET_LAYOUT);
+
+      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+          keyTableIter = keyTable.iterator()) {
+
+        while (keyTableIter.hasNext()) {
+          Table.KeyValue<String, OmKeyInfo> kv = keyTableIter.next();
+          OmKeyInfo keyInfo = kv.getValue();
+
+          setKeyParentID(keyInfo);
+
+          if (keyInfo.getKeyName().endsWith(OM_KEY_PREFIX)) {
+            OmDirectoryInfo directoryInfo =
+                new OmDirectoryInfo.Builder()
+                    .setName(keyInfo.getKeyName())
+                    .setObjectID(keyInfo.getObjectID())
+                    .setParentObjectID(keyInfo.getParentObjectID())
+                    .build();
+            handlePutDirEvent(directoryInfo, nsSummaryMap);
+          } else {
+            handlePutKeyEvent(keyInfo, nsSummaryMap);
+          }
+        }
+      }
+    } catch (IOException ioEx) {
+      LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ",
+          ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
+    }
+
+    try {
+      writeNSSummariesToDB(nsSummaryMap);
+    } catch (IOException e) {
+      LOG.error("Unable to write Namespace Summary data in Recon DB.", e);
+      return new ImmutablePair<>(getTaskName(), false);
+    }
+    LOG.info("Completed a reprocess run of NSSummaryTaskWithLegacy");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  private void setKeyParentID(OmKeyInfo keyInfo) throws IOException {
+    String[] keyPath = keyInfo.getKeyName().split(OM_KEY_PREFIX);
+
+    //if (keyPath > 1) there is one or more directories
+    if (keyPath.length > 1) {
+      String[] dirs = Arrays.copyOf(keyPath, keyPath.length - 1);
+      String parentKeyName = String.join(OM_KEY_PREFIX, dirs);
+      parentKeyName += OM_KEY_PREFIX;
+      String fullParentKeyName =
+          reconOMMetadataManager.getOzoneKey(keyInfo.getVolumeName(),
+              keyInfo.getBucketName(), parentKeyName);
+      OmKeyInfo parentKeyInfo = reconOMMetadataManager
+          .getKeyTable(BUCKET_LAYOUT)
+          .get(fullParentKeyName);
+
+      try {
+        keyInfo.setParentObjectID(parentKeyInfo.getObjectID());
+      } catch (NullPointerException e) {
+        LOG.error("ParentKeyInfo for NSSummaryTaskWithLegacy is null", e);
+      }
+    } else {
+      String bucketKey = reconOMMetadataManager
+          .getBucketKey(keyInfo.getVolumeName(), keyInfo.getBucketName());
+      OmBucketInfo parentBucketInfo =
+          reconOMMetadataManager.getBucketTable().get(bucketKey);
+
+      try {
+        keyInfo.setParentObjectID(parentBucketInfo.getObjectID());
+      } catch (NullPointerException e) {

Review Comment:
   why is the npe being swallowed?



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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #3746: HDDS-7121. Support namespace summaries (du, dist & counts) for legacy FS buckets

Posted by GitBox <gi...@apache.org>.
GeorgeJahad commented on code in PR #3746:
URL: https://github.com/apache/ozone/pull/3746#discussion_r979139835


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/handlers/DirectoryEntityHandler.java:
##########
@@ -89,7 +91,15 @@ public DUResponse getDuResponse(
     for (long subdirObjectId: subdirs) {
       NSSummary subdirNSSummary =
               getReconNamespaceSummaryManager().getNSSummary(subdirObjectId);
-      String subdirName = subdirNSSummary.getDirName();
+      // for the subdirName we need the filename, not the key name
+      Path subdirPath = Paths.get(subdirNSSummary.getDirName());
+      Path subdirFileName = subdirPath.getFileName();
+      String subdirName;
+      if (subdirFileName != null) {
+        subdirName = subdirFileName.toString();
+      } else {
+        throw new NullPointerException("Subdirectory file name is null.");
+      }

Review Comment:
   is this change to remove the trailing slash?  why not: OzoneFSUtils.removeTrailingSlashIfNeeded()



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