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

adoroszlai 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 348d4f4f126 HDDS-14467. Expose pending delete bytes/namespace in 
OzoneBucket (#9708)
348d4f4f126 is described below

commit 348d4f4f126ee96066352f280a88f898708e2900
Author: Sadanand Shenoy <[email protected]>
AuthorDate: Tue Mar 3 11:21:17 2026 +0530

    HDDS-14467. Expose pending delete bytes/namespace in OzoneBucket (#9708)
---
 .../apache/hadoop/ozone/client/OzoneBucket.java    | 30 ++++++++++
 .../apache/hadoop/ozone/client/rpc/RpcClient.java  |  4 ++
 .../ozone/client/rpc/OzoneRpcClientTests.java      | 68 ++++++++++++++++++++++
 3 files changed, 102 insertions(+)

diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
index 56642742422..75d7d82c5e1 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
+++ 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
@@ -151,6 +151,14 @@ public class OzoneBucket extends WithMetadata {
    * Bucket Owner.
    */
   private String owner;
+  /**
+   * Pending deletion bytes (Includes bytes retained by snapshots).
+   */
+  private long pendingDeleteBytes;
+  /**
+   * Pending deletion namespace (Includes keys retained by snapshots).
+   */
+  private long pendingDeleteNamespace;
 
   protected OzoneBucket(Builder builder) {
     super(builder);
@@ -167,6 +175,8 @@ protected OzoneBucket(Builder builder) {
     }
     this.usedBytes = builder.usedBytes;
     this.usedNamespace = builder.usedNamespace;
+    this.pendingDeleteBytes = builder.pendingDeleteBytes;
+    this.pendingDeleteNamespace = builder.pendingDeleteNamespace;
     this.creationTime = Instant.ofEpochMilli(builder.creationTime);
     if (builder.modificationTime != 0) {
       this.modificationTime = Instant.ofEpochMilli(builder.modificationTime);
@@ -610,6 +620,14 @@ public long getUsedNamespace() {
     return usedNamespace;
   }
 
+  public long getPendingDeleteBytes() {
+    return pendingDeleteBytes;
+  }
+
+  public long getPendingDeleteNamespace() {
+    return pendingDeleteNamespace;
+  }
+
   /**
    * Returns Iterator to iterate over all keys in the bucket.
    * The result can be restricted using key prefix, will return all
@@ -1127,6 +1145,8 @@ public static class Builder extends WithMetadata.Builder {
     private long quotaInNamespace;
     private BucketLayout bucketLayout;
     private String owner;
+    private long pendingDeleteBytes;
+    private long pendingDeleteNamespace;
 
     protected Builder() {
     }
@@ -1223,6 +1243,16 @@ public Builder setOwner(String owner) {
       return this;
     }
 
+    public Builder setPendingDeleteBytes(long pendingDeleteBytes) {
+      this.pendingDeleteBytes = pendingDeleteBytes;
+      return this;
+    }
+
+    public Builder setPendingDeleteNamespace(long pendingDeleteNamespace) {
+      this.pendingDeleteNamespace = pendingDeleteNamespace;
+      return this;
+    }
+
     public OzoneBucket build() {
       return new OzoneBucket(this);
     }
diff --git 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
index 3947e4b6818..1702e433b32 100644
--- 
a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
+++ 
b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
@@ -1299,6 +1299,8 @@ public OzoneBucket getBucketDetails(
         .setSourceBucket(bucketInfo.getSourceBucket())
         .setUsedBytes(bucketInfo.getTotalBucketSpace())
         .setUsedNamespace(bucketInfo.getTotalBucketNamespace())
+        .setPendingDeleteBytes(bucketInfo.getSnapshotUsedBytes())
+        .setPendingDeleteNamespace(bucketInfo.getSnapshotUsedNamespace())
         .setQuotaInBytes(bucketInfo.getQuotaInBytes())
         .setQuotaInNamespace(bucketInfo.getQuotaInNamespace())
         .setBucketLayout(bucketInfo.getBucketLayout())
@@ -1330,6 +1332,8 @@ public List<OzoneBucket> listBuckets(String volumeName, 
String bucketPrefix,
                 .setSourceBucket(bucket.getSourceBucket())
                 .setUsedBytes(bucket.getTotalBucketSpace())
                 .setUsedNamespace(bucket.getTotalBucketNamespace())
+                .setPendingDeleteBytes(bucket.getSnapshotUsedBytes())
+                .setPendingDeleteNamespace(bucket.getSnapshotUsedNamespace())
                 .setQuotaInBytes(bucket.getQuotaInBytes())
                 .setQuotaInNamespace(bucket.getQuotaInNamespace())
                 .setBucketLayout(bucket.getBucketLayout())
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java
index 31c826fabb0..61f3d230812 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/OzoneRpcClientTests.java
@@ -33,6 +33,7 @@
 import static 
org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE;
 import static 
org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
+import static 
org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL;
 import static org.apache.hadoop.ozone.OzoneConsts.DEFAULT_OM_UPDATE_ID;
 import static org.apache.hadoop.ozone.OzoneConsts.ETAG;
 import static org.apache.hadoop.ozone.OzoneConsts.GB;
@@ -211,6 +212,7 @@
 import org.junit.jupiter.params.provider.CsvSource;
 import org.junit.jupiter.params.provider.EnumSource;
 import org.junit.jupiter.params.provider.MethodSource;
+import org.junit.jupiter.params.provider.ValueSource;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -263,6 +265,7 @@ static void startCluster(OzoneConfiguration conf, 
MiniOzoneCluster.Builder build
     conf.setInt(OZONE_SCM_RATIS_PIPELINE_LIMIT, 10);
     conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 1, 
TimeUnit.SECONDS);
     conf.setTimeDuration(OZONE_DIR_DELETING_SERVICE_INTERVAL, 1, 
TimeUnit.SECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL, 1, 
TimeUnit.SECONDS);
 
     ClientConfigForTesting.newBuilder(StorageUnit.MB)
         .setDataStreamMinPacketSize(1)
@@ -1151,6 +1154,71 @@ public void testDeleteAuditLog() throws Exception {
         ", replicationConfig=EC{rs-3-2-1024k}}\",\"unDeletedKeysList\"");
   }
 
+  /**
+   * Verifies pendingDelete* fields are populated after key delete,
+   * with/without snapshot retention.
+   *
+   * @param withSnapshot whether to create a snapshot before deleting the key
+   * @throws Exception on failure
+   */
+  @ParameterizedTest
+  @ValueSource(booleans = {false, true})
+  public void testBucketPendingDeleteBytes(boolean withSnapshot) throws 
Exception {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    String keyName = UUID.randomUUID().toString();
+    String snapshotName = "snap-" + UUID.randomUUID();
+    String value = "sample value";
+    int valueLength = value.getBytes(UTF_8).length;
+
+    store.createVolume(volumeName);
+    OzoneVolume volume = store.getVolume(volumeName);
+    volume.createBucket(bucketName);
+    OzoneBucket bucket = volume.getBucket(bucketName);
+
+    writeKey(bucket, keyName, ONE, value, valueLength);
+
+    OzoneBucket bucketAfterKeyWrite = store.getVolume(volumeName)
+        .getBucket(bucketName);
+    assertThat(bucketAfterKeyWrite.getUsedBytes()).isEqualTo(valueLength);
+    assertThat(bucketAfterKeyWrite.getUsedNamespace()).isEqualTo(1);
+    assertThat(bucketAfterKeyWrite.getPendingDeleteBytes()).isEqualTo(0);
+    assertThat(bucketAfterKeyWrite.getPendingDeleteNamespace()).isEqualTo(0);
+
+    if (withSnapshot) {
+      store.createSnapshot(volumeName, bucketName, snapshotName);
+    }
+    bucket.deleteKey(keyName);
+    // After delete: usedBytes should still be totalBucketSpace.
+    OzoneBucket bucketAfterKeyDelete = store.getVolume(volumeName)
+        .getBucket(bucketName);
+    assertThat(bucketAfterKeyDelete.getUsedBytes()).isEqualTo(valueLength);
+    assertThat(bucketAfterKeyDelete.getUsedNamespace()).isEqualTo(1);
+    
assertThat(bucketAfterKeyDelete.getPendingDeleteBytes()).isEqualTo(valueLength);
+    assertThat(bucketAfterKeyDelete.getPendingDeleteNamespace()).isEqualTo(1);
+
+    if (withSnapshot) {
+      // if snapshot is present bytes won't be released until snapshot is 
deleted.
+      store.deleteSnapshot(volumeName, bucketName, snapshotName);
+    }
+
+    GenericTestUtils.waitFor(() -> {
+      OzoneBucket buck = null;
+      try {
+        buck = store.getVolume(volumeName).getBucket(bucketName);
+      } catch (IOException e) {
+        fail("Failed to get bucket details", e);
+      }
+      return buck.getUsedBytes() == 0 && buck.getUsedNamespace() == 0;
+    }, 1000, 30000);
+    OzoneBucket bucketAfterKeyPurge = store.getVolume(volumeName)
+        .getBucket(bucketName);
+    assertThat(bucketAfterKeyPurge.getUsedBytes()).isEqualTo(0);
+    assertThat(bucketAfterKeyPurge.getUsedNamespace()).isEqualTo(0);
+    assertThat(bucketAfterKeyPurge.getPendingDeleteBytes()).isEqualTo(0);
+    assertThat(bucketAfterKeyPurge.getPendingDeleteNamespace()).isEqualTo(0);
+  }
+
   protected void verifyReplication(String volumeName, String bucketName,
       String keyName, ReplicationConfig replication)
       throws IOException {


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

Reply via email to