You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by el...@apache.org on 2021/04/01 17:55:56 UTC

[ozone] branch master updated: HDDS-4506. Support query parameter based v4 auth in S3g (#1628)

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

elek pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 8ec5f43  HDDS-4506. Support query parameter based v4 auth in S3g (#1628)
8ec5f43 is described below

commit 8ec5f437b198ea4e984e535bef142fda957c74ab
Author: Elek, Márton <el...@users.noreply.github.com>
AuthorDate: Thu Apr 1 19:55:41 2021 +0200

    HDDS-4506. Support query parameter based v4 auth in S3g (#1628)
---
 .../hadoop/ozone/s3/AWSSignatureProcessor.java     | 462 ---------------------
 .../hadoop/ozone/s3/OzoneClientProducer.java       |  92 ++--
 .../hadoop/ozone/s3/endpoint/EndpointBase.java     |  14 -
 .../hadoop/ozone/s3/exception/OS3Exception.java    |  14 +-
 .../hadoop/ozone/s3/exception/S3ErrorTable.java    |   2 +-
 .../ozone/s3/signature/AWSSignatureProcessor.java  | 205 +++++++++
 .../AuthorizationV2HeaderParser.java}              |  70 ++--
 .../AuthorizationV4HeaderParser.java}              | 206 ++++-----
 .../s3/signature/AuthorizationV4QueryParser.java   |  98 +++++
 .../ozone/s3/{header => signature}/Credential.java |   8 +-
 .../hadoop/ozone/s3/signature/SignatureInfo.java   | 113 +++++
 .../SignatureParser.java}                          |  20 +-
 .../s3/{ => signature}/SignatureProcessor.java     |  30 +-
 .../ozone/s3/signature/StringToSignProducer.java   | 324 +++++++++++++++
 .../s3/{header => signature}/package-info.java     |   2 +-
 .../hadoop/ozone/s3/TestAWSSignatureProcessor.java | 141 -------
 .../hadoop/ozone/s3/TestOzoneClientProducer.java   | 100 ++---
 .../hadoop/ozone/s3/endpoint/TestBucketPut.java    |  31 +-
 .../hadoop/ozone/s3/endpoint/TestRootList.java     |  24 --
 .../TestAuthorizationV2HeaderParser.java}          |  35 +-
 .../TestAuthorizationV4HeaderParser.java}          | 153 ++++---
 .../signature/TestAuthorizationV4QueryParser.java  | 146 +++++++
 .../s3/signature/TestStringToSignProducer.java     |  90 ++++
 23 files changed, 1340 insertions(+), 1040 deletions(-)

diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/AWSSignatureProcessor.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/AWSSignatureProcessor.java
deleted file mode 100644
index 2bcecb5..0000000
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/AWSSignatureProcessor.java
+++ /dev/null
@@ -1,462 +0,0 @@
-/*
- * 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;
-
-import javax.annotation.PostConstruct;
-import javax.enterprise.context.RequestScoped;
-import javax.ws.rs.container.ContainerRequestContext;
-import javax.ws.rs.core.Context;
-import javax.ws.rs.core.MultivaluedMap;
-import java.io.UnsupportedEncodingException;
-import java.net.InetAddress;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URLEncoder;
-import java.net.UnknownHostException;
-import java.nio.charset.StandardCharsets;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.time.LocalDate;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.hadoop.ozone.s3.exception.OS3Exception;
-import org.apache.hadoop.ozone.s3.header.AuthorizationHeaderV2;
-import org.apache.hadoop.ozone.s3.header.AuthorizationHeaderV4;
-import org.apache.hadoop.ozone.s3.header.Credential;
-
-import com.google.common.annotations.VisibleForTesting;
-import static java.time.temporal.ChronoUnit.SECONDS;
-import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.S3_AUTHINFO_CREATION_ERROR;
-import org.apache.kerby.util.Hex;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * Parser to process AWS V2 & V4 auth request. Creates string to sign and auth
- * header. For more details refer to AWS documentation https://docs.aws
- * .amazon.com/general/latest/gr/sigv4-create-canonical-request.html.
- **/
-@RequestScoped
-public class AWSSignatureProcessor implements SignatureProcessor {
-
-  private static final Logger LOG =
-      LoggerFactory.getLogger(AWSSignatureProcessor.class);
-
-  @Context
-  private ContainerRequestContext context;
-
-  private Map<String, String> headers;
-  private MultivaluedMap<String, String> queryMap;
-  private String uri;
-  private String method;
-  private AuthorizationHeaderV4 v4Header;
-  private AuthorizationHeaderV2 v2Header;
-  private String stringToSign;
-  private Exception exception;
-
-  @PostConstruct
-  public void init()
-      throws Exception {
-    //header map is MUTABLE. It's better to save it here. (with lower case
-    // keys!!!)
-    this.headers = new LowerCaseKeyStringMap(new HashMap<>());
-    for (Entry<String, List<String>> headerEntry : context.getHeaders()
-        .entrySet()) {
-      if (0 < headerEntry.getValue().size()) {
-        String headerKey = headerEntry.getKey();
-        if (headers.containsKey(headerKey)) {
-          //mutiple headers from the same type are combined
-          headers.put(headerKey,
-              headers.get(headerKey) + "," + headerEntry.getValue().get(0));
-        } else {
-          headers.put(headerKey, headerEntry.getValue().get(0));
-        }
-      }
-    }
-    //in case of the HeaderPreprocessor executed before us, let's restore the
-    // original content type.
-    if (headers.containsKey(HeaderPreprocessor.ORIGINAL_CONTENT_TYPE)) {
-      headers.put(HeaderPreprocessor.CONTENT_TYPE,
-          headers.get(HeaderPreprocessor.ORIGINAL_CONTENT_TYPE));
-    }
-
-
-    this.queryMap = context.getUriInfo().getQueryParameters();
-    this.uri = context.getUriInfo().getRequestUri().getPath();
-
-    this.method = context.getMethod();
-    String authHeader = headers.get(AUTHORIZATION_HEADER);
-    try {
-      if (authHeader != null) {
-        String[] split = authHeader.split(" ");
-        if (split[0].equals(AuthorizationHeaderV2.IDENTIFIER)) {
-          if (v2Header == null) {
-            v2Header = new AuthorizationHeaderV2(authHeader);
-          }
-        } else {
-          if (v4Header == null) {
-            v4Header = new AuthorizationHeaderV4(authHeader);
-          }
-          parse();
-        }
-      } else { // no auth header
-        v4Header = null;
-        v2Header = null;
-      }
-    } catch (Exception ex) {
-      // During validation of auth header, create instance and set Exception.
-      // This way it can be handled in OzoneClientProducer creation of
-      // SignatureProcessor instance failure.
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Error during Validation of Auth Header:{}", authHeader);
-      }
-      this.exception = ex;
-    }
-  }
-
-
-  private void parse() throws Exception {
-
-    StringBuilder strToSign = new StringBuilder();
-    // According to AWS sigv4 documentation, authorization header should be
-    // in following format.
-    // Authorization: algorithm Credential=access key ID/credential scope,
-    // SignedHeaders=SignedHeaders, Signature=signature
-
-    // Construct String to sign in below format.
-    // StringToSign =
-    //    Algorithm + \n +
-    //    RequestDateTime + \n +
-    //    CredentialScope + \n +
-    //    HashedCanonicalRequest
-    String algorithm, requestDateTime, credentialScope, canonicalRequest;
-    algorithm = v4Header.getAlgorithm();
-    requestDateTime = headers.get(X_AMAZ_DATE);
-    Credential credential = v4Header.getCredentialObj();
-    credentialScope = String.format("%s/%s/%s/%s", credential.getDate(),
-        credential.getAwsRegion(), credential.getAwsService(),
-        credential.getAwsRequest());
-
-    // If the absolute path is empty, use a forward slash (/)
-    uri = (uri.trim().length() > 0) ? uri : "/";
-    // Encode URI and preserve forward slashes
-    strToSign.append(algorithm + NEWLINE);
-    strToSign.append(requestDateTime + NEWLINE);
-    strToSign.append(credentialScope + NEWLINE);
-
-    canonicalRequest = buildCanonicalRequest();
-    strToSign.append(hash(canonicalRequest));
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("canonicalRequest:[{}]", canonicalRequest);
-    }
-
-    if (LOG.isTraceEnabled()) {
-      headers.keySet().forEach(k -> LOG.trace("Header:{},value:{}", k,
-          headers.get(k)));
-    }
-
-    LOG.debug("StringToSign:[{}]", strToSign);
-    stringToSign = strToSign.toString();
-  }
-
-  @VisibleForTesting
-  protected String buildCanonicalRequest() throws OS3Exception {
-
-    Iterable<String> parts = split("/", uri);
-    List<String> encParts = new ArrayList<>();
-    for (String p : parts) {
-      encParts.add(urlEncode(p));
-    }
-    String canonicalUri = join("/", encParts);
-
-    String canonicalQueryStr = getQueryParamString();
-
-    StringBuilder canonicalHeaders = new StringBuilder();
-
-    for (String header : v4Header.getSignedHeaders()) {
-      canonicalHeaders.append(header.toLowerCase());
-      canonicalHeaders.append(":");
-      if (headers.containsKey(header)) {
-        String headerValue = headers.get(header);
-        canonicalHeaders.append(headerValue);
-        canonicalHeaders.append(NEWLINE);
-
-        // Set for testing purpose only to skip date and host validation.
-        validateSignedHeader(header, headerValue);
-
-      } else {
-        throw new RuntimeException("Header " + header + " not present in " +
-            "request but requested to be signed.");
-      }
-    }
-
-    String payloadHash;
-    if (UNSIGNED_PAYLOAD.equals(
-        headers.get(X_AMZ_CONTENT_SHA256))) {
-      payloadHash = UNSIGNED_PAYLOAD;
-    } else {
-      payloadHash = headers.get(X_AMZ_CONTENT_SHA256);
-    }
-
-    String signedHeaderStr = v4Header.getSignedHeaderString();
-    String canonicalRequest = method + NEWLINE
-        + canonicalUri + NEWLINE
-        + canonicalQueryStr + NEWLINE
-        + canonicalHeaders + NEWLINE
-        + signedHeaderStr + NEWLINE
-        + payloadHash;
-
-    return canonicalRequest;
-  }
-
-  @VisibleForTesting
-  void validateSignedHeader(String header, String headerValue)
-      throws OS3Exception {
-    switch (header) {
-    case HOST:
-      try {
-        String schema = context.getUriInfo().getRequestUri().getScheme();
-        URI hostUri = new URI(schema + "://" + headerValue);
-        InetAddress.getByName(hostUri.getHost());
-        // TODO: Validate if current request is coming from same host.
-      } catch (UnknownHostException|URISyntaxException e) {
-        LOG.error("Host value mentioned in signed header is not valid. " +
-            "Host:{}", headerValue);
-        throw S3_AUTHINFO_CREATION_ERROR;
-      }
-      break;
-    case X_AMAZ_DATE:
-      LocalDate date = LocalDate.parse(headerValue, TIME_FORMATTER);
-      LocalDate now = LocalDate.now();
-      if (date.isBefore(now.minus(PRESIGN_URL_MAX_EXPIRATION_SECONDS, SECONDS))
-          || date.isAfter(now.plus(PRESIGN_URL_MAX_EXPIRATION_SECONDS,
-          SECONDS))) {
-        LOG.error("AWS date not in valid range. Request timestamp:{} should " +
-                "not be older than {} seconds.", headerValue,
-            PRESIGN_URL_MAX_EXPIRATION_SECONDS);
-        throw S3_AUTHINFO_CREATION_ERROR;
-      }
-      break;
-    case X_AMZ_CONTENT_SHA256:
-      // TODO: Construct request payload and match HEX(SHA256(requestPayload))
-      break;
-    default:
-      break;
-    }
-  }
-
-  /**
-   * String join that also works with empty strings.
-   *
-   * @return joined string
-   */
-  private static String join(String glue, List<String> parts) {
-    StringBuilder result = new StringBuilder();
-    boolean addSeparator = false;
-    for (String p : parts) {
-      if (addSeparator) {
-        result.append(glue);
-      }
-      result.append(p);
-      addSeparator = true;
-    }
-    return result.toString();
-  }
-
-  /**
-   * Returns matching strings.
-   *
-   * @param regex Regular expression to split by
-   * @param whole The string to split
-   * @return pieces
-   */
-  private static Iterable<String> split(String regex, String whole) {
-    Pattern p = Pattern.compile(regex);
-    Matcher m = p.matcher(whole);
-    List<String> result = new ArrayList<>();
-    int pos = 0;
-    while (m.find()) {
-      result.add(whole.substring(pos, m.start()));
-      pos = m.end();
-    }
-    result.add(whole.substring(pos));
-    return result;
-  }
-
-  private String urlEncode(String str) {
-    try {
-
-      return URLEncoder.encode(str, StandardCharsets.UTF_8.name())
-          .replaceAll("\\+", "%20")
-          .replaceAll("%7E", "~");
-    } catch (UnsupportedEncodingException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  private String getQueryParamString() {
-    List<String> params = new ArrayList<>(queryMap.keySet());
-
-    // Sort by name, then by value
-    Collections.sort(params, (o1, o2) -> o1.equals(o2) ?
-        queryMap.getFirst(o1).compareTo(queryMap.getFirst(o2)) :
-        o1.compareTo(o2));
-
-    StringBuilder result = new StringBuilder();
-    for (String p : params) {
-      if (result.length() > 0) {
-        result.append("&");
-      }
-      result.append(urlEncode(p));
-      result.append('=');
-
-      result.append(urlEncode(queryMap.getFirst(p)));
-    }
-    return result.toString();
-  }
-
-  public static String hash(String payload) throws NoSuchAlgorithmException {
-    MessageDigest md = MessageDigest.getInstance("SHA-256");
-    md.update(payload.getBytes(StandardCharsets.UTF_8));
-    return Hex.encode(md.digest()).toLowerCase();
-  }
-
-  @Override
-  public String getAwsAccessId() {
-    return (v4Header != null ? v4Header.getAccessKeyID() :
-        v2Header != null ? v2Header.getAccessKeyID() : "");
-  }
-
-  @Override
-  public String getSignature() {
-    return (v4Header != null ? v4Header.getSignature() :
-        v2Header != null ? v2Header.getSignature() : "");
-  }
-
-  @Override
-  public String getStringToSign() throws Exception {
-    return stringToSign;
-  }
-
-  @VisibleForTesting
-  public void setContext(ContainerRequestContext context) {
-    this.context = context;
-  }
-
-  @VisibleForTesting
-  public void setV4Header(
-      AuthorizationHeaderV4 v4Header) {
-    this.v4Header = v4Header;
-  }
-
-  @VisibleForTesting
-  public void setV2Header(AuthorizationHeaderV2 v2Header) {
-    this.v2Header = v2Header;
-  }
-
-  @Override
-  public Exception getException() {
-    return this.exception;
-  }
-
-  /**
-   * A simple map which forces lower case key usage.
-   */
-  public static class LowerCaseKeyStringMap implements Map<String, String> {
-
-    private HashMap<String, String> delegate;
-
-    public LowerCaseKeyStringMap(
-        HashMap<String, String> delegate) {
-      this.delegate = delegate;
-    }
-
-    @Override
-    public int size() {
-      return delegate.size();
-    }
-
-    @Override
-    public boolean isEmpty() {
-      return delegate.isEmpty();
-    }
-
-    @Override
-    public boolean containsKey(Object key) {
-      return delegate.containsKey(key.toString().toLowerCase());
-    }
-
-    @Override
-    public boolean containsValue(Object value) {
-      return delegate.containsValue(value);
-    }
-
-    @Override
-    public String get(Object key) {
-      return delegate.get(key.toString().toLowerCase());
-    }
-
-    @Override
-    public String put(String key, String value) {
-      return delegate.put(key.toLowerCase(), value);
-    }
-
-    @Override
-    public String remove(Object key) {
-      return delegate.remove(key.toString());
-    }
-
-    @Override
-    public void putAll(Map<? extends String, ? extends String> m) {
-      for (Entry<? extends String, ? extends String> entry : m.entrySet()) {
-        put(entry.getKey().toLowerCase(), entry.getValue());
-      }
-    }
-
-    @Override
-    public void clear() {
-      delegate.clear();
-    }
-
-    @Override
-    public Set<String> keySet() {
-      return delegate.keySet();
-    }
-
-    @Override
-    public Collection<String> values() {
-      return delegate.values();
-    }
-
-    @Override
-    public Set<Entry<String, String>> entrySet() {
-      return delegate.entrySet();
-    }
-  }
-
-}
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java
index d0cca97..4264f41 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/OzoneClientProducer.java
@@ -21,8 +21,10 @@ import javax.annotation.PreDestroy;
 import javax.enterprise.context.RequestScoped;
 import javax.enterprise.inject.Produces;
 import javax.inject.Inject;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.Context;
 import java.io.IOException;
-import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
 import java.security.PrivilegedExceptionAction;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -31,16 +33,19 @@ import org.apache.hadoop.ozone.OzoneSecurityUtil;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientFactory;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.signature.SignatureInfo;
+import org.apache.hadoop.ozone.s3.signature.SignatureInfo.Version;
+import org.apache.hadoop.ozone.s3.signature.SignatureProcessor;
+import org.apache.hadoop.ozone.s3.signature.StringToSignProducer;
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 
 import com.google.common.annotations.VisibleForTesting;
-import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMTokenProto.Type.S3AUTHINFO;
 import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.INTERNAL_ERROR;
 import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.MALFORMED_HEADER;
-import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.S3_AUTHINFO_CREATION_ERROR;
+import org.jetbrains.annotations.NotNull;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,10 +57,11 @@ public class OzoneClientProducer {
 
   private static final Logger LOG =
       LoggerFactory.getLogger(OzoneClientProducer.class);
+
   private OzoneClient client;
 
   @Inject
-  private SignatureProcessor signatureParser;
+  private SignatureProcessor signatureProcessor;
 
   @Inject
   private OzoneConfiguration ozoneConfiguration;
@@ -66,13 +72,15 @@ public class OzoneClientProducer {
   @Inject
   private String omServiceID;
 
+  @Context
+  private ContainerRequestContext context;
 
   @Produces
   public OzoneClient createClient() throws OS3Exception, IOException {
     client = getClient(ozoneConfiguration);
     return client;
   }
-  
+
   @PreDestroy
   public void destroy() throws IOException {
     client.close();
@@ -82,45 +90,45 @@ public class OzoneClientProducer {
       throws OS3Exception {
     OzoneClient ozoneClient = null;
     try {
-      // Check if any error occurred during creation of signatureProcessor.
-      if (signatureParser.getException() != null) {
-        throw signatureParser.getException();
+      SignatureInfo signatureInfo = signatureProcessor.parseSignature();
+
+      String stringToSign = "";
+      if (signatureInfo.getVersion() == Version.V4) {
+        stringToSign =
+            StringToSignProducer.createSignatureBase(signatureInfo, context);
       }
-      String awsAccessId = signatureParser.getAwsAccessId();
+
+      String awsAccessId = signatureInfo.getAwsAccessId();
       validateAccessId(awsAccessId);
 
       UserGroupInformation remoteUser =
           UserGroupInformation.createRemoteUser(awsAccessId);
       if (OzoneSecurityUtil.isSecurityEnabled(config)) {
         LOG.debug("Creating s3 auth info for client.");
-        try {
-          OzoneTokenIdentifier identifier = new OzoneTokenIdentifier();
-          identifier.setTokenType(S3AUTHINFO);
-          identifier.setStrToSign(signatureParser.getStringToSign());
-          identifier.setSignature(signatureParser.getSignature());
-          identifier.setAwsAccessId(awsAccessId);
-          identifier.setOwner(new Text(awsAccessId));
-          if (LOG.isTraceEnabled()) {
-            LOG.trace("Adding token for service:{}", omService);
-          }
-          Token<OzoneTokenIdentifier> token = new Token(identifier.getBytes(),
-              identifier.getSignature().getBytes(UTF_8),
-              identifier.getKind(),
-              omService);
-          remoteUser.addToken(token);
-        } catch (OS3Exception | URISyntaxException ex) {
-          throw S3_AUTHINFO_CREATION_ERROR;
+
+        if (signatureInfo.getVersion() == Version.NONE) {
+          throw MALFORMED_HEADER;
+        }
+
+        OzoneTokenIdentifier identifier = new OzoneTokenIdentifier();
+        identifier.setTokenType(S3AUTHINFO);
+        identifier.setStrToSign(stringToSign);
+        identifier.setSignature(signatureInfo.getSignature());
+        identifier.setAwsAccessId(awsAccessId);
+        identifier.setOwner(new Text(awsAccessId));
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Adding token for service:{}", omService);
         }
+        Token<OzoneTokenIdentifier> token = new Token(identifier.getBytes(),
+            identifier.getSignature().getBytes(StandardCharsets.UTF_8),
+            identifier.getKind(),
+            omService);
+        remoteUser.addToken(token);
+
       }
       ozoneClient =
-          remoteUser.doAs((PrivilegedExceptionAction<OzoneClient>)() -> {
-            if (omServiceID == null) {
-              return OzoneClientFactory.getRpcClient(ozoneConfiguration);
-            } else {
-              // As in HA case, we need to pass om service ID.
-              return OzoneClientFactory.getRpcClient(omServiceID,
-                  ozoneConfiguration);
-            }
+          remoteUser.doAs((PrivilegedExceptionAction<OzoneClient>) () -> {
+            return createOzoneClient();
           });
     } catch (OS3Exception ex) {
       if (LOG.isDebugEnabled()) {
@@ -138,6 +146,18 @@ public class OzoneClientProducer {
     return ozoneClient;
   }
 
+  @NotNull
+  @VisibleForTesting
+  OzoneClient createOzoneClient() throws IOException {
+    if (omServiceID == null) {
+      return OzoneClientFactory.getRpcClient(ozoneConfiguration);
+    } else {
+      // As in HA case, we need to pass om service ID.
+      return OzoneClientFactory.getRpcClient(omServiceID,
+          ozoneConfiguration);
+    }
+  }
+
   // ONLY validate aws access id when needed.
   private void validateAccessId(String awsAccessId) throws Exception {
     if (awsAccessId == null || awsAccessId.equals("")) {
@@ -151,7 +171,7 @@ public class OzoneClientProducer {
   }
 
   @VisibleForTesting
-  public void setSignatureParser(SignatureProcessor signatureParser) {
-    this.signatureParser = signatureParser;
+  public void setSignatureParser(SignatureProcessor awsSignatureProcessor) {
+    this.signatureProcessor = awsSignatureProcessor;
   }
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java
index 360f4f4..137b8ea 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
-import org.apache.hadoop.ozone.s3.SignatureProcessor;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 
@@ -43,9 +42,6 @@ public class EndpointBase {
   @Inject
   private OzoneClient client;
 
-  @Inject
-  private SignatureProcessor signatureProcessor;
-
   protected OzoneBucket getBucket(OzoneVolume volume, String bucketName)
       throws OS3Exception, IOException {
     OzoneBucket bucket;
@@ -171,16 +167,6 @@ public class EndpointBase {
     }
   }
 
-  public SignatureProcessor getSignatureProcessor() {
-    return signatureProcessor;
-  }
-
-  @VisibleForTesting
-  public void setSignatureProcessor(
-      SignatureProcessor signatureProcessor) {
-    this.signatureProcessor = signatureProcessor;
-  }
-
   @VisibleForTesting
   public void setClient(OzoneClient ozoneClient) {
     this.client = ozoneClient;
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/OS3Exception.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/OS3Exception.java
index 12e6cfc..810aa20 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/OS3Exception.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/OS3Exception.java
@@ -17,6 +17,12 @@
  */
 package org.apache.hadoop.ozone.s3.exception;
 
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlTransient;
+
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.SerializationFeature;
 import com.fasterxml.jackson.dataformat.xml.XmlMapper;
@@ -24,12 +30,6 @@ import com.fasterxml.jackson.module.jaxb.JaxbAnnotationModule;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.xml.bind.annotation.XmlAccessType;
-import javax.xml.bind.annotation.XmlAccessorType;
-import javax.xml.bind.annotation.XmlElement;
-import javax.xml.bind.annotation.XmlTransient;
-import javax.xml.bind.annotation.XmlRootElement;
-
 
 /**
  * This class represents exceptions raised from Ozone S3 service.
@@ -38,7 +38,7 @@ import javax.xml.bind.annotation.XmlRootElement;
  */
 @XmlRootElement(name = "Error")
 @XmlAccessorType(XmlAccessType.NONE)
-public class OS3Exception extends  Exception {
+public class OS3Exception extends Exception {
   private static final Logger LOG =
       LoggerFactory.getLogger(OS3Exception.class);
   private static ObjectMapper mapper;
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java
index 9e795a7..7d9b55f 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java
@@ -125,7 +125,7 @@ public final class S3ErrorTable {
     OS3Exception err =  new OS3Exception(e.getCode(), e.getErrorMessage(),
         e.getHttpCode());
     err.setResource(resource);
-    LOG.error(err.toXml());
+    LOG.error(err.toXml(), e);
     return err;
   }
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AWSSignatureProcessor.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AWSSignatureProcessor.java
new file mode 100644
index 0000000..807b473
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AWSSignatureProcessor.java
@@ -0,0 +1,205 @@
+/*
+ * 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.signature;
+
+import javax.enterprise.context.RequestScoped;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MultivaluedMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.hadoop.ozone.s3.HeaderPreprocessor;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.signature.SignatureInfo.Version;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Parser to process AWS V2 & V4 auth request. Creates string to sign and auth
+ * header. For more details refer to AWS documentation https://docs.aws
+ * .amazon.com/general/latest/gr/sigv4-create-canonical-request.html.
+ **/
+@RequestScoped
+public class AWSSignatureProcessor implements SignatureProcessor {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AWSSignatureProcessor.class);
+
+  @Context
+  private ContainerRequestContext context;
+
+  public SignatureInfo parseSignature() throws OS3Exception {
+
+    LowerCaseKeyStringMap headers =
+        LowerCaseKeyStringMap.fromHeaderMap(context.getHeaders());
+
+    String authHeader = headers.get("Authorization");
+
+    List<SignatureParser> signatureParsers = new ArrayList<>();
+    signatureParsers.add(new AuthorizationV4HeaderParser(authHeader,
+        headers.get(StringToSignProducer.X_AMAZ_DATE)));
+    signatureParsers.add(new AuthorizationV4QueryParser(
+        StringToSignProducer.fromMultiValueToSingleValueMap(
+            context.getUriInfo().getQueryParameters())));
+    signatureParsers.add(new AuthorizationV2HeaderParser(authHeader));
+
+    SignatureInfo signatureInfo = null;
+    for (SignatureParser parser : signatureParsers) {
+      signatureInfo = parser.parseSignature();
+      if (signatureInfo != null) {
+        break;
+      }
+    }
+    if (signatureInfo == null) {
+      signatureInfo = new SignatureInfo(
+          Version.NONE,
+          "", "", "", "", "", "", "", false
+      );
+    }
+    return signatureInfo;
+  }
+
+  @VisibleForTesting
+  public void setContext(ContainerRequestContext context) {
+    this.context = context;
+  }
+
+  /**
+   * A simple map which forces lower case key usage.
+   */
+  public static class LowerCaseKeyStringMap implements Map<String, String> {
+
+    private Map<String, String> delegate;
+
+    public LowerCaseKeyStringMap() {
+      this.delegate = new HashMap<>();
+    }
+
+    public static LowerCaseKeyStringMap fromHeaderMap(
+        MultivaluedMap<String,
+            String> rawHeaders
+    ) {
+
+      //header map is MUTABLE. It's better to save it here. (with lower case
+      // keys!!!)
+      final LowerCaseKeyStringMap headers =
+          new LowerCaseKeyStringMap();
+
+      for (Entry<String, List<String>> headerEntry : rawHeaders.entrySet()) {
+        if (0 < headerEntry.getValue().size()) {
+          String headerKey = headerEntry.getKey();
+          if (headers.containsKey(headerKey)) {
+            //multiple headers from the same type are combined
+            headers.put(headerKey,
+                headers.get(headerKey) + "," + headerEntry.getValue().get(0));
+          } else {
+            headers.put(headerKey, headerEntry.getValue().get(0));
+          }
+        }
+      }
+
+      headers.fixContentType();
+
+      if (LOG.isTraceEnabled()) {
+        headers.keySet().forEach(k -> LOG.trace("Header:{},value:{}", k,
+            headers.get(k)));
+      }
+      return headers;
+    }
+
+    @VisibleForTesting
+    protected void fixContentType() {
+      //in case of the HeaderPreprocessor executed before us, let's restore the
+      // original content type.
+      if (containsKey(HeaderPreprocessor.ORIGINAL_CONTENT_TYPE)) {
+        put(HeaderPreprocessor.CONTENT_TYPE,
+            get(HeaderPreprocessor.ORIGINAL_CONTENT_TYPE));
+      }
+    }
+
+    @Override
+    public int size() {
+      return delegate.size();
+    }
+
+    @Override
+    public boolean isEmpty() {
+      return delegate.isEmpty();
+    }
+
+    @Override
+    public boolean containsKey(Object key) {
+      return delegate.containsKey(key.toString().toLowerCase());
+    }
+
+    @Override
+    public boolean containsValue(Object value) {
+      return delegate.containsValue(value);
+    }
+
+    @Override
+    public String get(Object key) {
+      return delegate.get(key.toString().toLowerCase());
+    }
+
+    @Override
+    public String put(String key, String value) {
+      return delegate.put(key.toLowerCase(), value);
+    }
+
+    @Override
+    public String remove(Object key) {
+      return delegate.remove(key.toString());
+    }
+
+    @Override
+    public void putAll(Map<? extends String, ? extends String> m) {
+      for (Entry<? extends String, ? extends String> entry : m.entrySet()) {
+        put(entry.getKey().toLowerCase(), entry.getValue());
+      }
+    }
+
+    @Override
+    public void clear() {
+      delegate.clear();
+    }
+
+    @Override
+    public Set<String> keySet() {
+      return delegate.keySet();
+    }
+
+    @Override
+    public Collection<String> values() {
+      return delegate.values();
+    }
+
+    @Override
+    public Set<Entry<String, String>> entrySet() {
+      return delegate.entrySet();
+    }
+  }
+
+}
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/AuthorizationHeaderV2.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV2HeaderParser.java
similarity index 66%
rename from hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/AuthorizationHeaderV2.java
rename to hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV2HeaderParser.java
index d29d7df..1681dea 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/AuthorizationHeaderV2.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV2HeaderParser.java
@@ -15,48 +15,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.hadoop.ozone.s3.signature;
 
-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 org.apache.hadoop.ozone.s3.signature.SignatureInfo.Version;
 
 import static org.apache.commons.lang3.StringUtils.isBlank;
 
 /**
- * Authorization Header v2.
+ * Class to parse V2 auth information from header.
  */
-public class AuthorizationHeaderV2 {
+public class AuthorizationV2HeaderParser implements SignatureParser {
 
   public static final 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();
+  private final String authHeader;
+
+  public AuthorizationV2HeaderParser(String authHeader) {
+    this.authHeader = authHeader;
   }
 
   /**
    * This method parses the authorization header.
-   *
+   * <p>
    * Authorization header sample:
    * AWS AKIAIOSFODNN7EXAMPLE:frJIUN8DYpKDtOLCwo//yllqDzg=
-   *
-   * @throws OS3Exception
    */
-  @SuppressWarnings("StringSplitter")
-  public void parseHeader() throws OS3Exception {
+  @Override
+  public SignatureInfo parseSignature() throws OS3Exception {
+    if (authHeader == null || !authHeader.startsWith(IDENTIFIER + " ")) {
+      return null;
+    }
     String[] split = authHeader.split(" ");
     if (split.length != 2) {
       throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
     }
 
-    identifier = split[0];
+    String identifier = split[0];
     if (!IDENTIFIER.equals(identifier)) {
       throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
     }
@@ -67,31 +63,21 @@ public class AuthorizationHeaderV2 {
       throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
     }
 
-    accessKeyID = remainingSplit[0];
-    signature = remainingSplit[1];
+    String accessKeyID = remainingSplit[0];
+    String signature = remainingSplit[1];
     if (isBlank(accessKeyID) || isBlank(signature)) {
       throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
     }
+    return new SignatureInfo(
+        Version.V2,
+        "",
+        "",
+        accessKeyID,
+        signature,
+        "",
+        "",
+        "",
+        false
+    );
   }
-
-  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;
-  }
-
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/AuthorizationHeaderV4.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV4HeaderParser.java
similarity index 57%
rename from hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/AuthorizationHeaderV4.java
rename to hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV4HeaderParser.java
index 4f4bff3..f62d0f1 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/AuthorizationHeaderV4.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV4HeaderParser.java
@@ -1,86 +1,80 @@
 /**
- * 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
+ * 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.
+ * 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.signature;
 
-package org.apache.hadoop.ozone.s3.header;
-
+import java.time.LocalDate;
+import java.util.Collection;
 
-import com.google.common.annotations.VisibleForTesting;
-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.ozone.s3.signature.SignatureInfo.Version;
 import org.apache.hadoop.util.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.time.LocalDate;
-import java.util.Collection;
 
+import com.google.common.annotations.VisibleForTesting;
 import static java.time.temporal.ChronoUnit.DAYS;
+import org.apache.commons.codec.DecoderException;
+import org.apache.commons.codec.binary.Hex;
 import static org.apache.commons.lang3.StringUtils.isEmpty;
 import static org.apache.commons.lang3.StringUtils.isNotEmpty;
 import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.MALFORMED_HEADER;
-import static org.apache.hadoop.ozone.s3.AWSSignatureProcessor.AWS4_SIGNING_ALGORITHM;
-import static org.apache.hadoop.ozone.s3.AWSSignatureProcessor.DATE_FORMATTER;
+import static org.apache.hadoop.ozone.s3.signature.SignatureProcessor.AWS4_SIGNING_ALGORITHM;
+import static org.apache.hadoop.ozone.s3.signature.SignatureProcessor.DATE_FORMATTER;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
- * S3 Authorization header.
- * Ref: https://docs.aws.amazon.com/AmazonS3/latest/API/sigv4-auth-using
- * -authorization-header.html
+ * Class to parse v4 auth information from header.
  */
-public class AuthorizationHeaderV4 {
+public class AuthorizationV4HeaderParser implements SignatureParser {
 
-  private static final Logger LOG = LoggerFactory.getLogger(
-      AuthorizationHeaderV4.class);
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AuthorizationV4HeaderParser.class);
 
   private static final String CREDENTIAL = "Credential=";
   private static final String SIGNEDHEADERS = "SignedHeaders=";
   private static final String SIGNATURE = "Signature=";
 
   private String authHeader;
-  private String algorithm;
-  private String credential;
-  private String signedHeadersStr;
-  private String signature;
-  private Credential credentialObj;
-  private Collection<String> signedHeaders;
 
-  /**
-   * Construct AuthorizationHeader object.
-   * @param header
-   */
-  public AuthorizationHeaderV4(String header) throws OS3Exception {
-    Preconditions.checkNotNull(header);
-    this.authHeader = header;
-    parseAuthHeader();
+  private String dateHeader;
+
+  public AuthorizationV4HeaderParser(String authHeader, String dateHeader) {
+    this.authHeader = authHeader;
+    this.dateHeader = dateHeader;
   }
 
   /**
    * This method parses authorization header.
+   * <p>
+   * 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
    *
-   *  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
    */
   @SuppressWarnings("StringSplitter")
-  public void parseAuthHeader() throws OS3Exception {
+  @Override
+  public SignatureInfo parseSignature() throws OS3Exception {
+    if (authHeader == null || !authHeader.startsWith("AWS4")) {
+      return null;
+    }
     int firstSep = authHeader.indexOf(' ');
     if (firstSep < 0) {
       throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
@@ -93,29 +87,39 @@ public class AuthorizationHeaderV4 {
       throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
     }
 
-    algorithm = authHeader.substring(0, firstSep);
-    validateAlgorithm();
-    credential = split[0];
-    signedHeadersStr = split[1];
-    signature = split[2];
-    validateCredentials();
-    validateSignedHeaders();
-    validateSignature();
-
+    String algorithm = parseAlgorithm(authHeader.substring(0, firstSep));
+    Credential credentialObj = parseCredentials(split[0]);
+    String signedHeaders = parseSignedHeaders(split[1]);
+    String signature = parseSignature(split[2]);
+    return new SignatureInfo(
+        Version.V4,
+        credentialObj.getDate(),
+        dateHeader,
+        credentialObj.getAccessKeyID(),
+        signature,
+        signedHeaders,
+        credentialObj.createScope(),
+        algorithm,
+        true
+    );
   }
 
   /**
    * Validate Signed headers.
-   * */
-  private void validateSignedHeaders() throws OS3Exception {
+   */
+  private String parseSignedHeaders(String signedHeadersStr)
+      throws OS3Exception {
     if (isNotEmpty(signedHeadersStr)
         && signedHeadersStr.startsWith(SIGNEDHEADERS)) {
-      signedHeadersStr = signedHeadersStr.substring(SIGNEDHEADERS.length());
-      signedHeaders = StringUtils.getStringCollection(signedHeadersStr, ";");
+      String parsedSignedHeaders =
+          signedHeadersStr.substring(SIGNEDHEADERS.length());
+      Collection<String> signedHeaders =
+          StringUtils.getStringCollection(parsedSignedHeaders, ";");
       if (signedHeaders.size() == 0) {
         LOG.error("No signed headers found. Authheader:{}", authHeader);
         throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
       }
+      return parsedSignedHeaders;
     } else {
       LOG.error("No signed headers found. Authheader:{}", authHeader);
       throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
@@ -124,21 +128,22 @@ public class AuthorizationHeaderV4 {
 
   /**
    * Validate signature.
-   * */
-  private void validateSignature() throws OS3Exception {
+   */
+  private String parseSignature(String signature) throws OS3Exception {
     if (signature.startsWith(SIGNATURE)) {
-      signature = signature.substring(SIGNATURE.length());
-      if (isEmpty(signature)) {
+      String parsedSignature = signature.substring(SIGNATURE.length());
+      if (isEmpty(parsedSignature)) {
         LOG.error("Signature can't be empty: {}", signature);
         throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
       }
       try {
-        Hex.decodeHex(signature);
+        Hex.decodeHex(parsedSignature);
       } catch (DecoderException e) {
         LOG.error("Signature:{} should be in hexa-decimal encoding.",
             signature);
         throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
       }
+      return parsedSignature;
     } else {
       LOG.error("No signature found: {}", signature);
       throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
@@ -147,8 +152,10 @@ public class AuthorizationHeaderV4 {
 
   /**
    * Validate credentials.
-   * */
-  private void validateCredentials() throws OS3Exception {
+   */
+  private Credential parseCredentials(String credential)
+      throws OS3Exception {
+    Credential credentialObj = null;
     if (isNotEmpty(credential) && credential.startsWith(CREDENTIAL)) {
       credential = credential.substring(CREDENTIAL.length());
       // Parse credential. Other parts of header are not validated yet. When
@@ -159,7 +166,8 @@ public class AuthorizationHeaderV4 {
     }
 
     if (credentialObj.getAccessKeyID().isEmpty()) {
-      LOG.error("AWS access id shouldn't be empty. credential:{}", credential);
+      LOG.error("AWS access id shouldn't be empty. credential:{}",
+          credential);
       throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
     }
     if (credentialObj.getAwsRegion().isEmpty()) {
@@ -178,82 +186,36 @@ public class AuthorizationHeaderV4 {
 
     // Date should not be empty and within valid range.
     if (!credentialObj.getDate().isEmpty()) {
-      validateDateRange();
+      validateDateRange(credentialObj);
     } else {
       LOG.error("AWS date shouldn't be empty. credential:{}", credential);
       throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
     }
+    return credentialObj;
   }
 
   @VisibleForTesting
-  public void validateDateRange() throws OS3Exception {
+  public void validateDateRange(Credential credentialObj) throws OS3Exception {
     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());
+          "than 1 day(i.e yesterday) and greater than 1 day(i.e " +
+          "tomorrow).", date);
       throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
     }
   }
 
   /**
    * Validate if algorithm is in expected format.
-   * */
-  private void validateAlgorithm() throws OS3Exception {
+   */
+  private String parseAlgorithm(String algorithm) throws OS3Exception {
     if (isEmpty(algorithm) || !algorithm.equals(AWS4_SIGNING_ALGORITHM)) {
       LOG.error("Unexpected hash algorithm. Algo:{}", algorithm);
       throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
     }
-  }
-
-  public String getAuthHeader() {
-    return authHeader;
-  }
-
-  public String getAlgorithm() {
     return algorithm;
   }
 
-  public String getCredential() {
-    return credential;
-  }
-
-  public String getSignedHeaderString() {
-    return signedHeadersStr;
-  }
-
-  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();
-  }
-
-  public Collection<String> getSignedHeaders() {
-    return signedHeaders;
-  }
-
-  public Credential getCredentialObj() {
-    return credentialObj;
-  }
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV4QueryParser.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV4QueryParser.java
new file mode 100644
index 0000000..fbf4594
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV4QueryParser.java
@@ -0,0 +1,98 @@
+/**
+ * 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.signature;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.time.ZonedDateTime;
+import java.util.Map;
+
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.signature.SignatureInfo.Version;
+
+import com.google.common.annotations.VisibleForTesting;
+import static java.time.temporal.ChronoUnit.SECONDS;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Parser for getting auth info from query parameters.
+ * <p>
+ * See: https://docs.aws.amazon
+ * .com/AmazonS3/latest/API/sigv4-query-string-auth.html
+ */
+public class AuthorizationV4QueryParser implements SignatureParser {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(AuthorizationV4QueryParser.class);
+
+  private final Map<String, String> queryParameters;
+
+  public AuthorizationV4QueryParser(
+      Map<String, String> queryParameters
+  ) {
+    this.queryParameters = queryParameters;
+  }
+
+  @Override
+  public SignatureInfo parseSignature() throws OS3Exception {
+
+    if (!queryParameters.containsKey("X-Amz-Signature")) {
+      return null;
+    }
+
+    validateDateAndExpires();
+
+    final String rawCredential = queryParameters.get("X-Amz-Credential");
+
+    Credential credential =
+        null;
+    try {
+      credential = new Credential(URLDecoder.decode(rawCredential, "UTF-8"));
+    } catch (UnsupportedEncodingException e) {
+      throw new IllegalArgumentException(
+          "X-Amz-Credential is not proper URL encoded");
+    }
+
+    return new SignatureInfo(
+        Version.V4,
+        credential.getDate(),
+        queryParameters.get("X-Amz-Date"),
+        credential.getAccessKeyID(),
+        queryParameters.get("X-Amz-Signature"),
+        queryParameters.get("X-Amz-SignedHeaders"),
+        credential.createScope(),
+        queryParameters.get("X-Amz-Algorithm"),
+        false
+    );
+  }
+
+  @VisibleForTesting
+  protected void validateDateAndExpires() {
+    final String dateString = queryParameters.get("X-Amz-Date");
+    final String expiresString = queryParameters.get("X-Amz-Expires");
+    if (expiresString != null && expiresString.length() > 0) {
+      final Long expires = Long.valueOf(expiresString);
+
+      if (ZonedDateTime.parse(dateString, StringToSignProducer.TIME_FORMATTER)
+          .plus(expires, SECONDS).isBefore(ZonedDateTime.now())) {
+        throw new IllegalArgumentException("Pre-signed S3 url is expired");
+      }
+    }
+  }
+}
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/Credential.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/Credential.java
similarity index 93%
rename from hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/Credential.java
rename to hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/Credential.java
index 883980a..14bf2a2 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/Credential.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/Credential.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.ozone.s3.header;
+package org.apache.hadoop.ozone.s3.signature;
 
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
@@ -107,4 +107,10 @@ public class Credential {
   public String getCredential() {
     return credential;
   }
+
+  public String createScope() {
+    return String.format("%s/%s/%s/%s", getDate(),
+        getAwsRegion(), getAwsService(),
+        getAwsRequest());
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureInfo.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureInfo.java
new file mode 100644
index 0000000..cadbd5d
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureInfo.java
@@ -0,0 +1,113 @@
+/**
+ * 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.signature;
+
+/**
+ * Signature and related information.
+ * <p>
+ * Required to create stringToSign and token.
+ */
+public class SignatureInfo {
+
+  private Version version;
+
+  /**
+   * Information comes from the credential (Date only).
+   */
+  private String date;
+
+  /**
+   * Information comes from header/query param (full timestamp).
+   */
+  private String dateTime;
+
+  private String awsAccessId;
+
+  private String signature;
+
+  private String signedHeaders;
+
+  private String credentialScope;
+
+  private String algorithm;
+
+  private boolean signPayload = true;
+
+  @SuppressWarnings("checkstyle:ParameterNumber")
+  public SignatureInfo(
+      Version version,
+      String date,
+      String dateTime,
+      String awsAccessId,
+      String signature,
+      String signedHeaders,
+      String credentialScope,
+      String algorithm,
+      boolean signPayload
+  ) {
+    this.version = version;
+    this.date = date;
+    this.dateTime = dateTime;
+    this.awsAccessId = awsAccessId;
+    this.signature = signature;
+    this.signedHeaders = signedHeaders;
+    this.credentialScope = credentialScope;
+    this.algorithm = algorithm;
+    this.signPayload = signPayload;
+  }
+
+  public String getAwsAccessId() {
+    return awsAccessId;
+  }
+
+  public String getSignature() {
+    return signature;
+  }
+
+  public String getDate() {
+    return date;
+  }
+
+  public String getSignedHeaders() {
+    return signedHeaders;
+  }
+
+  public String getCredentialScope() {
+    return credentialScope;
+  }
+
+  public String getAlgorithm() {
+    return algorithm;
+  }
+
+  public Version getVersion() {
+    return version;
+  }
+
+  public boolean isSignPayload() {
+    return signPayload;
+  }
+
+  public String getDateTime() {
+    return dateTime;
+  }
+
+  public enum Version {
+    NONE, V4, V2;
+  }
+}
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/package-info.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java
similarity index 66%
copy from hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/package-info.java
copy to hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java
index 40bc78b..4659573 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/package-info.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -6,17 +6,25 @@
  * 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
- *
+ * <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.signature;
+
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 
 /**
- * This package contains Ozone S3 Authorization header.
+ * Parser contract to extract signature information from header or query.
  */
-package org.apache.hadoop.ozone.s3.header;
\ No newline at end of file
+public interface SignatureParser {
+
+  String AUTHORIZATION_HEADER = "Authorization";
+
+  SignatureInfo parseSignature() throws OS3Exception;
+}
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/SignatureProcessor.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureProcessor.java
similarity index 63%
rename from hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/SignatureProcessor.java
rename to hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureProcessor.java
index 5e2e3fb..c5f1393 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/SignatureProcessor.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureProcessor.java
@@ -15,50 +15,30 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.ozone.s3;
+package org.apache.hadoop.ozone.s3.signature;
 
-import java.time.ZoneOffset;
 import java.time.format.DateTimeFormatter;
 
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+
 /**
  * Parser to request auth parser for http request.
  */
 public interface SignatureProcessor {
 
-  String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
-  String NEWLINE = "\n";
   String CONTENT_TYPE = "content-type";
-  String X_AMAZ_DATE = "X-Amz-Date";
+
   String CONTENT_MD5 = "content-md5";
-  String AUTHORIZATION_HEADER = "Authorization";
-  String X_AMZ_CONTENT_SHA256 = "X-Amz-Content-SHA256";
-  String HOST = "host";
 
   String AWS4_SIGNING_ALGORITHM = "AWS4-HMAC-SHA256";
 
-  /**
-   * Seconds in a week, which is the max expiration time Sig-v4 accepts.
-   */
-  long PRESIGN_URL_MAX_EXPIRATION_SECONDS =
-      60 * 60 * 24 * 7;
-
   String HOST_HEADER = "Host";
 
   DateTimeFormatter DATE_FORMATTER =
       DateTimeFormatter.ofPattern("yyyyMMdd");
 
-  DateTimeFormatter TIME_FORMATTER =
-      DateTimeFormatter.ofPattern("yyyyMMdd'T'HHmmss'Z'")
-          .withZone(ZoneOffset.UTC);
-
   /**
    * API to return string to sign.
    */
-  String getStringToSign() throws Exception;
-
-  String getSignature();
-
-  String getAwsAccessId();
-
-  Exception getException();
+  SignatureInfo parseSignature() throws OS3Exception;
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/StringToSignProducer.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/StringToSignProducer.java
new file mode 100644
index 0000000..3202a96
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/StringToSignProducer.java
@@ -0,0 +1,324 @@
+/**
+ * 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.signature;
+
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.MultivaluedMap;
+import java.io.UnsupportedEncodingException;
+import java.net.InetAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URLEncoder;
+import java.net.UnknownHostException;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.time.LocalDate;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.signature.AWSSignatureProcessor.LowerCaseKeyStringMap;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import static java.time.temporal.ChronoUnit.SECONDS;
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.S3_AUTHINFO_CREATION_ERROR;
+import org.apache.kerby.util.Hex;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Stateless utility to create stringToSign, the base of the signature.
+ */
+public final class StringToSignProducer {
+
+  public static final String X_AMZ_CONTENT_SHA256 = "X-Amz-Content-SHA256";
+  public static final String X_AMAZ_DATE = "X-Amz-Date";
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StringToSignProducer.class);
+  private static final Charset UTF_8 = StandardCharsets.UTF_8;
+  private static final String NEWLINE = "\n";
+  private static final String HOST = "host";
+  private static final String UNSIGNED_PAYLOAD = "UNSIGNED-PAYLOAD";
+  /**
+   * Seconds in a week, which is the max expiration time Sig-v4 accepts.
+   */
+  private static final long PRESIGN_URL_MAX_EXPIRATION_SECONDS =
+      60 * 60 * 24 * 7;
+  public static final DateTimeFormatter TIME_FORMATTER =
+      DateTimeFormatter.ofPattern("yyyyMMdd'T'HHmmss'Z'")
+          .withZone(ZoneOffset.UTC);
+
+  private StringToSignProducer() {
+  }
+
+  public static String createSignatureBase(
+      SignatureInfo signatureInfo,
+      ContainerRequestContext context
+  ) throws Exception {
+    return createSignatureBase(signatureInfo,
+        context.getUriInfo().getRequestUri().getScheme(),
+        context.getMethod(),
+        context.getUriInfo().getRequestUri().getPath(),
+        LowerCaseKeyStringMap.fromHeaderMap(context.getHeaders()),
+        fromMultiValueToSingleValueMap(
+            context.getUriInfo().getQueryParameters()));
+  }
+
+  @VisibleForTesting
+  public static String createSignatureBase(
+      SignatureInfo signatureInfo,
+      String scheme,
+      String method,
+      String uri,
+      LowerCaseKeyStringMap headers,
+      Map<String, String> queryParams
+  ) throws Exception {
+    StringBuilder strToSign = new StringBuilder();
+    // According to AWS sigv4 documentation, authorization header should be
+    // in following format.
+    // Authorization: algorithm Credential=access key ID/credential scope,
+    // SignedHeaders=SignedHeaders, Signature=signature
+
+    // Construct String to sign in below format.
+    // StringToSign =
+    //    Algorithm + \n +
+    //    RequestDateTime + \n +
+    //    CredentialScope + \n +
+    //    HashedCanonicalRequest
+    String credentialScope = signatureInfo.getCredentialScope();
+
+    // If the absolute path is empty, use a forward slash (/)
+    uri = (uri.trim().length() > 0) ? uri : "/";
+    // Encode URI and preserve forward slashes
+    strToSign.append(signatureInfo.getAlgorithm() + NEWLINE);
+    strToSign.append(signatureInfo.getDateTime() + NEWLINE);
+    strToSign.append(credentialScope + NEWLINE);
+
+    String canonicalRequest = buildCanonicalRequest(
+        scheme,
+        method,
+        uri,
+        signatureInfo.getSignedHeaders(),
+        headers,
+        queryParams,
+        !signatureInfo.isSignPayload());
+    strToSign.append(hash(canonicalRequest));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("canonicalRequest:[{}]", canonicalRequest);
+      LOG.debug("StringToSign:[{}]", strToSign);
+    }
+
+    return strToSign.toString();
+  }
+
+  public static Map<String, String> fromMultiValueToSingleValueMap(
+      MultivaluedMap<String, String> queryParameters
+  ) {
+    Map<String, String> result = new HashMap<>();
+    for (String key : queryParameters.keySet()) {
+      result.put(key, queryParameters.getFirst(key));
+    }
+    return result;
+  }
+
+  public static String hash(String payload) throws NoSuchAlgorithmException {
+    MessageDigest md = MessageDigest.getInstance("SHA-256");
+    md.update(payload.getBytes(UTF_8));
+    return Hex.encode(md.digest()).toLowerCase();
+  }
+
+  @VisibleForTesting
+  public static String buildCanonicalRequest(
+      String schema,
+      String method,
+      String uri,
+      String signedHeaders,
+      Map<String, String> headers,
+      Map<String, String> queryParams,
+      boolean unsignedPayload
+  ) throws OS3Exception {
+
+    Iterable<String> parts = split("/", uri);
+    List<String> encParts = new ArrayList<>();
+    for (String p : parts) {
+      encParts.add(urlEncode(p));
+    }
+    String canonicalUri = join("/", encParts);
+
+    String canonicalQueryStr = getQueryParamString(queryParams);
+
+    StringBuilder canonicalHeaders = new StringBuilder();
+
+    for (String header : StringUtils.getStringCollection(signedHeaders, ";")) {
+      canonicalHeaders.append(header.toLowerCase());
+      canonicalHeaders.append(":");
+      if (headers.containsKey(header)) {
+        String headerValue = headers.get(header);
+        canonicalHeaders.append(headerValue);
+        canonicalHeaders.append(NEWLINE);
+
+        // Set for testing purpose only to skip date and host validation.
+        validateSignedHeader(schema, header, headerValue);
+
+      } else {
+        throw new RuntimeException("Header " + header + " not present in " +
+            "request but requested to be signed.");
+      }
+    }
+
+    String payloadHash;
+    if (UNSIGNED_PAYLOAD.equals(
+        headers.get(X_AMZ_CONTENT_SHA256)) || unsignedPayload) {
+      payloadHash = UNSIGNED_PAYLOAD;
+    } else {
+      payloadHash = headers.get(X_AMZ_CONTENT_SHA256);
+    }
+    String canonicalRequest = method + NEWLINE
+        + canonicalUri + NEWLINE
+        + canonicalQueryStr + NEWLINE
+        + canonicalHeaders + NEWLINE
+        + signedHeaders + NEWLINE
+        + payloadHash;
+    return canonicalRequest;
+  }
+
+  /**
+   * String join that also works with empty strings.
+   *
+   * @return joined string
+   */
+  private static String join(String glue, List<String> parts) {
+    StringBuilder result = new StringBuilder();
+    boolean addSeparator = false;
+    for (String p : parts) {
+      if (addSeparator) {
+        result.append(glue);
+      }
+      result.append(p);
+      addSeparator = true;
+    }
+    return result.toString();
+  }
+
+  /**
+   * Returns matching strings.
+   *
+   * @param regex Regular expression to split by
+   * @param whole The string to split
+   * @return pieces
+   */
+  private static Iterable<String> split(String regex, String whole) {
+    Pattern p = Pattern.compile(regex);
+    Matcher m = p.matcher(whole);
+    List<String> result = new ArrayList<>();
+    int pos = 0;
+    while (m.find()) {
+      result.add(whole.substring(pos, m.start()));
+      pos = m.end();
+    }
+    result.add(whole.substring(pos));
+    return result;
+  }
+
+  private static String urlEncode(String str) {
+    try {
+
+      return URLEncoder.encode(str, UTF_8.name())
+          .replaceAll("\\+", "%20")
+          .replaceAll("%7E", "~");
+    } catch (UnsupportedEncodingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static String getQueryParamString(
+      Map<String, String> queryMap
+  ) {
+    List<String> params = new ArrayList<>(queryMap.keySet());
+
+    // Sort by name, then by value
+    Collections.sort(params, (o1, o2) -> o1.equals(o2) ?
+        queryMap.get(o1).compareTo(queryMap.get(o2)) :
+        o1.compareTo(o2));
+
+    StringBuilder result = new StringBuilder();
+    for (String p : params) {
+      if (!p.equals("X-Amz-Signature")) {
+
+        if (result.length() > 0) {
+          result.append("&");
+        }
+        result.append(urlEncode(p));
+        result.append('=');
+
+        result.append(urlEncode(queryMap.get(p)));
+      }
+    }
+    return result.toString();
+  }
+
+  @VisibleForTesting
+  static void validateSignedHeader(
+      String schema,
+      String header,
+      String headerValue
+  )
+      throws OS3Exception {
+    switch (header) {
+    case HOST:
+      try {
+        URI hostUri = new URI(schema + "://" + headerValue);
+        InetAddress.getByName(hostUri.getHost());
+        // TODO: Validate if current request is coming from same host.
+      } catch (UnknownHostException | URISyntaxException e) {
+        LOG.error("Host value mentioned in signed header is not valid. " +
+            "Host:{}", headerValue);
+        throw S3_AUTHINFO_CREATION_ERROR;
+      }
+      break;
+    case X_AMAZ_DATE:
+      LocalDate date = LocalDate.parse(headerValue, TIME_FORMATTER);
+      LocalDate now = LocalDate.now();
+      if (date.isBefore(now.minus(PRESIGN_URL_MAX_EXPIRATION_SECONDS, SECONDS))
+          || date.isAfter(now.plus(PRESIGN_URL_MAX_EXPIRATION_SECONDS,
+          SECONDS))) {
+        LOG.error("AWS date not in valid range. Request timestamp:{} should " +
+                "not be older than {} seconds.", headerValue,
+            PRESIGN_URL_MAX_EXPIRATION_SECONDS);
+        throw S3_AUTHINFO_CREATION_ERROR;
+      }
+      break;
+    case X_AMZ_CONTENT_SHA256:
+      // TODO: Construct request payload and match HEX(SHA256(requestPayload))
+      break;
+    default:
+      break;
+    }
+  }
+
+}
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/package-info.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/package-info.java
similarity index 95%
rename from hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/package-info.java
rename to hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/package-info.java
index 40bc78b..63f18a6 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/header/package-info.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/package-info.java
@@ -19,4 +19,4 @@
 /**
  * This package contains Ozone S3 Authorization header.
  */
-package org.apache.hadoop.ozone.s3.header;
\ No newline at end of file
+package org.apache.hadoop.ozone.s3.signature;
\ No newline at end of file
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestAWSSignatureProcessor.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestAWSSignatureProcessor.java
deleted file mode 100644
index 239e285..0000000
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestAWSSignatureProcessor.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/**
- * 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;
-
-import javax.ws.rs.container.ContainerRequestContext;
-import javax.ws.rs.core.MultivaluedHashMap;
-import javax.ws.rs.core.MultivaluedMap;
-import javax.ws.rs.core.UriInfo;
-import java.net.URI;
-
-import org.apache.hadoop.ozone.s3.exception.OS3Exception;
-import org.apache.hadoop.ozone.s3.header.AuthorizationHeaderV2;
-import org.apache.hadoop.ozone.s3.header.AuthorizationHeaderV4;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.Mockito;
-
-/**
- * Test the Auth parser.
- */
-public class TestAWSSignatureProcessor {
-
-  @Test
-  public void testV4Initialization() throws Exception {
-
-    MultivaluedMap<String, String> headers = new MultivaluedHashMap<>();
-    headers.putSingle("Content-Length", "123");
-    headers.putSingle("Host", "0.0.0.0:9878");
-    headers.putSingle("X-AMZ-Content-Sha256", "Content-SHA");
-    headers.putSingle("X-AMZ-Date", "123");
-    headers.putSingle("Content-Type", "ozone/mpu");
-    headers.putSingle(HeaderPreprocessor.ORIGINAL_CONTENT_TYPE, "streaming");
-
-    String authHeader =
-        "AWS4-HMAC-SHA256 Credential=AKIAJWFJK62WUTKNFJJA/20181009/us-east-1"
-            + "/s3/aws4_request, "
-            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date;"
-            + "content-type, "
-            + "Signature"
-            +
-            "=db81b057718d7c1b3b8dffa29933099551c51d787b3b13b9e0f9ebed45982bf2";
-    headers.putSingle("Authorization",
-        authHeader);
-
-    AuthorizationHeaderV4 parserAuthHeader =
-        new AuthorizationHeaderV4(authHeader) {
-          @Override
-          public void validateDateRange() throws OS3Exception {
-          }
-        };
-
-    MultivaluedMap<String, String> queryParameters = new MultivaluedHashMap<>();
-
-    UriInfo uriInfo = Mockito.mock(UriInfo.class);
-    Mockito.when(uriInfo.getQueryParameters()).thenReturn(queryParameters);
-    Mockito.when(uriInfo.getRequestUri())
-        .thenReturn(new URI("http://localhost/buckets"));
-
-    ContainerRequestContext mock = Mockito.mock(ContainerRequestContext.class);
-    Mockito.when(mock.getHeaders()).thenReturn(headers);
-    Mockito.when(mock.getMethod()).thenReturn("GET");
-    Mockito.when(mock.getUriInfo()).thenReturn(uriInfo);
-
-    AWSSignatureProcessor parser = new AWSSignatureProcessor() {
-      @Override
-      void validateSignedHeader(String header, String headerValue)
-          throws OS3Exception {
-        super.validateSignedHeader(header, headerValue);
-      }
-    };
-    parser.setV4Header(parserAuthHeader);
-    parser.setContext(mock);
-    parser.init();
-
-    Assert.assertTrue(
-        "the ozone/mpu header is not changed back before signature processing",
-        parser.buildCanonicalRequest().contains("content-type:streaming"));
-
-    Assert.assertEquals(
-        "String to sign is invalid",
-        "AWS4-HMAC-SHA256\n"
-            + "123\n"
-            + "20181009/us-east-1/s3/aws4_request\n"
-            +
-            "f20d4de80af2271545385e8d4c7df608cae70a791c69b97aab1527ed93a0d665",
-        parser.getStringToSign());
-  }
-
-  @Test
-  public void testV2Initialization() throws Exception {
-
-    MultivaluedMap<String, String> headers = new MultivaluedHashMap<>();
-    String authHeader = "AWS root:ixWQAgWvJDuqLUqgDG9o4b2HF7c=";
-    headers.putSingle("Authorization", authHeader);
-
-    AuthorizationHeaderV2 parserAuthHeader =
-        new AuthorizationHeaderV2(authHeader);
-
-    MultivaluedMap<String, String> queryParameters = new MultivaluedHashMap<>();
-
-    UriInfo uriInfo = Mockito.mock(UriInfo.class);
-    Mockito.when(uriInfo.getQueryParameters()).thenReturn(queryParameters);
-    Mockito.when(uriInfo.getRequestUri())
-        .thenReturn(new URI("http://localhost/buckets"));
-
-    ContainerRequestContext mock = Mockito.mock(ContainerRequestContext.class);
-    Mockito.when(mock.getHeaders()).thenReturn(headers);
-    Mockito.when(mock.getMethod()).thenReturn("GET");
-    Mockito.when(mock.getUriInfo()).thenReturn(uriInfo);
-
-    AWSSignatureProcessor parser = new AWSSignatureProcessor() {
-      @Override
-      void validateSignedHeader(String header, String headerValue)
-          throws OS3Exception {
-        super.validateSignedHeader(header, headerValue);
-      }
-    };
-    parser.setV2Header(parserAuthHeader);
-    parser.setContext(mock);
-    parser.init();
-
-    Assert.assertEquals("root", parser.getAwsAccessId());
-    Assert.assertEquals("ixWQAgWvJDuqLUqgDG9o4b2HF7c=", parser.getSignature());
-  }
-}
\ No newline at end of file
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestOzoneClientProducer.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestOzoneClientProducer.java
index d1b5e08..127bd4e 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestOzoneClientProducer.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestOzoneClientProducer.java
@@ -6,9 +6,9 @@
  * 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.
@@ -29,13 +29,14 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
-
-import static org.apache.hadoop.ozone.s3.SignatureProcessor.AUTHORIZATION_HEADER;
-import static org.apache.hadoop.ozone.s3.SignatureProcessor.CONTENT_MD5;
-import static org.apache.hadoop.ozone.s3.SignatureProcessor.CONTENT_TYPE;
-import static org.apache.hadoop.ozone.s3.SignatureProcessor.HOST_HEADER;
-import static org.apache.hadoop.ozone.s3.SignatureProcessor.X_AMAZ_DATE;
-import static org.apache.hadoop.ozone.s3.SignatureProcessor.X_AMZ_CONTENT_SHA256;
+import org.apache.hadoop.ozone.s3.signature.AWSSignatureProcessor;
+
+import static org.apache.hadoop.ozone.s3.signature.SignatureParser.AUTHORIZATION_HEADER;
+import static org.apache.hadoop.ozone.s3.signature.SignatureProcessor.CONTENT_MD5;
+import static org.apache.hadoop.ozone.s3.signature.SignatureProcessor.CONTENT_TYPE;
+import static org.apache.hadoop.ozone.s3.signature.SignatureProcessor.HOST_HEADER;
+import static org.apache.hadoop.ozone.s3.signature.StringToSignProducer.X_AMAZ_DATE;
+import static org.apache.hadoop.ozone.s3.signature.StringToSignProducer.X_AMZ_CONTENT_SHA256;
 import static org.junit.Assert.fail;
 
 import org.junit.Assert;
@@ -46,7 +47,7 @@ import org.mockito.Mockito;
 
 /**
  * Test class for @{@link OzoneClientProducer}.
- * */
+ */
 @RunWith(Parameterized.class)
 public class TestOzoneClientProducer {
 
@@ -59,13 +60,13 @@ public class TestOzoneClientProducer {
   private String amzContentSha256;
   private String date;
   private String contentType;
-
-
   private ContainerRequestContext context;
   private UriInfo uriInfo;
 
-  public TestOzoneClientProducer(String authHeader, String contentMd5,
-      String host, String amzContentSha256, String date, String contentType)
+  public TestOzoneClientProducer(
+      String authHeader, String contentMd5,
+      String host, String amzContentSha256, String date, String contentType
+  )
       throws Exception {
     this.authHeader = authHeader;
     this.contentMd5 = contentMd5;
@@ -85,44 +86,9 @@ public class TestOzoneClientProducer {
     producer.setOzoneConfiguration(config);
   }
 
-  @Test
-  public void testGetClientFailure() {
-    try {
-      producer.createClient();
-      fail("testGetClientFailure");
-    } catch (Exception ex) {
-      Assert.assertTrue(ex instanceof OS3Exception);
-    }
-  }
-
-  private void setupContext() throws Exception {
-    headerMap.putSingle(AUTHORIZATION_HEADER, authHeader);
-    headerMap.putSingle(CONTENT_MD5, contentMd5);
-    headerMap.putSingle(HOST_HEADER, host);
-    headerMap.putSingle(X_AMZ_CONTENT_SHA256, amzContentSha256);
-    headerMap.putSingle(X_AMAZ_DATE, date);
-    headerMap.putSingle(CONTENT_TYPE, contentType);
-
-    Mockito.when(uriInfo.getQueryParameters()).thenReturn(queryMap);
-    Mockito.when(uriInfo.getRequestUri()).thenReturn(new URI(""));
-
-    Mockito.when(context.getUriInfo()).thenReturn(uriInfo);
-    Mockito.when(context.getHeaders()).thenReturn(headerMap);
-    Mockito.when(context.getHeaderString(AUTHORIZATION_HEADER))
-        .thenReturn(authHeader);
-    Mockito.when(context.getUriInfo().getQueryParameters())
-        .thenReturn(queryMap);
-
-    AWSSignatureProcessor awsSignatureProcessor = new AWSSignatureProcessor();
-    awsSignatureProcessor.setContext(context);
-    awsSignatureProcessor.init();
-
-    producer.setSignatureParser(awsSignatureProcessor);
-  }
-
   @Parameterized.Parameters
   public static Collection<Object[]> data() {
-    return Arrays.asList(new Object[][]{
+    return Arrays.asList(new Object[][] {
         {
             "AWS4-HMAC-SHA256 Credential=testuser1/20190221/us-west-1/s3" +
                 "/aws4_request, SignedHeaders=content-md5;host;" +
@@ -156,4 +122,38 @@ public class TestOzoneClientProducer {
     });
   }
 
+  @Test
+  public void testGetClientFailure() {
+    try {
+      producer.createClient();
+      fail("testGetClientFailure");
+    } catch (Exception ex) {
+      Assert.assertTrue(ex instanceof OS3Exception);
+    }
+  }
+
+  private void setupContext() throws Exception {
+    headerMap.putSingle(AUTHORIZATION_HEADER, authHeader);
+    headerMap.putSingle(CONTENT_MD5, contentMd5);
+    headerMap.putSingle(HOST_HEADER, host);
+    headerMap.putSingle(X_AMZ_CONTENT_SHA256, amzContentSha256);
+    headerMap.putSingle(X_AMAZ_DATE, date);
+    headerMap.putSingle(CONTENT_TYPE, contentType);
+
+    Mockito.when(uriInfo.getQueryParameters()).thenReturn(queryMap);
+    Mockito.when(uriInfo.getRequestUri()).thenReturn(new URI(""));
+
+    Mockito.when(context.getUriInfo()).thenReturn(uriInfo);
+    Mockito.when(context.getHeaders()).thenReturn(headerMap);
+    Mockito.when(context.getHeaderString(AUTHORIZATION_HEADER))
+        .thenReturn(authHeader);
+    Mockito.when(context.getUriInfo().getQueryParameters())
+        .thenReturn(queryMap);
+
+    AWSSignatureProcessor awsSignatureProcessor = new AWSSignatureProcessor();
+    awsSignatureProcessor.setContext(context);
+
+    producer.setSignatureParser(awsSignatureProcessor);
+  }
+
 }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
index ca6e1bd..738bb1d 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestBucketPut.java
@@ -21,26 +21,22 @@
 package org.apache.hadoop.ozone.s3.endpoint;
 
 import javax.ws.rs.core.Response;
+import java.time.LocalDate;
 
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientStub;
-
-import org.apache.hadoop.ozone.s3.SignatureProcessor;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 
 import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
-import static org.apache.hadoop.ozone.s3.AWSSignatureProcessor.DATE_FORMATTER;
 import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.MALFORMED_HEADER;
+import static org.apache.hadoop.ozone.s3.signature.SignatureProcessor.DATE_FORMATTER;
+import org.junit.Assert;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
-
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.time.LocalDate;
-
 /**
  * This class test Create Bucket functionality.
  */
@@ -59,27 +55,6 @@ public class TestBucketPut {
     // Create HeadBucket and setClient to OzoneClientStub
     bucketEndpoint = new BucketEndpoint();
     bucketEndpoint.setClient(clientStub);
-    bucketEndpoint.setSignatureProcessor(new SignatureProcessor() {
-      @Override
-      public String getStringToSign() throws Exception {
-        return null;
-      }
-
-      @Override
-      public String getSignature() {
-        return null;
-      }
-
-      @Override
-      public String getAwsAccessId() {
-        return OzoneConsts.OZONE;
-      }
-
-      @Override
-      public Exception getException() {
-        return null;
-      }
-    });
   }
 
   @Test
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestRootList.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestRootList.java
index 7c5d2a5..19ab3bf 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestRootList.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestRootList.java
@@ -20,13 +20,10 @@
 
 package org.apache.hadoop.ozone.s3.endpoint;
 
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneClientStub;
 
 import static org.junit.Assert.assertEquals;
-
-import org.apache.hadoop.ozone.s3.SignatureProcessor;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -54,27 +51,6 @@ public class TestRootList {
   @Test
   public void testListBucket() throws Exception {
 
-    rootEndpoint.setSignatureProcessor(new SignatureProcessor() {
-      @Override
-      public String getStringToSign() {
-        return null;
-      }
-
-      @Override
-      public String getSignature() {
-        return null;
-      }
-
-      @Override
-      public String getAwsAccessId() {
-        return OzoneConsts.OZONE;
-      }
-
-      @Override
-      public Exception getException() {
-        return null;
-      }
-    });
     // List operation should succeed even there is no bucket.
     ListBucketResponse response =
         (ListBucketResponse) rootEndpoint.get().getEntity();
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/header/TestAuthorizationHeaderV2.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV2HeaderParser.java
similarity index 69%
rename from hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/header/TestAuthorizationHeaderV2.java
rename to hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV2HeaderParser.java
index 97f7fb4..3e2373b 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/header/TestAuthorizationHeaderV2.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV2HeaderParser.java
@@ -16,27 +16,28 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.ozone.s3.header;
+package org.apache.hadoop.ozone.s3.signature;
 
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
-import org.junit.Test;
-
 
+import org.junit.Assert;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
+import org.junit.Test;
 
 /**
  * This class tests Authorization header format v2.
  */
-public class TestAuthorizationHeaderV2 {
+public class TestAuthorizationV2HeaderParser {
 
   @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");
+      AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
+      final SignatureInfo signatureInfo = v2.parseSignature();
+      assertEquals(signatureInfo.getAwsAccessId(), "accessKey");
+      assertEquals(signatureInfo.getSignature(), "signature");
     } catch (OS3Exception ex) {
       fail("testAuthHeaderV2 failed");
     }
@@ -44,20 +45,18 @@ public class TestAuthorizationHeaderV2 {
 
   @Test
   public void testIncorrectHeader1() throws OS3Exception {
-    try {
-      String auth = "AAA accessKey:signature";
-      new AuthorizationHeaderV2(auth);
-      fail("testIncorrectHeader");
-    } catch (OS3Exception ex) {
-      assertEquals("AuthorizationHeaderMalformed", ex.getCode());
-    }
+    String auth = "AAA accessKey:signature";
+    AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
+    Assert.assertNull(v2.parseSignature());
+
   }
 
   @Test
   public void testIncorrectHeader2() throws OS3Exception {
     try {
       String auth = "AWS :accessKey";
-      new AuthorizationHeaderV2(auth);
+      AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
+      Assert.assertNull(v2.parseSignature());
       fail("testIncorrectHeader");
     } catch (OS3Exception ex) {
       assertEquals("AuthorizationHeaderMalformed", ex.getCode());
@@ -68,7 +67,8 @@ public class TestAuthorizationHeaderV2 {
   public void testIncorrectHeader3() throws OS3Exception {
     try {
       String auth = "AWS :signature";
-      new AuthorizationHeaderV2(auth);
+      AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
+      Assert.assertNull(v2.parseSignature());
       fail("testIncorrectHeader");
     } catch (OS3Exception ex) {
       assertEquals("AuthorizationHeaderMalformed", ex.getCode());
@@ -79,7 +79,8 @@ public class TestAuthorizationHeaderV2 {
   public void testIncorrectHeader4() throws OS3Exception {
     try {
       String auth = "AWS accessKey:";
-      new AuthorizationHeaderV2(auth);
+      AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
+      Assert.assertNull(v2.parseSignature());
       fail("testIncorrectHeader");
     } catch (OS3Exception ex) {
       assertEquals("AuthorizationHeaderMalformed", ex.getCode());
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/header/TestAuthorizationHeaderV4.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV4HeaderParser.java
similarity index 71%
rename from hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/header/TestAuthorizationHeaderV4.java
rename to hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV4HeaderParser.java
index ce80333..93a09c4 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/header/TestAuthorizationHeaderV4.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV4HeaderParser.java
@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -16,25 +16,29 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.ozone.s3.header;
+package org.apache.hadoop.ozone.s3.signature;
+
+import java.time.LocalDate;
 
 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.AWSSignatureProcessor.DATE_FORMATTER;
+import static org.apache.hadoop.ozone.s3.signature.SignatureProcessor.DATE_FORMATTER;
+import org.junit.Assert;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
+import org.junit.Before;
+import org.junit.Test;
 
 /**
  * This class tests Authorization header format v2.
  */
 
-public class TestAuthorizationHeaderV4 {
+public class TestAuthorizationV4HeaderParser {
+
+  private static final String SAMPLE_DATE = "20210202T144559Z";
+
   private String curDate;
 
   @Before
@@ -49,14 +53,14 @@ public class TestAuthorizationHeaderV4 {
         "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(curDate, v4.getDate());
-    assertEquals("us-east-1", v4.getAwsRegion());
-    assertEquals("aws4_request", v4.getAwsRequest());
-    assertEquals("host;range;x-amz-date", v4.getSignedHeaderString());
-    assertEquals("fe5f80f77d5fa3beca038a248ff027", v4.getSignature());
+    AuthorizationV4HeaderParser v4 =
+        new AuthorizationV4HeaderParser(auth, SAMPLE_DATE);
+    final SignatureInfo signatureInfo = v4.parseSignature();
+    assertEquals("ozone", signatureInfo.getAwsAccessId());
+    assertEquals(curDate, signatureInfo.getDate());
+    assertEquals("host;range;x-amz-date", signatureInfo.getSignedHeaders());
+    assertEquals("fe5f80f77d5fa3beca038a248ff027",
+        signatureInfo.getSignature());
   }
 
   @Test
@@ -65,7 +69,9 @@ public class TestAuthorizationHeaderV4 {
       String auth = "AWS4-HMAC-SHA256 " +
           "Credential=ozone/" + curDate + "/us-east-1/s3/aws4_request, " +
           "SignedHeaders=host;range;x-amz-date,";
-      new AuthorizationHeaderV4(auth);
+      AuthorizationV4HeaderParser v4 =
+          new AuthorizationV4HeaderParser(auth, SAMPLE_DATE);
+      v4.parseSignature();
       fail("Exception is expected in case of malformed header");
     } catch (OS3Exception ex) {
       assertEquals("AuthorizationHeaderMalformed", ex.getCode());
@@ -79,7 +85,9 @@ public class TestAuthorizationHeaderV4 {
           "Credential=" + curDate + "/us-east-1/s3/aws4_request, " +
           "SignedHeaders=host;range;x-amz-date, " +
           "Signature=fe5f80f77d5fa3beca038a248ff027";
-      new AuthorizationHeaderV4(auth);
+      AuthorizationV4HeaderParser v4 =
+          new AuthorizationV4HeaderParser(auth, SAMPLE_DATE);
+      v4.parseSignature();
       fail("Exception is expected in case of malformed header");
     } catch (OS3Exception ex) {
       assertEquals("AuthorizationHeaderMalformed", ex.getCode());
@@ -95,16 +103,17 @@ public class TestAuthorizationHeaderV4 {
             + "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(curDate, v4.getDate());
-    assertEquals("us-east-1", v4.getAwsRegion());
-    assertEquals("aws4_request", v4.getAwsRequest());
+    AuthorizationV4HeaderParser v4 = new AuthorizationV4HeaderParser(auth,
+        SAMPLE_DATE);
+    SignatureInfo signature = v4.parseSignature();
+
+    assertEquals("AWS4-HMAC-SHA256", signature.getAlgorithm());
+    assertEquals("ozone", signature.getAwsAccessId());
+    assertEquals(curDate, signature.getDate());
     assertEquals("host;x-amz-content-sha256;x-amz-date",
-        v4.getSignedHeaderString());
-    assertEquals("fe5f80f77d5fa3beca038a248ff027", v4.getSignature());
+        signature.getSignedHeaders());
+    assertEquals("fe5f80f77d5fa3beca038a248ff027", signature.getSignature());
 
   }
 
@@ -113,15 +122,15 @@ public class TestAuthorizationHeaderV4 {
     // Case 1: valid date within range.
     LocalDate now = LocalDate.now();
     String dateStr = DATE_FORMATTER.format(now);
-    validateResponse(dateStr);
+    testRequestWithSpecificDate(dateStr);
 
     // Case 2: Valid date with in range.
     dateStr = DATE_FORMATTER.format(now.plus(1, DAYS));
-    validateResponse(dateStr);
+    testRequestWithSpecificDate(dateStr);
 
     // Case 3: Valid date with in range.
     dateStr = DATE_FORMATTER.format(now.minus(1, DAYS));
-    validateResponse(dateStr);
+    testRequestWithSpecificDate(dateStr);
   }
 
   @Test
@@ -130,54 +139,57 @@ public class TestAuthorizationHeaderV4 {
     LocalDate now = LocalDate.now();
     String dateStr = "";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> validateResponse(dateStr));
+        () -> testRequestWithSpecificDate(dateStr));
 
     // Case 2: Date after yesterday.
     String dateStr2 = DATE_FORMATTER.format(now.plus(2, DAYS));
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> validateResponse(dateStr2));
+        () -> testRequestWithSpecificDate(dateStr2));
 
     // Case 3: Date before yesterday.
     String dateStr3 = DATE_FORMATTER.format(now.minus(2, DAYS));
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> validateResponse(dateStr3));
+        () -> testRequestWithSpecificDate(dateStr3));
   }
 
-  private void validateResponse(String dateStr) throws OS3Exception {
+  private void testRequestWithSpecificDate(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);
+    AuthorizationV4HeaderParser v4 =
+        new AuthorizationV4HeaderParser(auth, SAMPLE_DATE);
+    SignatureInfo signature = v4.parseSignature();
 
-    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("AWS4-HMAC-SHA256", signature.getAlgorithm());
+    assertEquals("ozone", signature.getAwsAccessId());
+    assertEquals(dateStr, signature.getDate());
     assertEquals("host;x-amz-content-sha256;x-amz-date",
-        v4.getSignedHeaderString());
-    assertEquals("fe5f80f77d5fa3beca038a248ff027", v4.getSignature());
+        signature.getSignedHeaders());
+    assertEquals("fe5f80f77d5fa3beca038a248ff027", signature.getSignature());
   }
 
   @Test
   public void testV4HeaderRegionValidationFailure() throws Exception {
     String auth =
-        "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "//s3/aws4_request,"
+        "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));
+        () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
+            .parseSignature());
     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));
+        () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
+            .parseSignature());
   }
 
   @Test
@@ -189,7 +201,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth));
+        () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
+            .parseSignature());
 
     String auth2 =
         "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1" +
@@ -198,7 +211,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth2));
+        () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
+            .parseSignature());
   }
 
   @Test
@@ -210,7 +224,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth));
+        () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
+            .parseSignature());
 
     String auth2 =
         "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
@@ -219,7 +234,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth2));
+        () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
+            .parseSignature());
 
     String auth3 =
         "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
@@ -228,7 +244,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth3));
+        () -> new AuthorizationV4HeaderParser(auth3, SAMPLE_DATE)
+            .parseSignature());
   }
 
   @Test
@@ -240,7 +257,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth));
+        () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
+            .parseSignature());
 
     String auth2 =
         "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
@@ -249,7 +267,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth2));
+        () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
+            .parseSignature());
 
     String auth3 =
         "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
@@ -258,7 +277,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth3));
+        () -> new AuthorizationV4HeaderParser(auth3, SAMPLE_DATE)
+            .parseSignature());
 
     String auth4 =
         "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
@@ -267,7 +287,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth4));
+        () -> new AuthorizationV4HeaderParser(auth4, SAMPLE_DATE)
+            .parseSignature());
   }
 
   @Test
@@ -279,7 +300,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027%";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth));
+        () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
+            .parseSignature());
 
     String auth2 =
         "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
@@ -288,7 +310,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth2));
+        () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
+            .parseSignature());
 
     String auth3 =
         "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
@@ -297,7 +320,8 @@ public class TestAuthorizationHeaderV4 {
             + ""
             + "=";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth3));
+        () -> new AuthorizationV4HeaderParser(auth3, SAMPLE_DATE)
+            .parseSignature());
   }
 
   @Test
@@ -309,7 +333,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth));
+        () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
+            .parseSignature());
 
     String auth2 =
         "SHA-256 Credential=ozone/" + curDate + "/us-east-1/s3" +
@@ -317,8 +342,8 @@ public class TestAuthorizationHeaderV4 {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth2));
+    Assert.assertNull(new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
+        .parseSignature());
 
     String auth3 =
         " Credential=ozone/" + curDate + "/us-east-1/s3" +
@@ -326,8 +351,8 @@ public class TestAuthorizationHeaderV4 {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth3));
+    Assert.assertNull(new AuthorizationV4HeaderParser(auth3, SAMPLE_DATE)
+        .parseSignature());
   }
 
   @Test
@@ -339,7 +364,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth));
+        () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
+            .parseSignature());
 
     String auth2 =
         "AWS4-HMAC-SHA =/" + curDate + "//" +
@@ -348,7 +374,8 @@ public class TestAuthorizationHeaderV4 {
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
     LambdaTestUtils.intercept(OS3Exception.class, "",
-        () -> new AuthorizationHeaderV4(auth2));
+        () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
+            .parseSignature());
   }
 
 }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV4QueryParser.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV4QueryParser.java
