You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by um...@apache.org on 2022/03/07 16:21:52 UTC

[ozone] branch HDDS-3816-ec updated: HDDS-6185: EC: EC keys can't be created via S3 interfaces (#3150)

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 58eac93  HDDS-6185: EC: EC keys can't be created via S3 interfaces (#3150)
58eac93 is described below

commit 58eac935cd735052aa00abe89144caacd206cbed
Author: Uma Maheswara Rao G <um...@apache.org>
AuthorDate: Mon Mar 7 08:21:34 2022 -0800

    HDDS-6185: EC: EC keys can't be created via S3 interfaces (#3150)
---
 .../apache/hadoop/fs/ozone/OzoneClientUtils.java   |   2 +-
 .../hadoop/ozone/s3/endpoint/ObjectEndpoint.java   |  62 +++++----
 .../org/apache/hadoop/ozone/s3/util/S3Utils.java   |  75 +++++++++++
 .../hadoop/ozone/client/OzoneBucketStub.java       |  47 ++++++-
 .../hadoop/ozone/s3/endpoint/TestObjectPut.java    |  31 ++++-
 .../ozone/s3/endpoint/TestPermissionCheck.java     |   2 +-
 .../apache/hadoop/ozone/s3/util/TestS3Utils.java   | 143 +++++++++++++++++++++
 7 files changed, 325 insertions(+), 37 deletions(-)

diff --git a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java
index 3a45c90..32220dd 100644
--- a/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java
+++ b/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientUtils.java
@@ -144,7 +144,7 @@ public final class OzoneClientUtils {
     return clientDeterminedReplConfig;
   }
 
