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

umamahesh pushed a commit to branch HDDS-3816-ec
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/HDDS-3816-ec by this push:
     new aac07ef  HDDS-6347. EC: Freon randomKeys EC key support (#3112)
aac07ef is described below

commit aac07ef790752e0726d96bbe200e968496f1aa57
Author: Kaijie Chen <[email protected]>
AuthorDate: Wed Feb 23 07:49:34 2022 +0800

    HDDS-6347. EC: Freon randomKeys EC key support (#3112)
---
 .../hadoop/ozone/freon/TestRandomKeyGenerator.java | 43 +++++++++-----
 .../hadoop/ozone/freon/RandomKeyGenerator.java     | 68 +++++++++++++++-------
 2 files changed, 75 insertions(+), 36 deletions(-)

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 da85364..689214f 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
@@ -76,14 +76,12 @@ public class TestRandomKeyGenerator {
   }
 
   @Test
-  public void defaultTest() throws Exception {
+  public void testDefault() throws Exception {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
+        new RandomKeyGenerator(cluster.getConf());
     randomKeyGenerator.setNumOfVolumes(2);
     randomKeyGenerator.setNumOfBuckets(5);
     randomKeyGenerator.setNumOfKeys(10);
-    randomKeyGenerator.setFactor(ReplicationFactor.THREE);
-    randomKeyGenerator.setType(ReplicationType.RATIS);
     randomKeyGenerator.call();
     Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCreated());
     Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
@@ -91,9 +89,24 @@ public class TestRandomKeyGenerator {
   }
 
   @Test
-  public void multiThread() throws Exception {
+  public void testECKey() throws Exception {
+    RandomKeyGenerator randomKeyGenerator =
+        new RandomKeyGenerator(cluster.getConf());
+    randomKeyGenerator.setNumOfVolumes(2);
+    randomKeyGenerator.setNumOfBuckets(5);
+    randomKeyGenerator.setNumOfKeys(10);
+    randomKeyGenerator.setReplication("rs-3-2-1024k");
+    randomKeyGenerator.setType(ReplicationType.EC);
+    randomKeyGenerator.call();
+    Assert.assertEquals(2, randomKeyGenerator.getNumberOfVolumesCreated());
+    Assert.assertEquals(10, randomKeyGenerator.getNumberOfBucketsCreated());
+    Assert.assertEquals(100, randomKeyGenerator.getNumberOfKeysAdded());
+  }
+
+  @Test
+  public void testMultiThread() throws Exception {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
+        new RandomKeyGenerator(cluster.getConf());
     randomKeyGenerator.setNumOfVolumes(10);
     randomKeyGenerator.setNumOfBuckets(1);
     randomKeyGenerator.setNumOfKeys(10);
@@ -108,9 +121,9 @@ public class TestRandomKeyGenerator {
   }
 
   @Test
-  public void ratisTest3() throws Exception {
+  public void testRatisKey() throws Exception {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
+        new RandomKeyGenerator(cluster.getConf());
     randomKeyGenerator.setNumOfVolumes(10);
     randomKeyGenerator.setNumOfBuckets(1);
     randomKeyGenerator.setNumOfKeys(10);
@@ -125,9 +138,9 @@ public class TestRandomKeyGenerator {
   }
 
   @Test
-  public void bigFileThan2GB() throws Exception {
+  public void testKeyLargerThan2GB() throws Exception {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
+        new RandomKeyGenerator(cluster.getConf());
     randomKeyGenerator.setNumOfVolumes(1);
     randomKeyGenerator.setNumOfBuckets(1);
     randomKeyGenerator.setNumOfKeys(1);
@@ -144,9 +157,9 @@ public class TestRandomKeyGenerator {
   }
 
   @Test
-  public void fileWithSizeZero() throws Exception {
+  public void testZeroSizeKey() throws Exception {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
+        new RandomKeyGenerator(cluster.getConf());
     randomKeyGenerator.setNumOfVolumes(1);
     randomKeyGenerator.setNumOfBuckets(1);
     randomKeyGenerator.setNumOfKeys(1);
@@ -165,7 +178,7 @@ public class TestRandomKeyGenerator {
   @Test
   public void testThreadPoolSize() throws Exception {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
+        new RandomKeyGenerator(cluster.getConf());
     randomKeyGenerator.setNumOfVolumes(1);
     randomKeyGenerator.setNumOfBuckets(1);
     randomKeyGenerator.setNumOfKeys(1);
@@ -179,9 +192,9 @@ public class TestRandomKeyGenerator {
 
   @Test
   @org.junit.Ignore("HDDS-5993")
-  public void cleanObjectsTest() throws Exception {
+  public void testCleanObjects() throws Exception {
     RandomKeyGenerator randomKeyGenerator =
-        new RandomKeyGenerator((OzoneConfiguration) cluster.getConf());
+        new RandomKeyGenerator(cluster.getConf());
     randomKeyGenerator.setNumOfVolumes(2);
     randomKeyGenerator.setNumOfBuckets(5);
     randomKeyGenerator.setNumOfKeys(10);
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 18ab210..07748f0 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
@@ -38,9 +38,11 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.LongSupplier;
 
+import org.apache.hadoop.fs.ozone.OzoneClientUtils;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.StringUtils;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -178,19 +180,25 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
   private String jsonDir;
 
   @Option(
-      names = {"--replication-type", "--replicationType"},
-      description = "Replication type (STAND_ALONE, RATIS). Full name " +
-          "--replicationType will be removed in later versions.",
-      defaultValue = "STAND_ALONE"
+      names = {"--type", "--replication-type", "--replicationType"},
+      description = "Replication type (STAND_ALONE, RATIS, EC). Full name " +
+          "--replicationType will be removed in later versions."
   )
-  private ReplicationType type = ReplicationType.STAND_ALONE;
+  private ReplicationType type;
+
+  @Option(names = {"--replication"},
+      description =
+          "Replication configuration of the new key."
+              + "(ONE, THREE) for RATIS or STAND_ALONE, "
+              + "(rs-3-2-1024k, rs-6-3-1024k or rs-10-4-1024k) for EC."
+  )
+  private String replication;
 
   @Option(
       names = "--factor",
-      description = "Replication factor (ONE, THREE)",
-      defaultValue = "ONE"
+      description = "[Deprecated] Replication factor (ONE, THREE)"
   )
-  private ReplicationFactor factor = ReplicationFactor.ONE;
+  private ReplicationFactor factor;
 
   @Option(
       names = "--om-service-id",
@@ -205,6 +213,8 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
   )
   private boolean cleanObjects = false;
 
+  private ReplicationConfig replicationConfig;
+
   private int threadPoolSize;
 
   private OzoneClient ozoneClient;
@@ -290,17 +300,28 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
 
   @Override
   public Void call() throws Exception {
-    if (ozoneConfiguration != null) {
-      if (!ozoneConfiguration.getBoolean(
-          HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA,
-          HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA_DEFAULT)) {
-        LOG.info("Override validateWrites to false, because "
-            + HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA + " is set to false.");
-        validateWrites = false;
+    if (ozoneConfiguration == null) {
+      ozoneConfiguration = freon.createOzoneConfiguration();
+    }
+    if (!ozoneConfiguration.getBoolean(
+        HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA,
+        HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA_DEFAULT)) {
+      LOG.info("Override validateWrites to false, because "
+          + HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA + " is set to false.");
+      validateWrites = false;
+    }
+    init(ozoneConfiguration);
+
+    if (factor != null) {
+      // for backward compatibility
+      if (type == null) {
+        type = ReplicationType.STAND_ALONE;
       }
-      init(ozoneConfiguration);
+      replicationConfig = ReplicationConfig.fromTypeAndFactor(type, factor);
     } else {
-      init(freon.createOzoneConfiguration());
+      replicationConfig = OzoneClientUtils
+          .validateAndGetClientReplicationConfig(type, replication,
+              ozoneConfiguration);
     }
 
     keyValueBuffer = StringUtils.string2Bytes(
@@ -491,8 +512,8 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
     out.println("Number of Volumes created: " + numberOfVolumesCreated);
     out.println("Number of Buckets created: " + numberOfBucketsCreated);
     out.println("Number of Keys added: " + numberOfKeysAdded);
-    out.println("Ratis replication factor: " + factor.name());
-    out.println("Ratis replication type: " + type.name());
+    out.println("Replication: " + replicationConfig.getReplication());
+    out.println("Replication type: " + replicationConfig.getReplicationType());
     out.println(
         "Average Time spent in volume creation: " + prettyAverageVolumeTime);
     out.println(
@@ -812,8 +833,8 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
     try {
       try (AutoCloseable scope = TracingUtil.createActivatedSpan("createKey")) 
{
         long keyCreateStart = System.nanoTime();
-        try (OzoneOutputStream os = bucket.createKey(keyName, keySize, type,
-            factor, new HashMap<>())) {
+        try (OzoneOutputStream os = bucket.createKey(keyName, keySize,
+            replicationConfig, new HashMap<>())) {
           long keyCreationDuration = System.nanoTime() - keyCreateStart;
           histograms.get(FreonOps.KEY_CREATE.ordinal())
               .update(keyCreationDuration);
@@ -1259,6 +1280,11 @@ public final class RandomKeyGenerator implements 
Callable<Void> {
   }
 
   @VisibleForTesting
+  public void setReplication(String replication) {
+    this.replication = replication;
+  }
+
+  @VisibleForTesting
   public void setFactor(ReplicationFactor factor) {
     this.factor = factor;
   }

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

Reply via email to