adoroszlai commented on code in PR #7796:
URL: https://github.com/apache/ozone/pull/7796#discussion_r1955079054


##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskOBS.java:
##########
@@ -0,0 +1,337 @@
+/*
+ * 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 com.google.inject.Inject;
+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.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.recon.ReconConstants;
+import org.apache.hadoop.ozone.recon.ReconUtils;
+import org.hadoop.ozone.recon.schema.UtilizationSchemaDefinition;
+import org.hadoop.ozone.recon.schema.tables.daos.FileCountBySizeDao;
+import org.hadoop.ozone.recon.schema.tables.pojos.FileCountBySize;
+import org.jooq.DSLContext;
+import org.jooq.Record3;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.List;
+import java.util.HashMap;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.KEY_TABLE;
+import static 
org.hadoop.ozone.recon.schema.tables.FileCountBySizeTable.FILE_COUNT_BY_SIZE;
+
+/**
+ * Class to iterate over the OM DB and store the counts of existing/new
+ * files binned into ranges (1KB, 2Kb..,4MB,.., 1TB,..1PB) to the Recon
+ * fileSize DB.
+ */
+public class FileSizeCountTaskOBS implements ReconOmTask {

Review Comment:
   Let's share common code in a parent class `FileSizeCount`.
   
   Better yet, if the two implementations are code-wise the same, and if it's 
possible to bind two _instances_ of `FileSizeCount`, then we can just set the 
differences as instance variables, like:
   
   ```java
   new FileSizeCountTask(BucketLayout.OBJECT_STORE, 
OmMetadataManagerImpl.KEY_TABLE)
   ```
   
   (Add any other differences as needed.)



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconConstants.java:
##########
@@ -90,4 +92,15 @@ private ReconConstants() {
       (double) MAX_CONTAINER_SIZE_UPPER_BOUND /
           MIN_CONTAINER_SIZE_UPPER_BOUND) /
       Math.log(2)) + 1;
+
+  // For file-size count reprocessing: ensure only one task truncates the 
table.
+  public static final AtomicBoolean TABLE_TRUNCATED = new AtomicBoolean(false);
+
+  /**
+   * Resets the table-truncated flag. This should be called once per reprocess 
cycle,
+   * for example by the OM task controller, before the tasks run.
+   */
+  public static void resetTableTruncatedFlag() {
+    TABLE_TRUNCATED.set(false);

Review Comment:
   I don't see this being called.



##########
hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/tasks/FileSizeCountTaskFSO.java:
##########
@@ -76,27 +75,25 @@ public FileSizeCountTask(FileCountBySizeDao 
fileCountBySizeDao,
    */
   @Override
   public Pair<String, Boolean> reprocess(OMMetadataManager omMetadataManager) {
-    // Map to store the count of files based on file size
     Map<FileSizeCountKey, Long> fileSizeCountMap = new HashMap<>();
+    long startTime = System.currentTimeMillis(); // Start time for execution
 
-    // Delete all records from FILE_COUNT_BY_SIZE table
-    int execute = dslContext.delete(FILE_COUNT_BY_SIZE).execute();
-    LOG.debug("Deleted {} records from {}", execute, FILE_COUNT_BY_SIZE);
-
-    // Call reprocessBucket method for FILE_SYSTEM_OPTIMIZED bucket layout
-    boolean statusFSO =
-        reprocessBucketLayout(BucketLayout.FILE_SYSTEM_OPTIMIZED,
-            omMetadataManager,
-            fileSizeCountMap);
-    // Call reprocessBucket method for LEGACY bucket layout
-    boolean statusOBS =
-        reprocessBucketLayout(BucketLayout.LEGACY, omMetadataManager,
-            fileSizeCountMap);
-    if (!statusFSO && !statusOBS) {
+    // Use the shared atomic flag to ensure only one task truncates the table.
+    if (ReconConstants.TABLE_TRUNCATED.compareAndSet(false, true)) {
+      int execute = dslContext.delete(FILE_COUNT_BY_SIZE).execute();

Review Comment:
   This is not atomic: one task may start adding records before the other one 
executes the delete statement.
   
   Also, the two tasks need to coordinate more than that: once both tasks are 
done, shared state needs to be updated to allow the next pair of tasks to 
truncate the table again.



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