new file mode 100644
index 0000000..5f66bf0
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV4QueryParser.java
@@ -0,0 +1,146 @@
+/*
+ * 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.signature;
+
+import java.time.ZonedDateTime;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.ozone.s3.signature.AWSSignatureProcessor.LowerCaseKeyStringMap;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestAuthorizationV4QueryParser {
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testExpiredHeaders() throws Exception {
+
+    //GIVEN
+    Map<String, String> parameters = new HashMap<>();
+    parameters.put("X-Amz-Algorithm", "AWS4-HMAC-SHA256");
+    parameters.put("X-Amz-Credential",
+        "AKIAIOSFODNN7EXAMPLE%2F20130524%2Fus-east-1%2Fs3%2Faws4_request");
+    parameters.put("X-Amz-Date", "20160801T083241Z");
+    parameters.put("X-Amz-Expires", "10000");
+    parameters.put("X-Amz-Signature",
+        "aeeed9bbccd4d02ee5c0109b86d86835f995330da4c265957d157751f604d404");
+
+    AuthorizationV4QueryParser parser =
+        new AuthorizationV4QueryParser(parameters);
+
+    //WHEN
+    parser.parseSignature();
+
+    //THEN
+    Assert.fail("Expired header is not detected");
+  }
+
+  @Test()
+  public void testUnExpiredHeaders() throws Exception {
+
+    //GIVEN
+    Map<String, String> parameters = new HashMap<>();
+    parameters.put("X-Amz-Algorithm", "AWS4-HMAC-SHA256");
+    parameters.put("X-Amz-Credential",
+        "AKIAIOSFODNN7EXAMPLE%2F20130524%2Fus-east-1%2Fs3%2Faws4_request");
+    parameters.put("X-Amz-Date",
+        ZonedDateTime.now().format(StringToSignProducer.TIME_FORMATTER));
+    parameters.put("X-Amz-Expires", "10000");
+    parameters.put("X-Amz-Signature",
+        "aeeed9bbccd4d02ee5c0109b86d86835f995330da4c265957d157751f604d404");
+
+
+    AuthorizationV4QueryParser parser =
+        new AuthorizationV4QueryParser(parameters);
+
+    //WHEN
+    parser.parseSignature();
+
+    //THEN
+    //passed
+  }
+
+  @Test()
+  public void testWithoutExpiration() throws Exception {
+
+    //GIVEN
+    Map<String, String> parameters = new HashMap<>();
+    parameters.put("X-Amz-Algorithm", "AWS4-HMAC-SHA256");
+    parameters.put("X-Amz-Credential",
+        "AKIAIOSFODNN7EXAMPLE%2F20130524%2Fus-east-1%2Fs3%2Faws4_request");
+    parameters.put("X-Amz-Date", "20130524T000000Z");
+    parameters.put("X-Amz-Signature",
+        "aeeed9bbccd4d02ee5c0109b86d86835f995330da4c265957d157751f604d404");
+
+    AuthorizationV4QueryParser parser =
+        new AuthorizationV4QueryParser(parameters);
+
+    //WHEN
+    parser.parseSignature();
+
+    //THEN
+    //passed
+  }
+
+  @Test
+  /**
+   * Based on https://docs.aws.amazon
+   * .com/AmazonS3/latest/API/sigv4-query-string-auth.html.
+   */
+  public void testWithAWSExample() throws Exception {
+
+    Map<String, String> queryParams = new HashMap<>();
+
+    queryParams.put("X-Amz-Algorithm", "AWS4-HMAC-SHA256");
+    queryParams.put("X-Amz-Credential",
+        "AKIAIOSFODNN7EXAMPLE/20130524/us-east-1/s3/aws4_request");
+    queryParams.put("X-Amz-Date", "20130524T000000Z");
+    queryParams.put("X-Amz-Expires", "86400");
+    queryParams.put("X-Amz-SignedHeaders", "host");
+    queryParams.put("X-Amz-Signature",
+        "aeeed9bbccd4d02ee5c0109b86d86835f995330da4c265957d157751f604d404");
+
+    AuthorizationV4QueryParser parser =
+        new AuthorizationV4QueryParser(queryParams) {
+          @Override
+          protected void validateDateAndExpires() {
+            //noop
+          }
+        };
+
+    final SignatureInfo signatureInfo = parser.parseSignature();
+
+    LowerCaseKeyStringMap headers = new LowerCaseKeyStringMap();
+    headers.put("host", "examplebucket.s3.amazonaws.com");
+
+    final String stringToSign =
+        StringToSignProducer.createSignatureBase(signatureInfo, "https", "GET",
+            "/test.txt", headers,
+            queryParams);
+
+    Assert.assertEquals("AWS4-HMAC-SHA256\n"
+            + "20130524T000000Z\n"
+            + "20130524/us-east-1/s3/aws4_request\n"
+            +
+            "3bfa292879f6447bbcda7001decf97f4a54dc650c8942174ae0a9121cf58ad04",
+        stringToSign);
+  }
+
+}
\ No newline at end of file
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestStringToSignProducer.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestStringToSignProducer.java
new file mode 100644
index 0000000..37fdbf0
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestStringToSignProducer.java
@@ -0,0 +1,90 @@
+/*
+ * 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.signature;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.ozone.s3.HeaderPreprocessor;
+import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.s3.signature.AWSSignatureProcessor.LowerCaseKeyStringMap;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test string2sign creation.
+ */
+public class TestStringToSignProducer {
+
+  @Test
+  public void test() throws Exception {
+
+    LowerCaseKeyStringMap headers = new LowerCaseKeyStringMap();
+    headers.put("Content-Length", "123");
+    headers.put("Host", "0.0.0.0:9878");
+    headers.put("X-AMZ-Content-Sha256", "Content-SHA");
+    headers.put("X-AMZ-Date", "123");
+    headers.put("Content-Type", "ozone/mpu");
+    headers.put(HeaderPreprocessor.ORIGINAL_CONTENT_TYPE, "streaming");
+
+    String authHeader =
+        "AWS4-HMAC-SHA256 Credential=AKIAJWFJK62WUTKNFJJA/20181009/us-east-1"
+            + "/s3/aws4_request, "
+            + "SignedHeaders=host;x-amz-content-sha256;x-amz-date;"
+            + "content-type, "
+            + "Signature"
+            +
+            "=db81b057718d7c1b3b8dffa29933099551c51d787b3b13b9e0f9ebed45982bf2";
+
+    headers.put("Authorization",
+        authHeader);
+
+    Map<String, String> queryParameters = new HashMap<>();
+
+    final SignatureInfo signatureInfo =
+        new AuthorizationV4HeaderParser(authHeader, "123") {
+          @Override
+          public void validateDateRange(Credential credentialObj)
+              throws OS3Exception {
+            //NOOP
+          }
+        }.parseSignature();
+
+    headers.fixContentType();
+
+    final String signatureBase =
+        StringToSignProducer.createSignatureBase(
+            signatureInfo,
+            "http",
+            "GET",
+            "/buckets",
+            headers,
+            queryParameters);
+
+    Assert.assertEquals(
+        "String to sign is invalid",
+        "AWS4-HMAC-SHA256\n"
+            + "123\n"
+            + "20181009/us-east-1/s3/aws4_request\n"
+            +
+            "f20d4de80af2271545385e8d4c7df608cae70a791c69b97aab1527ed93a0d665",
+        signatureBase);
+  }
+
+}
\ No newline at end of file

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