You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/04/11 13:28:10 UTC

[GitHub] [ozone] adoroszlai commented on a diff in pull request #3252: HDDS-6525. Add audit log for S3Gateway

adoroszlai commented on code in PR #3252:
URL: https://github.com/apache/ozone/pull/3252#discussion_r847300688


##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java:
##########
@@ -48,26 +49,41 @@
   @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.logWriteFailure(

Review Comment:
   `GET` is a read operation.



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java:
##########
@@ -38,7 +40,18 @@ public static String urlEncode(String str)
     return URLEncoder.encode(str, UTF_8.name());
   }
 
+  public static Map<String, String> genAuditParam(String... strs) {
+    if (strs.length / 2 == 0) {
+      throw new IllegalArgumentException();
+    }

Review Comment:
   I think this is intended to require even number of parameters, but it actually only rejects calls with 0 or 1 arguments.
   
   Shouldn't it be:
   
   ```suggestion
       if (strs.length % 2 == 1) {
         throw new IllegalArgumentException("Unexpected number of parameters: "
             + strs.length);
       }
   ```



##########
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=ACCEPT

Review Comment:
   I think we should `DENY` here until #3255 is approved.



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java:
##########
@@ -38,7 +40,18 @@ public static String urlEncode(String str)
     return URLEncoder.encode(str, UTF_8.name());
   }
 
