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 9f6d7f2  HDDS-5167. Add clean option for Freon randomkeys to clean 
generated objects (#2198)
9f6d7f2 is described below

commit 9f6d7f2437a1ba7ee0dcd246196daa87eb3a20fb
Author: Symious <[email protected]>
AuthorDate: Tue May 18 12:54:48 2021 +0800

    HDDS-5167. Add clean option for Freon randomkeys to clean generated objects 
(#2198)
---
 .../dist/src/main/smoketest/freon/remove.robot     |  43 +++++++
 .../hadoop/ozone/freon/TestRandomKeyGenerator.java |  19 +++
 .../om/request/bucket/OMBucketDeleteRequest.java   |   2 +-
 .../hadoop/ozone/freon/BaseFreonGenerator.java     |   7 ++
 .../java/org/apache/hadoop/ozone/freon/Freon.java  |   2 +
 .../hadoop/ozone/freon/OmBucketGenerator.java      |   2 +-
 ...OmBucketGenerator.java => OmBucketRemover.java} |  36 ++----
 ...etGenerator.java => OzoneClientKeyRemover.java} |  60 ++++-----
 .../hadoop/ozone/freon/RandomKeyGenerator.java     | 137 +++++++++++++++++++++
 .../apache/hadoop/ozone/freon/S3KeyGenerator.java  |   1 +
 10 files changed, 250 insertions(+), 59 deletions(-)

diff --git a/hadoop-ozone/dist/src/main/smoketest/freon/remove.robot 
b/hadoop-ozone/dist/src/main/smoketest/freon/remove.robot
new file mode 100644
index 0000000..087a3bd
--- /dev/null
+++ b/hadoop-ozone/dist/src/main/smoketest/freon/remove.robot
@@ -0,0 +1,43 @@
+# 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
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# 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.
+
+*** Settings ***
+Documentation       Test freon data remove commands
+Resource            ../lib/os.robot
+Test Timeout        5 minutes
+
+*** Variables ***
+${OCKR_PREFIX}    ockr
+${OMBR_PREFIX}    ombr
+
+*** Test Cases ***
+Ozone Client Key Remover
+    [Setup]            Ozone Client Key Generator For Remover    ${OCKR_PREFIX}
+    ${result} =        Execute                ozone freon ockr ${OM_HA_PARAM} 
-t=1 -n=1 -p ${OCKR_PREFIX}
+                       Should contain         ${result}   Successful 
executions: 1
+
+OM Bucket Remover
+    [Setup]            OM Bucket Generator For Remover           ${OMBR_PREFIX}
+    ${result} =        Execute                ozone freon ombr ${OM_HA_PARAM} 
-t=1 -n=1 -p ${OMBR_PREFIX}
+                       Should contain         ${result}   Successful 
executions: 1
+
+*** Keywords ***
+Ozone Client Key Generator For Remover
+    [Arguments]        ${PREFIX}
+    Execute            ozone freon ockg ${OM_HA_PARAM} -t=1 -n=1 -p ${PREFIX}
+
+OM Bucket Generator For Remover
+    [Arguments]        ${PREFIX}
+    Execute            ozone freon ombg ${OM_HA_PARAM} -t=1 -n=1 -p ${PREFIX}
\ No newline at end of file
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java
index beda422..4455625 100644
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java
@@ -179,4 +179,23 @@ public class TestRandomKeyGenerator {
     Assert.assertEquals(10, randomKeyGenerator.getThreadPoolSize());
     Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded());
   }
+
+  @Test
+  public void cleanObjectsTest() throws Exception {
+    RandomKeyGenerator randomKeyGenerator =
+        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
+    randomKeyGenerator.setNumOfVolumes(2);
+    randomKeyGenerator.setNumOfBuckets(5);
+    randomKeyGenerator.setNumOfKeys(10);
+    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
+    randomKeyGenerator.setType(ReplicationType.RATIS);
+    randomKeyGenerator.setNumOfThreads(10);
+    randomKeyGenerator.setCleanObjects(true);
+    randomKeyGenerator.call();
+    Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCreated());
+    Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
+    Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
+    Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCleaned());
+    Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCleaned());
+  }
 }
