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 2021/07/06 18:56:12 UTC

[GitHub] [ozone] yuangu002 opened a new pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

yuangu002 opened a new pull request #2389:
URL: https://github.com/apache/ozone/pull/2389


   ## What changes were proposed in this pull request?
   
   FileSizeCountTask counts the number of bins inside each buckets.
   Try to build on FileSizeCountTask to count what we need for NSSummary as well.
   [6.28] Added an independent Recon task to process namespace summary info per objectID in the attempt to decouple with other task modules.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-5386
   
   ## How was this patch tested?
   Unit test: org.apache.hadoop.ozone.recon.tasks.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] avijayanhwx commented on a change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r667140102



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+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.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+
+/**
+ * Task to query data from OMDB and write into Recon RocksDB.

Review comment:
       Maybe a more specific Javadoc?

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/NSSummary.java
##########
@@ -20,51 +20,67 @@
 
 import org.apache.hadoop.ozone.recon.ReconConstants;
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
 /**
  * Class to encapsulate namespace metadata summaries from OM.
  */
 
 public class NSSummary {
   private int numOfFiles;
-  private int sizeOfFiles;
+  private long sizeOfFiles;
   private int[] fileSizeBucket;
+  private List<Long> childDir;
 
   public NSSummary() {
     this.numOfFiles = 0;
-    this.sizeOfFiles = 0;
+    this.sizeOfFiles = 0L;
     this.fileSizeBucket = new int[ReconConstants.NUM_OF_BINS];
+    this.childDir = new ArrayList<>();
   }
 
-  public NSSummary(int numOfFiles, int sizeOfFiles, int[] bucket) {
+  public NSSummary(int numOfFiles,
+                   long sizeOfFiles,
+                   int[] bucket,
+                   List<Long> childDir) {
     this.numOfFiles = numOfFiles;
     this.sizeOfFiles = sizeOfFiles;
     setFileSizeBucket(bucket);
+    setChildDir(childDir);
   }
 
   public int getNumOfFiles() {
     return numOfFiles;
   }
 
-  public int getSizeOfFiles() {
+  public long getSizeOfFiles() {
     return sizeOfFiles;
   }
 
   public int[] getFileSizeBucket() {
     return Arrays.copyOf(this.fileSizeBucket, ReconConstants.NUM_OF_BINS);
   }
 
+  public List<Long> getChildDir() {
+    return new ArrayList<>(childDir);

Review comment:
       Why do we need to create a copy of the map here? We can avoid the setter in NSSummary#setChildDir if a copy is not being created here. Is there a use case for an NSSummary object to be immutable? If yes, then there should not be a set.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+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.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+
+/**
+ * Task to query data from OMDB and write into Recon RocksDB.
+ */
+public class NSSummaryTask implements ReconOmTask {
+  private static final Logger LOG =
+          LoggerFactory.getLogger(NSSummaryTask.class);
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  public NSSummaryTask(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager) {
+    this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
+  }
+
+  // We only listen to updates from FSO-enabled KeyTable(FileTable) and DirTable
+  public Collection<String> getTaskTables() {
+    return Arrays.asList(new String[]{FILE_TABLE, DIRECTORY_TABLE});
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    final Collection<String> taskTables = getTaskTables();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+              WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable and Dirtable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnFileTable = table.equals(FILE_TABLE);
+      if (!taskTables.contains(table)) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        if (updateOnFileTable) {
+          // key update on fileTable
+          OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+          OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+          OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case DELETE:
+            deleteOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              deleteOmKeyInfoOnNamespaceDB(oldKeyInfo);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                      updatedKey);
+            }
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                    omdbUpdateEvent.getAction());
+          }
+        } else {
+          // directory update on DirTable
+          OMDBUpdateEvent<String, OmDirectoryInfo> dirTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmDirectoryInfo>) omdbUpdateEvent;
+          OmDirectoryInfo updatedDirectoryInfo = dirTableUpdateEvent.getValue();
+          OmDirectoryInfo oldDirectoryInfo = dirTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case DELETE:
+            deleteOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case UPDATE:
+            // TODO: we may just want to ignore update event on table,
+            //  if objectId and parentObjectId cannot be modified.
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              deleteOmDirectoryInfoOnNamespaceDB(oldDirectoryInfo);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                      updatedKey);
+            }
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            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);
+      }
+    }
+    LOG.info("Completed a process run of NSSummaryTask");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    // actually fileTable with FSO
+    Table keyTable = omMetadataManager.getKeyTable();
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+            keyTableIter = keyTable.iterator();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.initNSSummaryTable();

Review comment:
       If we are truncating the table here, can we name method appropriately? init implies we are creating something new.

