You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ck...@apache.org on 2022/10/27 12:49:27 UTC

[ozone] 01/02: HDDS-7328. Improve Deletion of FSO Paths (#3844)

This is an automated email from the ASF dual-hosted git repository.

ckj pushed a commit to branch ozone-1.3
in repository https://gitbox.apache.org/repos/asf/ozone.git

commit 1eb6433aa74f3ead081f63c4a482e6841d7b5c7b
Author: Nibiru <ax...@qq.com>
AuthorDate: Thu Oct 27 19:23:22 2022 +0800

    HDDS-7328. Improve Deletion of FSO Paths (#3844)
---
 .../ozone/TestDirectoryDeletingServiceWithFSO.java | 107 ++++++-----
 .../ozone/om/service/DirectoryDeletingService.java | 195 +++++++++++++--------
 2 files changed, 181 insertions(+), 121 deletions(-)

diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestDirectoryDeletingServiceWithFSO.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestDirectoryDeletingServiceWithFSO.java
index 1c060fe5e2..b790284cfc 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestDirectoryDeletingServiceWithFSO.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestDirectoryDeletingServiceWithFSO.java
@@ -41,7 +41,6 @@ import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
 import org.apache.ozone.test.GenericTestUtils;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.AfterAll;
-import org.junit.Assert;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.Timeout;
@@ -54,10 +53,12 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.function.LongSupplier;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.fail;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
-import static org.junit.Assert.fail;
 
 /**
  * Directory deletion service test cases.
@@ -78,7 +79,7 @@ public class TestDirectoryDeletingServiceWithFSO {
   @BeforeAll
   public static void init() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
-    conf.setInt(OMConfigKeys.OZONE_DIR_DELETING_SERVICE_INTERVAL, 1);
+    conf.setInt(OMConfigKeys.OZONE_DIR_DELETING_SERVICE_INTERVAL, 2000);
     conf.setInt(OMConfigKeys.OZONE_PATH_DELETING_LIMIT_PER_TASK, 5);
     conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 100,
         TimeUnit.MILLISECONDS);
@@ -147,6 +148,7 @@ public class TestDirectoryDeletingServiceWithFSO {
     assertTableRowCount(dirTable, 2);
 
     assertSubPathsCount(dirDeletingService::getDeletedDirsCount, 0);
+    assertSubPathsCount(dirDeletingService::getMovedDirsCount, 0);
     assertSubPathsCount(dirDeletingService::getMovedFilesCount, 0);
 
     // Delete the appRoot, empty dir
@@ -159,13 +161,14 @@ public class TestDirectoryDeletingServiceWithFSO {
     assertTableRowCount(dirTable, 1);
 
     assertSubPathsCount(dirDeletingService::getDeletedDirsCount, 1);
+    assertSubPathsCount(dirDeletingService::getMovedDirsCount, 0);
     assertSubPathsCount(dirDeletingService::getMovedFilesCount, 0);
 
-    Assert.assertTrue(dirTable.iterator().hasNext());
-    Assert.assertEquals(root.getName(),
+    assertTrue(dirTable.iterator().hasNext());
+    assertEquals(root.getName(),
         dirTable.iterator().next().getValue().getName());
 
-    Assert.assertTrue(dirDeletingService.getRunCount() > 1);
+    assertTrue(dirDeletingService.getRunCount() > 1);
   }
 
   /**
@@ -211,8 +214,11 @@ public class TestDirectoryDeletingServiceWithFSO {
     assertTableRowCount(dirTable, 20);
 
     assertSubPathsCount(dirDeletingService::getMovedFilesCount, 0);
+    assertSubPathsCount(dirDeletingService::getMovedDirsCount, 0);
     assertSubPathsCount(dirDeletingService::getDeletedDirsCount, 0);
 
+    long preRunCount = dirDeletingService.getRunCount();
+
     // Delete the appRoot
     fs.delete(appRoot, true);
 
@@ -224,9 +230,14 @@ public class TestDirectoryDeletingServiceWithFSO {
     assertTableRowCount(dirTable, 1);
 
     assertSubPathsCount(dirDeletingService::getMovedFilesCount, 15);
+    // 15 subDir + 3 parentDir
+    assertSubPathsCount(dirDeletingService::getMovedDirsCount, 18);
     assertSubPathsCount(dirDeletingService::getDeletedDirsCount, 19);
 
-    Assert.assertTrue(dirDeletingService.getRunCount() > 1);
+    long elapsedRunCount = dirDeletingService.getRunCount() - preRunCount;
+    assertTrue(dirDeletingService.getRunCount() > 1);
+    // Ensure dir deleting speed, here provide a backup value for safe CI
+    assertTrue(elapsedRunCount == 8 || elapsedRunCount == 9);
   }
 
   @Test
@@ -258,6 +269,7 @@ public class TestDirectoryDeletingServiceWithFSO {
     assertTableRowCount(keyTable, 3);
 
     assertSubPathsCount(dirDeletingService::getMovedFilesCount, 0);
+    assertSubPathsCount(dirDeletingService::getMovedDirsCount, 0);
     assertSubPathsCount(dirDeletingService::getDeletedDirsCount, 0);
 
     // Delete the rootDir, which should delete all keys.
@@ -271,45 +283,10 @@ public class TestDirectoryDeletingServiceWithFSO {
     assertTableRowCount(dirTable, 0);
 
     assertSubPathsCount(dirDeletingService::getMovedFilesCount, 3);
+    assertSubPathsCount(dirDeletingService::getMovedDirsCount, 4);
     assertSubPathsCount(dirDeletingService::getDeletedDirsCount, 5);
 
-    Assert.assertTrue(dirDeletingService.getRunCount() > 1);
-  }
-
-  static void assertSubPathsCount(LongSupplier pathCount, long expectedCount)
-      throws TimeoutException, InterruptedException {
-    GenericTestUtils.waitFor(() -> pathCount.getAsLong() >= expectedCount,
-        1000, 120000);
-  }
-
-  private void assertTableRowCount(Table<String, ?> table, int count)
-      throws TimeoutException, InterruptedException {
-    GenericTestUtils.waitFor(() -> assertTableRowCount(count, table), 1000,
-        120000); // 2 minutes
-  }
-
-  private boolean assertTableRowCount(int expectedCount,
-                                      Table<String, ?> table) {
-    long count = 0L;
-    try {
-      count = cluster.getOzoneManager().getMetadataManager()
-          .countRowsInTable(table);
-      LOG.info("{} actual row count={}, expectedCount={}", table.getName(),
-          count, expectedCount);
-    } catch (IOException ex) {
-      fail("testDoubleBuffer failed with: " + ex);
-    }
-    return count == expectedCount;
-  }
-
-  private void checkPath(Path path) {
-    try {
-      fs.getFileStatus(path);
-      fail("testRecursiveDelete failed");
-    } catch (IOException ex) {
-      Assert.assertTrue(ex instanceof FileNotFoundException);
-      Assert.assertTrue(ex.getMessage().contains("No such file or directory"));
-    }
+    assertTrue(dirDeletingService.getRunCount() > 1);
   }
 
   @Test
@@ -367,10 +344,11 @@ public class TestDirectoryDeletingServiceWithFSO {
     assertTableRowCount(deletedKeyTable, 0);
 
     assertSubPathsCount(dirDeletingService::getMovedFilesCount, 0);
+    assertSubPathsCount(dirDeletingService::getMovedDirsCount, 0);
     assertSubPathsCount(dirDeletingService::getDeletedDirsCount, 0);
     // verify whether KeyDeletingService has purged the keys
     long currentDeletedKeyCount = keyDeletingService.getDeletedKeyCount().get();
-    Assert.assertEquals(prevDeletedKeyCount + 3, currentDeletedKeyCount);
+    assertEquals(prevDeletedKeyCount + 3, currentDeletedKeyCount);
 
 
     // Case-2) Delete dir, this will cleanup sub-files under the deleted dir.
@@ -385,10 +363,47 @@ public class TestDirectoryDeletingServiceWithFSO {
     assertTableRowCount(deletedKeyTable, 0);
 
     assertSubPathsCount(dirDeletingService::getMovedFilesCount, 2);
+    assertSubPathsCount(dirDeletingService::getMovedDirsCount, 0);
     assertSubPathsCount(dirDeletingService::getDeletedDirsCount, 1);
     // verify whether KeyDeletingService has purged the keys
     currentDeletedKeyCount = keyDeletingService.getDeletedKeyCount().get();
-    Assert.assertEquals(prevDeletedKeyCount + 5, currentDeletedKeyCount);
+    assertEquals(prevDeletedKeyCount + 5, currentDeletedKeyCount);
+  }
+
+  static void assertSubPathsCount(LongSupplier pathCount, long expectedCount)
+      throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(() -> pathCount.getAsLong() >= expectedCount,
+        1000, 120000);
+  }
+
+  private void assertTableRowCount(Table<String, ?> table, int count)
+      throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(() -> assertTableRowCount(count, table), 1000,
+        120000); // 2 minutes
+  }
+
+  private boolean assertTableRowCount(int expectedCount,
+                                      Table<String, ?> table) {
+    long count = 0L;
+    try {
+      count = cluster.getOzoneManager().getMetadataManager()
+          .countRowsInTable(table);
+      LOG.info("{} actual row count={}, expectedCount={}", table.getName(),
+          count, expectedCount);
+    } catch (IOException ex) {
+      fail("testDoubleBuffer failed with: " + ex);
+    }
+    return count == expectedCount;
+  }
+
+  private void checkPath(Path path) {
+    try {
+      fs.getFileStatus(path);
+      fail("testRecursiveDelete failed");
+    } catch (IOException ex) {
+      assertTrue(ex instanceof FileNotFoundException);
+      assertTrue(ex.getMessage().contains("No such file or directory"));
+    }
   }
 
   private static BucketLayout getFSOBucketLayout() {
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/DirectoryDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/DirectoryDeletingService.java
index b22341779e..8115b17608 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/DirectoryDeletingService.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/DirectoryDeletingService.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -24,20 +24,24 @@ 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.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.Table.KeyValue;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
 import org.apache.hadoop.ozone.ClientVersion;
 import org.apache.hadoop.ozone.om.OzoneManager;
 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.ozone.protocol.proto.OzoneManagerProtocolProtos.PurgePathRequest;
 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -63,10 +67,13 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_PATH_DELETING_LIMIT_
  * components of an orphan directory is visited.
  */
 public class DirectoryDeletingService extends BackgroundService {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DirectoryDeletingService.class);
 
   private final OzoneManager ozoneManager;
-  private AtomicLong deletedDirsCount;
-  private AtomicLong deletedFilesCount;
+  private final AtomicLong deletedDirsCount;
+  private final AtomicLong movedDirsCount;
+  private final AtomicLong movedFilesCount;
   private final AtomicLong runCount;
 
   private static ClientId clientId = ClientId.randomId();
@@ -86,7 +93,8 @@ public class DirectoryDeletingService extends BackgroundService {
         DIR_DELETING_CORE_POOL_SIZE, serviceTimeout);
     this.ozoneManager = ozoneManager;
     this.deletedDirsCount = new AtomicLong(0);
-    this.deletedFilesCount = new AtomicLong(0);
+    this.movedDirsCount = new AtomicLong(0);
+    this.movedFilesCount = new AtomicLong(0);
     this.runCount = new AtomicLong(0);
     this.pathLimitPerTask = configuration
         .getInt(OZONE_PATH_DELETING_LIMIT_PER_TASK,
@@ -125,73 +133,96 @@ public class DirectoryDeletingService extends BackgroundService {
     @Override
     public BackgroundTaskResult call() throws Exception {
       if (shouldRun()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Running DirectoryDeletingService");
+        }
         runCount.incrementAndGet();
-        long count = pathLimitPerTask;
+        int dirNum = 0;
+        int subDirNum = 0;
+        int subFileNum = 0;
+        long remainNum = pathLimitPerTask;
+        List<PurgePathRequest> purgePathRequestList = new ArrayList<>();
+
+        Table.KeyValue<String, OmKeyInfo> pendingDeletedDirInfo;
         try {
+          TableIterator<String, ? extends KeyValue<String, OmKeyInfo>>
+              deleteTableIterator = ozoneManager.getMetadataManager().
+              getDeletedDirTable().iterator();
+
           long startTime = Time.monotonicNow();
-          // step-1) Get one pending deleted directory
-          Table.KeyValue<String, OmKeyInfo> pendingDeletedDirInfo =
-              ozoneManager.getKeyManager().getPendingDeletionDir();
-          if (pendingDeletedDirInfo != null) {
+          while (remainNum > 0 && deleteTableIterator.hasNext()) {
+            pendingDeletedDirInfo = deleteTableIterator.next();
+            // step-0: Get one pending deleted directory
             if (LOG.isDebugEnabled()) {
               LOG.debug("Pending deleted dir name: {}",
                   pendingDeletedDirInfo.getValue().getKeyName());
             }
             final String[] keys = pendingDeletedDirInfo.getKey()
-                    .split(OM_KEY_PREFIX);
+                .split(OM_KEY_PREFIX);
             final long volumeId = Long.parseLong(keys[1]);
             final long bucketId = Long.parseLong(keys[2]);
 
             // step-1: get all sub directories under the deletedDir
-            List<OmKeyInfo> dirs = ozoneManager.getKeyManager()
+            List<OmKeyInfo> subDirs = ozoneManager.getKeyManager()
                 .getPendingDeletionSubDirs(volumeId, bucketId,
-                        pendingDeletedDirInfo.getValue(), 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());
+                    pendingDeletedDirInfo.getValue(), remainNum);
+            remainNum = remainNum - subDirs.size();
+
+            if (LOG.isDebugEnabled()) {
+              for (OmKeyInfo dirInfo : subDirs) {
+                LOG.debug("Moved sub dir name: {}", dirInfo.getKeyName());
               }
             }
 
             // step-2: get all sub files under the deletedDir
-            List<OmKeyInfo> purgeDeletedFiles = ozoneManager.getKeyManager()
+            List<OmKeyInfo> subFiles = ozoneManager.getKeyManager()
                 .getPendingDeletionSubFiles(volumeId, bucketId,
-                        pendingDeletedDirInfo.getValue(), count);
-            count = count - purgeDeletedFiles.size();
+                    pendingDeletedDirInfo.getValue(), remainNum);
+            remainNum = remainNum - subFiles.size();
 
             if (LOG.isDebugEnabled()) {
-              for (OmKeyInfo fileInfo : purgeDeletedFiles) {
-                LOG.debug("deleted sub file name: {}", fileInfo.getKeyName());
+              for (OmKeyInfo fileInfo : subFiles) {
+                LOG.debug("Moved 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.
-            final Optional<String> purgeDeletedDir = count > 0 ?
-                    Optional.of(pendingDeletedDirInfo.getKey()) :
-                    Optional.empty();
-
-            if (isRatisEnabled()) {
-              submitPurgePaths(volumeId, bucketId, purgeDeletedDir,
-                      purgeDeletedFiles, deletedSubDirList);
+          // 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.
+            String purgeDeletedDir = remainNum > 0 ?
+                pendingDeletedDirInfo.getKey() : null;
+
+            PurgePathRequest request = wrapPurgeRequest(volumeId, bucketId,
+                purgeDeletedDir, subFiles, subDirs);
+            purgePathRequestList.add(request);
+
+            // Count up the purgeDeletedDir, subDirs and subFiles
+            if (purgeDeletedDir != null) {
+              dirNum++;
             }
-            // TODO: need to handle delete with non-ratis
+            subDirNum += subDirs.size();
+            subFileNum += subFiles.size();
+          }
 
-            deletedDirsCount.incrementAndGet();
-            deletedFilesCount.addAndGet(purgeDeletedFiles.size());
-            if (LOG.isDebugEnabled()) {
-              LOG.debug("Number of dirs deleted: {}, Number of files moved:" +
-                      " {} to DeletedTable, elapsed time: {}ms",
-                  deletedDirsCount, deletedFilesCount,
-                  Time.monotonicNow() - startTime);
-            }
+          // TODO: need to handle delete with non-ratis
+          if (isRatisEnabled()) {
+            submitPurgePaths(purgePathRequestList);
+          }
+
+          if (dirNum != 0 || subDirNum != 0 || subFileNum != 0) {
+            deletedDirsCount.addAndGet(dirNum);
+            movedDirsCount.addAndGet(subDirNum);
+            movedFilesCount.addAndGet(subFileNum);
+            LOG.info("Number of dirs deleted: {}, Number of sub-files moved:" +
+                    " {} to DeletedTable, Number of sub-dirs moved {} to " +
+                    "DeletedDirectoryTable, iteration elapsed: {}ms," +
+                    " totalRunCount: {}",
+                dirNum, subFileNum, subDirNum,
+                Time.monotonicNow() - startTime, getRunCount());
           }
+
         } catch (IOException e) {
           LOG.error("Error while running delete directories and files " +
               "background task. Will retry at next run.", e);
@@ -213,6 +244,16 @@ public class DirectoryDeletingService extends BackgroundService {
     return deletedDirsCount.get();
   }
 
+  /**
+   * Returns the number of sub-dirs deleted by the background service.
+   *
+   * @return Long count.
+   */
+  @VisibleForTesting
+  public long getMovedDirsCount() {
+    return movedDirsCount.get();
+  }
+
   /**
    * Returns the number of files moved to DeletedTable by the background
    * service.
@@ -221,7 +262,7 @@ public class DirectoryDeletingService extends BackgroundService {
    */
   @VisibleForTesting
   public long getMovedFilesCount() {
-    return deletedFilesCount.get();
+    return movedFilesCount.get();
   }
 
   /**
@@ -234,33 +275,10 @@ public class DirectoryDeletingService extends BackgroundService {
     return runCount.get();
   }
 
-  private int submitPurgePaths(final long volumeId, final long bucketId,
-      final Optional<String> purgeDeletedDir,
-      final List<OmKeyInfo> purgeDeletedFiles,
-      final List<OmKeyInfo> markDirsAsDeleted) {
-    // Put all keys to be purged in a list
-    int deletedCount = 0;
-    OzoneManagerProtocolProtos.PurgePathRequest.Builder purgePathsRequest =
-        OzoneManagerProtocolProtos.PurgePathRequest.newBuilder();
-    purgePathsRequest.setVolumeId(volumeId);
-    purgePathsRequest.setBucketId(bucketId);
-    purgeDeletedDir.ifPresent(purgePathsRequest::setDeletedDir);
-    for (OmKeyInfo purgeFile : purgeDeletedFiles) {
-      purgePathsRequest.addDeletedSubFiles(
-          purgeFile.getProtobuf(true, ClientVersion.CURRENT_VERSION));
-    }
-
-    // Add these directories to deletedDirTable, so that its sub-paths will be
-    // traversed in next iteration to ensure cleanup all sub-children.
-    for (OmKeyInfo dir : markDirsAsDeleted) {
-      purgePathsRequest.addMarkDeletedSubDirs(
-          dir.getProtobuf(ClientVersion.CURRENT_VERSION));
-    }
-
+  private void submitPurgePaths(List<PurgePathRequest> requests) {
     OzoneManagerProtocolProtos.PurgeDirectoriesRequest.Builder purgeDirRequest =
-            OzoneManagerProtocolProtos.PurgeDirectoriesRequest.newBuilder();
-    purgeDirRequest.addDeletedPath(purgePathsRequest.build());
-
+        OzoneManagerProtocolProtos.PurgeDirectoriesRequest.newBuilder();
+    purgeDirRequest.addAllDeletedPath(requests);
 
     OzoneManagerProtocolProtos.OMRequest omRequest =
         OzoneManagerProtocolProtos.OMRequest.newBuilder()
@@ -277,9 +295,36 @@ public class DirectoryDeletingService extends BackgroundService {
           raftClientRequest);
     } catch (ServiceException e) {
       LOG.error("PurgePaths request failed. Will retry at next run.");
-      return 0;
     }
-    return deletedCount;
+  }
+
+  private PurgePathRequest wrapPurgeRequest(final long volumeId,
+      final long bucketId,
+      final String purgeDeletedDir,
+      final List<OmKeyInfo> purgeDeletedFiles,
+      final List<OmKeyInfo> markDirsAsDeleted) {
+    // Put all keys to be purged in a list
+    PurgePathRequest.Builder purgePathsRequest = PurgePathRequest.newBuilder();
+    purgePathsRequest.setVolumeId(volumeId);
+    purgePathsRequest.setBucketId(bucketId);
+
+    if (purgeDeletedDir != null) {
+      purgePathsRequest.setDeletedDir(purgeDeletedDir);
+    }
+
+    for (OmKeyInfo purgeFile : purgeDeletedFiles) {
+      purgePathsRequest.addDeletedSubFiles(
+          purgeFile.getProtobuf(true, ClientVersion.CURRENT_VERSION));
+    }
+
+    // Add these directories to deletedDirTable, so that its sub-paths will be
+    // traversed in next iteration to ensure cleanup all sub-children.
+    for (OmKeyInfo dir : markDirsAsDeleted) {
+      purgePathsRequest.addMarkDeletedSubDirs(
+          dir.getProtobuf(ClientVersion.CURRENT_VERSION));
+    }
+
+    return purgePathsRequest.build();
   }
 
 


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