Browse Source

HDDS-712. Use x-amz-storage-class to specify replication type and replication factor. Contributed by Bharat Viswanadham.

Márton Elek 6 years ago
parent
commit
11291d9c7a

+ 2 - 2
hadoop-ozone/dist/src/main/smoketest/s3/awss3.robot

@@ -29,9 +29,9 @@ ${BUCKET}             generated
 
 File upload and directory list
                         Execute                   date > /tmp/testfile
-    ${result} =         Execute AWSS3Cli          cp /tmp/testfile s3://${BUCKET}
+    ${result} =         Execute AWSS3Cli          cp --storage-class REDUCED_REDUNDANCY /tmp/testfile s3://${BUCKET}
                         Should contain            ${result}         upload
-    ${result} =         Execute AWSS3Cli          cp /tmp/testfile s3://${BUCKET}/dir1/dir2/file
+    ${result} =         Execute AWSS3Cli          cp --storage-class REDUCED_REDUNDANCY /tmp/testfile s3://${BUCKET}/dir1/dir2/file
                         Should contain            ${result}         upload
     ${result} =         Execute AWSS3Cli          ls s3://${BUCKET}
                         Should contain            ${result}         testfile

+ 9 - 9
hadoop-ozone/dist/src/main/smoketest/s3/objectcopy.robot

@@ -39,28 +39,28 @@ Create Dest Bucket
 Copy Object Happy Scenario
     Run Keyword if    '${DESTBUCKET}' == 'generated1'    Create Dest Bucket
                         Execute                    date > /tmp/copyfile
-    ${result} =         Execute AWSS3ApiCli        put-object --bucket ${BUCKET} --key copyobject/f1 --body /tmp/copyfile
+    ${result} =         Execute AWSS3ApiCli        put-object --storage-class REDUCED_REDUNDANCY --bucket ${BUCKET} --key copyobject/f1 --body /tmp/copyfile
     ${result} =         Execute AWSS3ApiCli        list-objects --bucket ${BUCKET} --prefix copyobject/
                         Should contain             ${result}         f1
 
-    ${result} =         Execute AWSS3ApiCli        copy-object --bucket ${DESTBUCKET} --key copyobject/f1 --copy-source ${BUCKET}/copyobject/f1
+    ${result} =         Execute AWSS3ApiCli        copy-object --storage-class REDUCED_REDUNDANCY --bucket ${DESTBUCKET} --key copyobject/f1 --copy-source ${BUCKET}/copyobject/f1
     ${result} =         Execute AWSS3ApiCli        list-objects --bucket ${DESTBUCKET} --prefix copyobject/
                         Should contain             ${result}         f1
     #copying again will not throw error
-    ${result} =         Execute AWSS3ApiCli        copy-object --bucket ${DESTBUCKET} --key copyobject/f1 --copy-source ${BUCKET}/copyobject/f1
+    ${result} =         Execute AWSS3ApiCli        copy-object --storage-class REDUCED_REDUNDANCY --bucket ${DESTBUCKET} --key copyobject/f1 --copy-source ${BUCKET}/copyobject/f1
     ${result} =         Execute AWSS3ApiCli        list-objects --bucket ${DESTBUCKET} --prefix copyobject/
                         Should contain             ${result}         f1
 
 Copy Object Where Bucket is not available
-    ${result} =         Execute AWSS3APICli and checkrc        copy-object --bucket dfdfdfdfdfnonexistent --key copyobject/f1 --copy-source ${BUCKET}/copyobject/f1      255
+    ${result} =         Execute AWSS3APICli and checkrc        copy-object --storage-class REDUCED_REDUNDANCY --bucket dfdfdfdfdfnonexistent --key copyobject/f1 --copy-source ${BUCKET}/copyobject/f1      255
                         Should contain             ${result}        NoSuchBucket
-    ${result} =         Execute AWSS3APICli and checkrc        copy-object --bucket ${DESTBUCKET} --key copyobject/f1 --copy-source dfdfdfdfdfnonexistent/copyobject/f1  255
+    ${result} =         Execute AWSS3APICli and checkrc        copy-object --storage-class REDUCED_REDUNDANCY --bucket ${DESTBUCKET} --key copyobject/f1 --copy-source dfdfdfdfdfnonexistent/copyobject/f1  255
                         Should contain             ${result}        NoSuchBucket
 
