Parcourir la source

HDDS-1177. Add validation to AuthorizationHeaderV4. Contributed by Ajay Kumar

Ajay Kumar il y a 6 ans
Parent
commit
625e93713b

+ 67 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/AWSConstants.java

@@ -0,0 +1,67 @@
+/**
+ * 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.classification.InterfaceAudience;
+
+import java.time.format.DateTimeFormatter;
+
+/**
+ * AWS constants.
+ */
+@InterfaceAudience.Private
+public final class AWSConstants {
+
+  private AWSConstants() {
+  }
+
+  public static final String LINE_SEPARATOR = "\n";
+
+  public static final String AWS4_TERMINATOR = "aws4_request";
+
+  public static final String AWS4_SIGNING_ALGORITHM = "AWS4-HMAC-SHA256";
+
+  /**
+   * Seconds in a week, which is the max expiration time Sig-v4 accepts.
+   */
+  public static final long PRESIGN_URL_MAX_EXPIRATION_SECONDS =
+      60 * 60 * 24 * 7;
+
+  public static final String X_AMZ_SECURITY_TOKEN = "X-Amz-Security-Token";
+
+  public static final String X_AMZ_CREDENTIAL = "X-Amz-Credential";
+
+  public static final String X_AMZ_DATE = "X-Amz-Date";
+
+  public static final String X_AMZ_EXPIRES = "X-Amz-Expires";
+
+  public static final String X_AMZ_SIGNED_HEADER = "X-Amz-SignedHeaders";
+
+  public static final String X_AMZ_CONTENT_SHA256 = "x-amz-content-sha256";
+
+  public static final String X_AMZ_SIGNATURE = "X-Amz-Signature";
+
+  public static final String X_AMZ_ALGORITHM = "X-Amz-Algorithm";
+
+  public static final String AUTHORIZATION = "Authorization";
+
+  public static final String HOST = "Host";
+
+  public static final DateTimeFormatter DATE_FORMATTER =
+      DateTimeFormatter.ofPattern("yyyyMMdd");
+
+}

+ 118 - 18
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/AuthorizationHeaderV4.java

@@ -20,8 +20,23 @@ package org.apache.hadoop.ozone.s3.header;
 
 
 import com.google.common.base.Preconditions;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.time.LocalDate;
