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/10/19 05:50:55 UTC

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

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