-Copy Object Where both source and dest are same
-     ${result} =         Execute AWSS3APICli and checkrc        copy-object --bucket ${DESTBUCKET} --key copyobject/f1 --copy-source ${DESTBUCKET}/copyobject/f1      255
-                         Should contain             ${result}        InvalidRequest
+Copy Object Where both source and dest are same with change to storageclass
+     ${result} =         Execute AWSS3APICli        copy-object --storage-class REDUCED_REDUNDANCY --bucket ${DESTBUCKET} --key copyobject/f1 --copy-source ${DESTBUCKET}/copyobject/f1
+                         Should contain             ${result}        ETag
 
 Copy Object Where Key not available
-    ${result} =         Execute AWSS3APICli and checkrc        copy-object --bucket ${DESTBUCKET} --key copyobject/f1 --copy-source ${BUCKET}/nonnonexistentkey       255
+    ${result} =         Execute AWSS3APICli and checkrc        copy-object --storage-class REDUCED_REDUNDANCY --bucket ${DESTBUCKET} --key copyobject/f1 --copy-source ${BUCKET}/nonnonexistentkey       255
                         Should contain             ${result}        NoSuchKey

+ 3 - 3
hadoop-ozone/dist/src/main/smoketest/s3/objectdelete.robot

@@ -28,7 +28,7 @@ ${BUCKET}             generated
 *** Test Cases ***
 Delete file with s3api
                         Execute                    date > /tmp/testfile
-    ${result} =         Execute AWSS3ApiCli        put-object --bucket ${BUCKET} --key deletetestapi/f1 --body /tmp/testfile
+    ${result} =         Execute AWSS3ApiCli        put-object --storage-class REDUCED_REDUNDANCY --bucket ${BUCKET} --key deletetestapi/f1 --body /tmp/testfile
     ${result} =         Execute AWSS3ApiCli        list-objects --bucket ${BUCKET} --prefix deletetestapi/
                         Should contain             ${result}         f1
     ${result} =         Execute AWSS3APICli        delete-object --bucket ${BUCKET} --key deletetestapi/f1
@@ -46,7 +46,7 @@ Delete file with s3api, file doesn't exist
 
 Delete dir with s3api
                         Execute                    date > /tmp/testfile
-    ${result} =         Execute AWSS3Cli           cp /tmp/testfile s3://${BUCKET}/deletetestapidir/f1
+    ${result} =         Execute AWSS3Cli           cp --storage-class REDUCED_REDUNDANCY /tmp/testfile s3://${BUCKET}/deletetestapidir/f1
     ${result} =         Execute AWSS3Cli           ls s3://${BUCKET}/deletetestapidir/
                         Should contain             ${result}         f1
     ${result} =         Execute AWSS3APICli        delete-object --bucket ${BUCKET} --key deletetestapidir/
@@ -57,7 +57,7 @@ Delete dir with s3api
 
 Delete file with s3api, file doesn't exist, prefix of a real file
                         Execute                    date > /tmp/testfile
-    ${result} =         Execute AWSS3Cli           cp /tmp/testfile s3://${BUCKET}/deletetestapiprefix/filefile
+    ${result} =         Execute AWSS3Cli           cp --storage-class REDUCED_REDUNDANCY /tmp/testfile s3://${BUCKET}/deletetestapiprefix/filefile
     ${result} =         Execute AWSS3Cli           ls s3://${BUCKET}/deletetestapiprefix/
                         Should contain             ${result}         filefile
     ${result} =         Execute AWSS3APICli        delete-object --bucket ${BUCKET} --key deletetestapiprefix/file

+ 3 - 3
hadoop-ozone/dist/src/main/smoketest/s3/objectmultidelete.robot

@@ -29,9 +29,9 @@ ${BUCKET}             generated
 
 Delete file with multi delete
                         Execute                    date > /tmp/testfile
-    ${result} =         Execute AWSS3ApiCli        put-object --bucket ${BUCKET} --key multidelete/f1 --body /tmp/testfile
-    ${result} =         Execute AWSS3ApiCli        put-object --bucket ${BUCKET} --key multidelete/f2 --body /tmp/testfile
-    ${result} =         Execute AWSS3ApiCli        put-object --bucket ${BUCKET} --key multidelete/f3 --body /tmp/testfile
+    ${result} =         Execute AWSS3ApiCli        put-object --storage-class REDUCED_REDUNDANCY --bucket ${BUCKET} --key multidelete/f1 --body /tmp/testfile
+    ${result} =         Execute AWSS3ApiCli        put-object --storage-class REDUCED_REDUNDANCY --bucket ${BUCKET} --key multidelete/f2 --body /tmp/testfile
+    ${result} =         Execute AWSS3ApiCli        put-object --storage-class REDUCED_REDUNDANCY --bucket ${BUCKET} --key multidelete/f3 --body /tmp/testfile
     ${result} =         Execute AWSS3ApiCli        list-objects --bucket ${BUCKET} --prefix multidelete/
                         Should contain             ${result}         multidelete/f1
                         Should contain             ${result}         multidelete/f2