+import java.util.Collection;
+
+import static java.time.temporal.ChronoUnit.DAYS;
+import static org.apache.commons.lang3.StringUtils.isAllEmpty;
+import static org.apache.commons.lang3.StringUtils.isNoneEmpty;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.MALFORMED_HEADER;
+import static org.apache.hadoop.ozone.s3.header.AWSConstants.AWS4_SIGNING_ALGORITHM;
+import static org.apache.hadoop.ozone.s3.header.AWSConstants.DATE_FORMATTER;
 
 /**
  * S3 Authorization header.
@@ -29,6 +44,8 @@ import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
  * -authorization-header.html
  */
 public class AuthorizationHeaderV4 {
+  private final static Logger LOG = LoggerFactory.getLogger(
+      AuthorizationHeaderV4.class);
 
   private final static String CREDENTIAL = "Credential=";
   private final static String SIGNEDHEADERS= "SignedHeaders=";
@@ -37,9 +54,10 @@ public class AuthorizationHeaderV4 {
   private String authHeader;
   private String algorithm;
   private String credential;
-  private String signedHeaders;
+  private String signedHeadersStr;
   private String signature;
   private Credential credentialObj;
+  private Collection<String> signedHeaders;
 
   /**
    * Construct AuthorizationHeader object.
@@ -64,43 +82,125 @@ public class AuthorizationHeaderV4 {
   public void parseAuthHeader() throws OS3Exception {
     int firstSep = authHeader.indexOf(' ');
     if (firstSep < 0) {
-      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
+      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
     }
 
     //split the value parts of the authorization header
     String[] split = authHeader.substring(firstSep + 1).trim().split(", *");
 
     if (split.length != 3) {
-      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
+      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
     }
 
     algorithm = authHeader.substring(0, firstSep);
+    validateAlgorithm();
     credential = split[0];
-    signedHeaders = split[1];
+    signedHeadersStr = split[1];
     signature = split[2];
+    validateCredentials();
+    validateSignedHeaders();
+    validateSignature();
 
-    if (credential.startsWith(CREDENTIAL)) {
-      credential = credential.substring(CREDENTIAL.length());
-    } else {
-      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
-    }
+  }
 
-    if (signedHeaders.startsWith(SIGNEDHEADERS)) {
-      signedHeaders = signedHeaders.substring(SIGNEDHEADERS.length());
+  /**
+   * Validate Signed headers.
+   * */
+  private void validateSignedHeaders() throws OS3Exception {
+    if (isNoneEmpty(signedHeadersStr)
+        && signedHeadersStr.startsWith(SIGNEDHEADERS)) {
+      signedHeadersStr = signedHeadersStr.substring(SIGNEDHEADERS.length());
+      signedHeaders = StringUtils.getStringCollection(signedHeadersStr, ";");
+      if (signedHeaders.size() == 0) {
+        LOG.error("No signed headers found. Authheader:{}", authHeader);
+        throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      }
     } else {
-      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
+      LOG.error("No signed headers found. Authheader:{}", authHeader);
+      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
     }
+  }
 
+  /**
+   * Validate signature.
+   * */
+  private void validateSignature() throws OS3Exception {
     if (signature.startsWith(SIGNATURE)) {
       signature = signature.substring(SIGNATURE.length());
+      if (!isNoneEmpty(signature)) {
+        LOG.error("Signature can't be empty.", signature);
+        throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      }
+      try {
+        Hex.decodeHex(signature);
+      } catch (DecoderException e) {
+        LOG.error("Signature:{} should be in hexa-decimal encoding.",
+            signature);
+        throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      }
     } else {
-      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
+      LOG.error("Signature can't be empty.", signature);
+      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
     }
+  }
 
-    // Parse credential. Other parts of header are not validated yet. When
-    // security comes, it needs to be completed.
-    credentialObj = new Credential(credential);
+  /**
+   * Validate credentials.
+   * */
+  private void validateCredentials() throws OS3Exception {
+    if (isNoneEmpty(credential) && credential.startsWith(CREDENTIAL)) {
+      credential = credential.substring(CREDENTIAL.length());
+      // Parse credential. Other parts of header are not validated yet. When
+      // security comes, it needs to be completed.
+      credentialObj = new Credential(credential);
+    } else {
+      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+    }
+
+    if (credentialObj.getAccessKeyID().isEmpty()) {
+      LOG.error("AWS access id shouldn't be empty. credential:{}", credential);
+      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+    }
+    if (credentialObj.getAwsRegion().isEmpty()) {
+      LOG.error("AWS region shouldn't be empty. credential:{}", credential);
+      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+    }
+    if (credentialObj.getAwsRequest().isEmpty()) {
+      LOG.error("AWS request shouldn't be empty. credential:{}", credential);
+      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+    }
+    if (credentialObj.getAwsService().isEmpty()) {
+      LOG.error("AWS service:{} shouldn't be empty. credential:{}",
+          credential);
+      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+    }
 
+    // Date should not be empty and within valid range.
+    if (!credentialObj.getDate().isEmpty()) {
+      LocalDate date = LocalDate.parse(credentialObj.getDate(), DATE_FORMATTER);
+      LocalDate now = LocalDate.now();
+      if (date.isBefore(now.minus(1, DAYS)) ||
+          date.isAfter(now.plus(1, DAYS))) {
+        LOG.error("AWS date not in valid range. Date:{} should not be older " +
+                "than 1 day(i.e yesterday) and greater than 1 day(i.e " +
+                "tomorrow).",
+            getDate());
+        throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      }
+    } else {
+      LOG.error("AWS date shouldn't be empty. credential:{}", credential);
+      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+    }
+  }
+
+  /**
+   * Validate if algorithm is in expected format.
+   * */
+  private void validateAlgorithm() throws OS3Exception {
+    if (isAllEmpty(algorithm) || !algorithm.equals(AWS4_SIGNING_ALGORITHM)) {
+      LOG.error("Unexpected hash algorithm. Algo:{}", algorithm);
+      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+    }
   }
 
   public String getAuthHeader() {
@@ -115,8 +215,8 @@ public class AuthorizationHeaderV4 {
     return credential;
   }
 
-  public String getSignedHeaders() {
-    return signedHeaders;
+  public String getSignedHeaderString() {
+    return signedHeadersStr;
   }
 
   public String getSignature() {

+ 10 - 6
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/Credential.java

@@ -19,7 +19,8 @@ package org.apache.hadoop.ozone.s3.header;
 
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
@@ -29,6 +30,7 @@ import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
  *
  */
 public class Credential {
+  private static final Logger LOG = LoggerFactory.getLogger(Credential.class);
 
   private String accessKeyID;
   private String date;
@@ -58,12 +60,14 @@ public class Credential {
   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];
+      accessKeyID = split[0].trim();
+      date = split[1].trim();
+      awsRegion = split[2].trim();
+      awsService = split[3].trim();
+      awsRequest = split[4].trim();
     } else {
+      LOG.error("Credentials not in expected format. credential:{}",
+          credential);
       throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, credential);
     }
   }

+ 265 - 8
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/header/TestAuthorizationHeaderV4.java

@@ -19,8 +19,14 @@
 package org.apache.hadoop.ozone.s3.header;
 
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.Before;
 import org.junit.Test;
 
+import java.time.LocalDate;
+
+import static java.time.temporal.ChronoUnit.DAYS;
+import static org.apache.hadoop.ozone.s3.header.AWSConstants.DATE_FORMATTER;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
@@ -29,20 +35,27 @@ import static org.junit.Assert.fail;
  */
 
 public class TestAuthorizationHeaderV4 {
+  private String curDate;
+
+  @Before
+  public void setup() {
+    LocalDate now = LocalDate.now();
+    curDate = DATE_FORMATTER.format(now);
+  }
 
   @Test
   public void testV4HeaderWellFormed() throws Exception {
     String auth = "AWS4-HMAC-SHA256 " +
-        "Credential=ozone/20130524/us-east-1/s3/aws4_request, " +
+        "Credential=ozone/" + curDate + "/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(curDate, v4.getDate());
     assertEquals("us-east-1", v4.getAwsRegion());
     assertEquals("aws4_request", v4.getAwsRequest());
-    assertEquals("host;range;x-amz-date", v4.getSignedHeaders());
+    assertEquals("host;range;x-amz-date", v4.getSignedHeaderString());
     assertEquals("fe5f80f77d5fa3beca038a248ff027", v4.getSignature());
   }
 
@@ -50,7 +63,7 @@ public class TestAuthorizationHeaderV4 {
   public void testV4HeaderMissingParts() {
     try {
       String auth = "AWS4-HMAC-SHA256 " +
-          "Credential=ozone/20130524/us-east-1/s3/aws4_request, " +
+          "Credential=ozone/" + curDate + "/us-east-1/s3/aws4_request, " +
           "SignedHeaders=host;range;x-amz-date,";
       AuthorizationHeaderV4 v4 = new AuthorizationHeaderV4(auth);
       fail("Exception is expected in case of malformed header");
@@ -63,7 +76,7 @@ public class TestAuthorizationHeaderV4 {
   public void testV4HeaderInvalidCredential() {
     try {
       String auth = "AWS4-HMAC-SHA256 " +
-          "Credential=20130524/us-east-1/s3/aws4_request, " +
+          "Credential=" + curDate + "/us-east-1/s3/aws4_request, " +
           "SignedHeaders=host;range;x-amz-date, " +
           "Signature=fe5f80f77d5fa3beca038a248ff027";
       AuthorizationHeaderV4 v4 = new AuthorizationHeaderV4(auth);
@@ -77,7 +90,8 @@ public class TestAuthorizationHeaderV4 {
   public void testV4HeaderWithoutSpace() throws OS3Exception {
 
     String auth =
-        "AWS4-HMAC-SHA256 Credential=ozone/20130524/us-east-1/s3/aws4_request,"
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/aws4_request,"
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
@@ -85,13 +99,256 @@ public class TestAuthorizationHeaderV4 {
 
     assertEquals("AWS4-HMAC-SHA256", v4.getAlgorithm());
     assertEquals("ozone", v4.getAccessKeyID());
-    assertEquals("20130524", v4.getDate());
+    assertEquals(curDate, v4.getDate());
     assertEquals("us-east-1", v4.getAwsRegion());
     assertEquals("aws4_request", v4.getAwsRequest());
     assertEquals("host;x-amz-content-sha256;x-amz-date",
-        v4.getSignedHeaders());
+        v4.getSignedHeaderString());
     assertEquals("fe5f80f77d5fa3beca038a248ff027", v4.getSignature());
 
   }
 
+  @Test
+  public void testV4HeaderDateValidationSuccess() throws OS3Exception {
+    // Case 1: valid date within range.
+    LocalDate now = LocalDate.now();
+    String dateStr = DATE_FORMATTER.format(now);
+    validateResponse(dateStr);
+
+    // Case 2: Valid date with in range.
+    dateStr = DATE_FORMATTER.format(now.plus(1, DAYS));
+    validateResponse(dateStr);
+
+    // Case 3: Valid date with in range.
+    dateStr = DATE_FORMATTER.format(now.minus(1, DAYS));
+    validateResponse(dateStr);
+  }
+
+  @Test
+  public void testV4HeaderDateValidationFailure() throws Exception {
+    // Case 1: Empty date.
+    LocalDate now = LocalDate.now();
+    String dateStr = "";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> validateResponse(dateStr));
+
+    // Case 2: Date after yesterday.
+    String dateStr2 = DATE_FORMATTER.format(now.plus(2, DAYS));
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> validateResponse(dateStr2));
+
+    // Case 3: Date before yesterday.
+    String dateStr3 = DATE_FORMATTER.format(now.minus(2, DAYS));
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> validateResponse(dateStr3));
+  }
+
+  private void validateResponse(String dateStr) throws OS3Exception {
+    String auth =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + dateStr + "/us-east-1/s3" +
+            "/aws4_request,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    AuthorizationHeaderV4 v4 = new AuthorizationHeaderV4(auth);
+
+    assertEquals("AWS4-HMAC-SHA256", v4.getAlgorithm());
+    assertEquals("ozone", v4.getAccessKeyID());
+    assertEquals(dateStr, v4.getDate());
+    assertEquals("us-east-1", v4.getAwsRegion());
+    assertEquals("aws4_request", v4.getAwsRequest());
+    assertEquals("host;x-amz-content-sha256;x-amz-date",
+        v4.getSignedHeaderString());
+    assertEquals("fe5f80f77d5fa3beca038a248ff027", v4.getSignature());
+  }
+
+  @Test
+  public void testV4HeaderRegionValidationFailure() throws Exception {
+    String auth =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "//s3/aws4_request,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027%";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth));
+    String auth2 =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "s3/aws4_request,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027%";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth2));
+  }
+
+  @Test
+  public void testV4HeaderServiceValidationFailure() throws Exception {
+    String auth =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1" +
+            "//aws4_request,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth));
+
+    String auth2 =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1" +
+            "/aws4_request,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth2));
+  }
+
+  @Test
+  public void testV4HeaderRequestValidationFailure() throws Exception {
+    String auth =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/   ,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth));
+
+    String auth2 =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth2));
+
+    String auth3 =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            ","
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth3));
+  }
+
+  @Test
+  public void testV4HeaderSignedHeaderValidationFailure() throws Exception {
+    String auth =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/aws4_request,"
+            + "SignedHeaders=;;,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth));
+
+    String auth2 =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/aws4_request,"
+            + "SignedHeaders=,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth2));
+
+    String auth3 =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/aws4_request,"
+            + "=x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth3));
+
+    String auth4 =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/aws4_request,"
+            + "=,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth4));
+  }
+
+  @Test
+  public void testV4HeaderSignatureValidationFailure() throws Exception {
+    String auth =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/aws4_request,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027%";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth));
+
+    String auth2 =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/aws4_request,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth2));
+
+    String auth3 =
+        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/aws4_request,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + ""
+            + "=";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth3));
+  }
+
+  @Test
+  public void testV4HeaderHashAlgoValidationFailure() throws Exception {
+    String auth =
+        "AWS4-HMAC-SHA Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/aws4_request,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth));
+
+    String auth2 =
+        "SHA-256 Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/aws4_request,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth2));
+
+    String auth3 =
+        " Credential=ozone/" + curDate + "/us-east-1/s3" +
+            "/aws4_request,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth3));
+  }
+
+  @Test
+  public void testV4HeaderCredentialValidationFailure() throws Exception {
+    String auth =
+        "AWS4-HMAC-SHA Credential=/" + curDate + "//" +
+            "/,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth));
+
+    String auth2 =
+        "AWS4-HMAC-SHA =/" + curDate + "//" +
+            "/,"
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
+            + "Signature"
+            + "=fe5f80f77d5fa3beca038a248ff027";
+    LambdaTestUtils.intercept(OS3Exception.class, "",
+        () -> new AuthorizationHeaderV4(auth2));
+  }
+
 }