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/04/19 10:50:45 UTC

[ozone] branch master updated: HDDS-6525. Add audit log for S3Gateway (#3252)

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 3fd7dc6c77 HDDS-6525. Add audit log for S3Gateway (#3252)
3fd7dc6c77 is described below

commit 3fd7dc6c77d2eb9a43635c91aa7654e0bdcdc72a
Author: Symious <yi...@foxmail.com>
AuthorDate: Tue Apr 19 18:50:40 2022 +0800

    HDDS-6525. Add audit log for S3Gateway (#3252)
---
 .../apache/hadoop/ozone/audit/AuditLoggerType.java |   3 +-
 .../audit/{AuditLoggerType.java => S3GAction.java} |  41 +++--
 .../dist/dev-support/bin/dist-layout-stitching     |   1 +
 .../src/shell/conf/s3g-audit-log4j2.properties     |  90 +++++++++++
 hadoop-ozone/dist/src/shell/ozone/ozone            |   1 +
 hadoop-ozone/s3gateway/pom.xml                     |   4 +
 .../org/apache/hadoop/ozone/s3/ClientIpFilter.java |  69 ++++++++
 .../hadoop/ozone/s3/endpoint/BucketEndpoint.java   | 173 +++++++++++++++------
 .../hadoop/ozone/s3/endpoint/EndpointBase.java     |  56 ++++++-
 .../hadoop/ozone/s3/endpoint/ObjectEndpoint.java   | 127 +++++++++++++--
 .../hadoop/ozone/s3/endpoint/RootEndpoint.java     |  53 +++++--
 .../org/apache/hadoop/ozone/s3/util/S3Utils.java   |  20 +++
 .../hadoop/ozone/s3/TestS3GatewayAuditLog.java     | 156 +++++++++++++++++++
 .../src/test/resources/auditlog.properties         |  76 +++++++++
 14 files changed, 785 insertions(+), 85 deletions(-)

diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java
index dbfde9f555..d37d22118d 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java
@@ -23,7 +23,8 @@ package org.apache.hadoop.ozone.audit;
 public enum AuditLoggerType {
   DNLOGGER("DNAudit"),
   OMLOGGER("OMAudit"),
-  SCMLOGGER("SCMAudit");
+  SCMLOGGER("SCMAudit"),
+  S3GLOGGER("S3GAudit");
 
   private String type;
 
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/S3GAction.java
similarity index 60%
copy from hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java
copy to hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/S3GAction.java
index dbfde9f555..e0cbea9983 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/AuditLoggerType.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/S3GAction.java
@@ -14,24 +14,41 @@
  * License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package org.apache.hadoop.ozone.audit;
 
 /**
- * Enumeration for defining types of Audit Loggers in Ozone.
+ * Enum to define Audit Action types for S3Gateway.
  */
-public enum AuditLoggerType {
-  DNLOGGER("DNAudit"),
-  OMLOGGER("OMAudit"),
-  SCMLOGGER("SCMAudit");
+public enum S3GAction implements AuditAction {
 
-  private String type;
+  //BucketEndpoint
+  GET_BUCKET,
+  CREATE_BUCKET,
+  HEAD_BUCKET,
+  DELETE_BUCKET,
+  GET_ACL,
+  PUT_ACL,
+  LIST_MULTIPART_UPLOAD,
+  MULTI_DELETE,
 
-  public String getType() {
-    return type;
-  }
+  //RootEndpoint
+  LIST_S3_BUCKETS,
 
-  AuditLoggerType(String type) {
-    this.type = type;
+  //ObjectEndpoint
+  CREATE_MULTIPART_KEY,
+  COPY_OBJECT,
+  CREATE_KEY,
+  LIST_PARTS,
+  GET_KEY,
+  HEAD_KEY,
+  INIT_MULTIPART_UPLOAD,
+  COMPLETE_MULTIPART_UPLOAD,
+  ABORT_MULTIPART_UPLOAD,
+  DELETE_KEY;
+
+  @Override
+  public String getAction() {
+    return this.toString();
   }
+
 }
diff --git a/hadoop-ozone/dist/dev-support/bin/dist-layout-stitching b/hadoop-ozone/dist/dev-support/bin/dist-layout-stitching
index 2491a1a565..078824c701 100755
--- a/hadoop-ozone/dist/dev-support/bin/dist-layout-stitching
+++ b/hadoop-ozone/dist/dev-support/bin/dist-layout-stitching
@@ -82,6 +82,7 @@ run cp -r "${ROOT}/hadoop-hdds/common/src/main/conf/" "etc/hadoop"
 run cp "${ROOT}/hadoop-ozone/dist/src/shell/conf/om-audit-log4j2.properties" "etc/hadoop"
 run cp "${ROOT}/hadoop-ozone/dist/src/shell/conf/dn-audit-log4j2.properties" "etc/hadoop"
 run cp "${ROOT}/hadoop-ozone/dist/src/shell/conf/scm-audit-log4j2.properties" "etc/hadoop"
+run cp "${ROOT}/hadoop-ozone/dist/src/shell/conf/s3g-audit-log4j2.properties" "etc/hadoop"
 run cp "${ROOT}/hadoop-ozone/dist/src/shell/conf/ozone-shell-log4j.properties" "etc/hadoop"
 run cp "${ROOT}/hadoop-ozone/dist/src/shell/conf/ozone-site.xml" "etc/hadoop"
 run cp -f "${ROOT}/hadoop-ozone/dist/src/shell/conf/log4j.properties" "etc/hadoop"
diff --git a/hadoop-ozone/dist/src/shell/conf/s3g-audit-log4j2.properties b/hadoop-ozone/dist/src/shell/conf/s3g-audit-log4j2.properties
new file mode 100644
index 0000000000..6d516628c5
--- /dev/null
+++ b/hadoop-ozone/dist/src/shell/conf/s3g-audit-log4j2.properties
@@ -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.
+#
+name=PropertiesConfig
+
+# Checks for config change periodically and reloads
+monitorInterval=30
+
+filter=read,write
+# filter.read.onMatch=DENY avoids logging all READ events
+# filter.read.onMatch=ACCEPT permits logging all READ events
+# The above two settings ignore the log levels in configuration
+# filter.read.onMatch=NEUTRAL permits logging of only those READ events
+# which are attempted at log level equal or greater than log level specified
+# in the configuration
+filter.read.type=MarkerFilter
+filter.read.marker=READ
+filter.read.onMatch=DENY
+filter.read.onMismatch=NEUTRAL
+
+# filter.write.onMatch=DENY avoids logging all WRITE events
+# filter.write.onMatch=ACCEPT permits logging all WRITE events
+# The above two settings ignore the log levels in configuration
+# filter.write.onMatch=NEUTRAL permits logging of only those WRITE events
+# which are attempted at log level equal or greater than log level specified
+# in the configuration
+filter.write.type=MarkerFilter
+filter.write.marker=WRITE
+filter.write.onMatch=NEUTRAL
+filter.write.onMismatch=NEUTRAL
+
+# Log Levels are organized from most specific to least:
+# OFF (most specific, no logging)
+# FATAL (most specific, little data)
+# ERROR
+# WARN
+# INFO
+# DEBUG
+# TRACE (least specific, a lot of data)
+# ALL (least specific, all data)
+
+# Uncomment following section to enable logging to console appender also
+#appenders=console, rolling
+#appender.console.type=Console
+#appender.console.name=STDOUT
+#appender.console.layout.type=PatternLayout
+#appender.console.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
+
+# Comment this line when using both console and rolling appenders
+appenders=rolling
+
+#Rolling File Appender with size & time thresholds.
+#Rolling is triggered when either threshold is breached.
+#The rolled over file is compressed by default
+#Time interval is specified in seconds 86400s=1 day
+appender.rolling.type=RollingFile
+appender.rolling.name=RollingFile
+appender.rolling.fileName =${sys:hadoop.log.dir}/s3g-audit-${hostName}.log
+appender.rolling.filePattern=${sys:hadoop.log.dir}/s3g-audit-${hostName}-%d{yyyy-MM-dd-HH-mm-ss}-%i.log.gz
+appender.rolling.layout.type=PatternLayout
+appender.rolling.layout.pattern=%d{DEFAULT} | %-5level | %c{1} | %msg | %throwable{3} %n
+appender.rolling.policies.type=Policies
+appender.rolling.policies.time.type=TimeBasedTriggeringPolicy
+appender.rolling.policies.time.interval=86400
+appender.rolling.policies.size.type=SizeBasedTriggeringPolicy
+appender.rolling.policies.size.size=64MB
+
+loggers=audit
+logger.audit.type=AsyncLogger
+logger.audit.name=S3GAudit
+logger.audit.level=INFO
+logger.audit.appenderRefs=rolling
+logger.audit.appenderRef.file.ref=RollingFile
+
+rootLogger.level=INFO
+#rootLogger.appenderRefs=stdout
+#rootLogger.appenderRef.stdout.ref=STDOUT
diff --git a/hadoop-ozone/dist/src/shell/ozone/ozone b/hadoop-ozone/dist/src/shell/ozone/ozone
index 3b5ac09a1a..72be8cfb45 100755
--- a/hadoop-ozone/dist/src/shell/ozone/ozone
+++ b/hadoop-ozone/dist/src/shell/ozone/ozone
@@ -167,6 +167,7 @@ function ozonecmd_case
     s3g)
       OZONE_SUBCMD_SUPPORTDAEMONIZATION="true"
       OZONE_CLASSNAME='org.apache.hadoop.ozone.s3.Gateway'
+      OZONE_S3G_OPTS="${OZONE_S3G_OPTS} -Dlog4j.configurationFile=${OZONE_CONF_DIR}/s3g-audit-log4j2.properties"
       OZONE_RUN_ARTIFACT_NAME="ozone-s3gateway"
     ;;
     csi)
diff --git a/hadoop-ozone/s3gateway/pom.xml b/hadoop-ozone/s3gateway/pom.xml
index ba9a7b5325..ec751ed414 100644
--- a/hadoop-ozone/s3gateway/pom.xml
+++ b/hadoop-ozone/s3gateway/pom.xml
@@ -192,6 +192,10 @@
       <artifactId>spotbugs</artifactId>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+    </dependency>
   </dependencies>
 
   <build>
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/ClientIpFilter.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/ClientIpFilter.java
new file mode 100644
index 0000000000..921b18d9b5
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/ClientIpFilter.java
@@ -0,0 +1,69 @@
+/**
+ * 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Priority;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.container.ContainerRequestFilter;
+import javax.ws.rs.container.PreMatching;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.ext.Provider;
+import java.io.IOException;
+
+/**
+ * Filter used to get ClientIP from HttpServletRequest.
+ */
+
+@Provider
+@PreMatching
+@Priority(ClientIpFilter.PRIORITY)
+public class ClientIpFilter implements ContainerRequestFilter {
+
+  public static final int PRIORITY = 200;
+
+  public static final String CLIENT_IP_HEADER = "client_ip";
+
+  private static final Logger LOG = LoggerFactory.getLogger(
+      ClientIpFilter.class);
+
+  @Context
+  private HttpServletRequest httpServletRequest;
+
+  @Override
+  public void filter(ContainerRequestContext request) throws IOException {
+    String clientIp = httpServletRequest.getHeader("x-real-ip");
+
+    if (clientIp == null || clientIp.isEmpty()) {
+      // extract from forward ips
+      String ipForwarded = httpServletRequest.getHeader("x-forwarded-for");
+      String[] ips = ipForwarded == null ? null : ipForwarded.split(",");
+      clientIp = (ips == null || ips.length == 0) ? null : ips[0];
+
+      // extract from remote addr
+      clientIp = (clientIp == null || clientIp.isEmpty()) ?
+          httpServletRequest.getRemoteAddr() : clientIp;
+    }
+    LOG.trace("Real Ip[{}]", clientIp);
+    request.getHeaders().putSingle(CLIENT_IP_HEADER, clientIp);
+  }
+
+}
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 9acd310a13..808a8f0e41 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
@@ -21,6 +21,7 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.ozone.OzoneAcl;
+import org.apache.hadoop.ozone.audit.S3GAction;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneKey;
 import org.apache.hadoop.ozone.client.OzoneMultipartUploadList;
@@ -37,6 +38,7 @@ import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 import org.apache.hadoop.ozone.s3.util.ContinueToken;
 import org.apache.hadoop.ozone.s3.util.S3StorageType;
+import org.apache.hadoop.ozone.s3.util.S3Utils;
 import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
 import org.apache.http.HttpStatus;
 import org.slf4j.Logger;
@@ -63,6 +65,7 @@ import java.util.BitSet;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import static org.apache.hadoop.ozone.OzoneAcl.AclScope.ACCESS;
@@ -87,7 +90,7 @@ public class BucketEndpoint extends EndpointBase {
    */
   @GET
   @SuppressFBWarnings
-  @SuppressWarnings("parameternumber")
+  @SuppressWarnings({"parameternumber", "methodlength"})
   public Response get(
       @PathParam("bucket") String bucketName,
       @QueryParam("delimiter") String delimiter,
@@ -100,32 +103,45 @@ public class BucketEndpoint extends EndpointBase {
       @QueryParam("uploads") String uploads,
       @QueryParam("acl") String aclMarker,
       @Context HttpHeaders hh) throws OS3Exception, IOException {
-
-    if (aclMarker != null) {
-      S3BucketAcl result = getAcl(bucketName);
-      getMetrics().incGetAclSuccess();
-      return Response.ok(result, MediaType.APPLICATION_XML_TYPE).build();
-    }
-
-    if (uploads != null) {
-      return listMultipartUploads(bucketName, prefix);
-    }
-
-    if (prefix == null) {
-      prefix = "";
-    }
-
-
+    S3GAction s3GAction = S3GAction.GET_BUCKET;
+    Map<String, String> auditParams = S3Utils.genAuditParam(
+        "bucket", bucketName,
+        "delimiter", delimiter,
+        "encoding-type", encodingType,
+        "marker", marker,
+        "max-keys", String.valueOf(maxKeys),
+        "prefix", prefix,
+        "continuation-token", continueToken,
+        "start-after", startAfter
+    );
     Iterator<? extends OzoneKey> ozoneKeyIterator;
-
     ContinueToken decodedToken =
         ContinueToken.decodeFromString(continueToken);
 
-    // Assign marker to startAfter. for the compatibility of aws api v1
-    if (startAfter == null && marker != null) {
-      startAfter = marker;
-    }
     try {
+      if (aclMarker != null) {
+        s3GAction = S3GAction.GET_ACL;
+        S3BucketAcl result = getAcl(bucketName);
+        getMetrics().incGetAclSuccess();
+        AUDIT.logReadSuccess(
+            buildAuditMessageForSuccess(s3GAction, auditParams));
+        return Response.ok(result, MediaType.APPLICATION_XML_TYPE).build();
+      }
+
+      if (uploads != null) {
+        s3GAction = S3GAction.LIST_MULTIPART_UPLOAD;
+        return listMultipartUploads(bucketName, prefix);
+      }
+
+      if (prefix == null) {
+        prefix = "";
+      }
+
+      // Assign marker to startAfter. for the compatibility of aws api v1
+      if (startAfter == null && marker != null) {
+        startAfter = marker;
+      }
+      
       OzoneBucket bucket = getBucket(bucketName);
       if (startAfter != null && continueToken != null) {
         // If continuation token and start after both are provided, then we
@@ -139,14 +155,18 @@ public class BucketEndpoint extends EndpointBase {
         ozoneKeyIterator = bucket.listKeys(prefix);
       }
     } catch (OMException ex) {
+      AUDIT.logReadFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
       getMetrics().incGetBucketFailure();
       if (ex.getResult() == ResultCodes.PERMISSION_DENIED) {
         throw newError(S3ErrorTable.ACCESS_DENIED, bucketName, ex);
       } else {
         throw ex;
       }
-    } catch (OS3Exception ex) {
+    } catch (Exception ex) {
       getMetrics().incGetBucketFailure();
+      AUDIT.logReadFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
       throw ex;
     }
 
@@ -218,6 +238,7 @@ public class BucketEndpoint extends EndpointBase {
       response.setTruncated(false);
     }
 
+    AUDIT.logReadSuccess(buildAuditMessageForSuccess(s3GAction, auditParams));
     getMetrics().incGetBucketSuccess();
     response.setKeyCount(
         response.getCommonPrefixes().size() + response.getContents().size());
@@ -229,16 +250,27 @@ public class BucketEndpoint extends EndpointBase {
                       @QueryParam("acl") String aclMarker,
                       @Context HttpHeaders httpHeaders,
                       InputStream body) throws IOException, OS3Exception {
-    if (aclMarker != null) {
-      return putAcl(bucketName, httpHeaders, body);
-    }
+    S3GAction s3GAction = S3GAction.CREATE_BUCKET;
+    Map<String, String> auditParams = S3Utils.genAuditParam(
+        "bucket", bucketName,
+        "acl", aclMarker
+    );
+
     try {
+      if (aclMarker != null) {
+        s3GAction = S3GAction.PUT_ACL;
+        return putAcl(bucketName, httpHeaders, body);
+      }
       String location = createS3Bucket(bucketName);
       LOG.info("Location is {}", location);
+      AUDIT.logWriteSuccess(
+          buildAuditMessageForSuccess(s3GAction, auditParams));
       getMetrics().incCreateBucketSuccess();
       return Response.status(HttpStatus.SC_OK).header("Location", location)
           .build();
     } catch (OMException exception) {
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, exception));
       getMetrics().incCreateBucketFailure();
       if (exception.getResult() == ResultCodes.INVALID_BUCKET_NAME) {
         throw newError(S3ErrorTable.INVALID_BUCKET_NAME, bucketName, exception);
@@ -246,6 +278,10 @@ public class BucketEndpoint extends EndpointBase {
       LOG.error("Error in Create Bucket Request for bucket: {}", bucketName,
           exception);
       throw exception;
+    } catch (Exception ex) {
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
+      throw ex;
     }
   }
 
@@ -253,33 +289,45 @@ public class BucketEndpoint extends EndpointBase {
       @PathParam("bucket") String bucketName,
       @QueryParam("prefix") String prefix)
       throws OS3Exception, IOException {
+    S3GAction s3GAction = S3GAction.LIST_MULTIPART_UPLOAD;
+    Map<String, String> auditParams = S3Utils.genAuditParam(
+        "bucket", bucketName,
+        "prefix", prefix
+    );
 
     OzoneBucket bucket = getBucket(bucketName);
 
-    OzoneMultipartUploadList ozoneMultipartUploadList;
     try {
-      ozoneMultipartUploadList = bucket.listMultipartUploads(prefix);
+      OzoneMultipartUploadList ozoneMultipartUploadList =
+          bucket.listMultipartUploads(prefix);
+
+      ListMultipartUploadsResult result = new ListMultipartUploadsResult();
+      result.setBucket(bucketName);
+
+      ozoneMultipartUploadList.getUploads().forEach(upload -> result.addUpload(
+          new ListMultipartUploadsResult.Upload(
+              upload.getKeyName(),
+              upload.getUploadId(),
+              upload.getCreationTime(),
+              S3StorageType.fromReplicationType(upload.getReplicationType(),
+                  upload.getReplicationFactor())
+          )));
+      AUDIT.logReadSuccess(buildAuditMessageForSuccess(s3GAction, auditParams));
+      getMetrics().incListMultipartUploadsSuccess();
+      return Response.ok(result).build();
     } catch (OMException exception) {
+      AUDIT.logReadFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, exception));
       getMetrics().incListMultipartUploadsFailure();
       if (exception.getResult() == ResultCodes.PERMISSION_DENIED) {
         throw newError(S3ErrorTable.ACCESS_DENIED, prefix, exception);
       }
       throw exception;
+    } catch (Exception ex) {
+      AUDIT.logReadFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
+      throw ex;
     }
-
-    ListMultipartUploadsResult result = new ListMultipartUploadsResult();
-    result.setBucket(bucketName);
-
-    ozoneMultipartUploadList.getUploads().forEach(upload -> result.addUpload(
-        new ListMultipartUploadsResult.Upload(
-            upload.getKeyName(),
-            upload.getUploadId(),
-            upload.getCreationTime(),
-            S3StorageType.fromReplicationType(upload.getReplicationType(),
-                upload.getReplicationFactor())
-        )));
-    getMetrics().incListMultipartUploadsSuccess();
-    return Response.ok(result).build();
   }
 
   /**
@@ -291,9 +339,20 @@ public class BucketEndpoint extends EndpointBase {
   @HEAD
   public Response head(@PathParam("bucket") String bucketName)
       throws OS3Exception, IOException {
-    getBucket(bucketName);
-    getMetrics().incHeadBucketSuccess();
-    return Response.ok().build();
+    S3GAction s3GAction = S3GAction.HEAD_BUCKET;
+    Map<String, String> auditParams = S3Utils.genAuditParam(
+        "bucket", bucketName);
+    try {
+      getBucket(bucketName);
+      AUDIT.logReadSuccess(
+          buildAuditMessageForSuccess(s3GAction, auditParams));
+      getMetrics().incHeadBucketSuccess();
+      return Response.ok().build();
+    } catch (Exception e) {
+      AUDIT.logReadFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, e));
+      throw e;
+    }
   }
 
   /**
@@ -305,10 +364,16 @@ public class BucketEndpoint extends EndpointBase {
   @DELETE
   public Response delete(@PathParam("bucket") String bucketName)
       throws IOException, OS3Exception {
+    S3GAction s3GAction = S3GAction.DELETE_BUCKET;
+    Map<String, String> auditParams = S3Utils.genAuditParam(
+        "bucket", bucketName
+    );
 
     try {
       deleteS3Bucket(bucketName);
     } catch (OMException ex) {
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
       getMetrics().incDeleteBucketFailure();
       if (ex.getResult() == ResultCodes.BUCKET_NOT_EMPTY) {
         throw newError(S3ErrorTable.BUCKET_NOT_EMPTY, bucketName, ex);
@@ -319,8 +384,13 @@ public class BucketEndpoint extends EndpointBase {
       } else {
         throw ex;
       }
+    } catch (Exception ex) {
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
+      throw ex;
     }
 
+    AUDIT.logWriteSuccess(buildAuditMessageForSuccess(s3GAction, auditParams));
     getMetrics().incDeleteBucketSuccess();
     return Response
         .status(HttpStatus.SC_NO_CONTENT)
@@ -340,6 +410,12 @@ public class BucketEndpoint extends EndpointBase {
                                          @QueryParam("delete") String delete,
                                          MultiDeleteRequest request)
       throws OS3Exception, IOException {
+    S3GAction s3GAction = S3GAction.MULTI_DELETE;
+    Map<String, String> auditParams = S3Utils.genAuditParam(
+        "bucket", bucketName,
+        "delete", delete
+    );
+
     OzoneBucket bucket = getBucket(bucketName);
     MultiDeleteResponse result = new MultiDeleteResponse();
     if (request.getObjects() != null) {
@@ -369,6 +445,13 @@ public class BucketEndpoint extends EndpointBase {
         }
       }
     }
+    if (result.getErrors().size() != 0) {
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(s3GAction, auditParams,
+          new Exception("MultiDelete Exception")));
+    } else {
+      AUDIT.logWriteSuccess(
+          buildAuditMessageForSuccess(s3GAction, auditParams));
+    }
     return result;
   }
 
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 5b051e099a..b6a928f289 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
@@ -19,12 +19,21 @@ package org.apache.hadoop.ozone.s3.endpoint;
 
 import javax.annotation.PostConstruct;
 import javax.inject.Inject;
+import javax.ws.rs.container.ContainerRequestContext;
+import javax.ws.rs.core.Context;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.Iterator;
+import java.util.Map;
 import java.util.function.Function;
 
 import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.ozone.audit.AuditAction;
+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.audit.Auditor;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneVolume;
@@ -39,20 +48,27 @@ import org.apache.hadoop.ozone.s3.metrics.S3GatewayMetrics;
 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;
 
 /**
  * Basic helpers for all the REST endpoints.
  */
-public abstract class EndpointBase {
+public abstract class EndpointBase implements Auditor {
 
   @Inject
   private OzoneClient client;
   @Inject
   private S3Auth s3Auth;
+  @Context
+  private ContainerRequestContext context;
+
   private static final Logger LOG =
       LoggerFactory.getLogger(EndpointBase.class);
 
+  protected static final AuditLogger AUDIT =
+      new AuditLogger(AuditLoggerType.S3GLOGGER);
+
   protected OzoneBucket getBucket(OzoneVolume volume, String bucketName)
       throws OS3Exception, IOException {
     OzoneBucket bucket;
@@ -192,6 +208,40 @@ public abstract class EndpointBase {
     }
   }
 
+  private AuditMessage.Builder auditMessageBaseBuilder(AuditAction op,
+      Map<String, String> auditMap) {
+    AuditMessage.Builder builder = new AuditMessage.Builder()
+        .forOperation(op)
+        .withParams(auditMap);
+    if (s3Auth != null &&
+        s3Auth.getAccessID() != null &&
+        !s3Auth.getAccessID().isEmpty()) {
+      builder.setUser(s3Auth.getAccessID());
+    }
+    if (context != null) {
+      builder.atIp(getClientIpAddress());
+    }
+    return builder;
+  }
+
+  @Override
+  public AuditMessage buildAuditMessageForSuccess(AuditAction op,
+      Map<String, String> auditMap) {
+    AuditMessage.Builder builder = auditMessageBaseBuilder(op, auditMap)
+        .withResult(AuditEventStatus.SUCCESS);
+    return builder.build();
+  }
+
+  @Override
+  public AuditMessage buildAuditMessageForFailure(AuditAction op,
+      Map<String, String> auditMap, Throwable throwable) {
+    AuditMessage.Builder builder = auditMessageBaseBuilder(op, auditMap)
+        .withResult(AuditEventStatus.FAILURE)
+        .withException(throwable);
+    return builder.build();
+  }
+
+
   @VisibleForTesting
   public void setClient(OzoneClient ozoneClient) {
     this.client = ozoneClient;
@@ -205,4 +255,8 @@ public abstract class EndpointBase {
   public S3GatewayMetrics getMetrics() {
     return S3GatewayMetrics.create();
   }
+
+  public String getClientIpAddress() {
+    return context.getHeaderString(CLIENT_IP_HEADER);
+  }
 }
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 145578b0f6..a852a3c2ab 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
@@ -58,6 +58,7 @@ import org.apache.hadoop.hdds.client.ReplicationType;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.conf.StorageUnit;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.S3GAction;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneKey;
 import org.apache.hadoop.ozone.client.OzoneKeyDetails;
@@ -105,6 +106,7 @@ import static org.apache.hadoop.ozone.s3.util.S3Consts.COPY_SOURCE_IF_UNMODIFIED
 import static org.apache.hadoop.ozone.s3.util.S3Consts.RANGE_HEADER;
 import static org.apache.hadoop.ozone.s3.util.S3Consts.RANGE_HEADER_SUPPORTED_UNIT;
 import static org.apache.hadoop.ozone.s3.util.S3Consts.STORAGE_CLASS_HEADER;
+import static org.apache.hadoop.ozone.s3.util.S3Utils.genAuditParam;
 import static org.apache.hadoop.ozone.s3.util.S3Utils.urlDecode;
 
 import org.apache.http.HttpStatus;
@@ -161,16 +163,29 @@ public class ObjectEndpoint extends EndpointBase {
       @QueryParam("uploadId") @DefaultValue("") String uploadID,
       InputStream body) throws IOException, OS3Exception {
 
-    OzoneOutputStream output = null;
-
-    if (uploadID != null && !uploadID.equals("")) {
-      // If uploadID is specified, it is a request for upload part
-      return createMultipartKey(bucketName, keyPath, length,
-          partNumber, uploadID, body);
+    S3GAction s3GAction = S3GAction.CREATE_KEY;
+    boolean auditSuccess = true;
+    Map<String, String> auditParams = genAuditParam(
+        "bucket", bucketName,
+        "path", keyPath,
+        "Content-Length", String.valueOf(length),
+        "partNumber", String.valueOf(partNumber)
+    );
+    if (partNumber != 0) {
+      auditParams.put("uploadId", uploadID);
     }
 
+    OzoneOutputStream output = null;
+
     String copyHeader = null, storageType = null;
     try {
+      if (uploadID != null && !uploadID.equals("")) {
+        s3GAction = S3GAction.CREATE_MULTIPART_KEY;
+        // If uploadID is specified, it is a request for upload part
+        return createMultipartKey(bucketName, keyPath, length,
+            partNumber, uploadID, body);
+      }
+
       copyHeader = headers.getHeaderString(COPY_SOURCE_HEADER);
       storageType = headers.getHeaderString(STORAGE_CLASS_HEADER);
 
@@ -188,6 +203,7 @@ public class ObjectEndpoint extends EndpointBase {
 
       if (copyHeader != null) {
         //Copy object, as copy source available.
+        s3GAction = S3GAction.COPY_OBJECT;
         CopyObjectResponse copyObjectResponse = copyObject(
             copyHeader, bucketName, keyPath, replicationType,
             replicationFactor, storageTypeDefault);
@@ -212,6 +228,9 @@ public class ObjectEndpoint extends EndpointBase {
       return Response.ok().status(HttpStatus.SC_OK)
           .build();
     } catch (OMException ex) {
+      auditSuccess = false;
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
       if (copyHeader != null) {
         getMetrics().incCopyObjectFailure();
       } else {
@@ -230,7 +249,16 @@ public class ObjectEndpoint extends EndpointBase {
       }
       LOG.error("Exception occurred in PutObject", ex);
       throw ex;
+    } catch (Exception ex) {
+      auditSuccess = false;
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
+      throw ex;
     } finally {
+      if (auditSuccess) {
+        AUDIT.logWriteSuccess(
+            buildAuditMessageForSuccess(s3GAction, auditParams));
+      }
       if (output != null) {
         output.close();
       }
@@ -254,10 +282,21 @@ public class ObjectEndpoint extends EndpointBase {
       @QueryParam("max-parts") @DefaultValue("1000") int maxParts,
       @QueryParam("part-number-marker") String partNumberMarker,
       InputStream body) throws IOException, OS3Exception {
-    try {
 
+    S3GAction s3GAction = S3GAction.GET_KEY;
+    boolean auditSuccess = true;
+    Map<String, String> auditParams = genAuditParam(
+        "bucket", bucketName,
+        "path", keyPath,
+        "uploadId", uploadId,
+        "max-parts", String.valueOf(maxParts),
+        "part-number-marker", partNumberMarker
+    );
+
+    try {
       if (uploadId != null) {
         // When we have uploadId, this is the request for list Parts.
+        s3GAction = S3GAction.LIST_PARTS;
         int partMarker = parsePartNumberMarker(partNumberMarker);
         return listParts(bucketName, keyPath, uploadId,
             partMarker, maxParts);
@@ -332,6 +371,10 @@ public class ObjectEndpoint extends EndpointBase {
       getMetrics().incGetKeySuccess();
       return responseBuilder.build();
     } catch (OMException ex) {
+      auditSuccess = false;
+      AUDIT.logReadFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex)
+      );
       if (uploadId != null) {
         getMetrics().incListPartsFailure();
       } else {
@@ -344,6 +387,18 @@ public class ObjectEndpoint extends EndpointBase {
       } else {
         throw ex;
       }
+    } catch (Exception ex) {
+      auditSuccess = false;
+      AUDIT.logReadFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex)
+      );
+      throw ex;
+    } finally {
+      if (auditSuccess) {
+        AUDIT.logReadSuccess(
+            buildAuditMessageForSuccess(s3GAction, auditParams)
+        );
+      }
     }
   }
 
@@ -369,12 +424,19 @@ public class ObjectEndpoint extends EndpointBase {
       @PathParam("bucket") String bucketName,
       @PathParam("path") String keyPath) throws IOException, OS3Exception {
 
-    OzoneKey key;
+    S3GAction s3GAction = S3GAction.HEAD_KEY;
+    Map<String, String> auditParams = genAuditParam(
+        "bucket", bucketName,
+        "keyPath", keyPath
+    );
 
+    OzoneKey key;
     try {
       key = getBucket(bucketName).headObject(keyPath);
       // TODO: return the specified range bytes of this object.
     } catch (OMException ex) {
+      AUDIT.logReadFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
       getMetrics().incHeadKeyFailure();
       if (ex.getResult() == ResultCodes.KEY_NOT_FOUND) {
         // Just return 404 with no content
@@ -384,6 +446,10 @@ public class ObjectEndpoint extends EndpointBase {
       } else {
         throw ex;
       }
+    } catch (Exception ex) {
+      AUDIT.logReadFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
+      throw ex;
     }
 
     ResponseBuilder response = Response.ok().status(HttpStatus.SC_OK)
@@ -392,6 +458,7 @@ public class ObjectEndpoint extends EndpointBase {
         .header("Content-Type", "binary/octet-stream");
     addLastModifiedDate(response, key);
     getMetrics().incHeadKeySuccess();
+    AUDIT.logReadSuccess(buildAuditMessageForSuccess(s3GAction, auditParams));
     return response.build();
   }
 
@@ -438,14 +505,24 @@ public class ObjectEndpoint extends EndpointBase {
       @QueryParam("uploadId") @DefaultValue("") String uploadId) throws
       IOException, OS3Exception {
 
+    S3GAction s3GAction = S3GAction.DELETE_KEY;
+    Map<String, String> auditParams = genAuditParam(
+        "bucket", bucketName,
+        "path", keyPath,
+        "uploadId", uploadId
+    );
+
     try {
       if (uploadId != null && !uploadId.equals("")) {
+        s3GAction = S3GAction.ABORT_MULTIPART_UPLOAD;
         return abortMultipartUpload(bucketName, keyPath, uploadId);
       }
       OzoneBucket bucket = getBucket(bucketName);
       bucket.getKey(keyPath);
       bucket.deleteKey(keyPath);
     } catch (OMException ex) {
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
       if (uploadId != null && !uploadId.equals("")) {
         getMetrics().incAbortMultiPartUploadFailure();
       } else {
@@ -466,9 +543,13 @@ public class ObjectEndpoint extends EndpointBase {
       } else {
         throw ex;
       }
-
+    } catch (Exception ex) {
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
+      throw ex;
     }
     getMetrics().incDeleteKeySuccess();
+    AUDIT.logWriteSuccess(buildAuditMessageForSuccess(s3GAction, auditParams));
     return Response
         .status(Status.NO_CONTENT)
         .build();
@@ -487,6 +568,12 @@ public class ObjectEndpoint extends EndpointBase {
       @PathParam("path") String key
   )
       throws IOException, OS3Exception {
+    S3GAction s3GAction = S3GAction.INIT_MULTIPART_UPLOAD;
+    Map<String, String> auditParams = genAuditParam(
+        "bucket", bucket,
+        "path", key
+    );
+
     try {
       OzoneBucket ozoneBucket = getBucket(bucket);
       String storageType = headers.getHeaderString(STORAGE_CLASS_HEADER);
@@ -510,10 +597,14 @@ public class ObjectEndpoint extends EndpointBase {
       multipartUploadInitiateResponse.setKey(key);
       multipartUploadInitiateResponse.setUploadID(multipartInfo.getUploadID());
 
+      AUDIT.logWriteSuccess(
+          buildAuditMessageForSuccess(s3GAction, auditParams));
       getMetrics().incInitMultiPartUploadSuccess();
       return Response.status(Status.OK).entity(
           multipartUploadInitiateResponse).build();
     } catch (OMException ex) {
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
       getMetrics().incInitMultiPartUploadFailure();
       if (ex.getResult() == ResultCodes.PERMISSION_DENIED) {
         throw newError(S3ErrorTable.ACCESS_DENIED, key, ex);
@@ -521,6 +612,10 @@ public class ObjectEndpoint extends EndpointBase {
       LOG.error("Error in Initiate Multipart Upload Request for bucket: {}, " +
           "key: {}", bucket, key, ex);
       throw ex;
+    } catch (Exception ex) {
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
+      throw ex;
     }
   }
 
@@ -534,6 +629,12 @@ public class ObjectEndpoint extends EndpointBase {
       @QueryParam("uploadId") @DefaultValue("") String uploadID,
       CompleteMultipartUploadRequest multipartUploadRequest)
       throws IOException, OS3Exception {
+    S3GAction s3GAction = S3GAction.COMPLETE_MULTIPART_UPLOAD;
+    Map<String, String> auditParams = genAuditParam(
+        "bucket", bucket,
+        "path", key,
+        "uploadId", uploadID
+    );
     OzoneBucket ozoneBucket = getBucket(bucket);
     // Using LinkedHashMap to preserve ordering of parts list.
     Map<Integer, String> partsMap = new LinkedHashMap<>();
@@ -559,10 +660,14 @@ public class ObjectEndpoint extends EndpointBase {
           .getHash());
       // Location also setting as bucket name.
       completeMultipartUploadResponse.setLocation(bucket);
+      AUDIT.logWriteSuccess(
+          buildAuditMessageForSuccess(s3GAction, auditParams));
       getMetrics().incCompleteMultiPartUploadSuccess();
       return Response.status(Status.OK).entity(completeMultipartUploadResponse)
           .build();
     } catch (OMException ex) {
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
       getMetrics().incCompleteMultiPartUploadFailure();
       if (ex.getResult() == ResultCodes.INVALID_PART) {
         throw newError(S3ErrorTable.INVALID_PART, key, ex);
@@ -590,6 +695,10 @@ public class ObjectEndpoint extends EndpointBase {
       LOG.error("Error in Complete Multipart Upload Request for bucket: {}, " +
           ", key: {}", bucket, key, ex);
       throw ex;
+    } catch (Exception ex) {
+      AUDIT.logWriteFailure(
+          buildAuditMessageForFailure(s3GAction, auditParams, ex));
+      throw ex;
     }
   }
 
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java
index 615fb01271..57d0d12cfa 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java
@@ -21,8 +21,10 @@ import javax.ws.rs.GET;
 import javax.ws.rs.Path;
 import javax.ws.rs.core.Response;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.Iterator;
 
+import org.apache.hadoop.ozone.audit.S3GAction;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.s3.commontypes.BucketMetadata;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
@@ -48,26 +50,43 @@ public class RootEndpoint extends EndpointBase {
   @GET
   public Response get()
       throws OS3Exception, IOException {
-    ListBucketResponse response = new ListBucketResponse();
-
-    Iterator<? extends OzoneBucket> bucketIterator;
+    boolean auditSuccess = true;
     try {
-      bucketIterator = listS3Buckets(null);
-    } catch (Exception e) {
-      getMetrics().incListS3BucketsFailure();
-      throw e;
-    }
+      ListBucketResponse response = new ListBucketResponse();
 
-    while (bucketIterator.hasNext()) {
-      OzoneBucket next = bucketIterator.next();
-      BucketMetadata bucketMetadata = new BucketMetadata();
-      bucketMetadata.setName(next.getName());
-      bucketMetadata.setCreationDate(next.getCreationTime());
-      response.addBucket(bucketMetadata);
-    }
+      Iterator<? extends OzoneBucket> bucketIterator;
+      try {
+        bucketIterator = listS3Buckets(null);
+      } catch (Exception e) {
+        getMetrics().incListS3BucketsFailure();
+        throw e;
+      }
 
-    getMetrics().incListS3BucketsSuccess();
-    return Response.ok(response).build();
+      while (bucketIterator.hasNext()) {
+        OzoneBucket next = bucketIterator.next();
+        BucketMetadata bucketMetadata = new BucketMetadata();
+        bucketMetadata.setName(next.getName());
+        bucketMetadata.setCreationDate(next.getCreationTime());
+        response.addBucket(bucketMetadata);
+      }
+
+      getMetrics().incListS3BucketsSuccess();
+      return Response.ok(response).build();
+    } catch (Exception ex) {
+      auditSuccess = false;
+      AUDIT.logReadFailure(
+          buildAuditMessageForFailure(S3GAction.LIST_S3_BUCKETS,
+              Collections.emptyMap(), ex)
+      );
+      throw ex;
+    } finally {
+      if (auditSuccess) {
+        AUDIT.logReadSuccess(
+            buildAuditMessageForSuccess(S3GAction.LIST_S3_BUCKETS,
+                Collections.emptyMap())
+        );
+      }
+    }
   }
 
   @Override
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
index c9e16d1b80..7a07a89791 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java
@@ -17,9 +17,13 @@
  */
 package org.apache.hadoop.ozone.s3.util;
 
+import org.apache.commons.lang3.StringUtils;
+
 import java.io.UnsupportedEncodingException;
 import java.net.URLDecoder;
 import java.net.URLEncoder;
+import java.util.Map;
+import java.util.TreeMap;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
@@ -38,6 +42,22 @@ public final class S3Utils {
     return URLEncoder.encode(str, UTF_8.name());
   }
 
+  public static Map<String, String> genAuditParam(String... strs) {
+    if (strs.length % 2 == 1) {
+      throw new IllegalArgumentException("Unexpected number of parameters: "
+          + strs.length);
+    }
+    Map<String, String> auditParams = new TreeMap<>();
+    for (int i = 0; i < strs.length; i++) {
+      if (StringUtils.isEmpty(strs[i]) || StringUtils.isEmpty(strs[i + 1])) {
+        ++i;
+        continue;
+      }
+      auditParams.put(strs[i], strs[++i]);
+    }
+    return auditParams;
+  }
+
   private S3Utils() {
     // no instances
   }
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestS3GatewayAuditLog.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestS3GatewayAuditLog.java
new file mode 100644
index 0000000000..a598bdea60
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/TestS3GatewayAuditLog.java
@@ -0,0 +1,156 @@
+/*
+ * 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 org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientStub;
+import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.s3.endpoint.BucketEndpoint;
+import org.apache.hadoop.ozone.s3.endpoint.ObjectEndpoint;
+import org.apache.hadoop.ozone.s3.endpoint.RootEndpoint;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for S3Gateway Audit Log.
+ */
+public class TestS3GatewayAuditLog {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestS3GatewayAuditLog.class.getName());
+
+  static {
+    System.setProperty("log4j.configurationFile", "auditlog.properties");
+    System.setProperty("log4j2.contextSelector",
+        "org.apache.logging.log4j.core.async.AsyncLoggerContextSelector");
+  }
+
+  private String bucketName = OzoneConsts.BUCKET;
+  private OzoneClient clientStub;
+  private BucketEndpoint bucketEndpoint;
+  private RootEndpoint rootEndpoint;
+  private ObjectEndpoint keyEndpoint;
+  private OzoneBucket bucket;
+
+  @Before
+  public void setup() throws Exception {
+
+    clientStub = new OzoneClientStub();
+    clientStub.getObjectStore().createS3Bucket(bucketName);
+    bucket = clientStub.getObjectStore().getS3Bucket(bucketName);
+
+    bucketEndpoint = new BucketEndpoint();
+    bucketEndpoint.setClient(clientStub);
+
+    rootEndpoint = new RootEndpoint();
+    rootEndpoint.setClient(clientStub);
+
+    keyEndpoint = new ObjectEndpoint();
+    keyEndpoint.setClient(clientStub);
+    keyEndpoint.setOzoneConfiguration(new OzoneConfiguration());
+
+  }
+
+  @AfterClass
+  public static void tearDown() {
+    File file = new File("audit.log");
+    if (FileUtils.deleteQuietly(file)) {
+      LOG.info("{} has been deleted as all tests have completed.",
+          file.getName());
+    } else {
+      LOG.info("audit.log could not be deleted.");
+    }
+  }
+
+  @Test
+  public void testHeadBucket() throws Exception {
+    bucketEndpoint.head(bucketName);
+    String expected = "INFO  | S3GAudit | ? | user=null | ip=null | " +
+        "op=HEAD_BUCKET {bucket=bucket} | ret=SUCCESS";
+    verifyLog(expected);
+  }
+
+  @Test
+  public void testListBucket() throws Exception {
+
+    rootEndpoint.get().getEntity();
+    String expected = "INFO  | S3GAudit | ? | user=null | ip=null | " +
+        "op=LIST_S3_BUCKETS {} | ret=SUCCESS";
+    verifyLog(expected);
+  }
+
+  @Test
+  public void testHeadObject() throws Exception {
+    String value = RandomStringUtils.randomAlphanumeric(32);
+    OzoneOutputStream out = bucket.createKey("key1",
+        value.getBytes(UTF_8).length, ReplicationType.RATIS,
+        ReplicationFactor.ONE, new HashMap<>());
+    out.write(value.getBytes(UTF_8));
+    out.close();
+
+
+    keyEndpoint.head(bucketName, "key1");
+    String expected = "INFO  | S3GAudit | ? | user=null | ip=null | " +
+        "op=HEAD_KEY {bucket=bucket, keyPath=key1} | ret=SUCCESS";
+    verifyLog(expected);
+
+  }
+
+  private void verifyLog(String expectedString) throws IOException {
+    File file = new File("audit.log");
+    List<String> lines = FileUtils.readLines(file, (String)null);
+    final int retry = 5;
+    int i = 0;
+    while (lines.isEmpty() && i < retry) {
+      lines = FileUtils.readLines(file, (String)null);
+      try {
+        Thread.sleep(500 * (i + 1));
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+        break;
+      }
+      i++;
+    }
+    assertEquals(lines.get(0), expectedString);
+
+    //empty the file
+    lines.clear();
+    FileUtils.writeLines(file, lines, false);
+  }
+
+}
diff --git a/hadoop-ozone/s3gateway/src/test/resources/auditlog.properties b/hadoop-ozone/s3gateway/src/test/resources/auditlog.properties
new file mode 100644
index 0000000000..18a6b47ac0
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/resources/auditlog.properties
@@ -0,0 +1,76 @@
+#
+# 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.
+#
+name=PropertiesConfig
+
+# Checks for config change periodically and reloads
+monitorInterval=5
+
+filter=read, write
+# filter.read.onMatch = DENY avoids logging all READ events
+# filter.read.onMatch = ACCEPT permits logging all READ events
+# The above two settings ignore the log levels in configuration
+# filter.read.onMatch = NEUTRAL permits logging of only those READ events
+# which are attempted at log level equal or greater than log level specified
+# in the configuration
+filter.read.type = MarkerFilter
+filter.read.marker = READ
+filter.read.onMatch = ACCEPT
+filter.read.onMismatch = NEUTRAL
+
+# filter.write.onMatch = DENY avoids logging all WRITE events
+# filter.write.onMatch = ACCEPT permits logging all WRITE events
+# The above two settings ignore the log levels in configuration
+# filter.write.onMatch = NEUTRAL permits logging of only those WRITE events
+# which are attempted at log level equal or greater than log level specified
+# in the configuration
+filter.write.type = MarkerFilter
+filter.write.marker = WRITE
+filter.write.onMatch = NEUTRAL
+filter.write.onMismatch = NEUTRAL
+
+# Log Levels are organized from most specific to least:
+# OFF (most specific, no logging)
+# FATAL (most specific, little data)
+# ERROR
+# WARN
+# INFO
+# DEBUG
+# TRACE (least specific, a lot of data)
+# ALL (least specific, all data)
+
+appenders = console, audit
+appender.console.type = Console
+appender.console.name = STDOUT
+appender.console.layout.type = PatternLayout
+appender.console.layout.pattern = %-5level | %c{1} | %msg%n
+
+appender.audit.type = File
+appender.audit.name = AUDITLOG
+appender.audit.fileName=audit.log
+appender.audit.layout.type=PatternLayout
+appender.audit.layout.pattern= %-5level | %c{1} | %C | %msg%n
+
+loggers=audit
+logger.audit.type=AsyncLogger
+logger.audit.name=S3GAudit
+logger.audit.level = INFO
+logger.audit.appenderRefs = audit
+logger.audit.appenderRef.file.ref = AUDITLOG
+
+rootLogger.level = INFO
+rootLogger.appenderRefs = stdout
+rootLogger.appenderRef.stdout.ref = STDOUT


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