+ 1 - 1
hadoop-ozone/dist/src/main/smoketest/s3/objectputget.robot

@@ -30,7 +30,7 @@ ${BUCKET}             generated
 
 Put object to s3
                         Execute                    date > /tmp/testfile
-    ${result} =         Execute AWSS3ApiCli        put-object --bucket ${BUCKET} --key putobject/f1 --body /tmp/testfile
+    ${result} =         Execute AWSS3ApiCli        put-object --storage-class REDUCED_REDUNDANCY --bucket ${BUCKET} --key putobject/f1 --body /tmp/testfile
     ${result} =         Execute AWSS3ApiCli        list-objects --bucket ${BUCKET} --prefix putobject/
                         Should contain             ${result}         f1
 

+ 52 - 16
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java

@@ -18,14 +18,12 @@
 package org.apache.hadoop.ozone.s3.endpoint;
 
 import javax.ws.rs.DELETE;
-import javax.ws.rs.DefaultValue;
 import javax.ws.rs.GET;
 import javax.ws.rs.HEAD;
 import javax.ws.rs.HeaderParam;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
-import javax.ws.rs.QueryParam;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Response;
@@ -53,11 +51,16 @@ import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.ozone.s3.util.S3StorageType;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.http.HttpStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.ozone.s3.util.S3Consts.COPY_SOURCE_HEADER;
+import static org.apache.hadoop.ozone.s3.util.S3Consts.STORAGE_CLASS_HEADER;
+
 /**
  * Key level rest endpoints.
  */
