Sfoglia il codice sorgente

HDDS-524. Implement PutBucket REST endpoint. Contributed by Bharat Viswanadham.

Bharat Viswanadham 6 anni fa
parent
commit
38fe447d93

+ 55 - 0
hadoop-ozone/dist/src/main/smoketest/s3/bucketv2.robot

@@ -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.
+
+*** Settings ***
+Documentation       S3 gateway test with aws cli for bucket operations
+Library             String
+Library             OperatingSystem
+Resource            commonawslib.robot
+
+*** Variables ***
+${ENDPOINT_URL}       http://s3g:9878
+${OZONE_TEST}         true
+${BUCKET}             generated
+*** Keywords ***
+
+Install aws s3 cli
+                        Execute                    sudo apt-get install -y awscli
+                        Set Environment Variable   AWS_ACCESS_KEY_ID       default
+                        Set Environment Variable   AWS_SECRET_ACCESS_KEY   defaultsecret
+    ${postfix1} =       Generate Random String  5  [NUMBERS]
+    Set Suite Variable  ${BUCKET}                  bucket-${postfix1}
+
+Check Volume
+    # as we know bucket to volume map. Volume name  bucket mapped is s3 + AWS_ACCESS_KEY_ID
+    ${result} =         Execute                     ozone sh volume info /s3default
+                        Should contain              ${result}         s3default
+                        Should not contain          ${result}         VOLUME_NOT_FOUND
+
+*** Test Cases ***
+
+Setup s3 Tests
+    Run Keyword if    '${OZONE_TEST}' == 'true'    Install aws s3 cli
+
+Create Bucket
+    ${result} =         Execute AWSS3APICli         create-bucket --bucket ${BUCKET}
+                        Should contain              ${result}         ${BUCKET}
+                        Should contain              ${result}         Location
+    # create an already existing bucket
+    ${result} =         Execute AWSS3APICli         create-bucket --bucket ${BUCKET}
+                        Should contain              ${result}         ${BUCKET}
+                        Should contain              ${result}         Location
+
+    Run Keyword if     '${OZONE_TEST}' == 'true'    Check Volume

+ 60 - 0
hadoop-ozone/dist/src/main/smoketest/s3/bucketv4.robot

@@ -0,0 +1,60 @@
+# 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.
+
+*** Settings ***
+Documentation       S3 gateway test with aws cli for bucket operations
+Library             String
+Library             OperatingSystem
+Resource            commonawslib.robot
+
+*** Variables ***
+${ENDPOINT_URL}       http://s3g:9878
+${OZONE_TEST}         true
+${BUCKET}             generated
+
+*** Keywords ***
+
+Install aws s3 cli
+                        Execute                    sudo apt-get install -y awscli
+                        Remove Environment Variable    AWS_ACCESS_KEY_ID
+                        Remove Environment Variable    AWS_SECRET_ACCESS_KEY
+                        Execute                    aws configure set default.s3.signature_version s3v4
+                        Execute                    aws configure set aws_access_key_id default1
+                        Execute                    aws configure set aws_secret_access_key defaultsecret
+                        Execute                    aws configure set region us-west-1
+    ${postfix1} =       Generate Random String  5  [NUMBERS]
+    Set Suite Variable  ${BUCKET}                  bucket-${postfix1}
+
+Check Volume
+    # as we know bucket to volume map. Volume name  bucket mapped is s3 + AWS_ACCESS_KEY_ID
+    ${result} =         Execute                     ozone sh volume info /s3default1
+                        Should contain              ${result}         s3default1
+                        Should not contain          ${result}         VOLUME_NOT_FOUND
+
+*** Test Cases ***
+
+Setup s3 Tests
+    Run Keyword if    '${OZONE_TEST}' == 'true'    Install aws s3 cli
+
+Create Bucket
+    ${result} =         Execute AWSS3APICli         create-bucket --bucket ${BUCKET}
+                        Should contain              ${result}         ${BUCKET}
+                        Should contain              ${result}         Location
+    # create an already existing bucket
+    ${result} =         Execute AWSS3APICli         create-bucket --bucket ${BUCKET}
+                        Should contain              ${result}         ${BUCKET}
+                        Should contain              ${result}         Location
+
+    Run Keyword if     '${OZONE_TEST}' == 'true'    Check Volume