-  static ReplicationConfig getClientConfiguredReplicationConfig(
+  public static ReplicationConfig getClientConfiguredReplicationConfig(
       ConfigurationSource config) {
     String replication = config.get(OZONE_REPLICATION);
     if (replication == null) {
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
index 863b1b0..96bfe73 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
@@ -53,6 +53,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.OptionalLong;
 
+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;
@@ -77,6 +78,7 @@ import org.apache.hadoop.ozone.s3.util.RFC1123Util;
 import org.apache.hadoop.ozone.s3.util.RangeHeader;
 import org.apache.hadoop.ozone.s3.util.RangeHeaderParserUtil;
 import org.apache.hadoop.ozone.s3.util.S3StorageType;
+import org.apache.hadoop.ozone.s3.util.S3Utils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.util.Time;
 
@@ -87,6 +89,9 @@ import org.apache.commons.io.IOUtils;
 
 import org.apache.commons.lang3.tuple.Pair;
 
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION_TYPE_DEFAULT;
 import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ENABLE_FILESYSTEM_PATHS;
 import static org.apache.hadoop.ozone.s3.S3GatewayConfigKeys.OZONE_S3G_CLIENT_BUFFER_SIZE_DEFAULT;
 import static org.apache.hadoop.ozone.s3.S3GatewayConfigKeys.OZONE_S3G_CLIENT_BUFFER_SIZE_KEY;
@@ -173,32 +178,34 @@ public class ObjectEndpoint extends EndpointBase {
       String copyHeader = headers.getHeaderString(COPY_SOURCE_HEADER);
       String storageType = headers.getHeaderString(STORAGE_CLASS_HEADER);
 
-      S3StorageType s3StorageType;
-      boolean storageTypeDefault;
+      // Normal put object
+      OzoneBucket bucket = getBucket(bucketName);
+      ReplicationConfig clientConfiguredReplicationConfig = null;
+      String replication = ozoneConfiguration.get(OZONE_REPLICATION);
+      if (replication != null) {
+        clientConfiguredReplicationConfig = ReplicationConfig.parse(
+            ReplicationType.valueOf(ozoneConfiguration
+                .get(OZONE_REPLICATION_TYPE, OZONE_REPLICATION_TYPE_DEFAULT)),
+            replication, ozoneConfiguration);
+      }
+      ReplicationConfig replicationConfig = S3Utils
+          .resolveS3ClientSideReplicationConfig(storageType,
+              clientConfiguredReplicationConfig, bucket.getReplicationConfig());
+      boolean storageTypeDefault = false;
       if (storageType == null || storageType.equals("")) {
-        s3StorageType = S3StorageType.getDefault(ozoneConfiguration);
         storageTypeDefault = true;
-      } else {
-        s3StorageType = toS3StorageType(storageType);
-        storageTypeDefault = false;
       }
-      ReplicationType replicationType = s3StorageType.getType();
-      ReplicationFactor replicationFactor = s3StorageType.getFactor();
 
       if (copyHeader != null) {
         //Copy object, as copy source available.
         CopyObjectResponse copyObjectResponse = copyObject(
-            copyHeader, bucketName, keyPath, replicationType,
-            replicationFactor, storageTypeDefault);
+            copyHeader, bucket, keyPath, replicationConfig, storageTypeDefault);
         return Response.status(Status.OK).entity(copyObjectResponse).header(
             "Connection", "close").build();
       }
 
-      // Normal put object
-      OzoneBucket bucket = getBucket(bucketName);
-
-      output = bucket.createKey(keyPath, length, replicationType,
-          replicationFactor, new HashMap<>());
+      output =
+          bucket.createKey(keyPath, length, replicationConfig, new HashMap<>());
 
       if ("STREAMING-AWS4-HMAC-SHA256-PAYLOAD"
           .equals(headers.getHeaderString("x-amz-content-sha256"))) {
@@ -473,7 +480,7 @@ public class ObjectEndpoint extends EndpointBase {
       if (storageType == null || storageType.equals("")) {
         s3StorageType = S3StorageType.getDefault(ozoneConfiguration);
       } else {
-        s3StorageType = toS3StorageType(storageType);
+        s3StorageType = S3Utils.toS3StorageType(storageType);
       }
       ReplicationType replicationType = s3StorageType.getType();
       ReplicationFactor replicationFactor = s3StorageType.getFactor();
@@ -720,10 +727,9 @@ public class ObjectEndpoint extends EndpointBase {
   }
 
   private CopyObjectResponse copyObject(String copyHeader,
-                                        String destBucket,
+                                        OzoneBucket destBucket,
                                         String destkey,
-                                        ReplicationType replicationType,
-                                        ReplicationFactor replicationFactor,
+                                        ReplicationConfig replicationConfig,
                                         boolean storageTypeDefault)
       throws OS3Exception, IOException {
 
@@ -737,7 +743,8 @@ public class ObjectEndpoint extends EndpointBase {
     try {
       // Checking whether we trying to copying to it self.
 
-      if (sourceBucket.equals(destBucket) && sourceKey.equals(destkey)) {
+      if (sourceBucket.equals(destBucket.getName()) && sourceKey
+          .equals(destkey)) {
         // When copying to same storage type when storage type is provided,
         // we should not throw exception, as aws cli checks if any of the
         // options like storage type are provided or not when source and
@@ -763,15 +770,15 @@ public class ObjectEndpoint extends EndpointBase {
 
 
       OzoneBucket sourceOzoneBucket = getBucket(sourceBucket);
-      OzoneBucket destOzoneBucket = getBucket(destBucket);
+      OzoneBucket destOzoneBucket = destBucket;
 
       OzoneKeyDetails sourceKeyDetails = sourceOzoneBucket.getKey(sourceKey);
       long sourceKeyLen = sourceKeyDetails.getDataSize();
 
       sourceInputStream = sourceOzoneBucket.readKey(sourceKey);
 
-      destOutputStream = destOzoneBucket.createKey(destkey, sourceKeyLen,
-          replicationType, replicationFactor, new HashMap<>());
+      destOutputStream = destOzoneBucket
+          .createKey(destkey, sourceKeyLen, replicationConfig, new HashMap<>());
 
       IOUtils.copy(sourceInputStream, destOutputStream);
 
@@ -838,15 +845,6 @@ public class ObjectEndpoint extends EndpointBase {
     }
   }
 
-  private static S3StorageType toS3StorageType(String storageType)
-      throws OS3Exception {
-    try {
-      return S3StorageType.valueOf(storageType);
-    } catch (IllegalArgumentException ex) {
-      throw newError(INVALID_ARGUMENT, storageType, ex);
-    }
-  }
-
   private static int parsePartNumberMarker(String partNumberMarker) {
     int partMarker = 0;
     if (partNumberMarker != null) {
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
index c9e16d1..d644162 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
@@ -17,11 +17,19 @@
  */
 package org.apache.hadoop.ozone.s3.util;
 
+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.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+
 import java.io.UnsupportedEncodingException;
 import java.net.URLDecoder;
 import java.net.URLEncoder;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.INVALID_ARGUMENT;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.newError;
 
 /**
  * Utilities.
@@ -41,4 +49,71 @@ public final class S3Utils {
   private S3Utils() {
     // no instances
   }
+
+  /**
+   * This API used to resolve the client side configuration preference for file
+   * system layer implementations.
+   *
+   * @param s3StorageTypeHeader        - s3 user passed storage type
+   *                                   header.
+   * @param clientConfiguredReplConfig - Client side configured replication
+   *                                   config.
+   * @param bucketReplConfig           - server side bucket default replication
+   *                                   config.
+   * @return client resolved replication config.
+   */
+  public static ReplicationConfig resolveS3ClientSideReplicationConfig(
+      String s3StorageTypeHeader, ReplicationConfig clientConfiguredReplConfig,
+      ReplicationConfig bucketReplConfig)
+      throws OS3Exception {
+    ReplicationConfig clientDeterminedReplConfig = null;
+
+    // Let's map the user provided s3 storage type header to ozone s3 storage
+    // type.
+    S3StorageType s3StorageType = null;
+    if (s3StorageTypeHeader != null && !s3StorageTypeHeader.equals("")) {
+      s3StorageType = toS3StorageType(s3StorageTypeHeader);
+    }
+
+    boolean isECBucket = bucketReplConfig != null && bucketReplConfig
+        .getReplicationType() == HddsProtos.ReplicationType.EC;
+
+    // if bucket replication config configured with EC, we will give high
+    // preference to server side bucket defaults.
+    // Why we give high preference to EC is, there is no way for file system
+    // interfaces to pass EC replication. So, if one configures EC at bucket,
+    // we consider EC to take preference. in short, keys created from file
+    // system under EC bucket will always be EC'd.
+    if (isECBucket) {
+      // if bucket is EC, don't bother client provided configs, let's pass
+      // bucket config.
+      clientDeterminedReplConfig = bucketReplConfig;
+    } else {
+      // Let's validate the client side available replication configs.
+      boolean isUserPassedReplicationInSupportedList =
+          s3StorageType != null && (s3StorageType.getFactor()
+              .getValue() == ReplicationFactor.ONE.getValue() || s3StorageType
+              .getFactor().getValue() == ReplicationFactor.THREE.getValue());
+      if (isUserPassedReplicationInSupportedList) {
+        clientDeterminedReplConfig = ReplicationConfig.fromProtoTypeAndFactor(
+            ReplicationType.toProto(s3StorageType.getType()),
+            ReplicationFactor.toProto(s3StorageType.getFactor()));
+      } else {
+        // API passed replication number is not in supported replication list.
+        // So, let's use whatever available in client side configured.
+        // By default it will be null, so server will use server defaults.
+        clientDeterminedReplConfig = clientConfiguredReplConfig;
+      }
+    }
+    return clientDeterminedReplConfig;
+  }
+
+  public static S3StorageType toS3StorageType(String storageType)
+      throws OS3Exception {
+    try {
+      return S3StorageType.valueOf(storageType);
+    } catch (IllegalArgumentException ex) {
+      throw newError(INVALID_ARGUMENT, storageType, ex);
+    }
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
index 6a0d428..0f42b3d 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
@@ -32,10 +32,10 @@ import java.util.UUID;
 import java.util.stream.Collectors;
 
 import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
 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.client.StandaloneReplicationConfig;
 import org.apache.hadoop.hdds.protocol.StorageType;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.ozone.OzoneAcl;
@@ -62,6 +62,7 @@ public class OzoneBucketStub extends OzoneBucket {
   private Map<String, Map<Integer, Part>> partList = new HashMap<>();
 
   private ArrayList<OzoneAcl> aclList = new ArrayList<>();
+  private ReplicationConfig replicationConfig;
 
   /**
    * Constructs OzoneBucket instance.
@@ -79,10 +80,11 @@ public class OzoneBucketStub extends OzoneBucket {
       long creationTime) {
     super(volumeName,
         bucketName,
-        new StandaloneReplicationConfig(HddsProtos.ReplicationFactor.ONE),
+        new RatisReplicationConfig(HddsProtos.ReplicationFactor.THREE),
         storageType,
         versioning,
         creationTime);
+    this.replicationConfig = super.getReplicationConfig();
   }
 
   @Override
@@ -119,6 +121,37 @@ public class OzoneBucketStub extends OzoneBucket {
   }
 
   @Override
+  public OzoneOutputStream createKey(String key, long size,
+      ReplicationConfig rConfig, Map<String, String> metadata)
+      throws IOException {
+    final ReplicationConfig repConfig;
+    if (rConfig == null) {
+      repConfig = getReplicationConfig();
+    } else {
+      repConfig = rConfig;
+    }
+    ReplicationConfig finalReplicationCon = repConfig;
+    ByteArrayOutputStream byteArrayOutputStream =
+        new ByteArrayOutputStream((int) size) {
+          @Override
+          public void close() throws IOException {
+            keyContents.put(key, toByteArray());
+            keyDetails.put(key, new OzoneKeyDetails(
+                getVolumeName(),
+                getName(),
+                key,
+                size,
+                System.currentTimeMillis(),
+                System.currentTimeMillis(),
+                new ArrayList<>(), finalReplicationCon, metadata, null
+            ));
+            super.close();
+          }
+        };
+    return new OzoneOutputStream(byteArrayOutputStream);
+  }
+
+  @Override
   public OzoneInputStream readKey(String key) throws IOException {
     return new OzoneInputStream(new ByteArrayInputStream(keyContents.get(key)));
   }
@@ -358,4 +391,14 @@ public class OzoneBucketStub extends OzoneBucket {
       return content.clone();
     }
   }
+
+  @Override
+  public void setReplicationConfig(ReplicationConfig replicationConfig) {
+    this.replicationConfig = replicationConfig;
+  }
+
+  @Override
+  public ReplicationConfig getReplicationConfig() {
+    return this.replicationConfig;
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java
index cf44854..170981ff 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectPut.java
@@ -25,6 +25,7 @@ import javax.ws.rs.core.Response;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.client.OzoneClient;
@@ -101,6 +102,34 @@ public class TestObjectPut {
   }
 
   @Test
+  public void testPutObjectWithECReplicationConfig()
+      throws IOException, OS3Exception {
+    //GIVEN
+    HttpHeaders headers = Mockito.mock(HttpHeaders.class);
+    ByteArrayInputStream body =
+        new ByteArrayInputStream(CONTENT.getBytes(UTF_8));
+    objectEndpoint.setHeaders(headers);
+    ECReplicationConfig ecReplicationConfig =
+        new ECReplicationConfig("rs-3-2-1024K");
+    clientStub.getObjectStore().getS3Bucket(bucketName)
+        .setReplicationConfig(ecReplicationConfig);
+    Response response = objectEndpoint.put(bucketName, keyName, CONTENT
+        .length(), 1, null, body);
+
+    Assert.assertEquals(ecReplicationConfig,
+        clientStub.getObjectStore().getS3Bucket(bucketName).getKey(keyName)
+            .getReplicationConfig());
+    OzoneInputStream ozoneInputStream =
+        clientStub.getObjectStore().getS3Bucket(bucketName)
+            .readKey(keyName);
+    String keyContent =
+        IOUtils.toString(ozoneInputStream, UTF_8);
+
+    Assert.assertEquals(200, response.getStatus());
+    Assert.assertEquals(CONTENT, keyContent);
+  }
+
+  @Test
   public void testPutObjectWithSignedChunks() throws IOException, OS3Exception {
     //GIVEN
     HttpHeaders headers = Mockito.mock(HttpHeaders.class);
@@ -251,11 +280,11 @@ public class TestObjectPut {
 
     objectEndpoint.put(bucketName, keyName, CONTENT
             .length(), 1, null, body);
-
     OzoneKeyDetails key =
         clientStub.getObjectStore().getS3Bucket(bucketName)
             .getKey(keyName);
 
+
     //default type is set
     Assert.assertEquals(ReplicationType.RATIS, key.getReplicationType());
   }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
index 237cc71..4aa5ef1 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java
@@ -268,7 +268,7 @@ public class TestPermissionCheck {
   public void testPutKey() throws IOException {
     Mockito.when(objectStore.getS3Bucket(anyString())).thenReturn(bucket);
     doThrow(exception).when(bucket)
-        .createKey(anyString(), anyLong(), any(), any(), any());
+        .createKey(anyString(), anyLong(), any(), any());
     ObjectEndpoint objectEndpoint = new ObjectEndpoint();
     objectEndpoint.setClient(client);
     objectEndpoint.setHeaders(headers);
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestS3Utils.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestS3Utils.java
new file mode 100644
index 0000000..55f2f38
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestS3Utils.java
@@ -0,0 +1,143 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.
+ */
+package org.apache.hadoop.ozone.s3.util;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests the S3Utils APIs.
+ */
+public class TestS3Utils {
+  private ReplicationConfig ecReplicationConfig =
+      new ECReplicationConfig("rs-3-2-1024K");
+  private ReplicationConfig ratis3ReplicationConfig =
+      new RatisReplicationConfig(HddsProtos.ReplicationFactor.THREE);
+  private ReplicationConfig ratis1ReplicationConfig =
+      new RatisReplicationConfig(HddsProtos.ReplicationFactor.ONE);
+
+  @Test
+  public void testResolveClientSideRepConfigWhenBucketHasEC()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(S3StorageType.STANDARD.name(),
+            null, ecReplicationConfig);
+    // Bucket default is EC.
+    Assert.assertEquals(ecReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket replication is null and it should respect user passed value.
+   */
+  @Test
+  public void testResolveClientSideRepConfigWhenBucketHasNull()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(S3StorageType.STANDARD.name(),
+            null, null);
+    // Passed replication is 3 - Ozone mapped replication is ratis THREE
+    Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket replication is null and it should return null if user passed
+   * value is invalid.
+   */
+  @Test
+  public void testResolveClientSideRepConfigWhenUserPassedReplicationIsEmpty()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig =
+        S3Utils.resolveS3ClientSideReplicationConfig("", null, null);
+    // client configured value also null.
+    // This API caller should leave the decision to server.
+    Assert.assertNull(replicationConfig);
+  }
+
+  /**
+   * When bucket default is non-EC and client side values are not valid, we
+   * would just return null, so servers can make decision in this case.
+   */
+  @Test
+  public void testResolveRepConfWhenUserPassedIsInvalidButBucketDefaultNonEC()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(null, null,
+            ratis3ReplicationConfig);
+    // Configured client config also null.
+    Assert.assertNull(replicationConfig);
+  }
+
+  /**
+   * When bucket default is non-EC and client side value is valid, we
+   * would should return client side valid value.
+   */
+  @Test
+  public void testResolveRepConfWhenUserPassedIsValidButBucketDefaultNonEC()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(
+            S3StorageType.REDUCED_REDUNDANCY.name(), null,
+            ratis3ReplicationConfig);
+    // Passed value is replication one - Ozone mapped value is ratis ONE
+    Assert.assertEquals(ratis1ReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket default is EC and client side value also valid, we would just
+   * return bucket default EC.
+   */
+  @Test
+  public void testResolveRepConfWhenUserPassedIsValidButBucketDefaultEC()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(S3StorageType.STANDARD.name(),
+            ratis3ReplicationConfig, ecReplicationConfig);
+    // Bucket default is EC
+    Assert.assertEquals(ecReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket default is non-EC and client side passed value also not valid
+   * but configured value is valid, we would just return configured value.
+   */
+  @Test
+  public void testResolveRepConfWhenUserPassedIsInvalidAndBucketDefaultNonEC()
+      throws OS3Exception {
+    ReplicationConfig replicationConfig = S3Utils
+        .resolveS3ClientSideReplicationConfig(null, ratis3ReplicationConfig,
+            ratis1ReplicationConfig);
+    // Configured value is ratis THREE
+    Assert.assertEquals(ratis3ReplicationConfig, replicationConfig);
+  }
+
+  /**
+   * When bucket default is non-EC and client side passed value also not valid
+   * but configured value is valid, we would just return configured value.
+   */
+  @Test(expected = OS3Exception.class)
+  public void testResolveRepConfWhenUserPassedIsInvalid() throws OS3Exception {
+    S3Utils.resolveS3ClientSideReplicationConfig("INVALID",
+        ratis3ReplicationConfig, ratis1ReplicationConfig);
+  }
+
+}

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@ozone.apache.org
For additional commands, e-mail: commits-help@ozone.apache.org