@@ -91,22 +94,37 @@ public class ObjectEndpoint extends EndpointBase {
   public Response put(
       @PathParam("bucket") String bucketName,
       @PathParam("path") String keyPath,
-      @DefaultValue("STAND_ALONE") @QueryParam("replicationType")
-          ReplicationType replicationType,
-      @DefaultValue("ONE") @QueryParam("replicationFactor")
-          ReplicationFactor replicationFactor,
       @HeaderParam("Content-Length") long length,
       InputStream body) throws IOException, OS3Exception {
 
     OzoneOutputStream output = null;
     try {
-      String copyHeader = headers.getHeaderString("x-amz-copy-source");
+      String copyHeader = headers.getHeaderString(COPY_SOURCE_HEADER);
+      String storageType = headers.getHeaderString(STORAGE_CLASS_HEADER);
+
+      ReplicationType replicationType;
+      ReplicationFactor replicationFactor;
+      boolean storageTypeDefault;
+      if (storageType == null) {
+        replicationType = S3StorageType.getDefault().getType();
+        replicationFactor = S3StorageType.getDefault().getFactor();
+        storageTypeDefault = true;
+      } else {
+        try {
+          replicationType = S3StorageType.valueOf(storageType).getType();
+          replicationFactor = S3StorageType.valueOf(storageType).getFactor();
+        } catch (IllegalArgumentException ex) {
+          throw S3ErrorTable.newError(S3ErrorTable.INVALID_ARGUMENT,
+              storageType);
+        }
+        storageTypeDefault = false;
+      }
 
       if (copyHeader != null) {
         //Copy object, as copy source available.
         CopyObjectResponse copyObjectResponse = copyObject(
             copyHeader, bucketName, keyPath, replicationType,
-            replicationFactor);
+            replicationFactor, storageTypeDefault);
         return Response.status(Status.OK).entity(copyObjectResponse).header(
             "Connection", "close").build();
       }
@@ -254,7 +272,8 @@ public class ObjectEndpoint extends EndpointBase {
                                         String destBucket,
                                         String destkey,
                                         ReplicationType replicationType,
-                                        ReplicationFactor replicationFactor)
+                                        ReplicationFactor replicationFactor,
+                                        boolean storageTypeDefault)
       throws OS3Exception, IOException {
 
     if (copyHeader.startsWith("/")) {
@@ -276,18 +295,35 @@ public class ObjectEndpoint extends EndpointBase {
     boolean closed = false;
     try {
       // Checking whether we trying to copying to it self.
+
       if (sourceBucket.equals(destBucket)) {
         if (sourceKey.equals(destkey)) {
-          OS3Exception ex = S3ErrorTable.newError(S3ErrorTable
-              .INVALID_REQUEST, copyHeader);
-          ex.setErrorMessage("This copy request is illegal because it is " +
-              "trying to copy an object to it self itself without changing " +
-              "the object's metadata, storage class, website redirect " +
-              "location or encryption attributes.");
-          throw ex;
+          // 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
+          // dest are given same
+          if (storageTypeDefault) {
+            OS3Exception ex = S3ErrorTable.newError(S3ErrorTable
+                .INVALID_REQUEST, copyHeader);
+            ex.setErrorMessage("This copy request is illegal because it is " +
+                "trying to copy an object to it self itself without changing " +
+                "the object's metadata, storage class, website redirect " +
+                "location or encryption attributes.");
+            throw ex;
+          } else {
+            // TODO: Actually here we should change storage type, as ozone
+            // still does not support this just returning dummy response
+            // for now
+            CopyObjectResponse copyObjectResponse = new CopyObjectResponse();
+            copyObjectResponse.setETag(OzoneUtils.getRequestID());
+            copyObjectResponse.setLastModified(Instant.ofEpochMilli(
+                Time.now()));
+            return copyObjectResponse;
+          }
         }
       }
 
+
       OzoneBucket sourceOzoneBucket = getBucket(sourceBucket);
       OzoneBucket destOzoneBucket = getBucket(destBucket);
 

+ 19 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Consts.java

@@ -0,0 +1,19 @@
+package org.apache.hadoop.ozone.s3.util;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Set of constants used for S3 implementation.
+ */
+@InterfaceAudience.Private
+public final class S3Consts {
+
+  //Never Constructed
+  private S3Consts() {
+
+  }
+
+  public static final String COPY_SOURCE_HEADER = "x-amz-copy-source";
+  public static final String STORAGE_CLASS_HEADER = "x-amz-storage-class";
+
+}

+ 55 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3StorageType.java

@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+
+package org.apache.hadoop.ozone.s3.util;
+
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+
+/**
+ * Maps S3 storage class values to Ozone replication values.
+ */
+
+public enum S3StorageType {
+
+  REDUCED_REDUNDANCY(ReplicationType.STAND_ALONE, ReplicationFactor.ONE),
+  STANDARD(ReplicationType.RATIS, ReplicationFactor.THREE);
+
+  private final ReplicationType type;
+  private final ReplicationFactor factor;
+
+  S3StorageType(
+      ReplicationType type,
+      ReplicationFactor factor) {
+    this.type = type;
+    this.factor = factor;
+  }
+
+  public ReplicationFactor getFactor() {
+    return factor;
+  }
+
+  public ReplicationType getType() {
+    return type;
+  }
+
+  public static S3StorageType getDefault() {
+    return STANDARD;
+  }
+
+}

+ 22 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains Ozone S3 Util classes.
+ */
+package org.apache.hadoop.ozone.s3.util;

+ 39 - 30
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPutObject.java

@@ -26,19 +26,21 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.nio.charset.Charset;
 
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.ozone.client.ObjectStore;
 import org.apache.hadoop.ozone.client.OzoneClientStub;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import static org.apache.hadoop.ozone.s3.util.S3Consts.COPY_SOURCE_HEADER;
+import static org.apache.hadoop.ozone.s3.util.S3Consts.STORAGE_CLASS_HEADER;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.when;
 
@@ -80,9 +82,9 @@ public class TestPutObject {
     objectEndpoint.setHeaders(headers);
 
     //WHEN
-    Response response = objectEndpoint.put(bucketName, keyName,
-        ReplicationType.STAND_ALONE, ReplicationFactor.ONE, CONTENT.length(),
-        body);
+    Response response = objectEndpoint.put(bucketName, keyName, CONTENT
+        .length(), body);
+
 
     //THEN
     String volumeName = clientStub.getObjectStore()
@@ -113,8 +115,6 @@ public class TestPutObject {
 
     //WHEN
     Response response = objectEndpoint.put(bucketName, keyName,
-        ReplicationType.STAND_ALONE,
-        ReplicationFactor.ONE,
         chunkedContent.length(),
         new ByteArrayInputStream(chunkedContent.getBytes()));
 
@@ -140,8 +140,7 @@ public class TestPutObject {
     keyName = "sourceKey";
 
     Response response = objectEndpoint.put(bucketName, keyName,
-        ReplicationType.STAND_ALONE, ReplicationFactor.ONE, CONTENT.length(),
-        body);
+        CONTENT.length(), body);
 
     String volumeName = clientStub.getObjectStore().getOzoneVolumeName(
         bucketName);
@@ -157,12 +156,10 @@ public class TestPutObject {
 
 
     // Add copy header, and then call put
-    when(headers.getHeaderString("x-amz-copy-source")).thenReturn(
+    when(headers.getHeaderString(COPY_SOURCE_HEADER)).thenReturn(
         bucketName  + "/" + keyName);
 
-    response = objectEndpoint.put(destBucket, destkey,
-        ReplicationType.STAND_ALONE, ReplicationFactor.ONE, CONTENT.length(),
-        body);
+    response = objectEndpoint.put(destBucket, destkey, CONTENT.length(), body);
 
     // Check destination key and response
     volumeName = clientStub.getObjectStore().getOzoneVolumeName(destBucket);
@@ -176,8 +173,7 @@ public class TestPutObject {
 
     // source and dest same
     try {
-      objectEndpoint.put(bucketName, keyName, ReplicationType.STAND_ALONE,
-          ReplicationFactor.ONE, CONTENT.length(), body);
+      objectEndpoint.put(bucketName, keyName, CONTENT.length(), body);
       fail("test copy object failed");
     } catch (OS3Exception ex) {
       Assert.assertTrue(ex.getErrorMessage().contains("This copy request is " +
@@ -186,10 +182,9 @@ public class TestPutObject {
 
     // source bucket not found
     try {
-      when(headers.getHeaderString("x-amz-copy-source")).thenReturn(
+      when(headers.getHeaderString(COPY_SOURCE_HEADER)).thenReturn(
           nonexist + "/"  + keyName);
-      response = objectEndpoint.put(destBucket, destkey,
-          ReplicationType.STAND_ALONE, ReplicationFactor.ONE, CONTENT.length(),
+      objectEndpoint.put(destBucket, destkey, CONTENT.length(),
           body);
       fail("test copy object failed");
     } catch (OS3Exception ex) {
@@ -198,11 +193,9 @@ public class TestPutObject {
 
     // dest bucket not found
     try {
-      when(headers.getHeaderString("x-amz-copy-source")).thenReturn(
+      when(headers.getHeaderString(COPY_SOURCE_HEADER)).thenReturn(
           bucketName + "/" + keyName);
-      response = objectEndpoint.put(nonexist, destkey,
-          ReplicationType.STAND_ALONE, ReplicationFactor.ONE, CONTENT.length(),
-          body);
+      objectEndpoint.put(nonexist, destkey, CONTENT.length(), body);
       fail("test copy object failed");
     } catch (OS3Exception ex) {
       Assert.assertTrue(ex.getCode().contains("NoSuchBucket"));
@@ -210,11 +203,9 @@ public class TestPutObject {
 
     //Both source and dest bucket not found
     try {
-      when(headers.getHeaderString("x-amz-copy-source")).thenReturn(
+      when(headers.getHeaderString(COPY_SOURCE_HEADER)).thenReturn(
           nonexist + "/" + keyName);
-      response = objectEndpoint.put(nonexist, destkey,
-          ReplicationType.STAND_ALONE, ReplicationFactor.ONE, CONTENT.length(),
-          body);
+      objectEndpoint.put(nonexist, destkey, CONTENT.length(), body);
       fail("test copy object failed");
     } catch (OS3Exception ex) {
       Assert.assertTrue(ex.getCode().contains("NoSuchBucket"));
@@ -222,15 +213,33 @@ public class TestPutObject {
 
     // source key not found
     try {
-      when(headers.getHeaderString("x-amz-copy-source")).thenReturn(
+      when(headers.getHeaderString(COPY_SOURCE_HEADER)).thenReturn(
           bucketName + "/" + nonexist);
-      response = objectEndpoint.put("nonexistent", keyName,
-          ReplicationType.STAND_ALONE, ReplicationFactor.ONE, CONTENT.length(),
-          body);
+      objectEndpoint.put("nonexistent", keyName, CONTENT.length(), body);
       fail("test copy object failed");
     } catch (OS3Exception ex) {
       Assert.assertTrue(ex.getCode().contains("NoSuchBucket"));
     }
 
   }
+
+  @Test
+  public void testInvalidStorageType() throws IOException {
+    HttpHeaders headers = Mockito.mock(HttpHeaders.class);
+    ByteArrayInputStream body = new ByteArrayInputStream(CONTENT.getBytes());
+    objectEndpoint.setHeaders(headers);
+    keyName = "sourceKey";
+    when(headers.getHeaderString(STORAGE_CLASS_HEADER)).thenReturn("random");
+
+    try {
+      Response response = objectEndpoint.put(bucketName, keyName,
+          CONTENT.length(), body);
+      fail("testInvalidStorageType");
+    } catch (OS3Exception ex) {
+      assertEquals(S3ErrorTable.INVALID_ARGUMENT.getErrorMessage(),
+          ex.getErrorMessage());
+      assertEquals("random", ex.getResource());
+    }
+
+  }
 }