+ 23 - 0
hadoop-ozone/dist/src/main/smoketest/s3/commonawslib.robot

@@ -0,0 +1,23 @@
+# 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.
+
+*** Settings ***
+Resource            ../commonlib.robot
+
+*** Keywords ***
+Execute AWSS3APICli
+    [Arguments]       ${command}
+    ${output} =       Execute          aws s3api --endpoint-url ${ENDPOINT_URL} ${command}
+    [return]          ${output}

+ 28 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/EndpointBase.java

@@ -79,6 +79,34 @@ public class EndpointBase {
     return volume;
   }
 
+  /**
+   * Create an S3Bucket, and also it creates mapping needed to access via
+   * ozone and S3.
+   * @param userName
+   * @param bucketName
+   * @return location of the S3Bucket.
+   * @throws IOException
+   */
+  protected String createS3Bucket(String userName, String bucketName) throws
+      IOException {
+    try {
+      client.getObjectStore().createS3Bucket(userName, bucketName);
+    } catch (IOException ex) {
+      LOG.error("createS3Bucket error:", ex);
+      if (!ex.getMessage().contains("ALREADY_EXISTS")) {
+        // S3 does not return error for bucket already exists, it just
+        // returns the location.
+        throw ex;
+      }
+    }
+
+    // Not required to call as bucketname is same, but calling now in future
+    // if mapping changes we get right location.
+    String location = client.getObjectStore().getOzoneBucketName(
+        bucketName);
+    return "/"+location;
+  }
+
   @VisibleForTesting
   public void setClient(OzoneClient ozoneClient) {
     this.client = ozoneClient;

+ 43 - 8
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/PutBucket.java

@@ -20,25 +20,60 @@ package org.apache.hadoop.ozone.s3.bucket;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.PathParam;
-import javax.ws.rs.Produces;
-import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.Response;
 import java.io.IOException;
 
 import org.apache.hadoop.ozone.s3.EndpointBase;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+import org.apache.hadoop.ozone.s3.header.AuthorizationHeaderV2;
+import org.apache.hadoop.ozone.s3.header.AuthorizationHeaderV4;
+import org.apache.http.HttpStatus;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 
 /**
  * Create new bucket.
  */
-@Path("/{volume}/{bucket}")
+@Path("/{bucket}")
 public class PutBucket extends EndpointBase {
 
+  private static final Logger LOG =
+      LoggerFactory.getLogger(PutBucket.class);
+
   @PUT
-  @Produces(MediaType.APPLICATION_XML)
-  public void put(
-      @PathParam("volume") String volumeName,
-      @PathParam("bucket") String bucketName) throws IOException {
+  public Response put(@PathParam("bucket") String bucketName, @Context
+                  HttpHeaders httpHeaders) throws IOException, OS3Exception {
+
+    String auth = httpHeaders.getHeaderString("Authorization");
+    LOG.info("Auth header string {}", auth);
+
+    if (auth == null) {
+      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, S3ErrorTable
+          .Resource.HEADER);
+    }
+
+    String userName;
+    if (auth.startsWith("AWS4")) {
+      LOG.info("V4 Header {}", auth);
+      AuthorizationHeaderV4 authorizationHeader = new AuthorizationHeaderV4(
+          auth);
+      userName = authorizationHeader.getAccessKeyID().toLowerCase();
+    } else {
+      LOG.info("V2 Header {}", auth);
+      AuthorizationHeaderV2 authorizationHeader = new AuthorizationHeaderV2(
+          auth);
+      userName = authorizationHeader.getAccessKeyID().toLowerCase();
+    }
+
+    String location = createS3Bucket(userName, bucketName);
 
-    getVolume(volumeName).createBucket(bucketName);
+    LOG.info("Location is {}", location);
+    return Response.status(HttpStatus.SC_OK).header("Location", location)
+       .build();
 
   }
 }

+ 7 - 1
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java

@@ -45,6 +45,11 @@ public final class S3ErrorTable {
   public static final OS3Exception NO_SUCH_OBJECT = new OS3Exception(
       "NoSuchObject", "The specified object does not exist", HTTP_NOT_FOUND);
 
+
+  public static final OS3Exception MALFORMED_HEADER = new OS3Exception(
+      "AuthorizationHeaderMalformed", "The authorization header you provided " +
+      "is invalid.", HTTP_NOT_FOUND);
+
   /**
    * Create a new instance of Error.
    * @param e Error Template
@@ -63,7 +68,8 @@ public final class S3ErrorTable {
    */
   public enum Resource {
     BUCKET("Bucket"),
-    OBJECT("Object");
+    OBJECT("Object"),
+    HEADER("header");
 
     private final String resource;
 

+ 100 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/AuthorizationHeaderV2.java

@@ -0,0 +1,100 @@
+/**
+ * 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.header;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+
+import static org.apache.commons.lang3.StringUtils.isBlank;
+
+/**
+ * Authorization Header v2.
+ */
+public class AuthorizationHeaderV2 {
+
+  private final static String IDENTIFIER = "AWS";
+  private String authHeader;
+  private String identifier;
+  private String accessKeyID;
+  private String signature;
+
+  public AuthorizationHeaderV2(String auth) throws OS3Exception {
+    Preconditions.checkNotNull(auth);
+    this.authHeader = auth;
+    parseHeader();
+  }
+
+  /**
+   * This method parses the authorization header.
+   *
+   * Authorization header sample:
+   * AWS AKIAIOSFODNN7EXAMPLE:frJIUN8DYpKDtOLCwo//yllqDzg=
+   *
+   * @throws OS3Exception
+   */
+  public void parseHeader() throws OS3Exception {
+    String[] split = authHeader.split(" ");
+    if (split.length != 2) {
+      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, S3ErrorTable
+          .Resource.HEADER);
+    }
+
+    identifier = split[0];
+    if (!IDENTIFIER.equals(identifier)) {
+      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, S3ErrorTable
+          .Resource.HEADER);
+    }
+
+    String[] remainingSplit = split[1].split(":");
+
+    if (remainingSplit.length != 2) {
+      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, S3ErrorTable
+          .Resource.HEADER);
+    }
+
+    accessKeyID = remainingSplit[0];
+    signature = remainingSplit[1];
+    if (isBlank(accessKeyID) || isBlank(signature)) {
+      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, S3ErrorTable
+          .Resource.HEADER);
+    }
+  }
+
+  public String getAuthHeader() {
+    return authHeader;
+  }
+
+  public void setAuthHeader(String authHeader) {
+    this.authHeader = authHeader;
+  }
+
+  public String getIdentifier() {
+    return identifier;
+  }
+
+  public String getAccessKeyID() {
+    return accessKeyID;
+  }
+
+  public String getSignature() {
+    return signature;
+  }
+
+}

