diff --git a/hadoop-ozone/freon/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java b/hadoop-ozone/freon/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java index 3c2e364668e5..e73ddc77f2b4 100644 --- a/hadoop-ozone/freon/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java +++ b/hadoop-ozone/freon/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java @@ -60,6 +60,7 @@ import org.apache.hadoop.hdds.conf.StorageSize; import org.apache.hadoop.hdds.tracing.TracingUtil; import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.client.BucketArgs; import org.apache.hadoop.ozone.client.ObjectStore; import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; @@ -67,6 +68,7 @@ import org.apache.hadoop.ozone.client.OzoneVolume; import org.apache.hadoop.ozone.client.io.OzoneInputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.hadoop.ozone.util.ShutdownHookManager; import org.apache.hadoop.util.Time; import org.apache.hadoop.util.VersionInfo; @@ -191,6 +193,12 @@ public final class RandomKeyGenerator implements Callable, FreonSubcommand ) private boolean cleanObjects = false; + @Option( + names = "--bucket-layout", + description = "Specifies the bucket layout (e.g., FILE_SYSTEM_OPTIMIZED, OBJECT_STORE, LEGACY)." + ) + private BucketLayout bucketLayout; + private ReplicationConfig replicationConfig; private int threadPoolSize; @@ -646,6 +654,16 @@ long getUnsuccessfulValidationCount() { return validateWrites ? writeValidationFailureCount.get() : 0; } + /** + * Returns the current size of the buckets map. + * + * @return number of buckets created and added to the map + */ + @VisibleForTesting + int getBucketMapSize() { + return buckets.size(); + } + /** * Wrapper to hold ozone keyValidate entry. */ @@ -757,7 +775,14 @@ private boolean createBucket(int globalBucketNumber) { .createActivatedSpan("createBucket")) { long start = System.nanoTime(); - volume.createBucket(bucketName); + if (bucketLayout != null) { + BucketArgs bucketArgs = BucketArgs.newBuilder() + .setBucketLayout(bucketLayout) + .build(); + volume.createBucket(bucketName, bucketArgs); + } else { + volume.createBucket(bucketName); + } long bucketCreationDuration = System.nanoTime() - start; histograms.get(FreonOps.BUCKET_CREATE.ordinal()) .update(bucketCreationDuration); @@ -880,7 +905,8 @@ private OzoneVolume getVolume(Integer volumeNumber) { return waitUntilAddedToMap(volumes, volumeNumber); } - private OzoneBucket getBucket(Integer bucketNumber) { + @VisibleForTesting + OzoneBucket getBucket(Integer bucketNumber) { return waitUntilAddedToMap(buckets, bucketNumber); } 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 e7a5bf2f7906..657ece8be81b 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 @@ -21,6 +21,8 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; +import org.apache.hadoop.ozone.client.OzoneBucket; +import org.apache.hadoop.ozone.om.helpers.BucketLayout; import org.apache.ozone.test.NonHATests; import org.junit.jupiter.api.Test; import picocli.CommandLine; @@ -185,4 +187,25 @@ void cleanObjectsTest() { assertEquals(2, randomKeyGenerator.getNumberOfVolumesCleaned()); assertEquals(6, randomKeyGenerator.getNumberOfBucketsCleaned()); } + + @Test + void testBucketLayoutOption() { + RandomKeyGenerator randomKeyGenerator = + new RandomKeyGenerator(cluster().getConf()); + CommandLine cmd = new CommandLine(randomKeyGenerator); + cmd.execute("--num-of-volumes", "1", + "--num-of-buckets", "1", + "--num-of-keys", "2", + "--bucket-layout", "OBJECT_STORE" + ); + + assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated()); + assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated()); + assertEquals(2, randomKeyGenerator.getNumberOfKeysAdded()); + assertEquals(1, randomKeyGenerator.getBucketMapSize()); + + // Fetch the bucket and check its layout + OzoneBucket bucket = randomKeyGenerator.getBucket(0); + assertEquals(BucketLayout.OBJECT_STORE, bucket.getBucketLayout()); + } }