+  public static Map<String, String> genAuditParam(String... strs) {
+    if (strs.length / 2 == 0) {
+      throw new IllegalArgumentException();
+    }
+    Map<String, String> auditParams = new TreeMap<>();
+    for (int i = 0; i < strs.length; i++) {
+      auditParams.put(strs[i], strs[++i]);
+    }
+    return auditParams;
+  }
+
   private S3Utils() {
-    // no instances
+      // no instances

Review Comment:
   Nit: unnecessary space change.



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java:
##########
@@ -48,26 +49,41 @@
   @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.logWriteFailure(
+          buildAuditMessageForFailure(S3GAction.LIST_S3_BUCKETS, null, ex)
+      );
+      throw ex;
+    } finally {
+      if (auditSuccess) {
+        AUDIT.logWriteSuccess(

Review Comment:
   `GET` is a read operation.



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java:
##########
@@ -191,6 +207,44 @@ public void deleteS3Bucket(String s3BucketName)
     }
   }
 
+  @Override
+  public AuditMessage buildAuditMessageForSuccess(AuditAction op,
+      Map<String, String> auditMap) {
+    AuditMessage.Builder builder = new AuditMessage.Builder()
+        .forOperation(op)
+        .withParams(auditMap)
+        .withResult(AuditEventStatus.SUCCESS);
+    if (s3Auth != null &&
+        s3Auth.getAccessID() != null &&
+        !s3Auth.getAccessID().isEmpty()) {
+      builder.setUser(s3Auth.getAccessID());
+    }
+    if (context != null) {
+      builder.atIp(getClientIpAddress());
+    }
+    return builder.build();
+  }
+
+  @Override
+  public AuditMessage buildAuditMessageForFailure(AuditAction op,
+      Map<String, String> auditMap, Throwable throwable) {
+    AuditMessage.Builder builder = new AuditMessage.Builder()
+        .forOperation(op)
+        .withParams(auditMap)
+        .withResult(AuditEventStatus.FAILURE)
+        .withException(throwable);
+    if (s3Auth != null &&
+        s3Auth.getAccessID() != null &&
+        !s3Auth.getAccessID().isEmpty()) {
+      builder.setUser(s3Auth.getAccessID());
+    }
+    if (context != null) {
+      builder.atIp(getClientIpAddress());
+    }

Review Comment:
   Nit: Can we avoid duplication with `buildAuditMessageForSuccess`?



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java:
##########
@@ -101,42 +104,57 @@ public Response get(
       @QueryParam("uploads") String uploads,
       @QueryParam("acl") String aclMarker,
       @Context HttpHeaders hh) throws OS3Exception, IOException {
+    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,
+        "browser", browser,
+        "continuation-token", continueToken,
+        "start-after", startAfter,
+        "uploads", uploads,
+        "acl", aclMarker
+    );
+    Iterator<? extends OzoneKey> ozoneKeyIterator;
+    ContinueToken decodedToken =
+        ContinueToken.decodeFromString(continueToken);
 
-    if (aclMarker != null) {
-      S3BucketAcl result = getAcl(bucketName);
-      getMetrics().incGetAclSuccess();
-      return Response.ok(result, MediaType.APPLICATION_XML_TYPE).build();
-    }
-
-    if (browser != null) {
-      InputStream browserPage = getClass()
-          .getResourceAsStream("/browser.html");
-      return Response.ok(browserPage,
-            MediaType.TEXT_HTML_TYPE)
-            .build();
-
-    }
+    try {
+      if (aclMarker != null) {
+        s3GAction = S3GAction.GET_ACL;
+        S3BucketAcl result = getAcl(bucketName);
+        getMetrics().incGetAclSuccess();
+        return Response.ok(result, MediaType.APPLICATION_XML_TYPE).build();

Review Comment:
   No audit log for `getAcl`?



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/ClientIpFilter.java:
##########
@@ -0,0 +1,70 @@
+/**
+ * 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
+

Review Comment:
   ```suggestion
   ```



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java:
##########
@@ -101,42 +104,57 @@ public Response get(
       @QueryParam("uploads") String uploads,
       @QueryParam("acl") String aclMarker,
       @Context HttpHeaders hh) throws OS3Exception, IOException {
+    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,
+        "browser", browser,
+        "continuation-token", continueToken,
+        "start-after", startAfter,
+        "uploads", uploads,
+        "acl", aclMarker

Review Comment:
   I think we should omit `uploads` and `aclMarker`, they are not real parameters.  Only used to distinguish some operations, which are already indicated in the audit log by the `op` field.



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Utils.java:
##########
@@ -38,7 +40,18 @@ public static String urlEncode(String str)
     return URLEncoder.encode(str, UTF_8.name());
   }
 
+  public static Map<String, String> genAuditParam(String... strs) {
+    if (strs.length / 2 == 0) {
+      throw new IllegalArgumentException();
+    }
+    Map<String, String> auditParams = new TreeMap<>();
+    for (int i = 0; i < strs.length; i++) {
+      auditParams.put(strs[i], strs[++i]);

Review Comment:
   Can we omit `null` and empty values?



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java:
##########
@@ -161,16 +163,27 @@ public Response put(
       @QueryParam("uploadId") @DefaultValue("") String uploadID,
       InputStream body) throws IOException, OS3Exception {
 
-    OzoneOutputStream output = null;
+    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),
+        "uploadId", uploadID

Review Comment:
   Should these be included in the map only for multipart upload?



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java:
##########
@@ -101,42 +104,57 @@ public Response get(
       @QueryParam("uploads") String uploads,
       @QueryParam("acl") String aclMarker,
       @Context HttpHeaders hh) throws OS3Exception, IOException {
+    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),

Review Comment:
   It would be nice if there was a way to get query params which were actually set, not coming from `DefaultValue`.



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java:
##########
@@ -48,26 +49,41 @@
   @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.logWriteFailure(
+          buildAuditMessageForFailure(S3GAction.LIST_S3_BUCKETS, null, ex)
+      );
+      throw ex;
+    } finally {
+      if (auditSuccess) {
+        AUDIT.logWriteSuccess(
+            buildAuditMessageForSuccess(S3GAction.LIST_S3_BUCKETS, null)

Review Comment:
   Also prefer `emptyMap()` here.



##########
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/RootEndpoint.java:
##########
@@ -48,26 +49,41 @@
   @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.logWriteFailure(
+          buildAuditMessageForFailure(S3GAction.LIST_S3_BUCKETS, null, ex)

Review Comment:
   I think it would be better to pass `emptyMap()` instead of `null`.



##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/S3GAction.java:
##########
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <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.audit;
+
+/**
+ * Enum to define Audit Action types for S3Gateway.
+ */
+public enum S3GAction implements AuditAction {
+
+  //BucketEndpoint
+  BROWSE,

Review Comment:
   Can be omitted, bucket browser support was recently dropped.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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