+ 147 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/AuthorizationHeaderV4.java

@@ -0,0 +1,147 @@
+/**
+ * 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.header;
+
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+
+/**
+ * S3 Authorization header.
+ * Ref: https://docs.aws.amazon.com/AmazonS3/latest/API/sigv4-auth-using
+ * -authorization-header.html
+ */
+public class AuthorizationHeaderV4 {
+
+  private final static String CREDENTIAL = "Credential=";
+  private final static String SIGNEDHEADERS= "SignedHeaders=";
+  private final static String SIGNATURE = "Signature=";
+
+  private String authHeader;
+  private String algorithm;
+  private String credential;
+  private String signedHeaders;
+  private String signature;
+  private Credential credentialObj;
+
+  /**
+   * Construct AuthorizationHeader object.
+   * @param header
+   */
+  public AuthorizationHeaderV4(String header) throws OS3Exception {
+    Preconditions.checkNotNull(header);
+    this.authHeader = header;
+    parseAuthHeader();
+  }
+
+  /**
+   * This method parses authorization header.
+   *
+   *  Authorization Header sample:
+   *  AWS4-HMAC-SHA256 Credential=AKIAJWFJK62WUTKNFJJA/20181009/us-east-1/s3
+   *  /aws4_request, SignedHeaders=host;x-amz-content-sha256;x-amz-date,
+   * Signature=db81b057718d7c1b3b8dffa29933099551c51d787b3b13b9e0f9ebed45982bf2
+   * @throws OS3Exception
+   */
+  public void parseAuthHeader() throws OS3Exception {
+    String[] split = authHeader.split(" ");
+
+    if (split.length != 4) {
+      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, S3ErrorTable
+          .Resource.HEADER);
+    }
+
+    algorithm = split[0];
+    credential = split[1];
+    signedHeaders = split[2];
+    signature = split[3];
+
+
+    if (credential.startsWith(CREDENTIAL)) {
+      credential = credential.substring(CREDENTIAL.length(), credential
+          .length() - 1);
+    } else {
+      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, S3ErrorTable
+          .Resource.HEADER);
+    }
+
+    if (signedHeaders.startsWith(SIGNEDHEADERS)) {
+      signedHeaders = signedHeaders.substring(SIGNEDHEADERS.length(),
+          signedHeaders.length() - 1);
+    } else {
+      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, S3ErrorTable
+          .Resource.HEADER);
+    }
+
+    if (signature.startsWith(SIGNATURE)) {
+      signature = signature.substring(SIGNATURE.length(), signature
+          .length());
+    } else {
+      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, S3ErrorTable
+          .Resource.HEADER);
+    }
+
+    // Parse credential. Other parts of header are not validated yet. When
+    // security comes, it needs to be completed.
+    credentialObj = new Credential(credential);
+
+  }
+
+  public String getAuthHeader() {
+    return authHeader;
+  }
+
+  public String getAlgorithm() {
+    return algorithm;
+  }
+
+  public String getCredential() {
+    return credential;
+  }
+
+  public String getSignedHeaders() {
+    return signedHeaders;
+  }
+
+  public String getSignature() {
+    return signature;
+  }
+
+  public String getAccessKeyID() {
+    return credentialObj.getAccessKeyID();
+  }
+
+  public String getDate() {
+    return credentialObj.getDate();
+  }
+
+  public String getAwsRegion() {
+    return credentialObj.getAwsRegion();
+  }
+
+  public String getAwsService() {
+    return credentialObj.getAwsService();
+  }
+
+  public String getAwsRequest() {
+    return credentialObj.getAwsRequest();
+  }
+
+}

