You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ad...@apache.org on 2022/07/19 06:18:13 UTC

[ozone] branch master updated: HDDS-6904. Cleanup customer related events in S3G logs (#3531)

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

adoroszlai 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 2395d04fcb HDDS-6904. Cleanup customer related events in S3G logs (#3531)
2395d04fcb is described below

commit 2395d04fcb47fba23d87c51e5125dbea20ac3e1b
Author: Duong Nguyen <du...@gmail.com>
AuthorDate: Mon Jul 18 23:18:09 2022 -0700

    HDDS-6904. Cleanup customer related events in S3G logs (#3531)
---
 .../org/apache/hadoop/ozone/audit/AuditLogger.java |  6 ++
 .../org/apache/hadoop/ozone/audit/AuditMarker.java |  3 +-
 .../hadoop/ozone/audit/TestOzoneAuditLogger.java   | 17 ++++
 .../hadoop/ozone/s3/endpoint/BucketEndpoint.java   | 12 +--
 .../hadoop/ozone/s3/endpoint/EndpointBase.java     | 30 +++----
 .../hadoop/ozone/s3/endpoint/ObjectEndpoint.java   | 21 ++---
 .../ozone/s3/signature/AWSSignatureProcessor.java  | 30 ++++++-
 .../{SignatureParser.java => AuthOperation.java}   | 31 +++++--
 .../s3/signature/AuthorizationV2HeaderParser.java  | 12 ++-
 .../s3/signature/AuthorizationV4HeaderParser.java  | 92 ++++++++++----------
 .../s3/signature/AuthorizationV4QueryParser.java   |  3 +-
 .../hadoop/ozone/s3/signature/Credential.java      | 14 ++--
 ...Parser.java => MalformedResourceException.java} | 27 ++++--
 .../hadoop/ozone/s3/signature/SignatureParser.java |  4 +-
 .../apache/hadoop/ozone/s3/util/AuditUtils.java    | 53 ++++++++++++
 .../hadoop/ozone/s3/TestOzoneClientProducer.java   |  4 +
 .../signature/TestAuthorizationV2HeaderParser.java | 70 ++++++----------
 .../signature/TestAuthorizationV4HeaderParser.java | 98 ++++++++++------------
 .../s3/signature/TestStringToSignProducer.java     |  4 +-
 19 files changed, 307 insertions(+), 224 deletions(-)

diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java
index 43fb4e4792..b27fc78771 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLogger.java
@@ -45,6 +45,7 @@ public class AuditLogger {
   private static final String FQCN = AuditLogger.class.getName();
   private static final Marker WRITE_MARKER = AuditMarker.WRITE.getMarker();
   private static final Marker READ_MARKER = AuditMarker.READ.getMarker();
+  private static final Marker AUTH_MARKER = AuditMarker.AUTH.getMarker();
   private final AtomicReference<Set<String>> debugCmdSetRef =
       new AtomicReference<>(new HashSet<>());
   public static final String AUDIT_LOG_DEBUG_CMD_LIST_PREFIX =
@@ -87,6 +88,11 @@ public class AuditLogger {
         msg.getThrowable());
   }
 
+  public void logAuthFailure(AuditMessage msg) {
+    this.logger.logIfEnabled(FQCN, Level.ERROR, AUTH_MARKER, msg,
+        msg.getThrowable());
+  }
+
   public void logReadSuccess(AuditMessage msg) {
     if (shouldLogAtDebug(msg)) {
       this.logger.logIfEnabled(FQCN, Level.DEBUG, READ_MARKER, msg, null);
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMarker.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMarker.java
index 3414aa403b..d9aed2bb65 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMarker.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditMarker.java
@@ -24,7 +24,8 @@ import org.apache.logging.log4j.MarkerManager;
  */
 public enum AuditMarker {
   WRITE(MarkerManager.getMarker("WRITE")),
-  READ(MarkerManager.getMarker("READ"));
+  READ(MarkerManager.getMarker("READ")),
+  AUTH(MarkerManager.getMarker("AUTH")),;
 
   private Marker marker;
 
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java
index d42795ede5..a96469659a 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/audit/TestOzoneAuditLogger.java
@@ -102,6 +102,15 @@ public class TestOzoneAuditLogger {
           .withResult(SUCCESS)
           .withException(null).build();
 
+  private static final AuditMessage AUTH_FAIL_MSG =
+      new AuditMessage.Builder()
+          .setUser(USER)
+          .atIp(IP_ADDRESS)
+          .forOperation(DummyAction.READ_VOLUME)
+          .withParams(PARAMS)
+          .withResult(FAILURE)
+          .withException(null).build();
+
   @AfterAll
   public static void tearDown() {
     File file = new File("audit.log");
@@ -163,6 +172,14 @@ public class TestOzoneAuditLogger {
     verifyLog(expected);
   }
 
+  @Test
+  public void verifyDefaultLogLevelForAuthFailure() throws IOException {
+    AUDIT.logAuthFailure(AUTH_FAIL_MSG);
+    String expected =
+        "ERROR | OMAudit | ? | " + AUTH_FAIL_MSG.getFormattedMessage();
+    verifyLog(expected);
+  }
+
   @Test
   public void messageIncludesAllParts() {
     String message = WRITE_FAIL_MSG.getFormattedMessage();
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
index bf65394654..0ece60bb40 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
@@ -250,22 +250,17 @@ public class BucketEndpoint extends EndpointBase {
         return response;
       }
       String location = createS3Bucket(bucketName);
-      LOG.info("Location is {}", location);
       AUDIT.logWriteSuccess(
           buildAuditMessageForSuccess(s3GAction, getAuditParameters()));
       getMetrics().incCreateBucketSuccess();
       return Response.status(HttpStatus.SC_OK).header("Location", location)
           .build();
     } catch (OMException exception) {
-      AUDIT.logWriteFailure(
-          buildAuditMessageForFailure(s3GAction, getAuditParameters(),
-              exception));
+      auditWriteFailure(s3GAction, exception);
       getMetrics().incCreateBucketFailure();
       if (exception.getResult() == ResultCodes.INVALID_BUCKET_NAME) {
         throw newError(S3ErrorTable.INVALID_BUCKET_NAME, bucketName, exception);
       }
-      LOG.error("Error in Create Bucket Request for bucket: {}", bucketName,
-          exception);
       throw exception;
     } catch (Exception ex) {
       AUDIT.logWriteFailure(
@@ -465,12 +460,12 @@ public class BucketEndpoint extends EndpointBase {
       return result;
     } catch (OMException ex) {
       getMetrics().incGetAclFailure();
+      auditReadFailure(S3GAction.GET_ACL, ex);
       if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) {
         throw newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName, ex);
       } else if (ex.getResult() == ResultCodes.PERMISSION_DENIED) {
         throw newError(S3ErrorTable.ACCESS_DENIED, bucketName, ex);
       } else {
-        LOG.error("Failed to get acl of Bucket " + bucketName, ex);
         throw newError(S3ErrorTable.INTERNAL_ERROR, bucketName, ex);
       }
     } catch (OS3Exception ex) {
@@ -567,13 +562,12 @@ public class BucketEndpoint extends EndpointBase {
       }
     } catch (OMException exception) {
       getMetrics().incPutAclFailure();
+      auditWriteFailure(S3GAction.PUT_ACL, exception);
       if (exception.getResult() == ResultCodes.BUCKET_NOT_FOUND) {
         throw newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName, exception);
       } else if (exception.getResult() == ResultCodes.PERMISSION_DENIED) {
         throw newError(S3ErrorTable.ACCESS_DENIED, bucketName, exception);
       }
-      LOG.error("Error in set ACL Request for bucket: {}", bucketName,
-          exception);
       throw exception;
     } catch (OS3Exception ex) {
       getMetrics().incPutAclFailure();
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 b3f4d914e3..7d875269d7 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
@@ -23,9 +23,7 @@ import javax.ws.rs.container.ContainerRequestContext;
 import javax.ws.rs.core.Context;
 import java.io.IOException;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.function.Function;
 
@@ -46,10 +44,10 @@ import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.ozone.s3.metrics.S3GatewayMetrics;
+import org.apache.hadoop.ozone.s3.util.AuditUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.hadoop.ozone.s3.ClientIpFilter.CLIENT_IP_HEADER;
 import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.newError;
 
 /**
@@ -252,7 +250,7 @@ public abstract class EndpointBase implements Auditor {
       builder.setUser(s3Auth.getAccessID());
     }
     if (context != null) {
-      builder.atIp(getClientIpAddress());
+      builder.atIp(AuditUtils.getClientIpAddress(context));
     }
     return builder;
   }
@@ -289,23 +287,17 @@ public abstract class EndpointBase implements Auditor {
     return S3GatewayMetrics.create();
   }
 
-  public String getClientIpAddress() {
-    return context.getHeaderString(CLIENT_IP_HEADER);
+  protected Map<String, String> getAuditParameters() {
+    return AuditUtils.getAuditParameters(context);
   }
 
-  protected Map<String, String> getAuditParameters() {
-    Map<String, String> res = new HashMap<>();
-    if (context != null) {
-      for (Map.Entry<String, List<String>> entry :
-          context.getUriInfo().getPathParameters().entrySet()) {
-        res.put(entry.getKey(), entry.getValue().toString());
+  protected void auditWriteFailure(AuditAction action, Throwable ex) {
+    AUDIT.logWriteFailure(
+        buildAuditMessageForFailure(action, getAuditParameters(), ex));
+  }
 
-      }
-      for (Map.Entry<String, List<String>> entry :
-          context.getUriInfo().getQueryParameters().entrySet()) {
-        res.put(entry.getKey(), entry.getValue().toString());
-      }
-    }
-    return res;
+  protected void auditReadFailure(AuditAction action, Exception ex) {
+    AUDIT.logReadFailure(
+        buildAuditMessageForFailure(action, getAuditParameters(), ex));
   }
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
index 3ef48ef155..54eee55fb6 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java
@@ -213,8 +213,7 @@ public class ObjectEndpoint extends EndpointBase {
           .build();
     } catch (OMException ex) {
       auditSuccess = false;
-      AUDIT.logWriteFailure(
-          buildAuditMessageForFailure(s3GAction, getAuditParameters(), ex));
+      auditWriteFailure(s3GAction, ex);
       if (copyHeader != null) {
         getMetrics().incCopyObjectFailure();
       } else {
@@ -231,18 +230,15 @@ public class ObjectEndpoint extends EndpointBase {
       } else if (ex.getResult() == ResultCodes.PERMISSION_DENIED) {
         throw newError(S3ErrorTable.ACCESS_DENIED, keyPath, ex);
       }
-      LOG.error("Exception occurred in PutObject", ex);
       throw ex;
     } catch (Exception ex) {
       auditSuccess = false;
-      AUDIT.logWriteFailure(buildAuditMessageForFailure(s3GAction,
-          getAuditParameters(), ex));
+      auditWriteFailure(s3GAction, ex);
       if (copyHeader != null) {
         getMetrics().incCopyObjectFailure();
       } else {
         getMetrics().incCreateKeyFailure();
       }
-      LOG.error("Exception occurred in PutObject", ex.getMessage());
       throw ex;
     } finally {
       if (auditSuccess) {
@@ -574,14 +570,11 @@ public class ObjectEndpoint extends EndpointBase {
       return Response.status(Status.OK).entity(
           multipartUploadInitiateResponse).build();
     } catch (OMException ex) {
-      AUDIT.logWriteFailure(
-          buildAuditMessageForFailure(s3GAction, getAuditParameters(), ex));
+      auditWriteFailure(s3GAction, ex);
       getMetrics().incInitMultiPartUploadFailure();
       if (ex.getResult() == ResultCodes.PERMISSION_DENIED) {
         throw newError(S3ErrorTable.ACCESS_DENIED, key, ex);
       }
-      LOG.error("Error in Initiate Multipart Upload Request for bucket: {}, " +
-          "key: {}", bucket, key, ex);
       throw ex;
     } catch (Exception ex) {
       AUDIT.logWriteFailure(
@@ -651,8 +644,7 @@ public class ObjectEndpoint extends EndpointBase {
       return Response.status(Status.OK).entity(completeMultipartUploadResponse)
           .build();
     } catch (OMException ex) {
-      AUDIT.logWriteFailure(
-          buildAuditMessageForFailure(s3GAction, getAuditParameters(), ex));
+      auditWriteFailure(s3GAction, ex);
       getMetrics().incCompleteMultiPartUploadFailure();
       if (ex.getResult() == ResultCodes.INVALID_PART) {
         throw newError(S3ErrorTable.INVALID_PART, key, ex);
@@ -677,12 +669,9 @@ public class ObjectEndpoint extends EndpointBase {
             "given KeyName caused failure for MPU");
         throw os3Exception;
       }
-      LOG.error("Error in Complete Multipart Upload Request for bucket: {}, " +
-          ", key: {}", bucket, key, ex);
       throw ex;
     } catch (Exception ex) {
-      AUDIT.logWriteFailure(
-          buildAuditMessageForFailure(s3GAction, getAuditParameters(), ex));
+      auditWriteFailure(s3GAction, ex);
       throw ex;
     }
   }
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
index 807b473c71..1ae8682186 100644
--- 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
@@ -28,14 +28,22 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.hadoop.ozone.audit.AuditEventStatus;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.AuditLoggerType;
+import org.apache.hadoop.ozone.audit.AuditMessage;
 import org.apache.hadoop.ozone.s3.HeaderPreprocessor;
 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.ozone.s3.util.AuditUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.hadoop.ozone.s3.exception.S3ErrorTable.MALFORMED_HEADER;
+
 /**
  * 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
@@ -47,6 +55,9 @@ public class AWSSignatureProcessor implements SignatureProcessor {
   private static final Logger LOG =
       LoggerFactory.getLogger(AWSSignatureProcessor.class);
 
+  private static final AuditLogger AUDIT =
+      new AuditLogger(AuditLoggerType.S3GLOGGER);
+
   @Context
   private ContainerRequestContext context;
 
@@ -67,7 +78,13 @@ public class AWSSignatureProcessor implements SignatureProcessor {
 
     SignatureInfo signatureInfo = null;
     for (SignatureParser parser : signatureParsers) {
-      signatureInfo = parser.parseSignature();
+      try {
+        signatureInfo = parser.parseSignature();
+      } catch (MalformedResourceException e) {
+        AuditMessage message = buildAuthFailureMessage(e);
+        AUDIT.logAuthFailure(message);
+        throw S3ErrorTable.newError(MALFORMED_HEADER, e.getResource());
+      }
       if (signatureInfo != null) {
         break;
       }
@@ -81,6 +98,17 @@ public class AWSSignatureProcessor implements SignatureProcessor {
     return signatureInfo;
   }
 
+  private AuditMessage buildAuthFailureMessage(MalformedResourceException e) {
+    AuditMessage message = new AuditMessage.Builder()
+        .forOperation(AuthOperation.fromContext(context))
+        .withParams(AuditUtils.getAuditParameters(context))
+        .atIp(AuditUtils.getClientIpAddress(context))
+        .withResult(AuditEventStatus.FAILURE)
+        .withException(e)
+        .build();
+    return message;
+  }
+
   @VisibleForTesting
   public void setContext(ContainerRequestContext context) {
     this.context = context;
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthOperation.java
similarity index 52%
copy from hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java
copy to hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthOperation.java
index 4659573820..a2f6215f98 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthOperation.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>
+ *
+ *     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.
@@ -17,14 +17,29 @@
  */
 package org.apache.hadoop.ozone.s3.signature;
 
-import org.apache.hadoop.ozone.s3.exception.OS3Exception;
+import org.apache.hadoop.ozone.audit.AuditAction;
+
+import javax.ws.rs.container.ContainerRequestContext;
 
 /**
- * Parser contract to extract signature information from header or query.
+ * Authentication/authorization actions for S3Gateway.
  */
-public interface SignatureParser {
+public class AuthOperation implements AuditAction {
+  private final String path;
+  private final String method;
+
+  public AuthOperation(final String path, final String method) {
+    this.path = path;
+    this.method = method;
+  }
 
-  String AUTHORIZATION_HEADER = "Authorization";
+  public static AuthOperation fromContext(ContainerRequestContext context) {
+    return new AuthOperation("/" + context.getUriInfo().getPath(),
+        context.getMethod());
+  }
 
-  SignatureInfo parseSignature() throws OS3Exception;
+  @Override
+  public String getAction() {
+    return String.format("AUTH(%s %s)", method, path);
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV2HeaderParser.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV2HeaderParser.java
index 1681dea8e1..1384456005 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV2HeaderParser.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV2HeaderParser.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.ozone.s3.signature;
 
-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;
@@ -43,30 +41,30 @@ public class AuthorizationV2HeaderParser implements SignatureParser {
    * AWS AKIAIOSFODNN7EXAMPLE:frJIUN8DYpKDtOLCwo//yllqDzg=
    */
   @Override
-  public SignatureInfo parseSignature() throws OS3Exception {
+  public SignatureInfo parseSignature() throws MalformedResourceException {
     if (authHeader == null || !authHeader.startsWith(IDENTIFIER + " ")) {
       return null;
     }
     String[] split = authHeader.split(" ");
     if (split.length != 2) {
-      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(authHeader);
     }
 
     String identifier = split[0];
     if (!IDENTIFIER.equals(identifier)) {
-      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(authHeader);
     }
 
     String[] remainingSplit = split[1].split(":");
 
     if (remainingSplit.length != 2) {
-      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(authHeader);
     }
 
     String accessKeyID = remainingSplit[0];
     String signature = remainingSplit[1];
     if (isBlank(accessKeyID) || isBlank(signature)) {
-      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(authHeader);
     }
     return new SignatureInfo(
         Version.V2,
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV4HeaderParser.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV4HeaderParser.java
index f62d0f1e3e..628745d198 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV4HeaderParser.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/AuthorizationV4HeaderParser.java
@@ -20,20 +20,21 @@ package org.apache.hadoop.ozone.s3.signature;
 import java.time.LocalDate;
 import java.util.Collection;
 
-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 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.signature.SignatureProcessor.AWS4_SIGNING_ALGORITHM;
 import static org.apache.hadoop.ozone.s3.signature.SignatureProcessor.DATE_FORMATTER;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -66,25 +67,23 @@ public class AuthorizationV4HeaderParser implements SignatureParser {
    * /aws4_request, SignedHeaders=host;x-amz-content-sha256;x-amz-date,
    * Signature
    * =db81b057718d7c1b3b8dffa29933099551c51d787b3b13b9e0f9ebed45982bf2
-   *
-   * @throws OS3Exception
    */
   @SuppressWarnings("StringSplitter")
   @Override
-  public SignatureInfo parseSignature() throws OS3Exception {
+  public SignatureInfo parseSignature() throws MalformedResourceException {
     if (authHeader == null || !authHeader.startsWith("AWS4")) {
       return null;
     }
     int firstSep = authHeader.indexOf(' ');
     if (firstSep < 0) {
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(authHeader);
     }
 
     //split the value parts of the authorization header
     String[] split = authHeader.substring(firstSep + 1).trim().split(", *");
 
     if (split.length != 3) {
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(authHeader);
     }
 
     String algorithm = parseAlgorithm(authHeader.substring(0, firstSep));
@@ -108,7 +107,7 @@ public class AuthorizationV4HeaderParser implements SignatureParser {
    * Validate Signed headers.
    */
   private String parseSignedHeaders(String signedHeadersStr)
-      throws OS3Exception {
+      throws MalformedResourceException {
     if (isNotEmpty(signedHeadersStr)
         && signedHeadersStr.startsWith(SIGNEDHEADERS)) {
       String parsedSignedHeaders =
@@ -116,37 +115,39 @@ public class AuthorizationV4HeaderParser implements SignatureParser {
       Collection<String> signedHeaders =
           StringUtils.getStringCollection(parsedSignedHeaders, ";");
       if (signedHeaders.size() == 0) {
-        LOG.error("No signed headers found. Authheader:{}", authHeader);
-        throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+        throw new MalformedResourceException("No signed headers found.",
+            authHeader);
       }
       return parsedSignedHeaders;
     } else {
-      LOG.error("No signed headers found. Authheader:{}", authHeader);
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException("No signed headers found.",
+          authHeader);
     }
   }
 
   /**
    * Validate signature.
    */
-  private String parseSignature(String signature) throws OS3Exception {
+  private String parseSignature(String signature)
+      throws MalformedResourceException {
     if (signature.startsWith(SIGNATURE)) {
       String parsedSignature = signature.substring(SIGNATURE.length());
       if (isEmpty(parsedSignature)) {
-        LOG.error("Signature can't be empty: {}", signature);
-        throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+        throw new MalformedResourceException(
+            "Signature can't be empty: " + signature,
+            authHeader);
       }
       try {
         Hex.decodeHex(parsedSignature);
       } catch (DecoderException e) {
-        LOG.error("Signature:{} should be in hexa-decimal encoding.",
-            signature);
-        throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+        throw new MalformedResourceException(
+            "Signature:" + signature + " should be in hexa-decimal encoding.",
+            authHeader);
       }
       return parsedSignature;
     } else {
-      LOG.error("No signature found: {}", signature);
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException("No signature found: " + signature,
+          authHeader);
     }
   }
 
@@ -154,7 +155,7 @@ public class AuthorizationV4HeaderParser implements SignatureParser {
    * Validate credentials.
    */
   private Credential parseCredentials(String credential)
-      throws OS3Exception {
+      throws MalformedResourceException {
     Credential credentialObj = null;
     if (isNotEmpty(credential) && credential.startsWith(CREDENTIAL)) {
       credential = credential.substring(CREDENTIAL.length());
@@ -162,58 +163,63 @@ public class AuthorizationV4HeaderParser implements SignatureParser {
       // security comes, it needs to be completed.
       credentialObj = new Credential(credential);
     } else {
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(authHeader);
     }
 
     if (credentialObj.getAccessKeyID().isEmpty()) {
-      LOG.error("AWS access id shouldn't be empty. credential:{}",
-          credential);
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(
+          "AWS access id shouldn't be empty. credential: " + credential,
+          authHeader);
     }
     if (credentialObj.getAwsRegion().isEmpty()) {
-      LOG.error("AWS region shouldn't be empty. credential:{}", credential);
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(
+          "AWS region shouldn't be empty. credential: " + credential,
+          authHeader);
     }
     if (credentialObj.getAwsRequest().isEmpty()) {
-      LOG.error("AWS request shouldn't be empty. credential:{}", credential);
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(
+          "AWS request shouldn't be empty. credential:" + credential,
+          authHeader);
     }
     if (credentialObj.getAwsService().isEmpty()) {
-      LOG.error("AWS service shouldn't be empty. credential:{}",
-          credential);
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(
+          "AWS service shouldn't be empty. credential:" + credential,
+          authHeader);
     }
 
     // Date should not be empty and within valid range.
     if (!credentialObj.getDate().isEmpty()) {
       validateDateRange(credentialObj);
     } else {
-      LOG.error("AWS date shouldn't be empty. credential:{}", credential);
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(
+          "AWS date shouldn't be empty. credential:{}" + credential,
+          authHeader);
     }
     return credentialObj;
   }
 
   @VisibleForTesting
-  public void validateDateRange(Credential credentialObj) throws OS3Exception {
+  public void validateDateRange(Credential credentialObj)
+      throws MalformedResourceException {
     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).", date);
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException(
+          "AWS date not in valid range. Date: " + date + " should not be older "
+              + "than 1 day(i.e yesterday) and greater than 1 day(i.e " +
+              "tomorrow).", authHeader);
     }
   }
 
   /**
    * Validate if algorithm is in expected format.
    */
-  private String parseAlgorithm(String algorithm) throws OS3Exception {
+  private String parseAlgorithm(String algorithm)
+      throws MalformedResourceException {
     if (isEmpty(algorithm) || !algorithm.equals(AWS4_SIGNING_ALGORITHM)) {
-      LOG.error("Unexpected hash algorithm. Algo:{}", algorithm);
-      throw S3ErrorTable.newError(MALFORMED_HEADER, authHeader);
+      throw new MalformedResourceException("Unexpected hash algorithm. Algo:"
+          + algorithm, authHeader);
     }
     return algorithm;
   }
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
index 40f954274f..769375c306 100644
--- 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
@@ -21,7 +21,6 @@ import java.io.UnsupportedEncodingException;
 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;
@@ -51,7 +50,7 @@ public class AuthorizationV4QueryParser implements SignatureParser {
   }
 
   @Override
-  public SignatureInfo parseSignature() throws OS3Exception {
+  public SignatureInfo parseSignature() throws MalformedResourceException {
 
     if (!queryParameters.containsKey("X-Amz-Signature")) {
       return null;
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/Credential.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/Credential.java
index 1783b58731..be9ecce7c0 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/Credential.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/Credential.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.ozone.s3.signature;
 
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
-import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -27,7 +26,6 @@ import org.slf4j.LoggerFactory;
  * Credential in the AWS authorization header.
  * Ref: https://docs.aws.amazon.com/AmazonS3/latest/API/
  * sigv4-auth-using-authorization-header.html
- *
  */
 public class Credential {
   private static final Logger LOG = LoggerFactory.getLogger(Credential.class);
@@ -41,23 +39,24 @@ public class Credential {
 
   /**
    * Construct Credential Object.
+   *
    * @param cred
    */
-  Credential(String cred) throws OS3Exception {
+  Credential(String cred) throws MalformedResourceException {
     this.credential = cred;
     parseCredential();
   }
 
   /**
    * Parse credential value.
-   *
+   * <p>
    * Sample credential value:
    * Credential=AKIAIOSFODNN7EXAMPLE/20130524/us-east-1/s3/aws4_request
    *
    * @throws OS3Exception
    */
   @SuppressWarnings("StringSplitter")
-  public void parseCredential() throws OS3Exception {
+  public void parseCredential() throws MalformedResourceException {
     String[] split = credential.split("/");
     switch (split.length) {
     case 5:
@@ -78,9 +77,8 @@ public class Credential {
       awsRequest = split[5].trim();
       return;
     default:
-      LOG.error("Credentials not in expected format. credential:{}",
-          credential);
-      throw S3ErrorTable.newError(S3ErrorTable.MALFORMED_HEADER, credential);
+      throw new MalformedResourceException(
+          "Credentials not in expected format.", credential);
     }
   }
 
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/MalformedResourceException.java
similarity index 61%
copy from hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java
copy to hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/MalformedResourceException.java
index 4659573820..8cc8497568 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/MalformedResourceException.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>
+ *
+ *     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.
@@ -17,14 +17,23 @@
  */
 package org.apache.hadoop.ozone.s3.signature;
 
-import org.apache.hadoop.ozone.s3.exception.OS3Exception;
-
 /**
- * Parser contract to extract signature information from header or query.
+ * This exception is used to communicate validation errors when parsing
+ * signatures.
  */
-public interface SignatureParser {
+public class MalformedResourceException extends Exception {
+  private final String resource;
+
+  public MalformedResourceException(String resource) {
+    this.resource = resource;
+  }
 
-  String AUTHORIZATION_HEADER = "Authorization";
+  public MalformedResourceException(String message, String resource) {
+    super(message);
+    this.resource = resource;
+  }
 
-  SignatureInfo parseSignature() throws OS3Exception;
+  public String getResource() {
+    return resource;
+  }
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java
index 4659573820..f4ed9da70e 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/signature/SignatureParser.java
@@ -17,8 +17,6 @@
  */
 package org.apache.hadoop.ozone.s3.signature;
 
-import org.apache.hadoop.ozone.s3.exception.OS3Exception;
-
 /**
  * Parser contract to extract signature information from header or query.
  */
@@ -26,5 +24,5 @@ public interface SignatureParser {
 
   String AUTHORIZATION_HEADER = "Authorization";
 
-  SignatureInfo parseSignature() throws OS3Exception;
+  SignatureInfo parseSignature() throws MalformedResourceException;
 }
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/AuditUtils.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/AuditUtils.java
new file mode 100644
index 0000000000..c036322b4f
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/AuditUtils.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.s3.util;
+
+import javax.ws.rs.container.ContainerRequestContext;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.s3.ClientIpFilter.CLIENT_IP_HEADER;
+
+/**
+ * Common utilities for operation auditing purposes.
+ */
+public final class AuditUtils {
+  private AuditUtils() {
+  }
+
+  public static Map<String, String> getAuditParameters(
+      ContainerRequestContext context) {
+    Map<String, String> res = new HashMap<>();
+    if (context != null) {
+      for (Map.Entry<String, List<String>> entry :
+          context.getUriInfo().getPathParameters().entrySet()) {
+        res.put(entry.getKey(), entry.getValue().toString());
+      }
+      for (Map.Entry<String, List<String>> entry :
+          context.getUriInfo().getQueryParameters().entrySet()) {
+        res.put(entry.getKey(), entry.getValue().toString());
+      }
+    }
+    return res;
+  }
+
+  public static String getClientIpAddress(ContainerRequestContext context) {
+    return context.getHeaderString(CLIENT_IP_HEADER);
+  }
+}
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 d02c3ccde8..e1fa059548 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
@@ -58,6 +58,7 @@ public class TestOzoneClientProducer {
   private OzoneClientProducer producer;
   private MultivaluedMap<String, String> headerMap;
   private MultivaluedMap<String, String> queryMap;
+  private MultivaluedMap<String, String> pathParamsMap;
   private String authHeader;
   private String contentMd5;
   private String host;
@@ -81,6 +82,7 @@ public class TestOzoneClientProducer {
     producer = new OzoneClientProducer();
     headerMap = new MultivaluedHashMap<>();
     queryMap = new MultivaluedHashMap<>();
+    pathParamsMap = new MultivaluedHashMap<>();
     uriInfo = Mockito.mock(UriInfo.class);
     context = Mockito.mock(ContainerRequestContext.class);
     OzoneConfiguration config = new OzoneConfiguration();
@@ -219,6 +221,8 @@ public class TestOzoneClientProducer {
         .thenReturn(authHeader);
     Mockito.when(context.getUriInfo().getQueryParameters())
         .thenReturn(queryMap);
+    Mockito.when(context.getUriInfo().getPathParameters())
+        .thenReturn(pathParamsMap);
 
     AWSSignatureProcessor awsSignatureProcessor = new AWSSignatureProcessor();
     awsSignatureProcessor.setContext(context);
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV2HeaderParser.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV2HeaderParser.java
index 3e2373ba8d..c2533f5536 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV2HeaderParser.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV2HeaderParser.java
@@ -18,11 +18,11 @@
 
 package org.apache.hadoop.ozone.s3.signature;
 
-import org.apache.hadoop.ozone.s3.exception.OS3Exception;
-
 import org.junit.Assert;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
+
 import org.junit.Test;
 
 /**
@@ -31,59 +31,43 @@ import org.junit.Test;
 public class TestAuthorizationV2HeaderParser {
 
   @Test
-  public void testAuthHeaderV2() throws OS3Exception {
-    try {
-      String auth = "AWS accessKey: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");
-    }
+  public void testAuthHeaderV2() throws MalformedResourceException {
+    String auth = "AWS accessKey:signature";
+    AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
+    final SignatureInfo signatureInfo = v2.parseSignature();
+    assertEquals(signatureInfo.getAwsAccessId(), "accessKey");
+    assertEquals(signatureInfo.getSignature(), "signature");
   }
 
   @Test
-  public void testIncorrectHeader1() throws OS3Exception {
+  public void testIncorrectHeader1() throws MalformedResourceException {
     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";
-      AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
-      Assert.assertNull(v2.parseSignature());
-      fail("testIncorrectHeader");
-    } catch (OS3Exception ex) {
-      assertEquals("AuthorizationHeaderMalformed", ex.getCode());
-    }
+  @Test(expected = MalformedResourceException.class)
+  public void testIncorrectHeader2() throws MalformedResourceException {
+    String auth = "AWS :accessKey";
+    AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
+    Assert.assertNull(v2.parseSignature());
+    fail("testIncorrectHeader");
   }
 
-  @Test
-  public void testIncorrectHeader3() throws OS3Exception {
-    try {
-      String auth = "AWS :signature";
-      AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
-      Assert.assertNull(v2.parseSignature());
-      fail("testIncorrectHeader");
-    } catch (OS3Exception ex) {
-      assertEquals("AuthorizationHeaderMalformed", ex.getCode());
-    }
+  @Test(expected = MalformedResourceException.class)
+  public void testIncorrectHeader3() throws MalformedResourceException {
+    String auth = "AWS :signature";
+    AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
+    Assert.assertNull(v2.parseSignature());
+    fail("testIncorrectHeader");
   }
 
-  @Test
-  public void testIncorrectHeader4() throws OS3Exception {
-    try {
-      String auth = "AWS accessKey:";
-      AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
-      Assert.assertNull(v2.parseSignature());
-      fail("testIncorrectHeader");
-    } catch (OS3Exception ex) {
-      assertEquals("AuthorizationHeaderMalformed", ex.getCode());
-    }
+  @Test(expected = MalformedResourceException.class)
+  public void testIncorrectHeader4() throws MalformedResourceException {
+    String auth = "AWS accessKey:";
+    AuthorizationV2HeaderParser v2 = new AuthorizationV2HeaderParser(auth);
+    Assert.assertNull(v2.parseSignature());
+    fail("testIncorrectHeader");
   }
 }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV4HeaderParser.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV4HeaderParser.java
index 7df737d91e..1dc02ef2ae 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV4HeaderParser.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/signature/TestAuthorizationV4HeaderParser.java
@@ -20,14 +20,15 @@ package org.apache.hadoop.ozone.s3.signature;
 
 import java.time.LocalDate;
 
-import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.ozone.test.LambdaTestUtils;
 
 import static java.time.temporal.ChronoUnit.DAYS;
 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;
 
@@ -63,39 +64,30 @@ public class TestAuthorizationV4HeaderParser {
         signatureInfo.getSignature());
   }
 
-  @Test
-  public void testV4HeaderMissingParts() {
-    try {
-      String auth = "AWS4-HMAC-SHA256 " +
-          "Credential=ozone/" + curDate + "/us-east-1/s3/aws4_request, " +
-          "SignedHeaders=host;range;x-amz-date,";
-      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());
-    }
+  @Test(expected = MalformedResourceException.class)
+  public void testV4HeaderMissingParts() throws MalformedResourceException {
+    String auth = "AWS4-HMAC-SHA256 " +
+        "Credential=ozone/" + curDate + "/us-east-1/s3/aws4_request, " +
+        "SignedHeaders=host;range;x-amz-date,";
+    AuthorizationV4HeaderParser v4 =
+        new AuthorizationV4HeaderParser(auth, SAMPLE_DATE);
+    v4.parseSignature();
   }
 
-  @Test
-  public void testV4HeaderInvalidCredential() {
-    try {
-      String auth = "AWS4-HMAC-SHA256 " +
-          "Credential=" + curDate + "/us-east-1/s3/aws4_request, " +
-          "SignedHeaders=host;range;x-amz-date, " +
-          "Signature=fe5f80f77d5fa3beca038a248ff027";
-      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());
-    }
+  @Test(expected = MalformedResourceException.class)
+  public void testV4HeaderInvalidCredential()
+      throws MalformedResourceException {
+    String auth = "AWS4-HMAC-SHA256 " +
+        "Credential=" + curDate + "/us-east-1/s3/aws4_request, " +
+        "SignedHeaders=host;range;x-amz-date, " +
+        "Signature=fe5f80f77d5fa3beca038a248ff027";
+    AuthorizationV4HeaderParser v4 =
+        new AuthorizationV4HeaderParser(auth, SAMPLE_DATE);
+    v4.parseSignature();
   }
 
   @Test
-  public void testV4HeaderWithoutSpace() throws OS3Exception {
+  public void testV4HeaderWithoutSpace() throws MalformedResourceException {
 
     String auth =
         "AWS4-HMAC-SHA256 Credential=ozone/" + curDate + "/us-east-1/s3" +
@@ -118,7 +110,8 @@ public class TestAuthorizationV4HeaderParser {
   }
 
   @Test
-  public void testV4HeaderDateValidationSuccess() throws OS3Exception {
+  public void testV4HeaderDateValidationSuccess()
+      throws MalformedResourceException {
     // Case 1: valid date within range.
     LocalDate now = LocalDate.now();
     String dateStr = DATE_FORMATTER.format(now);
@@ -138,21 +131,22 @@ public class TestAuthorizationV4HeaderParser {
     // Case 1: Empty date.
     LocalDate now = LocalDate.now();
     String dateStr = "";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> testRequestWithSpecificDate(dateStr));
 
     // Case 2: Date after yesterday.
     String dateStr2 = DATE_FORMATTER.format(now.plus(2, DAYS));
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> testRequestWithSpecificDate(dateStr2));
 
     // Case 3: Date before yesterday.
     String dateStr3 = DATE_FORMATTER.format(now.minus(2, DAYS));
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> testRequestWithSpecificDate(dateStr3));
   }
 
-  private void testRequestWithSpecificDate(String dateStr) throws OS3Exception {
+  private void testRequestWithSpecificDate(String dateStr)
+      throws MalformedResourceException {
     String auth =
         "AWS4-HMAC-SHA256 Credential=ozone/" + dateStr + "/us-east-1/s3" +
             "/aws4_request,"
@@ -179,7 +173,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027%";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
             .parseSignature());
     String auth2 =
@@ -187,7 +181,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027%";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
             .parseSignature());
   }
@@ -200,7 +194,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
             .parseSignature());
 
@@ -210,7 +204,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
             .parseSignature());
   }
@@ -223,7 +217,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
             .parseSignature());
 
@@ -233,7 +227,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
             .parseSignature());
 
@@ -243,7 +237,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth3, SAMPLE_DATE)
             .parseSignature());
   }
@@ -256,7 +250,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=;;,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
             .parseSignature());
 
@@ -266,7 +260,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
             .parseSignature());
 
@@ -276,7 +270,7 @@ public class TestAuthorizationV4HeaderParser {
             + "=x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth3, SAMPLE_DATE)
             .parseSignature());
 
@@ -286,7 +280,7 @@ public class TestAuthorizationV4HeaderParser {
             + "=,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth4, SAMPLE_DATE)
             .parseSignature());
   }
@@ -299,7 +293,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027%";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
             .parseSignature());
 
@@ -309,7 +303,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
             .parseSignature());
 
@@ -319,7 +313,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + ""
             + "=";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth3, SAMPLE_DATE)
             .parseSignature());
   }
@@ -332,7 +326,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
             .parseSignature());
 
@@ -363,7 +357,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth, SAMPLE_DATE)
             .parseSignature());
 
@@ -373,7 +367,7 @@ public class TestAuthorizationV4HeaderParser {
             + "SignedHeaders=host;x-amz-content-sha256;x-amz-date,"
             + "Signature"
             + "=fe5f80f77d5fa3beca038a248ff027";
-    LambdaTestUtils.intercept(OS3Exception.class, "",
+    LambdaTestUtils.intercept(MalformedResourceException.class, "",
         () -> new AuthorizationV4HeaderParser(auth2, SAMPLE_DATE)
             .parseSignature());
   }
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
index 37fdbf0751..e0b7c1042b 100644
--- 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
@@ -21,7 +21,6 @@ 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;
@@ -60,8 +59,7 @@ public class TestStringToSignProducer {
     final SignatureInfo signatureInfo =
         new AuthorizationV4HeaderParser(authHeader, "123") {
           @Override
-          public void validateDateRange(Credential credentialObj)
-              throws OS3Exception {
+          public void validateDateRange(Credential credentialObj) {
             //NOOP
           }
         }.parseSignature();


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