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 5198ac3e7d6..7cfd1fef81f 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 @@ -26,7 +26,6 @@ import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.HashMap; import java.util.Map; -import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; @@ -263,9 +262,7 @@ public final class RandomKeyGenerator implements Callable { // Compute the common initial digest for all keys without their UUID if (validateWrites) { commonInitialMD = DigestUtils.getDigest(DIGEST_ALGORITHM); - int uuidLength = UUID.randomUUID().toString().length(); - keySize = Math.max(uuidLength, keySize); - for (long nrRemaining = keySize - uuidLength; nrRemaining > 0; + for (long nrRemaining = keySize; nrRemaining > 0; nrRemaining -= bufferSize) { int curSize = (int)Math.min(bufferSize, nrRemaining); commonInitialMD.update(keyValueBuffer, 0, curSize); @@ -682,7 +679,6 @@ public final class RandomKeyGenerator implements Callable { + RandomStringUtils.randomNumeric(5); LOG.trace("Adding key: {} in bucket: {} of volume: {}", keyName, bucketName, volumeName); - byte[] randomValue = DFSUtil.string2Bytes(UUID.randomUUID().toString()); try { try (Scope scope = GlobalTracer.get().buildSpan("createKey") .startActive(true)) { @@ -697,12 +693,11 @@ public final class RandomKeyGenerator implements Callable { try (Scope writeScope = GlobalTracer.get().buildSpan("writeKeyData") .startActive(true)) { long keyWriteStart = System.nanoTime(); - for (long nrRemaining = keySize - randomValue.length; + for (long nrRemaining = keySize; nrRemaining > 0; nrRemaining -= bufferSize) { int curSize = (int) Math.min(bufferSize, nrRemaining); os.write(keyValueBuffer, 0, curSize); } - os.write(randomValue); os.close(); long keyWriteDuration = System.nanoTime() - keyWriteStart; @@ -716,7 +711,6 @@ public final class RandomKeyGenerator implements Callable { if (validateWrites) { MessageDigest tmpMD = (MessageDigest) commonInitialMD.clone(); - tmpMD.update(randomValue); boolean validate = validationQueue.offer( new KeyValidate(bucket, keyName, tmpMD.digest())); if (validate) { diff --git a/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java b/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java index 748972eafe5..45ea23d5570 100644 --- a/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java +++ b/hadoop-ozone/tools/src/test/java/org/apache/hadoop/ozone/freon/TestRandomKeyGenerator.java @@ -128,6 +128,25 @@ public class TestRandomKeyGenerator { Assert.assertEquals(1, randomKeyGenerator.getSuccessfulValidationCount()); } + @Test + public void fileWithSizeZero() throws Exception { + RandomKeyGenerator randomKeyGenerator = + new RandomKeyGenerator((OzoneConfiguration) cluster.getConf()); + randomKeyGenerator.setNumOfVolumes(1); + randomKeyGenerator.setNumOfBuckets(1); + randomKeyGenerator.setNumOfKeys(1); + randomKeyGenerator.setNumOfThreads(1); + randomKeyGenerator.setKeySize(0); + randomKeyGenerator.setFactor(ReplicationFactor.THREE); + randomKeyGenerator.setType(ReplicationType.RATIS); + randomKeyGenerator.setValidateWrites(true); + randomKeyGenerator.call(); + Assert.assertEquals(1, randomKeyGenerator.getNumberOfVolumesCreated()); + Assert.assertEquals(1, randomKeyGenerator.getNumberOfBucketsCreated()); + Assert.assertEquals(1, randomKeyGenerator.getNumberOfKeysAdded()); + Assert.assertEquals(1, randomKeyGenerator.getSuccessfulValidationCount()); + } + @Test public void testThreadPoolSize() throws Exception { RandomKeyGenerator randomKeyGenerator =