+ 94 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/Credential.java

@@ -0,0 +1,94 @@
+/**
+ * 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.header;
+
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+
+
+
+/**
+ * Credential in the AWS authorization header.
+ * Ref: https://docs.aws.amazon.com/AmazonS3/latest/API/
+ * sigv4-auth-using-authorization-header.html
+ *
+ */
+public class Credential {
+
+  private String accessKeyID;
+  private String date;
+  private String awsRegion;
+  private String awsService;
+  private String awsRequest;
+  private String credential;
+
+  /**
+   * Construct Credential Object.
+   * @param cred
+   */
+  Credential(String cred) throws OS3Exception {
+    this.credential = cred;
+    parseCredential();
+  }
+
+  /**
+   * Parse credential value.
+   *
+   * Sample credential value:
+   * Credential=AKIAIOSFODNN7EXAMPLE/20130524/us-east-1/s3/aws4_request
+   *
+   * @throws OS3Exception
+   */
+  public void parseCredential() throws OS3Exception {
+    String[] split = credential.split("/");
+    if (split.length == 5) {
+      accessKeyID = split[0];
+      date = split[1];
+      awsRegion = split[2];
+      awsService = split[3];
+      awsRequest = split[4];
+    } else {
+      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, S3ErrorTable
+          .Resource.HEADER);
+    }
+  }
+
+  public String getAccessKeyID() {
+    return accessKeyID;
+  }
+
+  public String getDate() {
+    return date;
+  }
+
+  public String getAwsRegion() {
+    return awsRegion;
+  }
+
+  public String getAwsService() {
+    return awsService;
+  }
+
+  public String getAwsRequest() {
+    return awsRequest;
+  }
+
+  public String getCredential() {
+    return credential;
+  }
+}

+ 22 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/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 Authorization header.
+ */
+package org.apache.hadoop.ozone.s3.header;

+ 88 - 0
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/header/TestAuthorizationHeaderV2.java