##########
File path: hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java
##########
@@ -0,0 +1,475 @@
+/**
+ * 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.protocol.proto.HddsProtos;
+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.ratis.utils.OzoneManagerRatisUtils;
+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.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProviderWithFSO;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.initializeNewOmMetadataManager;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeDirToOm;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm;
+
+/**
+ * Test for NSSummaryTask.
+ */
+public class TestNSSummaryTask {
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private OMMetadataManager omMetadataManager;
+  private ReconOMMetadataManager reconOMMetadataManager;
+  private OzoneManagerServiceProviderImpl ozoneManagerServiceProvider;
+
+  // Object names in FSO-enabled format
+  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_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 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;
+
+  @Before
+  public void setUp() throws Exception {
+    omMetadataManager = initializeNewOmMetadataManager(
+            temporaryFolder.newFolder());
+    ozoneManagerServiceProvider =
+            getMockOzoneManagerServiceProviderWithFSO();
+    reconOMMetadataManager = getTestReconOmMetadataManager(omMetadataManager,
+            temporaryFolder.newFolder());
+
+    ReconTestInjector reconTestInjector =
+            new ReconTestInjector.Builder(temporaryFolder)
+                    .withReconOm(reconOMMetadataManager)
+                    .withOmServiceProvider(ozoneManagerServiceProvider)
+                    .withReconSqlDb()
+                    .withContainerDB()
+                    .build();
+    reconNamespaceSummaryManager =
+            reconTestInjector.getInstance(ReconNamespaceSummaryManager.class);
+    OzoneManagerRatisUtils.setBucketFSOptimized(true);
+  }
+
+  @Test
+  public void testReprocess() throws Exception {
+    NSSummary nonExistentSummary =
+            reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID);
+    Assert.assertNull(nonExistentSummary);
+
+    populateOMDB();
+
+    NSSummaryTask nsSummaryTask = new NSSummaryTask(

Review comment:
       Can we write an NS summary prior to reprocess and verify it got cleaned up after?

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+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.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+
+/**
+ * Task to query data from OMDB and write into Recon RocksDB.
+ */
+public class NSSummaryTask implements ReconOmTask {
+  private static final Logger LOG =
+          LoggerFactory.getLogger(NSSummaryTask.class);
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  public NSSummaryTask(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager) {
+    this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
+  }
+
+  // We only listen to updates from FSO-enabled KeyTable(FileTable) and DirTable
+  public Collection<String> getTaskTables() {
+    return Arrays.asList(new String[]{FILE_TABLE, DIRECTORY_TABLE});
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    final Collection<String> taskTables = getTaskTables();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+              WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable and Dirtable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnFileTable = table.equals(FILE_TABLE);
+      if (!taskTables.contains(table)) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        if (updateOnFileTable) {
+          // key update on fileTable
+          OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+          OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+          OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case DELETE:
+            deleteOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              deleteOmKeyInfoOnNamespaceDB(oldKeyInfo);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                      updatedKey);
+            }
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                    omdbUpdateEvent.getAction());
+          }
+        } else {
+          // directory update on DirTable
+          OMDBUpdateEvent<String, OmDirectoryInfo> dirTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmDirectoryInfo>) omdbUpdateEvent;
+          OmDirectoryInfo updatedDirectoryInfo = dirTableUpdateEvent.getValue();
+          OmDirectoryInfo oldDirectoryInfo = dirTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case DELETE:
+            deleteOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case UPDATE:
+            // TODO: we may just want to ignore update event on table,
+            //  if objectId and parentObjectId cannot be modified.
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              deleteOmDirectoryInfoOnNamespaceDB(oldDirectoryInfo);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                      updatedKey);
+            }
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            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);
+      }
+    }
+    LOG.info("Completed a process run of NSSummaryTask");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    // actually fileTable with FSO
+    Table keyTable = omMetadataManager.getKeyTable();
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+            keyTableIter = keyTable.iterator();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.initNSSummaryTable();
+
+      while (keyTableIter.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> kv = keyTableIter.next();
+        OmKeyInfo keyInfo = kv.getValue();
+        writeOmKeyInfoOnNamespaceDB(keyInfo);
+      }
+
+      Table dirTable = omMetadataManager.getDirectoryTable();
+      TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+              dirTableIter = dirTable.iterator();
+
+      while (dirTableIter.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> kv = dirTableIter.next();
+        OmDirectoryInfo directoryInfo = kv.getValue();
+        writeOmDirectoryInfoOnNamespaceDB(directoryInfo);

Review comment:
       Why not go down the hierarchy tree instead of starting with leaves? (Create directories first and then the files)

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+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.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+
+/**
+ * Task to query data from OMDB and write into Recon RocksDB.
+ */
+public class NSSummaryTask implements ReconOmTask {
+  private static final Logger LOG =
+          LoggerFactory.getLogger(NSSummaryTask.class);
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  public NSSummaryTask(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager) {
+    this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
+  }
+
+  // We only listen to updates from FSO-enabled KeyTable(FileTable) and DirTable
+  public Collection<String> getTaskTables() {
+    return Arrays.asList(new String[]{FILE_TABLE, DIRECTORY_TABLE});
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    final Collection<String> taskTables = getTaskTables();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+              WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable and Dirtable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnFileTable = table.equals(FILE_TABLE);
+      if (!taskTables.contains(table)) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        if (updateOnFileTable) {
+          // key update on fileTable
+          OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+          OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+          OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case DELETE:
+            deleteOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              deleteOmKeyInfoOnNamespaceDB(oldKeyInfo);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                      updatedKey);
+            }
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                    omdbUpdateEvent.getAction());
+          }
+        } else {
+          // directory update on DirTable
+          OMDBUpdateEvent<String, OmDirectoryInfo> dirTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmDirectoryInfo>) omdbUpdateEvent;
+          OmDirectoryInfo updatedDirectoryInfo = dirTableUpdateEvent.getValue();
+          OmDirectoryInfo oldDirectoryInfo = dirTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case DELETE:
+            deleteOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case UPDATE:
+            // TODO: we may just want to ignore update event on table,
+            //  if objectId and parentObjectId cannot be modified.
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              deleteOmDirectoryInfoOnNamespaceDB(oldDirectoryInfo);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                      updatedKey);
+            }
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            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);
+      }
+    }
+    LOG.info("Completed a process run of NSSummaryTask");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    // actually fileTable with FSO
+    Table keyTable = omMetadataManager.getKeyTable();
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+            keyTableIter = keyTable.iterator();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.initNSSummaryTable();
+
+      while (keyTableIter.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> kv = keyTableIter.next();
+        OmKeyInfo keyInfo = kv.getValue();
+        writeOmKeyInfoOnNamespaceDB(keyInfo);
+      }
+
+      Table dirTable = omMetadataManager.getDirectoryTable();
+      TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+              dirTableIter = dirTable.iterator();
+
+      while (dirTableIter.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> kv = dirTableIter.next();
+        OmDirectoryInfo directoryInfo = kv.getValue();
+        writeOmDirectoryInfoOnNamespaceDB(directoryInfo);
+      }
+
+    } catch (IOException ioEx) {
+      LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ",
+              ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
+    }
+
+    LOG.info("Completed a reprocess run of NSSummaryTask");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  private void writeOmKeyInfoOnNamespaceDB(OmKeyInfo keyInfo)
+          throws IOException {
+    long parentObjectId = keyInfo.getParentObjectID();
+    NSSummary nsSummary = reconNamespaceSummaryManager
+            .getNSSummary(parentObjectId);
+    if (nsSummary == null) {
+      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);
+
+    // make sure the file is within our scope of tracking.
+    if (binIndex >= 0 && binIndex < ReconConstants.NUM_OF_BINS) {

Review comment:
       The FileSizeCountTask was supposed to map all files >1PB to the last bucket. Hence, there is no overflow of bin index conceptually. Is that not the 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] yuangu002 commented on a change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r665768508



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.OmKeyInfo;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+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.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Task to query data from OMDB and write into Recon RocksDB.
+ */
+public class NSSummaryTask implements ReconOmTask {
+  private static final Logger LOG =
+          LoggerFactory.getLogger(NSSummaryTask.class);
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  public NSSummaryTask(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager) {
+    this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
+  }
+
+  public Collection<String> getTaskTables() {
+    return Collections.singletonList(KEY_TABLE);
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    final Collection<String> taskTables = getTaskTables();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, OmKeyInfo> omdbUpdateEvent = eventIterator.next();
+      // we only process updates on OM's KeyTable.
+      if (!taskTables.contains(omdbUpdateEvent.getTable())) {

Review comment:
       For testing I think it's fine (as long as we set parent ID correct, we don't need to enable FSO in testing).
   But for the newer version of this branch, I am going to only track the changes in the FileTable (because FSO-enabled ozone doesn't have a KeyTable anymore. Instead, DirTable and FileTable replace it, according to [this](https://ci-hadoop.apache.org/view/Hadoop%20Ozone/job/ozone-doc-master/lastSuccessfulBuild/artifact/hadoop-hdds/docs/public/feature/prefixfso.html)).
   According to the existing code, if the FSO is enabled, OmMetadataManagerImpl.getKeyTable() will simply return the fileTable:
   ![Screen Shot 2021-07-07 at 7 25 53 PM](https://user-images.githubusercontent.com/53324985/124840558-41906e80-df59-11eb-9eaa-e93ed770d43e.png)
   Let me know if it makes sense to you.




-- 
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] yuangu002 commented on a change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r668232714



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/NSSummary.java
##########
@@ -20,51 +20,67 @@
 
 import org.apache.hadoop.ozone.recon.ReconConstants;
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
 /**
  * Class to encapsulate namespace metadata summaries from OM.
  */
 
 public class NSSummary {
   private int numOfFiles;
-  private int sizeOfFiles;
+  private long sizeOfFiles;
   private int[] fileSizeBucket;
+  private List<Long> childDir;

Review comment:
       Yeah makes sense, but maybe on the next patch? otherwise there'll be more conflicts to resolve.




-- 
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 change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r667228188



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/OMDBUpdateEvent.java
##########
@@ -83,6 +83,7 @@ public boolean equals(Object o) {
     }
     OMDBUpdateEvent that = (OMDBUpdateEvent) o;
     return this.updatedKey.equals(that.updatedKey) &&
+            this.table.equals(that.table) &&

Review comment:
       nit: alignment
   
   ```suggestion
           this.table.equals(that.table) &&
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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



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


[GitHub] [ozone] smengcl commented on pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

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


   Merged. Thanks @yuangu002 for the contribution. Thanks @avijayanhwx 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] yuangu002 commented on pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#issuecomment-875005944


   @smengcl @avijayanhwx @vivekratnavel Could you guys 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] yuangu002 commented on a change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r665768508



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.OmKeyInfo;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+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.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Task to query data from OMDB and write into Recon RocksDB.
+ */
+public class NSSummaryTask implements ReconOmTask {
+  private static final Logger LOG =
+          LoggerFactory.getLogger(NSSummaryTask.class);
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  public NSSummaryTask(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager) {
+    this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
+  }
+
+  public Collection<String> getTaskTables() {
+    return Collections.singletonList(KEY_TABLE);
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    final Collection<String> taskTables = getTaskTables();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, OmKeyInfo> omdbUpdateEvent = eventIterator.next();
+      // we only process updates on OM's KeyTable.
+      if (!taskTables.contains(omdbUpdateEvent.getTable())) {

Review comment:
       For testing I think it's fine (as long as we set parent ID correct, we don't need to enable FSO in testing).
   But for the newer version of this branch, I am going to only track the changes in the FileTable (because FSO-enabled ozone doesn't have a KeyTable anymore. Instead, DirTable and FileTable replace it, according to [this](https://ci-hadoop.apache.org/view/Hadoop%20Ozone/job/ozone-doc-master/lastSuccessfulBuild/artifact/hadoop-hdds/docs/public/feature/prefixfso.html)).
   According to the existing code, if the FSO is enabled, `OmMetadataManagerImpl.getKeyTable()` will simply return the fileTable:
   ![Screen Shot 2021-07-07 at 7 25 53 PM](https://user-images.githubusercontent.com/53324985/124840558-41906e80-df59-11eb-9eaa-e93ed770d43e.png)
   Let me know if it makes sense to you.




-- 
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 #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

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


   


-- 
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 change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r667200199



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/NSSummary.java
##########
@@ -20,51 +20,67 @@
 
 import org.apache.hadoop.ozone.recon.ReconConstants;
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
 /**
  * Class to encapsulate namespace metadata summaries from OM.
  */
 
 public class NSSummary {
   private int numOfFiles;
-  private int sizeOfFiles;
+  private long sizeOfFiles;
   private int[] fileSizeBucket;
+  private List<Long> childDir;
 
   public NSSummary() {
     this.numOfFiles = 0;
-    this.sizeOfFiles = 0;
+    this.sizeOfFiles = 0L;
     this.fileSizeBucket = new int[ReconConstants.NUM_OF_BINS];
+    this.childDir = new ArrayList<>();
   }
 
-  public NSSummary(int numOfFiles, int sizeOfFiles, int[] bucket) {
+  public NSSummary(int numOfFiles,
+                   long sizeOfFiles,
+                   int[] bucket,
+                   List<Long> childDir) {
     this.numOfFiles = numOfFiles;
     this.sizeOfFiles = sizeOfFiles;
     setFileSizeBucket(bucket);
+    setChildDir(childDir);
   }
 
   public int getNumOfFiles() {
     return numOfFiles;
   }
 
-  public int getSizeOfFiles() {
+  public long getSizeOfFiles() {
     return sizeOfFiles;
   }
 
   public int[] getFileSizeBucket() {
     return Arrays.copyOf(this.fileSizeBucket, ReconConstants.NUM_OF_BINS);
   }
 
+  public List<Long> getChildDir() {
+    return new ArrayList<>(childDir);

Review comment:
       I agree returning a copy of the array every time is no good for perf.
   
   Since this is internal maybe we can suppress the bugcheck warning with an annotation.




-- 
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 change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r667199196



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/NSSummary.java
##########
@@ -20,51 +20,67 @@
 
 import org.apache.hadoop.ozone.recon.ReconConstants;
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
 /**
  * Class to encapsulate namespace metadata summaries from OM.
  */
 
 public class NSSummary {
   private int numOfFiles;
-  private int sizeOfFiles;
+  private long sizeOfFiles;
   private int[] fileSizeBucket;
+  private List<Long> childDir;
 
   public NSSummary() {
     this.numOfFiles = 0;
-    this.sizeOfFiles = 0;
+    this.sizeOfFiles = 0L;
     this.fileSizeBucket = new int[ReconConstants.NUM_OF_BINS];
+    this.childDir = new ArrayList<>();
   }
 
-  public NSSummary(int numOfFiles, int sizeOfFiles, int[] bucket) {
+  public NSSummary(int numOfFiles,
+                   long sizeOfFiles,
+                   int[] bucket,
+                   List<Long> childDir) {
     this.numOfFiles = numOfFiles;
     this.sizeOfFiles = sizeOfFiles;
     setFileSizeBucket(bucket);
+    setChildDir(childDir);
   }
 
   public int getNumOfFiles() {
     return numOfFiles;
   }
 
-  public int getSizeOfFiles() {
+  public long getSizeOfFiles() {
     return sizeOfFiles;
   }
 
   public int[] getFileSizeBucket() {
     return Arrays.copyOf(this.fileSizeBucket, ReconConstants.NUM_OF_BINS);
   }
 
+  public List<Long> getChildDir() {
+    return new ArrayList<>(childDir);

Review comment:
       @yuangu002 mentioned a `bugcheck` complaint by just returning the array ptr itself. As the accessor can inadvertently change its content.




-- 
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] yuangu002 commented on a change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
yuangu002 commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r668118330



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+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.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+
+/**
+ * Task to query data from OMDB and write into Recon RocksDB.
+ */
+public class NSSummaryTask implements ReconOmTask {
+  private static final Logger LOG =
+          LoggerFactory.getLogger(NSSummaryTask.class);
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  public NSSummaryTask(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager) {
+    this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
+  }
+
+  // We only listen to updates from FSO-enabled KeyTable(FileTable) and DirTable
+  public Collection<String> getTaskTables() {
+    return Arrays.asList(new String[]{FILE_TABLE, DIRECTORY_TABLE});
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    final Collection<String> taskTables = getTaskTables();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+              WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable and Dirtable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnFileTable = table.equals(FILE_TABLE);
+      if (!taskTables.contains(table)) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        if (updateOnFileTable) {
+          // key update on fileTable
+          OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+          OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+          OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case DELETE:
+            deleteOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              deleteOmKeyInfoOnNamespaceDB(oldKeyInfo);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                      updatedKey);
+            }
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                    omdbUpdateEvent.getAction());
+          }
+        } else {
+          // directory update on DirTable
+          OMDBUpdateEvent<String, OmDirectoryInfo> dirTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmDirectoryInfo>) omdbUpdateEvent;
+          OmDirectoryInfo updatedDirectoryInfo = dirTableUpdateEvent.getValue();
+          OmDirectoryInfo oldDirectoryInfo = dirTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case DELETE:
+            deleteOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case UPDATE:
+            // TODO: we may just want to ignore update event on table,
+            //  if objectId and parentObjectId cannot be modified.
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              deleteOmDirectoryInfoOnNamespaceDB(oldDirectoryInfo);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                      updatedKey);
+            }
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            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);
+      }
+    }
+    LOG.info("Completed a process run of NSSummaryTask");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    // actually fileTable with FSO
+    Table keyTable = omMetadataManager.getKeyTable();
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+            keyTableIter = keyTable.iterator();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.initNSSummaryTable();
+
+      while (keyTableIter.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> kv = keyTableIter.next();
+        OmKeyInfo keyInfo = kv.getValue();
+        writeOmKeyInfoOnNamespaceDB(keyInfo);
+      }
+
+      Table dirTable = omMetadataManager.getDirectoryTable();
+      TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+              dirTableIter = dirTable.iterator();
+
+      while (dirTableIter.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> kv = dirTableIter.next();
+        OmDirectoryInfo directoryInfo = kv.getValue();
+        writeOmDirectoryInfoOnNamespaceDB(directoryInfo);
+      }
+
+    } catch (IOException ioEx) {
+      LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ",
+              ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
+    }
+
+    LOG.info("Completed a reprocess run of NSSummaryTask");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  private void writeOmKeyInfoOnNamespaceDB(OmKeyInfo keyInfo)
+          throws IOException {
+    long parentObjectId = keyInfo.getParentObjectID();
+    NSSummary nsSummary = reconNamespaceSummaryManager
+            .getNSSummary(parentObjectId);
+    if (nsSummary == null) {
+      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);
+
+    // make sure the file is within our scope of tracking.
+    if (binIndex >= 0 && binIndex < ReconConstants.NUM_OF_BINS) {

Review comment:
       I have a separate function to compute bin index, but yes, I can move this check to the function.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+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.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+
+/**
+ * Task to query data from OMDB and write into Recon RocksDB.
+ */
+public class NSSummaryTask implements ReconOmTask {
+  private static final Logger LOG =
+          LoggerFactory.getLogger(NSSummaryTask.class);
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  public NSSummaryTask(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager) {
+    this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
+  }
+
+  // We only listen to updates from FSO-enabled KeyTable(FileTable) and DirTable
+  public Collection<String> getTaskTables() {
+    return Arrays.asList(new String[]{FILE_TABLE, DIRECTORY_TABLE});
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    final Collection<String> taskTables = getTaskTables();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+              WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable and Dirtable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnFileTable = table.equals(FILE_TABLE);
+      if (!taskTables.contains(table)) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        if (updateOnFileTable) {
+          // key update on fileTable
+          OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+          OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+          OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case DELETE:
+            deleteOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              deleteOmKeyInfoOnNamespaceDB(oldKeyInfo);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                      updatedKey);
+            }
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                    omdbUpdateEvent.getAction());
+          }
+        } else {
+          // directory update on DirTable
+          OMDBUpdateEvent<String, OmDirectoryInfo> dirTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmDirectoryInfo>) omdbUpdateEvent;
+          OmDirectoryInfo updatedDirectoryInfo = dirTableUpdateEvent.getValue();
+          OmDirectoryInfo oldDirectoryInfo = dirTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case DELETE:
+            deleteOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case UPDATE:
+            // TODO: we may just want to ignore update event on table,
+            //  if objectId and parentObjectId cannot be modified.
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              deleteOmDirectoryInfoOnNamespaceDB(oldDirectoryInfo);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                      updatedKey);
+            }
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            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);
+      }
+    }
+    LOG.info("Completed a process run of NSSummaryTask");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    // actually fileTable with FSO
+    Table keyTable = omMetadataManager.getKeyTable();
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+            keyTableIter = keyTable.iterator();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.initNSSummaryTable();

Review comment:
       Renamed to `clearNSSummaryTable`.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.WithParentObjectId;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+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.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.DIRECTORY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.FILE_TABLE;
+
+/**
+ * Task to query data from OMDB and write into Recon RocksDB.
+ */
+public class NSSummaryTask implements ReconOmTask {
+  private static final Logger LOG =
+          LoggerFactory.getLogger(NSSummaryTask.class);
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  public NSSummaryTask(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager) {
+    this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
+  }
+
+  // We only listen to updates from FSO-enabled KeyTable(FileTable) and DirTable
+  public Collection<String> getTaskTables() {
+    return Arrays.asList(new String[]{FILE_TABLE, DIRECTORY_TABLE});
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    final Collection<String> taskTables = getTaskTables();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, ? extends
+              WithParentObjectId> omdbUpdateEvent = eventIterator.next();
+      OMDBUpdateEvent.OMDBUpdateAction action = omdbUpdateEvent.getAction();
+
+      // we only process updates on OM's KeyTable and Dirtable
+      String table = omdbUpdateEvent.getTable();
+      boolean updateOnFileTable = table.equals(FILE_TABLE);
+      if (!taskTables.contains(table)) {
+        continue;
+      }
+
+      String updatedKey = omdbUpdateEvent.getKey();
+
+      try {
+        if (updateOnFileTable) {
+          // key update on fileTable
+          OMDBUpdateEvent<String, OmKeyInfo> keyTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmKeyInfo>) omdbUpdateEvent;
+          OmKeyInfo updatedKeyInfo = keyTableUpdateEvent.getValue();
+          OmKeyInfo oldKeyInfo = keyTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case DELETE:
+            deleteOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          case UPDATE:
+            if (oldKeyInfo != null) {
+              // delete first, then put
+              deleteOmKeyInfoOnNamespaceDB(oldKeyInfo);
+            } else {
+              LOG.warn("Update event does not have the old keyInfo for {}.",
+                      updatedKey);
+            }
+            writeOmKeyInfoOnNamespaceDB(updatedKeyInfo);
+            break;
+
+          default:
+            LOG.debug("Skipping DB update event : {}",
+                    omdbUpdateEvent.getAction());
+          }
+        } else {
+          // directory update on DirTable
+          OMDBUpdateEvent<String, OmDirectoryInfo> dirTableUpdateEvent =
+                  (OMDBUpdateEvent<String, OmDirectoryInfo>) omdbUpdateEvent;
+          OmDirectoryInfo updatedDirectoryInfo = dirTableUpdateEvent.getValue();
+          OmDirectoryInfo oldDirectoryInfo = dirTableUpdateEvent.getOldValue();
+
+          switch (action) {
+          case PUT:
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case DELETE:
+            deleteOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            break;
+
+          case UPDATE:
+            // TODO: we may just want to ignore update event on table,
+            //  if objectId and parentObjectId cannot be modified.
+            if (oldDirectoryInfo != null) {
+              // delete first, then put
+              deleteOmDirectoryInfoOnNamespaceDB(oldDirectoryInfo);
+            } else {
+              LOG.warn("Update event does not have the old dirInfo for {}.",
+                      updatedKey);
+            }
+            writeOmDirectoryInfoOnNamespaceDB(updatedDirectoryInfo);
+            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);
+      }
+    }
+    LOG.info("Completed a process run of NSSummaryTask");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    // actually fileTable with FSO
+    Table keyTable = omMetadataManager.getKeyTable();
+
+    TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>>
+            keyTableIter = keyTable.iterator();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.initNSSummaryTable();
+
+      while (keyTableIter.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> kv = keyTableIter.next();
+        OmKeyInfo keyInfo = kv.getValue();
+        writeOmKeyInfoOnNamespaceDB(keyInfo);
+      }
+
+      Table dirTable = omMetadataManager.getDirectoryTable();
+      TableIterator<String, ? extends Table.KeyValue<String, OmDirectoryInfo>>
+              dirTableIter = dirTable.iterator();
+
+      while (dirTableIter.hasNext()) {
+        Table.KeyValue<String, OmDirectoryInfo> kv = dirTableIter.next();
+        OmDirectoryInfo directoryInfo = kv.getValue();
+        writeOmDirectoryInfoOnNamespaceDB(directoryInfo);

Review comment:
       I think we have already addressed that, but yeah it makes sense intuitively.

##########
File path: hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/tasks/TestNSSummaryTask.java
##########
@@ -0,0 +1,475 @@
+/**
+ * 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.protocol.proto.HddsProtos;
+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.ratis.utils.OzoneManagerRatisUtils;
+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.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getMockOzoneManagerServiceProviderWithFSO;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.getTestReconOmMetadataManager;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.initializeNewOmMetadataManager;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeDirToOm;
+import static org.apache.hadoop.ozone.recon.OMMetadataManagerTestUtils.writeKeyToOm;
+
+/**
+ * Test for NSSummaryTask.
+ */
+public class TestNSSummaryTask {
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+  private OMMetadataManager omMetadataManager;
+  private ReconOMMetadataManager reconOMMetadataManager;
+  private OzoneManagerServiceProviderImpl ozoneManagerServiceProvider;
+
+  // Object names in FSO-enabled format
+  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_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 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;
+
+  @Before
+  public void setUp() throws Exception {
+    omMetadataManager = initializeNewOmMetadataManager(
+            temporaryFolder.newFolder());
+    ozoneManagerServiceProvider =
+            getMockOzoneManagerServiceProviderWithFSO();
+    reconOMMetadataManager = getTestReconOmMetadataManager(omMetadataManager,
+            temporaryFolder.newFolder());
+
+    ReconTestInjector reconTestInjector =
+            new ReconTestInjector.Builder(temporaryFolder)
+                    .withReconOm(reconOMMetadataManager)
+                    .withOmServiceProvider(ozoneManagerServiceProvider)
+                    .withReconSqlDb()
+                    .withContainerDB()
+                    .build();
+    reconNamespaceSummaryManager =
+            reconTestInjector.getInstance(ReconNamespaceSummaryManager.class);
+    OzoneManagerRatisUtils.setBucketFSOptimized(true);
+  }
+
+  @Test
+  public void testReprocess() throws Exception {
+    NSSummary nonExistentSummary =
+            reconNamespaceSummaryManager.getNSSummary(BUCKET_ONE_OBJECT_ID);
+    Assert.assertNull(nonExistentSummary);
+
+    populateOMDB();
+
+    NSSummaryTask nsSummaryTask = new NSSummaryTask(

Review comment:
       Fixed.




-- 
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 change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r667200199



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/NSSummary.java
##########
@@ -20,51 +20,67 @@
 
 import org.apache.hadoop.ozone.recon.ReconConstants;
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
 /**
  * Class to encapsulate namespace metadata summaries from OM.
  */
 
 public class NSSummary {
   private int numOfFiles;
-  private int sizeOfFiles;
+  private long sizeOfFiles;
   private int[] fileSizeBucket;
+  private List<Long> childDir;
 
   public NSSummary() {
     this.numOfFiles = 0;
-    this.sizeOfFiles = 0;
+    this.sizeOfFiles = 0L;
     this.fileSizeBucket = new int[ReconConstants.NUM_OF_BINS];
+    this.childDir = new ArrayList<>();
   }
 
-  public NSSummary(int numOfFiles, int sizeOfFiles, int[] bucket) {
+  public NSSummary(int numOfFiles,
+                   long sizeOfFiles,
+                   int[] bucket,
+                   List<Long> childDir) {
     this.numOfFiles = numOfFiles;
     this.sizeOfFiles = sizeOfFiles;
     setFileSizeBucket(bucket);
+    setChildDir(childDir);
   }
 
   public int getNumOfFiles() {
     return numOfFiles;
   }
 
-  public int getSizeOfFiles() {
+  public long getSizeOfFiles() {
     return sizeOfFiles;
   }
 
   public int[] getFileSizeBucket() {
     return Arrays.copyOf(this.fileSizeBucket, ReconConstants.NUM_OF_BINS);
   }
 
+  public List<Long> getChildDir() {
+    return new ArrayList<>(childDir);

Review comment:
       I agree returning a copy of the array every time is no good for perf.
   
   Since this is internal maybe we can suppress the bugcheck warning with an annotation for now.
   TODO: Best if mitigated in other ways that doesn't break the encapsulation.




-- 
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 change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r665334841



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java
##########
@@ -299,4 +299,28 @@ public static void upsertGlobalStatsTable(Configuration sqlConfiguration,
       globalStatsDao.update(newRecord);
     }
   }
+
+  public static long getFileSizeUpperBound(long fileSize) {
+    if (fileSize >= ReconConstants.MAX_FILE_SIZE_UPPER_BOUND) {
+      return Long.MAX_VALUE;
+    }
+    // The smallest file size being tracked for count
+    // is 1 KB i.e. 1024 = 2 ^ 10.
+    int binIndex = getBinIndex(fileSize);
+    return (long) Math.pow(2, (10 + binIndex));
+  }
+
+  public static int getBinIndex(long fileSize) {
+    int index = nextClosestPowerIndexOfTwo(fileSize);
+    return index < 10 ? 0 : index - 10;
+  }
+
+  private static int nextClosestPowerIndexOfTwo(long dataSize) {
+    int index = 0;
+    while(dataSize != 0) {

Review comment:
       nit
   ```suggestion
       while (dataSize != 0) {
   ```

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.OmKeyInfo;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+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.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Task to query data from OMDB and write into Recon RocksDB.
+ */
+public class NSSummaryTask implements ReconOmTask {
+  private static final Logger LOG =
+          LoggerFactory.getLogger(NSSummaryTask.class);
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  public NSSummaryTask(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager) {
+    this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
+  }
+
+  public Collection<String> getTaskTables() {
+    return Collections.singletonList(KEY_TABLE);
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    final Collection<String> taskTables = getTaskTables();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, OmKeyInfo> omdbUpdateEvent = eventIterator.next();
+      // we only process updates on OM's KeyTable.
+      if (!taskTables.contains(omdbUpdateEvent.getTable())) {
+        continue;
+      }
+      String updatedKey = omdbUpdateEvent.getKey();
+      OmKeyInfo updatedKeyValue = omdbUpdateEvent.getValue();
+
+      try {
+        switch (omdbUpdateEvent.getAction()) {
+        case PUT:
+          writeOmKeyInfoOnNamespaceDB(updatedKeyValue);
+          break;
+
+        case DELETE:
+          deleteOmKeyInfoOnNamespaceDB(updatedKeyValue);
+          break;
+
+        case UPDATE:
+          if (omdbUpdateEvent.getOldValue() != null) {
+            // delete first, then put
+            deleteOmKeyInfoOnNamespaceDB(omdbUpdateEvent.getOldValue());
+          } else {
+            LOG.warn("Update event does not have the old Key Info for {}.",
+                    updatedKey);
+          }
+          writeOmKeyInfoOnNamespaceDB(updatedKeyValue);
+          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);
+      }
+    }
+    LOG.info("Completed a process run of NSSummaryTask");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  @Override
+  public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
+    Table keyTable = omMetadataManager.getKeyTable();
+    TableIterator<String, ? extends
+            Table.KeyValue<String, OmKeyInfo>> tableIter = keyTable.iterator();
+
+    try {
+      // reinit Recon RocksDB's namespace CF.
+      reconNamespaceSummaryManager.initNSSummaryTable();
+
+      while (tableIter.hasNext()) {
+        Table.KeyValue<String, OmKeyInfo> kv = tableIter.next();
+        OmKeyInfo keyInfo = kv.getValue();
+        writeOmKeyInfoOnNamespaceDB(keyInfo);
+      }
+    } catch (IOException ioEx) {
+      LOG.error("Unable to reprocess Namespace Summary data in Recon DB. ",
+              ioEx);
+      return new ImmutablePair<>(getTaskName(), false);
+    }
+
+    LOG.info("Completed a reprocess run of NSSummaryTask");
+    return new ImmutablePair<>(getTaskName(), true);
+  }
+
+  private void writeOmKeyInfoOnNamespaceDB(OmKeyInfo keyInfo)
+          throws IOException {
+    long parentObjectId = keyInfo.getParentObjectID();
+    NSSummary nsSummary = reconNamespaceSummaryManager
+            .getNSSummary(parentObjectId);
+    if (nsSummary == null) {
+      nsSummary = getEmptyNSSummary();
+    }
+    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);
+
+    // make sure the file is within our scope of tracking.
+    if (binIndex >= 0 && binIndex < ReconConstants.NUM_OF_BINS) {
+      ++fileBucket[binIndex];
+      nsSummary.setFileSizeBucket(fileBucket);
+    } else {
+      LOG.warn("File size beyond our tracking scope.");

Review comment:
       Better throw here? Warnings are often overlooked.

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/NSSummaryTask.java
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.OmKeyInfo;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.apache.hadoop.ozone.recon.api.types.NSSummary;
+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.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+
+/**
+ * Task to query data from OMDB and write into Recon RocksDB.
+ */
+public class NSSummaryTask implements ReconOmTask {
+  private static final Logger LOG =
+          LoggerFactory.getLogger(NSSummaryTask.class);
+  private ReconNamespaceSummaryManager reconNamespaceSummaryManager;
+
+  @Inject
+  public NSSummaryTask(ReconNamespaceSummaryManager
+                                 reconNamespaceSummaryManager) {
+    this.reconNamespaceSummaryManager = reconNamespaceSummaryManager;
+  }
+
+  @Override
+  public String getTaskName() {
+    return "NSSummaryTask";
+  }
+
+  public Collection<String> getTaskTables() {
+    return Collections.singletonList(KEY_TABLE);
+  }
+
+  @Override
+  public Pair<String, Boolean> process(OMUpdateEventBatch events) {
+    Iterator<OMDBUpdateEvent> eventIterator = events.getIterator();
+    final Collection<String> taskTables = getTaskTables();
+
+    while (eventIterator.hasNext()) {
+      OMDBUpdateEvent<String, OmKeyInfo> omdbUpdateEvent = eventIterator.next();
+      // we only process updates on OM's KeyTable.
+      if (!taskTables.contains(omdbUpdateEvent.getTable())) {

Review comment:
       Have we made sure that we are only tracking FSO-only buckets here?

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/codec/NSSummaryCodec.java
##########
@@ -36,15 +37,18 @@
 
   private final Codec<Integer> integerCodec = new IntegerCodec();
   private final Codec<Short> shortCodec = new ShortCodec();
-  // 2 int fields + 41-length int array
-  private static final int NUM_OF_INTS = 2 + ReconConstants.NUM_OF_BINS;
+  private final Codec<Long> longCodec = new LongCodec();
+  // 1 int fields + 41-length int array
+  private static final int NUM_OF_INTS = 1 + ReconConstants.NUM_OF_BINS;

Review comment:
       One field changed from int to long hence -1.




-- 
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] avijayanhwx commented on a change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r667234091



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/NSSummary.java
##########
@@ -20,51 +20,67 @@
 
 import org.apache.hadoop.ozone.recon.ReconConstants;
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
 /**
  * Class to encapsulate namespace metadata summaries from OM.
  */
 
 public class NSSummary {
   private int numOfFiles;
-  private int sizeOfFiles;
+  private long sizeOfFiles;
   private int[] fileSizeBucket;
+  private List<Long> childDir;
 
   public NSSummary() {
     this.numOfFiles = 0;
-    this.sizeOfFiles = 0;
+    this.sizeOfFiles = 0L;
     this.fileSizeBucket = new int[ReconConstants.NUM_OF_BINS];
+    this.childDir = new ArrayList<>();
   }
 
-  public NSSummary(int numOfFiles, int sizeOfFiles, int[] bucket) {
+  public NSSummary(int numOfFiles,
+                   long sizeOfFiles,
+                   int[] bucket,
+                   List<Long> childDir) {
     this.numOfFiles = numOfFiles;
     this.sizeOfFiles = sizeOfFiles;
     setFileSizeBucket(bucket);
+    setChildDir(childDir);
   }
 
   public int getNumOfFiles() {
     return numOfFiles;
   }
 
-  public int getSizeOfFiles() {
+  public long getSizeOfFiles() {
     return sizeOfFiles;
   }
 
   public int[] getFileSizeBucket() {
     return Arrays.copyOf(this.fileSizeBucket, ReconConstants.NUM_OF_BINS);
   }
 
+  public List<Long> getChildDir() {
+    return new ArrayList<>(childDir);

Review comment:
       Instead if we really want to keep the collection internal, we could provide addChild, removeChild APIs as well. instead of setChilden(Collection)




-- 
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 change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r668227618



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/NSSummary.java
##########
@@ -20,51 +20,67 @@
 
 import org.apache.hadoop.ozone.recon.ReconConstants;
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
 /**
  * Class to encapsulate namespace metadata summaries from OM.
  */
 
 public class NSSummary {
   private int numOfFiles;
-  private int sizeOfFiles;
+  private long sizeOfFiles;
   private int[] fileSizeBucket;
+  private List<Long> childDir;

Review comment:
       Just a note: If we are implementing `appendToChildDir`/`removeFromChildDir`, maybe use `Set<>` here instead?




-- 
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 change in pull request #2389: HDDS-5386. Add a NSSummaryTask to write NSSummary info into RDB

Posted by GitBox <gi...@apache.org>.
smengcl commented on a change in pull request #2389:
URL: https://github.com/apache/ozone/pull/2389#discussion_r667200199



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/NSSummary.java
##########
@@ -20,51 +20,67 @@
 
 import org.apache.hadoop.ozone.recon.ReconConstants;
 
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.List;
 
 /**
  * Class to encapsulate namespace metadata summaries from OM.
  */
 
 public class NSSummary {
   private int numOfFiles;
-  private int sizeOfFiles;
+  private long sizeOfFiles;
   private int[] fileSizeBucket;
+  private List<Long> childDir;
 
   public NSSummary() {
     this.numOfFiles = 0;
-    this.sizeOfFiles = 0;
+    this.sizeOfFiles = 0L;
     this.fileSizeBucket = new int[ReconConstants.NUM_OF_BINS];
+    this.childDir = new ArrayList<>();
   }
 
-  public NSSummary(int numOfFiles, int sizeOfFiles, int[] bucket) {
+  public NSSummary(int numOfFiles,
+                   long sizeOfFiles,
+                   int[] bucket,
+                   List<Long> childDir) {
     this.numOfFiles = numOfFiles;
     this.sizeOfFiles = sizeOfFiles;
     setFileSizeBucket(bucket);
+    setChildDir(childDir);
   }
 
   public int getNumOfFiles() {
     return numOfFiles;
   }
 
-  public int getSizeOfFiles() {
+  public long getSizeOfFiles() {
     return sizeOfFiles;
   }
 
   public int[] getFileSizeBucket() {
     return Arrays.copyOf(this.fileSizeBucket, ReconConstants.NUM_OF_BINS);
   }
 
+  public List<Long> getChildDir() {
+    return new ArrayList<>(childDir);

Review comment:
       I agree returning a copy of the array every time is no good for perf.
   
   Since this is internal maybe we can suppress the bugcheck warning with an annotation for now.
   TODO (later): Best if mitigated in other ways that doesn't break the encapsulation.




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