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/03/31 04:11:45 UTC

[GitHub] [ozone] rakeshadr commented on a change in pull request #2093: HDDS-4495. [FSO]Delete : Implement async cleanup of garbage and orphan sub-dirs/files

rakeshadr commented on a change in pull request #2093:
URL: https://github.com/apache/ozone/pull/2093#discussion_r604580646



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/DirectoryDeletingService.java
##########
@@ -0,0 +1,285 @@
+/**
+ * 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.om;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.ClientVersions.CURRENT_VERSION;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_PATH_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_PATH_DELETING_LIMIT_PER_TASK_DEFAULT;
+
+/**
+ * This is a background service to delete orphan directories and its
+ * sub paths(sub-dirs and sub-files).
+ *
+ * <p>
+ * This will scan the metadata of om periodically to get the orphan dirs from
+ * DeletedDirectoryTable and find its sub paths. It will fetch all sub-files
+ * from KeyTable and move those to DeletedTable so that OM's
+ * KeyDeletingService will cleanup those files later. It will fetch all
+ * sub-directories from the DirectoryTable and move those to
+ * DeletedDirectoryTable so that these will be visited in next iterations.
+ *
+ * <p>
+ * After moving all sub-files and sub-dirs the parent orphan directory will be
+ * deleted by this service. It will continue traversing until all the leaf path
+ * components of an orphan directory is visited.
+ */
+public class DirectoryDeletingService extends BackgroundService {
+
+  private final KeyManager keyManager;
+  private final OzoneManager ozoneManager;
+  private AtomicLong deletedDirsCount;
+  private AtomicLong deletedFilesCount;
+  private final AtomicLong runCount;
+
+  private static ClientId clientId = ClientId.randomId();
+
+  // Use only a single thread for DirDeletion. Multiple threads would read
+  // or write to same tables and can send deletion requests for same key
+  // multiple times.
+  private static final int DIR_DELETING_CORE_POOL_SIZE = 1;
+
+  // Number of items(dirs/files) to be batched in an iteration.
+  private final long pathLimitPerTask;
+
+  public DirectoryDeletingService(long interval, TimeUnit unit,
+      long serviceTimeout, OzoneManager ozoneManager) {
+    super("DirectoryDeletingService", interval, unit,
+        DIR_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.keyManager = ozoneManager.getKeyManager();
+    this.ozoneManager = ozoneManager;
+    this.deletedDirsCount = new AtomicLong(0);
+    this.deletedFilesCount = new AtomicLong(0);
+    this.runCount = new AtomicLong(0);
+    this.pathLimitPerTask = ozoneManager.getConfiguration()
+        .getInt(OZONE_PATH_DELETING_LIMIT_PER_TASK,
+            OZONE_PATH_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private boolean shouldRun() {
+    if (ozoneManager == null) {
+      // OzoneManager can be null for testing
+      return true;
+    }
+    return ozoneManager.isLeaderReady();
+  }
+
+  private boolean isRatisEnabled() {
+    if (ozoneManager == null) {
+      return false;
+    }
+    return ozoneManager.isRatisEnabled();
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+    queue.add(new DirectoryDeletingService.DirDeletingTask());
+    return queue;
+  }
+
+  private class DirDeletingTask implements BackgroundTask {
+
+    @Override
+    public int getPriority() {
+      return 0;
+    }
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (shouldRun()) {
+        runCount.incrementAndGet();
+        long count = pathLimitPerTask;
+        try {
+          long startTime = Time.monotonicNow();
+          // step-1) Get one pending deleted directory
+          OmKeyInfo pendingDeletedDirInfo = keyManager.getPendingDeletionDir();
+          if (pendingDeletedDirInfo != null) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Pending deleted dir name: {}",
+                  pendingDeletedDirInfo.getKeyName());
+            }
+            // step-1: get all sub directories under the deletedDir
+            List<OmKeyInfo> dirs =
+                keyManager.getPendingDeletionSubDirs(pendingDeletedDirInfo,
+                    --count);
+            count = count - dirs.size();
+            List<OmKeyInfo> deletedSubDirList = new ArrayList<>();
+            for (OmKeyInfo dirInfo : dirs) {
+              deletedSubDirList.add(dirInfo);
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("deleted sub dir name: {}",
+                    dirInfo.getKeyName());
+              }
+            }
+
+            // step-2: get all sub files under the deletedDir
+            List<OmKeyInfo> purgeDeletedFiles =
+                keyManager.getPendingDeletionSubFiles(pendingDeletedDirInfo,
+                    --count);
+
+            if (LOG.isDebugEnabled()) {
+              for (OmKeyInfo fileInfo : purgeDeletedFiles) {
+                LOG.debug("deleted sub file name: {}", fileInfo.getKeyName());
+              }
+            }
+
+            // step-3: Since there is a boundary condition of 'numEntries' in
+            // each batch, check whether the sub paths count reached batch size
+            // limit. If count reached limit then there can be some more child
+            // paths to be visited and will keep the parent deleted directory
+            // for one more pass.
+            List<String> purgeDeletedDirs = new ArrayList<>();
+            if (count > 0) {

Review comment:
       Thanks @linyiqun for the reviews. Yes, you are perfectly correct. 
   
   **Design Thoughts:** I've plans to enhance the implementations by adding more directories by recursively traversing to it sub-paths if there are less number of sub-path components in the tree. Please find the below TODO comment in the patch. I'm pasting the same thing here as well for our reference.
   
   ```
               if (count > 0) {
                 // TODO: Now, there is only one entry in this list. Maintained
                 //  list data structure becuase this can be extended to add
                 //  more directories within the batchSize limit.
                 purgeDeletedDirs.add(pendingDeletedDirInfo.getPath());
               }
   ```
   
   Assume below is the FS dir structure at OM and batch size is 1000. Again, assume user has deleted dir 'a'. 
   With recursive listing in place, `DirDeletingService` can collect dirs [a, b1, b2, b3, c1, c2, d1, d2, d3, e1, e2, e3] in first pass then delete these from `DirectoryTable` and files [d21.txt, d22.txt, e11.txt] will be added to `DeletedTable` so that KDS can take care its block(s) deletions. 
   This way I can avoid adding the sub-directory child entries into `DeletedDirectoryTable` and save IO(OM Database) operations. Since this involves some complexity I am planning to implement it via another jira task. Probably, this along with other performance optimizations can be done after feature branch merge to master. Let's see how quickly we can complete the major functionality parts and reach to a logical completion of the feature:-)
   
   By keeping all these thoughts in mind I've used list data structure:-)
   Does this make sense to you and shall I keep list data structure?
   
   ```
        *                  buck-1
        *                    |
        *                    a
        *                    |
        *      -----------------------------------
        *     |           |                       |
        *     b1          b2                      b3
        *   -----       --------               ----------
        *   |    |      |    |   |             |    |     |
        *  c1   c2     d1   d2  d3             e1   e2   e3
        *                   |                  |
        *               --------               |
        *              |        |              |
        *           d21.txt   d22.txt        e11.txt
   ```
   




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

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