@@ -0,0 +1,88 @@
+/**
+ * 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.header;
+
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.junit.Test;
+
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * This class tests Authorization header format v2.
+ */
+public class TestAuthorizationHeaderV2 {
+
+  @Test
+  public void testAuthHeaderV2() throws OS3Exception {
+    try {
+      String auth = "AWS accessKey:signature";
+      AuthorizationHeaderV2 v2 = new AuthorizationHeaderV2(auth);
+      assertEquals(v2.getAccessKeyID(), "accessKey");
+      assertEquals(v2.getSignature(), "signature");
+    } catch (OS3Exception ex) {
+      fail("testAuthHeaderV2 failed");
+    }
+  }
+
+  @Test
+  public void testIncorrectHeader1() throws OS3Exception {
+    try {
+      String auth = "AAA accessKey:signature";
+      new AuthorizationHeaderV2(auth);
+      fail("testIncorrectHeader");
+    } catch (OS3Exception ex) {
+      assertEquals("AuthorizationHeaderMalformed", ex.getCode());
+    }
+  }
+
+  @Test
+  public void testIncorrectHeader2() throws OS3Exception {
+    try {
+      String auth = "AWS :accessKey";
+      new AuthorizationHeaderV2(auth);
+      fail("testIncorrectHeader");
+    } catch (OS3Exception ex) {
+      assertEquals("AuthorizationHeaderMalformed", ex.getCode());
+    }
+  }
+
+  @Test
+  public void testIncorrectHeader3() throws OS3Exception {
+    try {
+      String auth = "AWS :signature";
+      new AuthorizationHeaderV2(auth);
+      fail("testIncorrectHeader");
+    } catch (OS3Exception ex) {
+      assertEquals("AuthorizationHeaderMalformed", ex.getCode());
+    }
+  }
+
+  @Test
+  public void testIncorrectHeader4() throws OS3Exception {
+    try {
+      String auth = "AWS accessKey:";
+      new AuthorizationHeaderV2(auth);
+      fail("testIncorrectHeader");
+    } catch (OS3Exception ex) {
+      assertEquals("AuthorizationHeaderMalformed", ex.getCode());
+    }
+  }
+}

+ 81 - 0
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/header/TestAuthorizationHeaderV4.java

@@ -0,0 +1,81 @@
+/**
+ * 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.header;
+
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+/**
+ * This class tests Authorization header format v2.
+ */
+
+public class TestAuthorizationHeaderV4 {
+
+  @Test
+  public void testV4Header1() {
+    try {
+      String auth = "AWS4-HMAC-SHA256 " +
+          "Credential=ozone/20130524/us-east-1/s3/aws4_request, " +
+          "SignedHeaders=host;range;x-amz-date, " +
+          "Signature=fe5f80f77d5fa3beca038a248ff027";
+      AuthorizationHeaderV4 v4 = new AuthorizationHeaderV4(auth);
+      assertEquals("AWS4-HMAC-SHA256", v4.getAlgorithm());
+      assertEquals("ozone", v4.getAccessKeyID());
+      assertEquals("20130524", v4.getDate());
+      assertEquals("us-east-1", v4.getAwsRegion());
+      assertEquals("aws4_request", v4.getAwsRequest());
+      assertEquals("host;range;x-amz-date", v4.getSignedHeaders());
+      assertEquals("fe5f80f77d5fa3beca038a248ff027", v4.getSignature());
+    } catch (OS3Exception ex) {
+      fail("testV4Header");
+    }
+
+  }
+
+  @Test
+  public void testV4Header2() {
+    try {
+      String auth = "AWS4-HMAC-SHA256 " +
+          "Credential=ozone/20130524/us-east-1/s3/aws4_request, " +
+          "SignedHeaders=host;range;x-amz-date,";
+      AuthorizationHeaderV4 v4 = new AuthorizationHeaderV4(auth);
+      fail("testV4Header2");
+    } catch (OS3Exception ex) {
+      assertEquals("AuthorizationHeaderMalformed", ex.getCode());
+    }
+  }
+
+
+  @Test
+  public void testV4Header3() {
+    try {
+      String auth = "AWS4-HMAC-SHA256 " +
+          "Credential=20130524/us-east-1/s3/aws4_request, " +
+          "SignedHeaders=host;range;x-amz-date, " +
+          "Signature=fe5f80f77d5fa3beca038a248ff027";
+      AuthorizationHeaderV4 v4 = new AuthorizationHeaderV4(auth);
+    } catch (OS3Exception ex) {
+      assertEquals("AuthorizationHeaderMalformed", ex.getCode());
+    }
+  }
+
+}