diff --git 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java
 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java
index 7b165b2..9049f59 100644
--- 
a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java
+++ 
b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketDeleteRequest.java
@@ -116,7 +116,7 @@ public class OMBucketDeleteRequest extends OMClientRequest {
 
       if (!omMetadataManager.getBucketTable().isExist(bucketKey)) {
         LOG.debug("bucket: {} not found ", bucketName);
-        throw new OMException("Bucket already exist", BUCKET_NOT_FOUND);
+        throw new OMException("Bucket not exists", BUCKET_NOT_FOUND);
       }
 
       //Check if bucket is empty
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java
index 9c67f32..3323335 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/BaseFreonGenerator.java
@@ -374,6 +374,13 @@ public class BaseFreonGenerator {
   }
 
   /**
+   * Generate a bucket name based on the prefix and counter.
+   */
+  public String generateBucketName(long counter) {
+    return getPrefix() + counter;
+  }
+
+  /**
    * Create missing target volume/bucket.
    */
   public void ensureVolumeAndBucketExist(OzoneClient rpcClient,
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/Freon.java 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/Freon.java
index 21714c9..2cf5725 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/Freon.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/Freon.java
@@ -42,8 +42,10 @@ import picocli.CommandLine.Option;
         RandomKeyGenerator.class,
         OzoneClientKeyGenerator.class,
         OzoneClientKeyValidator.class,
+        OzoneClientKeyRemover.class,
         OmKeyGenerator.class,
         OmBucketGenerator.class,
+        OmBucketRemover.class,
         HadoopFsGenerator.class,
         HadoopNestedDirGenerator.class,
         HadoopDirTreeGenerator.class,
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketGenerator.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketGenerator.java
index da9227e..de7f057 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketGenerator.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketGenerator.java
@@ -86,7 +86,7 @@ public class OmBucketGenerator extends BaseFreonGenerator
   private void createBucket(long index) throws Exception {
 
     OmBucketInfo bucketInfo = new OmBucketInfo.Builder()
-        .setBucketName(getPrefix()+index)
+        .setBucketName(generateBucketName(index))
         .setVolumeName(volumeName)
         .setStorageType(StorageType.DISK)
         .build();
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketGenerator.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketRemover.java
similarity index 71%
copy from 
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketGenerator.java
copy to 
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketRemover.java
index da9227e..18b2b43 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketGenerator.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketRemover.java
@@ -16,29 +16,25 @@
  */
 package org.apache.hadoop.ozone.freon;
 
-import java.util.concurrent.Callable;
-
+import com.codahale.metrics.Timer;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.StorageType;
-import org.apache.hadoop.ozone.client.OzoneClient;
-import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
-
-import com.codahale.metrics.Timer;
 import picocli.CommandLine.Command;
 import picocli.CommandLine.Option;
 
+import java.util.concurrent.Callable;
+
 /**
  * Data generator tool test om performance.
  */
-@Command(name = "ombg",
-    aliases = "om-bucket-generator",
-    description = "Generate ozone buckets on OM side.",
+@Command(name = "ombr",
+    aliases = "om-bucket-remover",
+    description = "Remove ozone buckets on OM side.",
     versionProvider = HddsVersionProvider.class,
     mixinStandardHelpOptions = true,
     showDefaultValues = true)
-public class OmBucketGenerator extends BaseFreonGenerator
+public class OmBucketRemover extends BaseFreonGenerator
     implements Callable<Void> {
 
   @Option(names = {"-v", "--volume"},
@@ -64,15 +60,13 @@ public class OmBucketGenerator extends BaseFreonGenerator
 
     OzoneConfiguration ozoneConfiguration = createOzoneConfiguration();
 
-    try (OzoneClient rpcClient = createOzoneClient(omServiceID,
-        ozoneConfiguration)) {
-      ensureVolumeExists(rpcClient, volumeName);
+    try {
 
       ozoneManagerClient = createOmClient(ozoneConfiguration, omServiceID);
 
-      bucketCreationTimer = getMetrics().timer("bucket-create");
+      bucketCreationTimer = getMetrics().timer("bucket-remove");
 
-      runTests(this::createBucket);
+      runTests(this::removeBucket);
 
     } finally {
       if (ozoneManagerClient != null) {
@@ -83,16 +77,12 @@ public class OmBucketGenerator extends BaseFreonGenerator
     return null;
   }
 
-  private void createBucket(long index) throws Exception {
+  private void removeBucket(long index) throws Exception {
 
-    OmBucketInfo bucketInfo = new OmBucketInfo.Builder()
-        .setBucketName(getPrefix()+index)
-        .setVolumeName(volumeName)
-        .setStorageType(StorageType.DISK)
-        .build();
+    String bucketName = generateBucketName(index);
 
     bucketCreationTimer.time(() -> {
-      ozoneManagerClient.createBucket(bucketInfo);
+      ozoneManagerClient.deleteBucket(volumeName, bucketName);
       return null;
     });
   }
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketGenerator.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyRemover.java
similarity index 62%
copy from 
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketGenerator.java
copy to 
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyRemover.java
index da9227e..528ae17 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OmBucketGenerator.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/OzoneClientKeyRemover.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
@@ -16,29 +16,26 @@
  */
 package org.apache.hadoop.ozone.freon;
 
-import java.util.concurrent.Callable;
-
+import com.codahale.metrics.Timer;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.StorageType;
+import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneClient;
-import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
-import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
-
-import com.codahale.metrics.Timer;
 import picocli.CommandLine.Command;
 import picocli.CommandLine.Option;
 
+import java.util.concurrent.Callable;
+
 /**
- * Data generator tool test om performance.
+ * Data remover tool test om performance.
  */
-@Command(name = "ombg",
-    aliases = "om-bucket-generator",
-    description = "Generate ozone buckets on OM side.",
+@Command(name = "ockr",
+    aliases = "ozone-client-key-remover",
+    description = "Remove keys with the help of the ozone clients.",
     versionProvider = HddsVersionProvider.class,
     mixinStandardHelpOptions = true,
     showDefaultValues = true)
-public class OmBucketGenerator extends BaseFreonGenerator
+public class OzoneClientKeyRemover extends BaseFreonGenerator
     implements Callable<Void> {
 
   @Option(names = {"-v", "--volume"},
@@ -47,52 +44,47 @@ public class OmBucketGenerator extends BaseFreonGenerator
       defaultValue = "vol1")
   private String volumeName;
 
+  @Option(names = {"-b", "--bucket"},
+      description = "Name of the bucket which contains the test data. Will be"
+          + " created if missing.",
+      defaultValue = "bucket1")
+  private String bucketName;
+
   @Option(
       names = "--om-service-id",
       description = "OM Service ID"
   )
   private String omServiceID = null;
 
-  private OzoneManagerProtocol ozoneManagerClient;
+  private Timer timer;
 
-  private Timer bucketCreationTimer;
+  private OzoneBucket ozoneBucket;
 
   @Override
   public Void call() throws Exception {
 
     init();
-
     OzoneConfiguration ozoneConfiguration = createOzoneConfiguration();
 
     try (OzoneClient rpcClient = createOzoneClient(omServiceID,
         ozoneConfiguration)) {
-      ensureVolumeExists(rpcClient, volumeName);
-
-      ozoneManagerClient = createOmClient(ozoneConfiguration, omServiceID);
+      ozoneBucket = rpcClient.getObjectStore().getVolume(volumeName)
+          .getBucket(bucketName);
 
-      bucketCreationTimer = getMetrics().timer("bucket-create");
+      timer = getMetrics().timer("remove");
 
-      runTests(this::createBucket);
+      runTests(this::removeKey);
 
-    } finally {
-      if (ozoneManagerClient != null) {
-        ozoneManagerClient.close();
-      }
     }
 
     return null;
   }
 
-  private void createBucket(long index) throws Exception {
-
-    OmBucketInfo bucketInfo = new OmBucketInfo.Builder()
-        .setBucketName(getPrefix()+index)
-        .setVolumeName(volumeName)
-        .setStorageType(StorageType.DISK)
-        .build();
+  private void removeKey(long counter) throws Exception {
+    final String key = generateObjectName(counter);
 
-    bucketCreationTimer.time(() -> {
-      ozoneManagerClient.createBucket(bucketInfo);
+    timer.time(() -> {
+      ozoneBucket.deleteKey(key);
       return null;
     });
   }
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java
index affb36e..ab83154 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java
@@ -189,6 +189,13 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
   )
   private String omServiceID = null;
 
+  @Option(
+      names = "--clean-objects",
+      description = "Specifies whether to clean the random generated " +
+          "volumes, buckets and keys."
+  )
+  private boolean cleanObjects = false;
+
   private int threadPoolSize;
 
   private OzoneClient ozoneClient;
@@ -217,6 +224,10 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
   private AtomicInteger numberOfBucketsCreated;
   private AtomicLong numberOfKeysAdded;
 
+  private AtomicInteger cleanedBucketCounter;
+  private AtomicInteger numberOfBucketsCleaned;
+  private AtomicInteger numberOfVolumesCleaned;
+
   private Long totalWritesValidated;
   private Long writeValidationSuccessCount;
   private Long writeValidationFailureCount;
@@ -251,6 +262,9 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
     keyCounter = new AtomicLong();
     volumes = new ConcurrentHashMap<>();
     buckets = new ConcurrentHashMap<>();
+    cleanedBucketCounter = new AtomicInteger();
+    numberOfBucketsCleaned = new AtomicInteger();
+    numberOfVolumesCleaned = new AtomicInteger();
     if (omServiceID != null) {
       ozoneClient = OzoneClientFactory.getRpcClient(omServiceID, 
configuration);
     } else {
@@ -307,6 +321,7 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
     LOG.info("Key size: {} bytes", keySize);
     LOG.info("Buffer size: {} bytes", bufferSize);
     LOG.info("validateWrites : {}", validateWrites);
+    LOG.info("cleanObjects : {}", cleanObjects);
     for (int i = 0; i < numOfThreads; i++) {
       executor.execute(new ObjectCreator());
     }
@@ -352,6 +367,9 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
     if (validator != null) {
       validator.join();
     }
+    if (cleanObjects && exception == null) {
+      doCleanObjects();
+    }
     ozoneClient.close();
     if (exception != null) {
       throw new RuntimeException(exception);
@@ -371,6 +389,48 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
           }
         }));
   }
+
+  private void doCleanObjects() throws InterruptedException {
+    // Clean Buckets first
+    executor = Executors.newFixedThreadPool(threadPoolSize);
+    for (int i = 0; i < numOfThreads; i++) {
+      executor.execute(new BucketCleaner());
+    }
+    LongSupplier currentValue = numberOfBucketsCleaned::get;
+    progressbar = new ProgressBar(System.out, totalBucketCount, currentValue);
+
+    LOG.info("Starting clean progress bar Thread.");
+    progressbar.start();
+
+    try {
+      // wait until all Buckets are cleaned or exception occurred.
+      while ((numberOfBucketsCleaned.get() != totalBucketCount)
+          && exception == null) {
+        try {
+          Thread.sleep(CHECK_INTERVAL_MILLIS);
+        } catch (InterruptedException e) {
+          throw e;
+        }
+      }
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+
+    executor.shutdown();
+    executor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS);
+
+    // Clean Volume after cleaning Bucket
+    for (int v = 0; v < numOfVolumes; v++) {
+      cleanVolume(v);
+    }
+
+    if (exception != null) {
+      progressbar.terminate();
+    } else {
+      progressbar.shutdown();
+    }
+  }
+
   /**
    * Prints stats of {@link Freon} run to the PrintStream.
    *
@@ -527,6 +587,25 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
   }
 
   /**
+   * Returns the number of volumes cleaned.
+   *
+   * @return cleaned volume count.
+   */
+  @VisibleForTesting
+  int getNumberOfVolumesCleaned() {
+    return numberOfVolumesCleaned.get();
+  }
+
+  /**
+   * Returns the number of buckets cleaned.
+   *
+   * @return cleaned bucket count.
+   */
+  @VisibleForTesting
+  int getNumberOfBucketsCleaned() {
+    return numberOfBucketsCleaned.get();
+  }
+  /**
    * Returns true if random validation of write is enabled.
    *
    * @return validateWrites
@@ -625,6 +704,18 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
     }
   }
 
+  private class BucketCleaner implements Runnable {
+    @Override
+    public void run() {
+      int b;
+      while ((b = cleanedBucketCounter.getAndIncrement()) < totalBucketCount) {
+        if (!cleanBucket(b)) {
+          return;
+        }
+      }
+    }
+  }
+
   private boolean createVolume(int volumeNumber) {
     String volumeName = "vol-" + volumeNumber + "-"
         + RandomStringUtils.randomNumeric(5);
@@ -744,6 +835,47 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
     }
   }
 
+  private boolean cleanVolume(int volumeNumber) {
+    OzoneVolume volume = getVolume(volumeNumber);
+    String volumeName = volume.getName();
+    LOG.trace("Cleaning volume: {}", volumeName);
+    try (AutoCloseable scope = TracingUtil
+        .createActivatedSpan("cleanVolume")) {
+      objectStore.deleteVolume(volumeName);
+      numberOfVolumesCleaned.getAndIncrement();
+      return true;
+    } catch (Throwable e) {
+      exception = e;
+      LOG.error("Could not clean volume", e);
+      return false;
+    }
+  }
+
+  private boolean cleanBucket(int globalBucketNumber) {
+    int volumeNumber = globalBucketNumber % numOfVolumes;
+    OzoneVolume volume = getVolume(volumeNumber);
+    OzoneBucket bucket = getBucket(globalBucketNumber);
+    String bucketName = bucket.getName();
+    if (volume == null) {
+      LOG.error("Could not find volume {}", volumeNumber);
+      return false;
+    }
+    LOG.trace("Cleaning bucket: {} in volume: {}",
+        bucketName, volume.getName());
+    ArrayList<String> keys = new ArrayList<>();
+    try {
+      bucket.listKeys(null).forEachRemaining(x -> keys.add(x.getName()));
+      bucket.deleteKeys(keys);
+      volume.deleteBucket(bucketName);
+      numberOfBucketsCleaned.getAndIncrement();
+      return true;
+    } catch (Throwable e) {
+      exception = e;
+      LOG.error("Could not clean bucket ", e);
+      return false;
+    }
+  }
+
   private OzoneVolume getVolume(Integer volumeNumber) {
     return waitUntilAddedToMap(volumes, volumeNumber);
   }
@@ -1119,4 +1251,9 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
   public int getThreadPoolSize() {
     return threadPoolSize;
   }
+
+  @VisibleForTesting
+  public void setCleanObjects(boolean cleanObjects) {
+    this.cleanObjects = cleanObjects;
+  }
 }
diff --git 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/S3KeyGenerator.java
 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/S3KeyGenerator.java
index 035f195..292b9f1 100644
--- 
a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/S3KeyGenerator.java
+++ 
b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/S3KeyGenerator.java
@@ -39,6 +39,7 @@ import com.amazonaws.services.s3.model.UploadPartResult;
 import com.codahale.metrics.Timer;
 import org.apache.commons.lang3.RandomStringUtils;
 import static org.apache.hadoop.ozone.OzoneConsts.OM_MULTIPART_MIN_SIZE;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import picocli.CommandLine.Command;

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

Reply via email to