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


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java:
##########
@@ -0,0 +1,474 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.tasks;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.db.RDBBatchOperation;
+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.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
+import org.apache.hadoop.ozone.recon.api.types.KeyPrefixContainer;
+import org.apache.hadoop.ozone.recon.spi.ReconContainerMetadataManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class that encapsulates the common logic for 
ContainerKeyMapperTaskFSO and ContainerKeyMapperTaskOBS.
+ */
+public abstract class ContainerKeyMapperHelper {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ContainerKeyMapperHelper.class);
+
+  // Static lock to guard table truncation.
+  private static final Object TRUNCATE_LOCK = new Object();
+
+  /**
+   * Ensures that the container key tables are truncated only once before 
reprocessing.
+   * Uses an AtomicBoolean to track if truncation has already been performed.
+   *
+   * @param reconContainerMetadataManager The metadata manager instance 
responsible for DB operations.
+   */
+  public static void truncateTablesIfNeeded(ReconContainerMetadataManager 
reconContainerMetadataManager) {
+    synchronized (TRUNCATE_LOCK) {
+      if (ReconConstants.CONTAINER_KEY_TABLES_TRUNCATED.compareAndSet(false, 
true)) {
+        try {
+          // Perform table truncation
+          reconContainerMetadataManager.reinitWithNewContainerDataFromOm(new 
HashMap<>());
+          LOG.info("Successfully truncated container key tables.");
+        } catch (IOException e) {
+          // Reset the flag so truncation can be retried
+          ReconConstants.CONTAINER_KEY_TABLES_TRUNCATED.set(false);
+          LOG.error("Error while truncating container key tables. Resetting 
flag.", e);
+          throw new RuntimeException("Table truncation failed", e);
+        }
+      } else {
+        LOG.info("Container key tables already truncated by another task, 
waiting for truncation to complete.");

Review Comment:
   If some task thread is able to take a lock and enter here in `else` block , 
then it means other task thread has already released lock and truncate is over, 
so the above log message is mis-leading. So I think, we can just have:
   ```suggestion
           LOG.info("Container key tables already truncated by another task");
   ```



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java:
##########
@@ -0,0 +1,474 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.tasks;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.db.RDBBatchOperation;
+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.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
+import org.apache.hadoop.ozone.recon.api.types.KeyPrefixContainer;
+import org.apache.hadoop.ozone.recon.spi.ReconContainerMetadataManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class that encapsulates the common logic for 
ContainerKeyMapperTaskFSO and ContainerKeyMapperTaskOBS.
+ */
+public abstract class ContainerKeyMapperHelper {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ContainerKeyMapperHelper.class);
+
+  // Static lock to guard table truncation.
+  private static final Object TRUNCATE_LOCK = new Object();
+
+  /**
+   * Ensures that the container key tables are truncated only once before 
reprocessing.
+   * Uses an AtomicBoolean to track if truncation has already been performed.
+   *
+   * @param reconContainerMetadataManager The metadata manager instance 
responsible for DB operations.
+   */
+  public static void truncateTablesIfNeeded(ReconContainerMetadataManager 
reconContainerMetadataManager) {
+    synchronized (TRUNCATE_LOCK) {
+      if (ReconConstants.CONTAINER_KEY_TABLES_TRUNCATED.compareAndSet(false, 
true)) {
+        try {
+          // Perform table truncation
+          reconContainerMetadataManager.reinitWithNewContainerDataFromOm(new 
HashMap<>());
+          LOG.info("Successfully truncated container key tables.");
+        } catch (IOException e) {

Review Comment:
   Having just `IOException` in catch may not guarantee the handle the truncate 
tables failure. I think we should catch with `Exception`



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconControllerModule.java:
##########
@@ -131,7 +132,8 @@ static class ReconOmTaskBindingModule extends 
AbstractModule {
     protected void configure() {
       Multibinder<ReconOmTask> taskBinder =
           Multibinder.newSetBinder(binder(), ReconOmTask.class);
-      taskBinder.addBinding().to(ContainerKeyMapperTask.class);
+      taskBinder.addBinding().to(ContainerKeyMapperTaskFSO.class);
+      taskBinder.addBinding().to(ContainerKeyMapperTaskOBS.class);

Review Comment:
   If we are splitting into multiple tasks, then we should consider increasing 
the thread count of executor service from 5 to 8 atleast because our 
independent recon OM tasks have increased.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java:
##########
@@ -0,0 +1,474 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.tasks;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.db.RDBBatchOperation;
+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.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
+import org.apache.hadoop.ozone.recon.api.types.KeyPrefixContainer;
+import org.apache.hadoop.ozone.recon.spi.ReconContainerMetadataManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class that encapsulates the common logic for 
ContainerKeyMapperTaskFSO and ContainerKeyMapperTaskOBS.
+ */
+public abstract class ContainerKeyMapperHelper {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ContainerKeyMapperHelper.class);
+
+  // Static lock to guard table truncation.
+  private static final Object TRUNCATE_LOCK = new Object();
+
+  /**
+   * Ensures that the container key tables are truncated only once before 
reprocessing.
+   * Uses an AtomicBoolean to track if truncation has already been performed.
+   *
+   * @param reconContainerMetadataManager The metadata manager instance 
responsible for DB operations.
+   */
+  public static void truncateTablesIfNeeded(ReconContainerMetadataManager 
reconContainerMetadataManager) {
+    synchronized (TRUNCATE_LOCK) {
+      if (ReconConstants.CONTAINER_KEY_TABLES_TRUNCATED.compareAndSet(false, 
true)) {
+        try {
+          // Perform table truncation
+          reconContainerMetadataManager.reinitWithNewContainerDataFromOm(new 
HashMap<>());
+          LOG.info("Successfully truncated container key tables.");
+        } catch (IOException e) {
+          // Reset the flag so truncation can be retried
+          ReconConstants.CONTAINER_KEY_TABLES_TRUNCATED.set(false);
+          LOG.error("Error while truncating container key tables. Resetting 
flag.", e);
+          throw new RuntimeException("Table truncation failed", e);
+        }
+      } else {
+        LOG.info("Container key tables already truncated by another task, 
waiting for truncation to complete.");
+      }
+    }
+  }
+
+  public static Pair<String, Boolean> reprocess(OMMetadataManager 
omMetadataManager,
+                                                ReconContainerMetadataManager 
reconContainerMetadataManager,
+                                                BucketLayout bucketLayout,
+                                                String taskName,
+                                                long 
containerKeyFlushToDBMaxThreshold) {
+    long omKeyCount = 0;
+    Map<ContainerKeyPrefix, Integer> containerKeyMap = new HashMap<>();
+    Map<Long, Long> containerKeyCountMap = new HashMap<>();
+
+    try {
+      LOG.info("Starting a 'reprocess' run for {}.", taskName);
+      Instant start = Instant.now();
+
+      // Ensure the tables are truncated only once
+      truncateTablesIfNeeded(reconContainerMetadataManager);
+
+      // Get the appropriate table based on BucketLayout
+      Table<String, OmKeyInfo> omKeyInfoTable = 
omMetadataManager.getKeyTable(bucketLayout);
+
+      // Iterate through the table and process keys
+      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>> 
keyIter = omKeyInfoTable.iterator()) {
+        while (keyIter.hasNext()) {
+          Table.KeyValue<String, OmKeyInfo> kv = keyIter.next();
+          handleKeyReprocess(kv.getKey(), kv.getValue(), containerKeyMap, 
containerKeyCountMap,
+              reconContainerMetadataManager);
+          omKeyCount++;
+
+          // Check and flush data if it reaches the batch threshold
+          if (!checkAndCallFlushToDB(containerKeyMap, 
containerKeyFlushToDBMaxThreshold,

Review Comment:
   Pls check this flush method call, while reviewing this code, it seems there 
was a bug in existing code also. This may seem create inconsistent data when 
`containerKeyMap` is flushed after size reaches to 
`containerKeyFlushToDBMaxThreshold` and if `containerKeyMap` fills with same 
`containerKeyPrefix` in next few iterations , then duplicate data in 
`containerKeyMap` will be flushed to DB again.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/ContainerKeyMapperHelper.java:
##########
@@ -0,0 +1,474 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.tasks;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.utils.db.RDBBatchOperation;
+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.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.api.types.ContainerKeyPrefix;
+import org.apache.hadoop.ozone.recon.api.types.KeyPrefixContainer;
+import org.apache.hadoop.ozone.recon.spi.ReconContainerMetadataManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Helper class that encapsulates the common logic for 
ContainerKeyMapperTaskFSO and ContainerKeyMapperTaskOBS.
+ */
+public abstract class ContainerKeyMapperHelper {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ContainerKeyMapperHelper.class);
+
+  // Static lock to guard table truncation.
+  private static final Object TRUNCATE_LOCK = new Object();
+
+  /**
+   * Ensures that the container key tables are truncated only once before 
reprocessing.
+   * Uses an AtomicBoolean to track if truncation has already been performed.
+   *
+   * @param reconContainerMetadataManager The metadata manager instance 
responsible for DB operations.
+   */
+  public static void truncateTablesIfNeeded(ReconContainerMetadataManager 
reconContainerMetadataManager) {
+    synchronized (TRUNCATE_LOCK) {
+      if (ReconConstants.CONTAINER_KEY_TABLES_TRUNCATED.compareAndSet(false, 
true)) {
+        try {
+          // Perform table truncation
+          reconContainerMetadataManager.reinitWithNewContainerDataFromOm(new 
HashMap<>());
+          LOG.info("Successfully truncated container key tables.");
+        } catch (IOException e) {
+          // Reset the flag so truncation can be retried
+          ReconConstants.CONTAINER_KEY_TABLES_TRUNCATED.set(false);
+          LOG.error("Error while truncating container key tables. Resetting 
flag.", e);
+          throw new RuntimeException("Table truncation failed", e);
+        }
+      } else {
+        LOG.info("Container key tables already truncated by another task, 
waiting for truncation to complete.");
+      }
+    }
+  }
+
+  public static Pair<String, Boolean> reprocess(OMMetadataManager 
omMetadataManager,
+                                                ReconContainerMetadataManager 
reconContainerMetadataManager,
+                                                BucketLayout bucketLayout,
+                                                String taskName,
+                                                long 
containerKeyFlushToDBMaxThreshold) {
+    long omKeyCount = 0;
+    Map<ContainerKeyPrefix, Integer> containerKeyMap = new HashMap<>();
+    Map<Long, Long> containerKeyCountMap = new HashMap<>();
+
+    try {
+      LOG.info("Starting a 'reprocess' run for {}.", taskName);
+      Instant start = Instant.now();
+
+      // Ensure the tables are truncated only once
+      truncateTablesIfNeeded(reconContainerMetadataManager);
+
+      // Get the appropriate table based on BucketLayout
+      Table<String, OmKeyInfo> omKeyInfoTable = 
omMetadataManager.getKeyTable(bucketLayout);
+
+      // Iterate through the table and process keys
+      try (TableIterator<String, ? extends Table.KeyValue<String, OmKeyInfo>> 
keyIter = omKeyInfoTable.iterator()) {
+        while (keyIter.hasNext()) {
+          Table.KeyValue<String, OmKeyInfo> kv = keyIter.next();
+          handleKeyReprocess(kv.getKey(), kv.getValue(), containerKeyMap, 
containerKeyCountMap,
+              reconContainerMetadataManager);
+          omKeyCount++;
+
+          // Check and flush data if it reaches the batch threshold
+          if (!checkAndCallFlushToDB(containerKeyMap, 
containerKeyFlushToDBMaxThreshold,
+              reconContainerMetadataManager)) {
+            LOG.error("Failed to flush container key data for {}", taskName);
+            return new ImmutablePair<>(taskName, false);
+          }
+        }
+      }
+
+      // Flush and commit changes
+      if (!flushAndCommitContainerKeyInfoToDB(containerKeyMap, 
containerKeyCountMap, reconContainerMetadataManager)) {
+        LOG.error("Failed to flush Container Key data to DB for {}", taskName);
+        return new ImmutablePair<>(taskName, false);
+      }
+
+      LOG.info("Completed 'reprocess' for {}. Processed {} keys.", taskName, 
omKeyCount);
+      Instant end = Instant.now();
+      long duration = Duration.between(start, end).toMillis();
+      LOG.info("Total time: {} seconds.", (double) duration / 1000.0);

Review Comment:
   Merge this log line and log line at 125 line number. Try to add as metric 
also.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to