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

ashishkr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 3e5414d0101 Revert "HDDS-13213. KeyDeletingService should limit task 
size by both key count and serialized size. (#8757)" (#8957)
3e5414d0101 is described below

commit 3e5414d0101cec1621aebb5cf4d952071c6e11b2
Author: Aryan Gupta <44232823+aryangupta1...@users.noreply.github.com>
AuthorDate: Wed Aug 20 09:17:25 2025 +0530

    Revert "HDDS-13213. KeyDeletingService should limit task size by both key 
count and serialized size. (#8757)" (#8957)
---
 .../hadoop/hdds/utils/MapBackedTableIterator.java  | 12 ++----
 .../org/apache/hadoop/ozone/om/TestKeyPurging.java |  9 +----
 .../org/apache/hadoop/ozone/om/KeyManager.java     |  8 ++--
 .../org/apache/hadoop/ozone/om/KeyManagerImpl.java | 44 +++++-----------------
 .../ozone/om/service/DirectoryDeletingService.java |  5 +--
 .../ozone/om/service/KeyDeletingService.java       | 35 +++++------------
 .../ozone/om/service/SnapshotDeletingService.java  |  7 ++--
 .../apache/hadoop/ozone/om/TestKeyManagerImpl.java | 23 ++---------
 .../ozone/om/service/TestKeyDeletingService.java   | 20 +++-------
 9 files changed, 42 insertions(+), 121 deletions(-)

diff --git 
a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/MapBackedTableIterator.java
 
b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/MapBackedTableIterator.java
index d3fbb3cb9f2..5af0e671d51 100644
--- 
a/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/MapBackedTableIterator.java
+++ 
b/hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/utils/MapBackedTableIterator.java
@@ -17,7 +17,6 @@
 
 package org.apache.hadoop.hdds.utils;
 
-import java.nio.charset.StandardCharsets;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.TreeMap;
@@ -42,12 +41,7 @@ public MapBackedTableIterator(TreeMap<String, V> values, 
String prefix) {
   public void seekToFirst() {
     this.itr = this.values.entrySet().stream()
         .filter(e -> prefix == null || e.getKey().startsWith(prefix))
-        .map(e -> {
-          V value = e.getValue();
-          int size = value != null ? 
value.toString().getBytes(StandardCharsets.UTF_8).length : 0;
-          return Table.newKeyValue(e.getKey(), value, size);
-        })
-        .iterator();
+        .map(e -> Table.newKeyValue(e.getKey(), e.getValue())).iterator();
   }
 
   @Override
@@ -59,8 +53,8 @@ public void seekToLast() {
   public Table.KeyValue<String, V> seek(String s) {
     this.itr = this.values.entrySet().stream()
         .filter(e -> prefix == null || e.getKey().startsWith(prefix))
-        .filter(e -> e.getKey().compareTo(s) >= 0).map(e -> 
Table.newKeyValue(e.getKey(), e.getValue(),
-            
e.getValue().toString().getBytes(StandardCharsets.UTF_8).length)).iterator();
+        .filter(e -> e.getKey().compareTo(s) >= 0)
+        .map(e -> Table.newKeyValue(e.getKey(), e.getValue())).iterator();
     Map.Entry<String, V> firstEntry = values.ceilingEntry(s);
     return firstEntry == null ? null : Table.newKeyValue(firstEntry.getKey(), 
firstEntry.getValue());
   }
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java
index 399453d6433..fa59754b67f 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyPurging.java
@@ -29,7 +29,6 @@
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.utils.IOUtils;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.TestDataUtil;
@@ -57,7 +56,6 @@ public class TestKeyPurging {
   private static final int NUM_KEYS = 10;
   private static final int KEY_SIZE = 100;
   private OzoneClient client;
-  private int ratisLimit;
 
   @BeforeEach
   public void setup() throws Exception {
@@ -76,11 +74,6 @@ public void setup() throws Exception {
     client = OzoneClientFactory.getRpcClient(conf);
     store = client.getObjectStore();
     om = cluster.getOzoneManager();
-    int limit = (int) conf.getStorageSize(
-        OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT,
-        OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT,
-        StorageUnit.BYTES);
-    ratisLimit = (int) (limit * 0.9);
   }
 
   @AfterEach
@@ -133,7 +126,7 @@ public void testKeysPurgingByKeyDeletingService() throws 
Exception {
     GenericTestUtils.waitFor(
         () -> {
           try {
-            return keyManager.getPendingDeletionKeys((kv) -> true, 
Integer.MAX_VALUE, ratisLimit)
+            return keyManager.getPendingDeletionKeys((kv) -> true, 
Integer.MAX_VALUE)
                 .getKeyBlocksList().isEmpty();
           } catch (IOException e) {
             return false;
diff --git 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java
 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java
index d69839a1c8d..7e76885c49b 100644
--- 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java
+++ 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java
@@ -124,7 +124,7 @@ ListKeysResult listKeys(String volumeName, String 
bucketName, String startKey,
    * @throws IOException if an I/O error occurs while fetching the keys.
    */
   PendingKeysDeletion getPendingDeletionKeys(
-      CheckedFunction<Table.KeyValue<String, OmKeyInfo>, Boolean, IOException> 
filter, int count, int ratisByteLimit)
+      CheckedFunction<Table.KeyValue<String, OmKeyInfo>, Boolean, IOException> 
filter, int count)
       throws IOException;
 
   /**
@@ -142,7 +142,7 @@ PendingKeysDeletion getPendingDeletionKeys(
    */
   PendingKeysDeletion getPendingDeletionKeys(
       String volume, String bucket, String startKey,
-      CheckedFunction<Table.KeyValue<String, OmKeyInfo>, Boolean, IOException> 
filter, int count, int ratisByteLimit)
+      CheckedFunction<Table.KeyValue<String, OmKeyInfo>, Boolean, IOException> 
filter, int count)
       throws IOException;
 
   /**
@@ -156,7 +156,7 @@ PendingKeysDeletion getPendingDeletionKeys(
    */
   List<Table.KeyValue<String, String>> getRenamesKeyEntries(
       String volume, String bucket, String startKey,
-      CheckedFunction<Table.KeyValue<String, String>, Boolean, IOException> 
filter, int count, int ratisLimit)
+      CheckedFunction<Table.KeyValue<String, String>, Boolean, IOException> 
filter, int count)
       throws IOException;
 
 
@@ -190,7 +190,7 @@ CheckedFunction<KeyManager, OmKeyInfo, IOException> 
getPreviousSnapshotOzoneKeyI
   List<Table.KeyValue<String, List<OmKeyInfo>>> getDeletedKeyEntries(
       String volume, String bucket, String startKey,
       CheckedFunction<Table.KeyValue<String, RepeatedOmKeyInfo>, Boolean, 
IOException> filter,
-      int count, int ratisLimit) throws IOException;
+      int count) throws IOException;
 
   /**
    * Returns the names of up to {@code count} open keys whose age is
diff --git 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
index 10c29b960ea..809974d9d7b 100644
--- 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
+++ 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
@@ -731,18 +731,17 @@ public ListKeysResult listKeys(String volumeName, String 
bucketName,
 
   @Override
   public PendingKeysDeletion getPendingDeletionKeys(
-      final CheckedFunction<KeyValue<String, OmKeyInfo>, Boolean, IOException> 
filter, final int count,
-      int ratisByteLimit) throws IOException {
-    return getPendingDeletionKeys(null, null, null, filter, count, 
ratisByteLimit);
+      final CheckedFunction<KeyValue<String, OmKeyInfo>, Boolean, IOException> 
filter, final int count)
+      throws IOException {
+    return getPendingDeletionKeys(null, null, null, filter, count);
   }
 
   @Override
   public PendingKeysDeletion getPendingDeletionKeys(
       String volume, String bucket, String startKey,
       CheckedFunction<KeyValue<String, OmKeyInfo>, Boolean, IOException> 
filter,
-      int count, int ratisByteLimit) throws IOException {
+      int count) throws IOException {
     List<BlockGroup> keyBlocksList = Lists.newArrayList();
-    long serializedSize = 0;
     Map<String, RepeatedOmKeyInfo> keysToModify = new HashMap<>();
     Map<String, Long> keyBlockReplicatedSize = new HashMap<>();
     int notReclaimableKeyCount = 0;
@@ -759,7 +758,6 @@ public PendingKeysDeletion getPendingDeletionKeys(
         delKeyIter.seek(startKey);
       }
       int currentCount = 0;
-      boolean maxReqSizeExceeded = false;
       while (delKeyIter.hasNext() && currentCount < count) {
         RepeatedOmKeyInfo notReclaimableKeyInfo = new RepeatedOmKeyInfo();
         KeyValue<String, RepeatedOmKeyInfo> kv = delKeyIter.next();
@@ -776,18 +774,6 @@ public PendingKeysDeletion getPendingDeletionKeys(
                       .map(b -> new BlockID(b.getContainerID(), 
b.getLocalID()))).collect(Collectors.toList());
               BlockGroup keyBlocks = 
BlockGroup.newBuilder().setKeyName(kv.getKey())
                   .addAllBlockIDs(blockIDS).build();
-              int keyBlockSerializedSize = 
keyBlocks.getProto().getSerializedSize();
-              serializedSize += keyBlockSerializedSize;
-              if (serializedSize > ratisByteLimit) {
-                maxReqSizeExceeded = true;
-                if (LOG.isDebugEnabled()) {
-                  LOG.debug(
-                      "Total size of cumulative keys and rename entries in the 
snapshotRenamedTable in a cycle " +
-                          "crossed 90% ratis limit, serialized size of keys: 
{}",
-                      serializedSize);
-                }
-                break;
-              }
               keyBlockReplicatedSize.put(keyBlocks.getGroupID(), 
info.getReplicatedSize());
               blockGroupList.add(keyBlocks);
               currentCount++;
@@ -795,9 +781,6 @@ public PendingKeysDeletion getPendingDeletionKeys(
               notReclaimableKeyInfo.addOmKeyInfo(info);
             }
           }
-          if (maxReqSizeExceeded) {
-            break;
-          }
 
           List<OmKeyInfo> notReclaimableKeyInfoList = 
notReclaimableKeyInfo.getOmKeyInfoList();
 
@@ -818,9 +801,8 @@ private <V, R> List<KeyValue<String, R>> 
getTableEntries(String startKey,
           TableIterator<String, ? extends KeyValue<String, V>> tableIterator,
           Function<V, R> valueFunction,
           CheckedFunction<KeyValue<String, V>, Boolean, IOException> filter,
-          int size, int ratisLimit) throws IOException {
+          int size) throws IOException {
     List<KeyValue<String, R>> entries = new ArrayList<>();
-    int consumedSize = 0;
     /* Seek to the start key if it's not null. The next key in queue is 
ensured to start with the bucket
          prefix, {@link 
org.apache.hadoop.hdds.utils.db.Table#iterator(bucketPrefix)} would ensure this.
     */
@@ -833,13 +815,8 @@ private <V, R> List<KeyValue<String, R>> 
getTableEntries(String startKey,
     while (tableIterator.hasNext() && currentCount < size) {
       KeyValue<String, V> kv = tableIterator.next();
       if (kv != null && filter.apply(kv)) {
-        consumedSize += kv.getValueByteSize();
-        entries.add(Table.newKeyValue(kv.getKey(), 
valueFunction.apply(kv.getValue()), kv.getValueByteSize()));
+        entries.add(Table.newKeyValue(kv.getKey(), 
valueFunction.apply(kv.getValue())));
         currentCount++;
-        if (consumedSize > ratisLimit) {
-          LOG.info("Serialized size exceeded the ratis limit, current 
serailized size : {}", consumedSize);
-          break;
-        }
       }
     }
     return entries;
@@ -860,12 +837,11 @@ private Optional<String> getBucketPrefix(String 
volumeName, String bucketName, b
   @Override
   public List<KeyValue<String, String>> getRenamesKeyEntries(
       String volume, String bucket, String startKey,
-      CheckedFunction<KeyValue<String, String>, Boolean, IOException> filter, 
int size, int ratisLimit)
-      throws IOException {
+      CheckedFunction<KeyValue<String, String>, Boolean, IOException> filter, 
int size) throws IOException {
     Optional<String> bucketPrefix = getBucketPrefix(volume, bucket, false);
     try (TableIterator<String, ? extends KeyValue<String, String>>
              renamedKeyIter = 
metadataManager.getSnapshotRenamedTable().iterator(bucketPrefix.orElse(""))) {
-      return getTableEntries(startKey, renamedKeyIter, Function.identity(), 
filter, size, ratisLimit);
+      return getTableEntries(startKey, renamedKeyIter, Function.identity(), 
filter, size);
     }
   }
 
@@ -911,11 +887,11 @@ private <T> CheckedFunction<KeyManager, T, IOException> 
getPreviousSnapshotOzone
   public List<KeyValue<String, List<OmKeyInfo>>> getDeletedKeyEntries(
       String volume, String bucket, String startKey,
       CheckedFunction<KeyValue<String, RepeatedOmKeyInfo>, Boolean, 
IOException> filter,
-      int size, int ratisLimit) throws IOException {
+      int size) throws IOException {
     Optional<String> bucketPrefix = getBucketPrefix(volume, bucket, false);
     try (TableIterator<String, ? extends KeyValue<String, RepeatedOmKeyInfo>>
              delKeyIter = 
metadataManager.getDeletedTable().iterator(bucketPrefix.orElse(""))) {
-      return getTableEntries(startKey, delKeyIter, 
RepeatedOmKeyInfo::cloneOmKeyInfoList, filter, size, ratisLimit);
+      return getTableEntries(startKey, delKeyIter, 
RepeatedOmKeyInfo::cloneOmKeyInfoList, filter, size);
     }
   }
 
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 24c5e1f096f..90ad878c640 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
@@ -296,9 +296,8 @@ void optimizeDirDeletesAndSubmitRequest(
         break;
       }
     }
-    if (purgePathRequestList.isEmpty() ||
-        submitPurgePaths(purgePathRequestList, snapTableKey, 
expectedPreviousSnapshotId) == null) {
-      return;
+    if (!purgePathRequestList.isEmpty()) {
+      submitPurgePaths(purgePathRequestList, snapTableKey, 
expectedPreviousSnapshotId);
     }
 
     if (dirNum != 0 || subDirNum != 0 || subFileNum != 0) {
diff --git 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java
 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java
index a106d1f6b22..bf429ad01dd 100644
--- 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java
+++ 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/KeyDeletingService.java
@@ -40,7 +40,6 @@
 import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hadoop.hdds.HddsUtils;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
-import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
 import org.apache.hadoop.hdds.utils.BackgroundTask;
 import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
@@ -89,8 +88,6 @@ public class KeyDeletingService extends 
AbstractKeyDeletingService {
   private final AtomicLong deletedKeyCount;
   private final boolean deepCleanSnapshots;
   private final SnapshotChainManager snapshotChainManager;
-  private int ratisByteLimit;
-  private static final double RATIS_LIMIT_FACTOR = 0.9;
   // Track metrics for current task execution
   private long latestRunTimestamp = 0L;
   private final DeletionStats aosDeletionStats = new DeletionStats();
@@ -111,12 +108,6 @@ public KeyDeletingService(OzoneManager ozoneManager,
     this.deepCleanSnapshots = deepCleanSnapshots;
     this.snapshotChainManager = 
((OmMetadataManagerImpl)ozoneManager.getMetadataManager()).getSnapshotChainManager();
     this.scmClient = scmClient;
-    int limit = (int) ozoneManager.getConfiguration().getStorageSize(
-        OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT,
-        OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT,
-        StorageUnit.BYTES);
-    // always go to 90% of max limit for request as other header will be added
-    this.ratisByteLimit = (int) (limit * RATIS_LIMIT_FACTOR);
   }
 
   /**
@@ -362,7 +353,7 @@ private 
OzoneManagerProtocolProtos.SetSnapshotPropertyRequest getSetSnapshotRequ
      * @param keyManager KeyManager of the underlying store.
      */
     private void processDeletedKeysForStore(SnapshotInfo currentSnapshotInfo, 
KeyManager keyManager,
-        int remainNum, int ratisLimit) throws IOException, 
InterruptedException {
+        int remainNum) throws IOException, InterruptedException {
       String volume = null, bucket = null, snapshotTableKey = null;
       if (currentSnapshotInfo != null) {
         volume = currentSnapshotInfo.getVolumeName();
@@ -394,24 +385,16 @@ private void processDeletedKeysForStore(SnapshotInfo 
currentSnapshotInfo, KeyMan
              ReclaimableRenameEntryFilter renameEntryFilter = new 
ReclaimableRenameEntryFilter(
                  getOzoneManager(), omSnapshotManager, snapshotChainManager, 
currentSnapshotInfo,
                  keyManager, lock)) {
-          List<Table.KeyValue<String, String>> renameKeyEntries =
-              keyManager.getRenamesKeyEntries(volume, bucket, null, 
renameEntryFilter, remainNum, ratisLimit);
-
-          List<String> renamedTableEntries = new 
ArrayList<>(renameKeyEntries.size());
-          int serializedSize = 0;
-
-          for (Table.KeyValue<String, String> kv : renameKeyEntries) {
-            renamedTableEntries.add(kv.getKey());
-            serializedSize += kv.getValueByteSize();
-          }
-
+          List<String> renamedTableEntries =
+              keyManager.getRenamesKeyEntries(volume, bucket, null, 
renameEntryFilter, remainNum).stream()
+                  .map(Table.KeyValue::getKey)
+                  .collect(Collectors.toList());
           remainNum -= renamedTableEntries.size();
-          ratisLimit -= serializedSize;
 
           // Get pending keys that can be deleted
-          PendingKeysDeletion pendingKeysDeletion = currentSnapshotInfo == 
null ?
-              keyManager.getPendingDeletionKeys(reclaimableKeyFilter, 
remainNum, ratisLimit) :
-              keyManager.getPendingDeletionKeys(volume, bucket, null, 
reclaimableKeyFilter, remainNum, ratisLimit);
+          PendingKeysDeletion pendingKeysDeletion = currentSnapshotInfo == null
+              ? keyManager.getPendingDeletionKeys(reclaimableKeyFilter, 
remainNum)
+              : keyManager.getPendingDeletionKeys(volume, bucket, null, 
reclaimableKeyFilter, remainNum);
           List<BlockGroup> keyBlocksList = 
pendingKeysDeletion.getKeyBlocksList();
           //submit purge requests if there are renamed entries to be purged or 
keys to be purged.
           if (!renamedTableEntries.isEmpty() || keyBlocksList != null && 
!keyBlocksList.isEmpty()) {
@@ -510,7 +493,7 @@ public BackgroundTaskResult call() {
                   snapInfo.getName())) {
             KeyManager keyManager = snapInfo == null ? 
getOzoneManager().getKeyManager()
                 : omSnapshot.get().getKeyManager();
-            processDeletedKeysForStore(snapInfo, keyManager, remainNum, 
ratisByteLimit);
+            processDeletedKeysForStore(snapInfo, keyManager, remainNum);
           }
         } catch (IOException e) {
           LOG.error("Error while running delete files background task for 
store {}. Will retry at next run.",
diff --git 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java
 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java
index 1b3199ec987..75e9a20cdf1 100644
--- 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java
+++ 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java
@@ -177,16 +177,15 @@ public BackgroundTaskResult call() throws 
InterruptedException {
             // Get all entries from deletedKeyTable.
             List<Table.KeyValue<String, List<OmKeyInfo>>> deletedKeyEntries =
                 
snapshotKeyManager.getDeletedKeyEntries(snapInfo.getVolumeName(), 
snapInfo.getBucketName(),
-                    null, (kv) -> true, remaining, ratisByteLimit);
+                    null, (kv) -> true, remaining);
             moveCount += deletedKeyEntries.size();
             // Get all entries from deletedDirTable.
             List<Table.KeyValue<String, OmKeyInfo>> deletedDirEntries = 
snapshotKeyManager.getDeletedDirEntries(
                 snapInfo.getVolumeName(), snapInfo.getBucketName(), remaining 
- moveCount);
             moveCount += deletedDirEntries.size();
             // Get all entries from snapshotRenamedTable.
-            List<Table.KeyValue<String, String>> renameEntries =
-                
snapshotKeyManager.getRenamesKeyEntries(snapInfo.getVolumeName(), 
snapInfo.getBucketName(), null,
-                    (kv) -> true, remaining - moveCount, ratisByteLimit);
+            List<Table.KeyValue<String, String>> renameEntries = 
snapshotKeyManager.getRenamesKeyEntries(
+                snapInfo.getVolumeName(), snapInfo.getBucketName(), null, (kv) 
-> true, remaining - moveCount);
             moveCount += renameEntries.size();
             if (moveCount > 0) {
               List<SnapshotMoveKeyInfos> deletedKeys = new 
ArrayList<>(deletedKeyEntries.size());
diff --git 
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
 
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
index 447be08cafc..d021cc75250 100644
--- 
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
+++ 
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
@@ -30,7 +30,6 @@
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.utils.MapBackedTableIterator;
 import org.apache.hadoop.hdds.utils.db.Table;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
@@ -122,11 +121,6 @@ public void testGetDeletedKeyEntries(int numberOfVolumes, 
int numberOfBucketsPer
     String bucketNamePrefix = "bucket";
     String keyPrefix = "key";
     OzoneConfiguration configuration = new OzoneConfiguration();
-    int limit = (int) configuration.getStorageSize(
-        OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT,
-        OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT,
-        StorageUnit.BYTES);
-    limit = (int) (limit * 0.9);
     OMMetadataManager metadataManager = Mockito.mock(OMMetadataManager.class);
     when(metadataManager.getBucketKeyPrefix(anyString(), 
anyString())).thenAnswer(i ->
         "/" + i.getArguments()[0] + "/" + i.getArguments()[1] + "/");
@@ -152,12 +146,10 @@ public void testGetDeletedKeyEntries(int numberOfVolumes, 
int numberOfBucketsPer
         : (String.format("/%s%010d/%s%010d/%s%010d", volumeNamePrefix, 
startVolumeNumber, bucketNamePrefix,
         startBucketNumber, keyPrefix, startKeyNumber));
     if (expectedException != null) {
-      int finalLimit = limit;
       assertThrows(expectedException, () -> 
km.getDeletedKeyEntries(volumeName, bucketName, startKey, filter,
-          numberOfEntries, finalLimit));
+          numberOfEntries));
     } else {
-      assertEquals(expectedEntries,
-          km.getDeletedKeyEntries(volumeName, bucketName, startKey, filter, 
numberOfEntries, limit));
+      assertEquals(expectedEntries, km.getDeletedKeyEntries(volumeName, 
bucketName, startKey, filter, numberOfEntries));
     }
   }
 
@@ -173,11 +165,6 @@ public void testGetRenameKeyEntries(int numberOfVolumes, 
int numberOfBucketsPerV
     String bucketNamePrefix = "bucket";
     String keyPrefix = "";
     OzoneConfiguration configuration = new OzoneConfiguration();
-    int limit = (int) configuration.getStorageSize(
-        OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT,
-        OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT,
-        StorageUnit.BYTES);
-    limit = (int) (limit * 0.9);
     OMMetadataManager metadataManager = Mockito.mock(OMMetadataManager.class);
     when(metadataManager.getBucketKeyPrefix(anyString(), 
anyString())).thenAnswer(i ->
         "/" + i.getArguments()[0] + "/" + i.getArguments()[1] + "/");
@@ -196,12 +183,10 @@ public void testGetRenameKeyEntries(int numberOfVolumes, 
int numberOfBucketsPerV
         : (String.format("/%s%010d/%s%010d/%s%010d", volumeNamePrefix, 
startVolumeNumber, bucketNamePrefix,
         startBucketNumber, keyPrefix, startKeyNumber));
     if (expectedException != null) {
-      int finalLimit = limit;
       assertThrows(expectedException, () -> 
km.getRenamesKeyEntries(volumeName, bucketName, startKey,
-          filter, numberOfEntries, finalLimit));
+          filter, numberOfEntries));
     } else {
-      assertEquals(expectedEntries,
-          km.getRenamesKeyEntries(volumeName, bucketName, startKey, filter, 
numberOfEntries, limit));
+      assertEquals(expectedEntries, km.getRenamesKeyEntries(volumeName, 
bucketName, startKey, filter, numberOfEntries));
     }
   }
 
diff --git 
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java
 
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java
index fe27f72d93f..505b5d4845c 100644
--- 
a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java
+++ 
b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestKeyDeletingService.java
@@ -65,7 +65,6 @@
 import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
 import org.apache.hadoop.hdds.server.ServerUtils;
 import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
@@ -75,7 +74,6 @@
 import org.apache.hadoop.ozone.om.DeletingServiceMetrics;
 import org.apache.hadoop.ozone.om.KeyManager;
 import org.apache.hadoop.ozone.om.KeyManagerImpl;
-import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
 import org.apache.hadoop.ozone.om.OmSnapshot;
@@ -146,7 +144,6 @@ class TestKeyDeletingService extends OzoneTestBase {
   private KeyDeletingService keyDeletingService;
   private DirectoryDeletingService directoryDeletingService;
   private ScmBlockLocationTestingClient scmBlockTestingClient;
-  private int ratisLimit;
   private DeletingServiceMetrics metrics;
 
   @BeforeAll
@@ -185,11 +182,6 @@ private void createSubject() throws Exception {
     writeClient = omTestManagers.getWriteClient();
     om = omTestManagers.getOzoneManager();
     metadataManager = omTestManagers.getMetadataManager();
-    int limit = (int) conf.getStorageSize(
-        OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT,
-        OMConfigKeys.OZONE_OM_RATIS_LOG_APPENDER_QUEUE_BYTE_LIMIT_DEFAULT,
-        StorageUnit.BYTES);
-    ratisLimit = (int) (limit * 0.9);
     metrics = keyDeletingService.getMetrics();
   }
 
@@ -242,7 +234,7 @@ void checkIfDeleteServiceIsDeletingKeys()
       assertThat(getRunCount()).isGreaterThan(initialRunCount);
       assertThat(keyManager.getPendingDeletionKeys(new 
ReclaimableKeyFilter(om, om.getOmSnapshotManager(),
               
((OmMetadataManagerImpl)om.getMetadataManager()).getSnapshotChainManager(), 
null,
-              keyManager, om.getMetadataManager().getLock()), 
Integer.MAX_VALUE, ratisLimit).getKeyBlocksList())
+              keyManager, om.getMetadataManager().getLock()), 
Integer.MAX_VALUE).getKeyBlocksList())
           .isEmpty();
     }
 
@@ -271,7 +263,7 @@ void checkDeletionForKeysWithMultipleVersions() throws 
Exception {
           1000, 10000);
       assertThat(getRunCount())
           .isGreaterThan(initialRunCount);
-      assertThat(keyManager.getPendingDeletionKeys((kv) -> true, 
Integer.MAX_VALUE, ratisLimit).getKeyBlocksList())
+      assertThat(keyManager.getPendingDeletionKeys((kv) -> true, 
Integer.MAX_VALUE).getKeyBlocksList())
           .isEmpty();
 
       // The 1st version of the key has 1 block and the 2nd version has 2
@@ -316,7 +308,7 @@ void checkDeletedTableCleanUpForSnapshot() throws Exception 
{
       assertThat(keyManager.getPendingDeletionKeys(new 
ReclaimableKeyFilter(om, om.getOmSnapshotManager(),
               
((OmMetadataManagerImpl)om.getMetadataManager()).getSnapshotChainManager(), 
null,
               keyManager, om.getMetadataManager().getLock()),
-          Integer.MAX_VALUE, ratisLimit).getKeyBlocksList())
+          Integer.MAX_VALUE).getKeyBlocksList())
           .isEmpty();
 
       // deletedTable should have deleted key of the snapshot bucket
@@ -424,7 +416,7 @@ public void 
testAOSKeyDeletingWithSnapshotCreateParallelExecution()
           Assertions.assertNotEquals(deletePathKey[0], group.getGroupID());
         }
         return pendingKeysDeletion;
-      }).when(km).getPendingDeletionKeys(any(), anyInt(), anyInt());
+      }).when(km).getPendingDeletionKeys(any(), anyInt());
       service.runPeriodicalTaskNow();
       service.runPeriodicalTaskNow();
       assertTableRowCount(snapshotInfoTable, initialSnapshotCount + 2, 
metadataManager);
@@ -1217,7 +1209,7 @@ private long getRunCount() {
 
   private int countKeysPendingDeletion() {
     try {
-      final int count = keyManager.getPendingDeletionKeys((kv) -> true, 
Integer.MAX_VALUE, ratisLimit)
+      final int count = keyManager.getPendingDeletionKeys((kv) -> true, 
Integer.MAX_VALUE)
           .getKeyBlocksList().size();
       LOG.debug("KeyManager keys pending deletion: {}", count);
       return count;
@@ -1228,7 +1220,7 @@ private int countKeysPendingDeletion() {
 
   private long countBlocksPendingDeletion() {
     try {
-      return keyManager.getPendingDeletionKeys((kv) -> true, 
Integer.MAX_VALUE, ratisLimit)
+      return keyManager.getPendingDeletionKeys((kv) -> true, Integer.MAX_VALUE)
           .getKeyBlocksList()
           .stream()
           .map(BlockGroup::getBlockIDList)


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

Reply via email to