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/03/24 17:03:08 UTC

[ozone] branch master updated: HDDS-6481. Add metrics for S3 gateway (#3221)

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 75f5501  HDDS-6481. Add metrics for S3 gateway (#3221)
75f5501 is described below

commit 75f55016028938874eafdb578fa0732fadc7878a
Author: Symious <yi...@foxmail.com>
AuthorDate: Fri Mar 25 01:02:39 2022 +0800

    HDDS-6481. Add metrics for S3 gateway (#3221)
---
 .../dist/src/main/compose/ozone/prometheus.yml     |   1 +
 .../java/org/apache/hadoop/ozone/s3/Gateway.java   |   9 +-
 .../hadoop/ozone/s3/endpoint/BucketEndpoint.java   |  13 +
 .../hadoop/ozone/s3/endpoint/EndpointBase.java     |   6 +
 .../hadoop/ozone/s3/endpoint/ObjectEndpoint.java   |  34 ++-
 .../hadoop/ozone/s3/endpoint/RootEndpoint.java     |   9 +-
 .../hadoop/ozone/s3/metrics/S3GatewayMetrics.java  | 320 +++++++++++++++++++++
 .../hadoop/ozone/s3/metrics/package-info.java      |  22 ++
 .../ozone/s3/metrics/TestS3GatewayMetrics.java     | 113 ++++++++
 9 files changed, 523 insertions(+), 4 deletions(-)

diff --git a/hadoop-ozone/dist/src/main/compose/ozone/prometheus.yml b/hadoop-ozone/dist/src/main/compose/ozone/prometheus.yml
index 48e6ac5..357dbcc 100644
--- a/hadoop-ozone/dist/src/main/compose/ozone/prometheus.yml
+++ b/hadoop-ozone/dist/src/main/compose/ozone/prometheus.yml
@@ -23,6 +23,7 @@ scrape_configs:
      - targets:
         - "scm:9876"
         - "om:9874"
+        - "s3g:9878"
         - "ozone_datanode_1:9882"
         - "ozone_datanode_2:9882"
         - "ozone_datanode_3:9882"
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java
index 0ba014e..3792f59 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java
@@ -24,7 +24,9 @@ import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.hdds.cli.HddsVersionProvider;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.tracing.TracingUtil;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
 import org.apache.hadoop.ozone.OzoneSecurityUtil;
+import org.apache.hadoop.ozone.s3.metrics.S3GatewayMetrics;
 import org.apache.hadoop.ozone.util.OzoneVersionInfo;
 
 import org.apache.hadoop.ozone.util.ShutdownHookManager;
@@ -51,6 +53,8 @@ public class Gateway extends GenericCli {
   private static final Logger LOG = LoggerFactory.getLogger(Gateway.class);
 
   private S3GatewayHttpServer httpServer;
+  private S3GatewayMetrics metrics;
+  private OzoneConfiguration ozoneConfiguration;
 
   public static void main(String[] args) throws Exception {
     new Gateway().run(args);
@@ -58,12 +62,13 @@ public class Gateway extends GenericCli {
 
   @Override
   public Void call() throws Exception {
-    OzoneConfiguration ozoneConfiguration = createOzoneConfiguration();
+    ozoneConfiguration = createOzoneConfiguration();
     TracingUtil.initTracing("S3gateway", ozoneConfiguration);
     OzoneConfigurationHolder.setConfiguration(ozoneConfiguration);
     UserGroupInformation.setConfiguration(ozoneConfiguration);
     loginS3GUser(ozoneConfiguration);
     httpServer = new S3GatewayHttpServer(ozoneConfiguration, "s3gateway");
+    metrics = S3GatewayMetrics.create();
     start();
 
     ShutdownHookManager.get().addShutdownHook(() -> {
@@ -84,11 +89,13 @@ public class Gateway extends GenericCli {
 
     LOG.info("Starting Ozone S3 gateway");
     httpServer.start();
+    HddsServerUtil.initializeMetrics(ozoneConfiguration, "S3Gateway");
   }
 
   public void stop() throws Exception {
     LOG.info("Stopping Ozone S3 gateway");
     httpServer.stop();
+    S3GatewayMetrics.unRegister();
   }
 
   private static void loginS3GUser(OzoneConfiguration conf)
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 949227b..2f3cb5c 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
@@ -104,6 +104,7 @@ public class BucketEndpoint extends EndpointBase {
 
     if (aclMarker != null) {
       S3BucketAcl result = getAcl(bucketName);
+      getMetrics().incGetAclSuccess();
       return Response.ok(result, MediaType.APPLICATION_XML_TYPE).build();
     }
 
@@ -148,6 +149,7 @@ public class BucketEndpoint extends EndpointBase {
         ozoneKeyIterator = bucket.listKeys(prefix);
       }
     } catch (OMException ex) {
+      getMetrics().incGetBucketFailure();
       if (ex.getResult() == ResultCodes.PERMISSION_DENIED) {
         throw newError(S3ErrorTable.ACCESS_DENIED, bucketName, ex);
       } else {
@@ -223,6 +225,7 @@ public class BucketEndpoint extends EndpointBase {
       response.setTruncated(false);
     }
 
+    getMetrics().incGetBucketSuccess();
     response.setKeyCount(
         response.getCommonPrefixes().size() + response.getContents().size());
     return Response.ok(response).build();
@@ -239,9 +242,11 @@ public class BucketEndpoint extends EndpointBase {
     try {
       String location = createS3Bucket(bucketName);
       LOG.info("Location is {}", location);
+      getMetrics().incCreateBucketSuccess();
       return Response.status(HttpStatus.SC_OK).header("Location", location)
           .build();
     } catch (OMException exception) {
+      getMetrics().incCreateBucketFailure();
       if (exception.getResult() == ResultCodes.INVALID_BUCKET_NAME) {
         throw newError(S3ErrorTable.INVALID_BUCKET_NAME, bucketName, exception);
       }
@@ -262,6 +267,7 @@ public class BucketEndpoint extends EndpointBase {
     try {
       ozoneMultipartUploadList = bucket.listMultipartUploads(prefix);
     } catch (OMException exception) {
+      getMetrics().incListMultipartUploadsFailure();
       if (exception.getResult() == ResultCodes.PERMISSION_DENIED) {
         throw newError(S3ErrorTable.ACCESS_DENIED, prefix, exception);
       }
@@ -279,6 +285,7 @@ public class BucketEndpoint extends EndpointBase {
             S3StorageType.fromReplicationType(upload.getReplicationType(),
                 upload.getReplicationFactor())
         )));
+    getMetrics().incListMultipartUploadsSuccess();
     return Response.ok(result).build();
   }
   /**
@@ -291,6 +298,7 @@ public class BucketEndpoint extends EndpointBase {
   public Response head(@PathParam("bucket") String bucketName)
       throws OS3Exception, IOException {
     getBucket(bucketName);
+    getMetrics().incHeadBucketSuccess();
     return Response.ok().build();
   }
 
@@ -307,6 +315,7 @@ public class BucketEndpoint extends EndpointBase {
     try {
       deleteS3Bucket(bucketName);
     } catch (OMException ex) {
+      getMetrics().incDeleteBucketFailure();
       if (ex.getResult() == ResultCodes.BUCKET_NOT_EMPTY) {
         throw newError(S3ErrorTable.BUCKET_NOT_EMPTY, bucketName, ex);
       } else if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) {
@@ -318,6 +327,7 @@ public class BucketEndpoint extends EndpointBase {
       }
     }
 
+    getMetrics().incDeleteBucketSuccess();
     return Response
         .status(HttpStatus.SC_NO_CONTENT)
         .build();
@@ -398,6 +408,7 @@ public class BucketEndpoint extends EndpointBase {
           new S3BucketAcl.AccessControlList(grantList));
       return result;
     } catch (OMException ex) {
+      getMetrics().incGetAclFailure();
       if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) {
         throw newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName, ex);
       } else if (ex.getResult() == ResultCodes.PERMISSION_DENIED) {
@@ -497,6 +508,7 @@ public class BucketEndpoint extends EndpointBase {
         volume.addAcl(acl);
       }
     } catch (OMException exception) {
+      getMetrics().incPutAclFailure();
       if (exception.getResult() == ResultCodes.BUCKET_NOT_FOUND) {
         throw newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName, exception);
       } else if (exception.getResult() == ResultCodes.PERMISSION_DENIED) {
@@ -506,6 +518,7 @@ public class BucketEndpoint extends EndpointBase {
           exception);
       throw exception;
     }
+    getMetrics().incPutAclSuccess();
     return Response.status(HttpStatus.SC_OK).build();
   }
 
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 b933363..a9411dd 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
@@ -35,6 +35,7 @@ import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.ozone.s3.metrics.S3GatewayMetrics;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -198,4 +199,9 @@ public abstract class EndpointBase {
   public OzoneClient getClient() {
     return client;
   }
+
+  @VisibleForTesting
+  public S3GatewayMetrics getMetrics() {
+    return S3GatewayMetrics.create();
+  }
 }
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 863b1b0..145578b 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
@@ -169,9 +169,10 @@ public class ObjectEndpoint extends EndpointBase {
           partNumber, uploadID, body);
     }
 
+    String copyHeader = null, storageType = null;
     try {
-      String copyHeader = headers.getHeaderString(COPY_SOURCE_HEADER);
-      String storageType = headers.getHeaderString(STORAGE_CLASS_HEADER);
+      copyHeader = headers.getHeaderString(COPY_SOURCE_HEADER);
+      storageType = headers.getHeaderString(STORAGE_CLASS_HEADER);
 
       S3StorageType s3StorageType;
       boolean storageTypeDefault;
@@ -207,9 +208,15 @@ public class ObjectEndpoint extends EndpointBase {
 
       IOUtils.copy(body, output);
 
+      getMetrics().incCreateKeySuccess();
       return Response.ok().status(HttpStatus.SC_OK)
           .build();
     } catch (OMException ex) {
+      if (copyHeader != null) {
+        getMetrics().incCopyObjectFailure();
+      } else {
+        getMetrics().incCreateKeyFailure();
+      }
       if (ex.getResult() == ResultCodes.NOT_A_FILE) {
         OS3Exception os3Exception = newError(INVALID_REQUEST, keyPath, ex);
         os3Exception.setErrorMessage("An error occurred (InvalidRequest) " +
@@ -322,8 +329,14 @@ public class ObjectEndpoint extends EndpointBase {
         }
       }
       addLastModifiedDate(responseBuilder, keyDetails);
+      getMetrics().incGetKeySuccess();
       return responseBuilder.build();
     } catch (OMException ex) {
+      if (uploadId != null) {
+        getMetrics().incListPartsFailure();
+      } else {
+        getMetrics().incGetKeyFailure();
+      }
       if (ex.getResult() == ResultCodes.KEY_NOT_FOUND) {
         throw newError(S3ErrorTable.NO_SUCH_KEY, keyPath, ex);
       } else if (ex.getResult() == ResultCodes.PERMISSION_DENIED) {
@@ -362,6 +375,7 @@ public class ObjectEndpoint extends EndpointBase {
       key = getBucket(bucketName).headObject(keyPath);
       // TODO: return the specified range bytes of this object.
     } catch (OMException ex) {
+      getMetrics().incHeadKeyFailure();
       if (ex.getResult() == ResultCodes.KEY_NOT_FOUND) {
         // Just return 404 with no content
         return Response.status(Status.NOT_FOUND).build();
@@ -377,6 +391,7 @@ public class ObjectEndpoint extends EndpointBase {
         .header("Content-Length", key.getDataSize())
         .header("Content-Type", "binary/octet-stream");
     addLastModifiedDate(response, key);
+    getMetrics().incHeadKeySuccess();
     return response.build();
   }
 
@@ -400,6 +415,7 @@ public class ObjectEndpoint extends EndpointBase {
       }
       throw ex;
     }
+    getMetrics().incAbortMultiPartUploadSuccess();
     return Response
         .status(Status.NO_CONTENT)
         .build();
@@ -430,6 +446,11 @@ public class ObjectEndpoint extends EndpointBase {
       bucket.getKey(keyPath);
       bucket.deleteKey(keyPath);
     } catch (OMException ex) {
+      if (uploadId != null && !uploadId.equals("")) {
+        getMetrics().incAbortMultiPartUploadFailure();
+      } else {
+        getMetrics().incDeleteKeyFailure();
+      }
       if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) {
         throw newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName, ex);
       } else if (ex.getResult() == ResultCodes.KEY_NOT_FOUND) {
@@ -447,6 +468,7 @@ public class ObjectEndpoint extends EndpointBase {
       }
 
     }
+    getMetrics().incDeleteKeySuccess();
     return Response
         .status(Status.NO_CONTENT)
         .build();
@@ -488,9 +510,11 @@ public class ObjectEndpoint extends EndpointBase {
       multipartUploadInitiateResponse.setKey(key);
       multipartUploadInitiateResponse.setUploadID(multipartInfo.getUploadID());
 
+      getMetrics().incInitMultiPartUploadSuccess();
       return Response.status(Status.OK).entity(
           multipartUploadInitiateResponse).build();
     } catch (OMException ex) {
+      getMetrics().incInitMultiPartUploadFailure();
       if (ex.getResult() == ResultCodes.PERMISSION_DENIED) {
         throw newError(S3ErrorTable.ACCESS_DENIED, key, ex);
       }
@@ -535,9 +559,11 @@ public class ObjectEndpoint extends EndpointBase {
           .getHash());
       // Location also setting as bucket name.
       completeMultipartUploadResponse.setLocation(bucket);
+      getMetrics().incCompleteMultiPartUploadSuccess();
       return Response.status(Status.OK).entity(completeMultipartUploadResponse)
           .build();
     } catch (OMException ex) {
+      getMetrics().incCompleteMultiPartUploadFailure();
       if (ex.getResult() == ResultCodes.INVALID_PART) {
         throw newError(S3ErrorTable.INVALID_PART, key, ex);
       } else if (ex.getResult() == ResultCodes.INVALID_PART_ORDER) {
@@ -638,6 +664,7 @@ public class ObjectEndpoint extends EndpointBase {
           ozoneOutputStream.getCommitUploadPartInfo();
       String eTag = omMultipartCommitUploadPartInfo.getPartName();
 
+      getMetrics().incCreateMultipartKeySuccess();
       if (copyHeader != null) {
         return Response.ok(new CopyPartResult(eTag)).build();
       } else {
@@ -646,6 +673,7 @@ public class ObjectEndpoint extends EndpointBase {
       }
 
     } catch (OMException ex) {
+      getMetrics().incCreateMultipartKeyFailure();
       if (ex.getResult() == ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR) {
         throw newError(NO_SUCH_UPLOAD, uploadID, ex);
       } else if (ex.getResult() == ResultCodes.PERMISSION_DENIED) {
@@ -711,6 +739,7 @@ public class ObjectEndpoint extends EndpointBase {
       }
       throw ex;
     }
+    getMetrics().incListPartsSuccess();
     return Response.status(Status.OK).entity(listPartsResponse).build();
   }
 
@@ -783,6 +812,7 @@ public class ObjectEndpoint extends EndpointBase {
 
       OzoneKeyDetails destKeyDetails = destOzoneBucket.getKey(destkey);
 
+      getMetrics().incCopyObjectSuccess();
       CopyObjectResponse copyObjectResponse = new CopyObjectResponse();
       copyObjectResponse.setETag(OzoneUtils.getRequestID());
       copyObjectResponse.setLastModified(destKeyDetails.getModificationTime());
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 814d87b..615fb01 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
@@ -50,7 +50,13 @@ public class RootEndpoint extends EndpointBase {
       throws OS3Exception, IOException {
     ListBucketResponse response = new ListBucketResponse();
 
-    Iterator<? extends OzoneBucket> bucketIterator = listS3Buckets(null);
+    Iterator<? extends OzoneBucket> bucketIterator;
+    try {
+      bucketIterator = listS3Buckets(null);
+    } catch (Exception e) {
+      getMetrics().incListS3BucketsFailure();
+      throw e;
+    }
 
     while (bucketIterator.hasNext()) {
       OzoneBucket next = bucketIterator.next();
@@ -60,6 +66,7 @@ public class RootEndpoint extends EndpointBase {
       response.addBucket(bucketMetadata);
     }
 
+    getMetrics().incListS3BucketsSuccess();
     return Response.ok(response).build();
   }
 
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/metrics/S3GatewayMetrics.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/metrics/S3GatewayMetrics.java
new file mode 100644
index 0000000..012da2a
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/metrics/S3GatewayMetrics.java
@@ -0,0 +1,320 @@
+/**
+ * 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.metrics;
+
+import org.apache.hadoop.hdds.annotation.InterfaceAudience;
+import org.apache.hadoop.metrics2.MetricsCollector;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
+import org.apache.hadoop.metrics2.MetricsSource;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MetricsRegistry;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.ozone.OzoneConsts;
+
+/**
+ * This class maintains S3 Gateway related metrics.
+ */
+@InterfaceAudience.Private
+@Metrics(about = "S3 Gateway Metrics", context = OzoneConsts.OZONE)
+public final class S3GatewayMetrics implements MetricsSource {
+
+  public static final String SOURCE_NAME =
+      S3GatewayMetrics.class.getSimpleName();
+
+  private MetricsRegistry registry;
+  private static S3GatewayMetrics instance;
+
+  // BucketEndpoint
+  private @Metric MutableCounterLong getBucketSuccess;
+  private @Metric MutableCounterLong getBucketFailure;
+  private @Metric MutableCounterLong createBucketSuccess;
+  private @Metric MutableCounterLong createBucketFailure;
+  private @Metric MutableCounterLong headBucketSuccess;
+  private @Metric MutableCounterLong deleteBucketSuccess;
+  private @Metric MutableCounterLong deleteBucketFailure;
+  private @Metric MutableCounterLong getAclSuccess;
+  private @Metric MutableCounterLong getAclFailure;
+  private @Metric MutableCounterLong putAclSuccess;
+  private @Metric MutableCounterLong putAclFailure;
+  private @Metric MutableCounterLong listMultipartUploadsSuccess;
+  private @Metric MutableCounterLong listMultipartUploadsFailure;
+
+  // RootEndpoint
+  private @Metric MutableCounterLong listS3BucketsSuccess;
+  private @Metric MutableCounterLong listS3BucketsFailure;
+
+  // ObjectEndpoint
+  private @Metric MutableCounterLong createMultipartKeySuccess;
+  private @Metric MutableCounterLong createMultipartKeyFailure;
+  private @Metric MutableCounterLong copyObjectSuccess;
+  private @Metric MutableCounterLong copyObjectFailure;
+  private @Metric MutableCounterLong createKeySuccess;
+  private @Metric MutableCounterLong createKeyFailure;
+  private @Metric MutableCounterLong listPartsSuccess;
+  private @Metric MutableCounterLong listPartsFailure;
+  private @Metric MutableCounterLong getKeySuccess;
+  private @Metric MutableCounterLong getKeyFailure;
+  private @Metric MutableCounterLong headKeySuccess;
+  private @Metric MutableCounterLong headKeyFailure;
+  private @Metric MutableCounterLong initMultiPartUploadSuccess;
+  private @Metric MutableCounterLong initMultiPartUploadFailure;
+  private @Metric MutableCounterLong completeMultiPartUploadSuccess;
+  private @Metric MutableCounterLong completeMultiPartUploadFailure;
+  private @Metric MutableCounterLong abortMultiPartUploadSuccess;
+  private @Metric MutableCounterLong abortMultiPartUploadFailure;
+  private @Metric MutableCounterLong deleteKeySuccess;
+  private @Metric MutableCounterLong deleteKeyFailure;
+
+
+  /**
+   * Private constructor.
+   */
+  private S3GatewayMetrics() {
+    this.registry = new MetricsRegistry(SOURCE_NAME);
+  }
+
+  /**
+   * Create and returns SCMPipelineMetrics instance.
+   *
+   * @return SCMPipelineMetrics
+   */
+  public static synchronized S3GatewayMetrics create() {
+    if (instance != null) {
+      return instance;
+    }
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    instance = ms.register(SOURCE_NAME, "S3 Gateway Metrics",
+        new S3GatewayMetrics());
+    return instance;
+  }
+
+  /**
+   * Unregister the metrics instance.
+   */
+  public static void unRegister() {
+    instance = null;
+    MetricsSystem ms = DefaultMetricsSystem.instance();
+    ms.unregisterSource(SOURCE_NAME);
+  }
+
+  @Override
+  public void getMetrics(MetricsCollector collector, boolean all) {
+    MetricsRecordBuilder recordBuilder = collector.addRecord(SOURCE_NAME);
+
+    // BucketEndpoint
+    getBucketSuccess.snapshot(recordBuilder, true);
+    getBucketFailure.snapshot(recordBuilder, true);
+    createBucketSuccess.snapshot(recordBuilder, true);
+    createBucketFailure.snapshot(recordBuilder, true);
+    headBucketSuccess.snapshot(recordBuilder, true);
+    deleteBucketSuccess.snapshot(recordBuilder, true);
+    deleteBucketFailure.snapshot(recordBuilder, true);
+    getAclSuccess.snapshot(recordBuilder, true);
+    getAclFailure.snapshot(recordBuilder, true);
+    putAclSuccess.snapshot(recordBuilder, true);
+    putAclFailure.snapshot(recordBuilder, true);
+    listMultipartUploadsSuccess.snapshot(recordBuilder, true);
+    listMultipartUploadsFailure.snapshot(recordBuilder, true);
+
+    // RootEndpoint
+    listS3BucketsSuccess.snapshot(recordBuilder, true);
+    listS3BucketsFailure.snapshot(recordBuilder, true);
+
+    // ObjectEndpoint
+    createMultipartKeySuccess.snapshot(recordBuilder, true);
+    createMultipartKeyFailure.snapshot(recordBuilder, true);
+    copyObjectSuccess.snapshot(recordBuilder, true);
+    copyObjectFailure.snapshot(recordBuilder, true);
+    createKeySuccess.snapshot(recordBuilder, true);
+    createKeyFailure.snapshot(recordBuilder, true);
+    listPartsSuccess.snapshot(recordBuilder, true);
+    listPartsFailure.snapshot(recordBuilder, true);
+    getKeySuccess.snapshot(recordBuilder, true);
+    getKeyFailure.snapshot(recordBuilder, true);
+    headKeySuccess.snapshot(recordBuilder, true);
+    headKeyFailure.snapshot(recordBuilder, true);
+    initMultiPartUploadSuccess.snapshot(recordBuilder, true);
+    initMultiPartUploadFailure.snapshot(recordBuilder, true);
+    completeMultiPartUploadSuccess.snapshot(recordBuilder, true);
+    completeMultiPartUploadFailure.snapshot(recordBuilder, true);
+    abortMultiPartUploadSuccess.snapshot(recordBuilder, true);
+    abortMultiPartUploadFailure.snapshot(recordBuilder, true);
+    deleteKeySuccess.snapshot(recordBuilder, true);
+    deleteKeyFailure.snapshot(recordBuilder, true);
+  }
+
+  // INC
+  public void incGetBucketSuccess() {
+    getBucketSuccess.incr();
+  }
+
+  public void incGetBucketFailure() {
+    getBucketFailure.incr();
+  }
+
+  public void incListS3BucketsSuccess() {
+    listS3BucketsSuccess.incr();
+  }
+
+
+  public void incListS3BucketsFailure() {
+    listS3BucketsFailure.incr();
+  }
+
+
+  public void incCreateBucketSuccess() {
+    createBucketSuccess.incr();
+  }
+
+  public void incCreateBucketFailure() {
+    createBucketFailure.incr();
+  }
+
+  public void incPutAclSuccess() {
+    putAclSuccess.incr();
+  }
+
+  public void incPutAclFailure() {
+    putAclFailure.incr();
+  }
+
+  public void incGetAclSuccess() {
+    getAclSuccess.incr();
+  }
+
+  public void incGetAclFailure() {
+    getAclFailure.incr();
+  }
+
+  public void incListMultipartUploadsSuccess() {
+    listMultipartUploadsSuccess.incr();
+  }
+
+  public void incListMultipartUploadsFailure() {
+    listMultipartUploadsFailure.incr();
+  }
+
+  public void incHeadBucketSuccess() {
+    headBucketSuccess.incr();
+  }
+
+
+  public void incDeleteBucketSuccess() {
+    deleteBucketSuccess.incr();
+  }
+
+  public void incDeleteBucketFailure() {
+    deleteBucketFailure.incr();
+  }
+
+  public void incCreateMultipartKeySuccess() {
+    createMultipartKeySuccess.incr();
+  }
+
+  public void incCreateMultipartKeyFailure() {
+    createMultipartKeyFailure.incr();
+  }
+
+  public void incCopyObjectSuccess() {
+    copyObjectSuccess.incr();
+  }
+
+  public void incCopyObjectFailure() {
+    copyObjectFailure.incr();
+  }
+
+  public void incCreateKeySuccess() {
+    createKeySuccess.incr();
+  }
+
+  public void incCreateKeyFailure() {
+    createKeyFailure.incr();
+  }
+
+  public void incListPartsSuccess() {
+    listPartsSuccess.incr();
+  }
+
+  public void incListPartsFailure() {
+    listPartsFailure.incr();
+  }
+
+  public void incGetKeySuccess() {
+    getKeySuccess.incr();
+  }
+
+  public void incGetKeyFailure() {
+    getKeyFailure.incr();
+  }
+
+  public void incHeadKeySuccess() {
+    headKeySuccess.incr();
+  }
+
+  public void incHeadKeyFailure() {
+    headKeyFailure.incr();
+  }
+
+  public void incAbortMultiPartUploadSuccess() {
+    abortMultiPartUploadSuccess.incr();
+  }
+
+  public void incAbortMultiPartUploadFailure() {
+    abortMultiPartUploadFailure.incr();
+  }
+
+  public void incDeleteKeySuccess() {
+    deleteKeySuccess.incr();
+  }
+
+  public void incDeleteKeyFailure() {
+    deleteKeyFailure.incr();
+  }
+
+  public void incInitMultiPartUploadSuccess() {
+    initMultiPartUploadSuccess.incr();
+  }
+
+  public void incInitMultiPartUploadFailure() {
+    initMultiPartUploadFailure.incr();
+  }
+
+  public void incCompleteMultiPartUploadSuccess() {
+    completeMultiPartUploadSuccess.incr();
+  }
+
+  public void incCompleteMultiPartUploadFailure() {
+    completeMultiPartUploadFailure.incr();
+  }
+
+  // GET
+  public long getListS3BucketsSuccess() {
+    return listS3BucketsSuccess.value();
+  }
+
+  public long getHeadBucketSuccess() {
+    return headBucketSuccess.value();
+  }
+
+  public long getHeadKeySuccess() {
+    return headKeySuccess.value();
+  }
+}
\ No newline at end of file
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/metrics/package-info.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/metrics/package-info.java
new file mode 100644
index 0000000..3620462
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/metrics/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains Ozone S3 Metrics.
+ */
+package org.apache.hadoop.ozone.s3.metrics;
\ No newline at end of file
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/metrics/TestS3GatewayMetrics.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/metrics/TestS3GatewayMetrics.java
new file mode 100644
index 0000000..c5d6606
--- /dev/null
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/metrics/TestS3GatewayMetrics.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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.metrics;
+
+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.Before;
+import org.junit.Test;
+
+import java.util.HashMap;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link S3GatewayMetrics}.
+ */
+public class TestS3GatewayMetrics {
+
+  private String bucketName = OzoneConsts.BUCKET;
+  private OzoneClient clientStub;
+  private BucketEndpoint bucketEndpoint;
+  private RootEndpoint rootEndpoint;
+  private ObjectEndpoint keyEndpoint;
+  private OzoneBucket bucket;
+
+  private S3GatewayMetrics metrics;
+
+  @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());
+
+    metrics = bucketEndpoint.getMetrics();
+  }
+
+  @Test
+  public void testHeadBucket() throws Exception {
+
+    long oriMetric = metrics.getHeadBucketSuccess();
+
+    bucketEndpoint.head(bucketName);
+
+    long curMetric = metrics.getHeadBucketSuccess();
+    assertEquals(1L, curMetric - oriMetric);
+  }
+
+  @Test
+  public void testListBucket() throws Exception {
+
+    long oriMetric = metrics.getListS3BucketsSuccess();
+
+    rootEndpoint.get().getEntity();
+
+    long curMetric = metrics.getListS3BucketsSuccess();
+    assertEquals(1L, curMetric - oriMetric);
+  }
+
+  @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();
+
+    long oriMetric = metrics.getHeadKeySuccess();
+
+    keyEndpoint.head(bucketName, "key1");
+
+    long curMetric = metrics.getHeadKeySuccess();
+    assertEquals(1L, curMetric - oriMetric);
+  }
+}

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