Sfoglia il codice sorgente

HDDS-585. Handle common request identifiers in a transparent way. Contributed by Elek Marton.

(cherry picked from commit d5dd6f31fc35b890cfa241d5fce404d6774e98c6)
Bharat Viswanadham 6 anni fa
parent
commit
308c614d4a

+ 9 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/CommonHeadersContainerResponseFilter.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.ozone.s3;
 
+import javax.inject.Inject;
 import javax.ws.rs.container.ContainerRequestContext;
 import javax.ws.rs.container.ContainerResponseContext;
 import javax.ws.rs.container.ContainerResponseFilter;
@@ -30,10 +31,18 @@ import java.io.IOException;
 public class CommonHeadersContainerResponseFilter implements
     ContainerResponseFilter {
 
+  @Inject
+  private RequestIdentifier requestIdentifier;
+
   @Override
   public void filter(ContainerRequestContext containerRequestContext,
       ContainerResponseContext containerResponseContext) throws IOException {
+
     containerResponseContext.getHeaders().add("Server", "Ozone");
+    containerResponseContext.getHeaders()
+        .add("x-amz-id-2", requestIdentifier.getAmzId());
+    containerResponseContext.getHeaders()
+        .add("x-amz-request-id", requestIdentifier.getRequestId());
 
   }
 }

+ 4 - 19
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/EndpointBase.java

@@ -24,12 +24,11 @@ import java.io.IOException;
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.client.OzoneVolume;
-
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable.Resource;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -42,7 +41,6 @@ public class EndpointBase {
       LoggerFactory.getLogger(EndpointBase.class);
   @Inject
   private OzoneClient client;
-  private String requestId;
 
   protected OzoneBucket getBucket(String volumeName, String bucketName)
       throws IOException {
@@ -57,8 +55,8 @@ public class EndpointBase {
     } catch (IOException ex) {
       LOG.error("Error occurred is {}", ex);
       if (ex.getMessage().contains("NOT_FOUND")) {
-        OS3Exception oex = S3ErrorTable.newError(S3ErrorTable.NO_SUCH_BUCKET,
-            OzoneUtils.getRequestID(), Resource.BUCKET);
+        OS3Exception oex =
+            S3ErrorTable.newError(S3ErrorTable.NO_SUCH_BUCKET, Resource.BUCKET);
         throw oex;
       } else {
         throw ex;
@@ -86,17 +84,4 @@ public class EndpointBase {
     this.client = ozoneClient;
   }
 
-  /**
-   * Set the requestId.
-   * @param id
-   */
-  protected void setRequestId(String id) {
-    this.requestId = id;
-  }
-
-  @VisibleForTesting
-  public String getRequestId() {
-    return requestId;
-  }
-
 }

+ 48 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/RequestIdentifier.java

@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.hadoop.ozone.s3;
+
+import javax.enterprise.context.RequestScoped;
+
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+
+import org.apache.commons.lang3.RandomStringUtils;
+
+/**
+ * Request specific identifiers.
+ */
+@RequestScoped
+public class RequestIdentifier {
+
+  private final String requestId;
+
+  private final String amzId;
+
+  public RequestIdentifier() {
+    amzId = RandomStringUtils.randomAlphanumeric(8, 16);
+    requestId = OzoneUtils.getRequestID();
+  }
+
+  public String getRequestId() {
+    return requestId;
+  }
+
+  public String getAmzId() {
+    return amzId;
+  }
+}

+ 6 - 9
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/DeleteBucket.java

@@ -26,11 +26,10 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import java.io.IOException;
 
-import org.apache.commons.lang3.RandomStringUtils;
 import org.apache.hadoop.ozone.s3.EndpointBase;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+
 import org.apache.http.HttpStatus;
 
 /**
@@ -45,27 +44,25 @@ public class DeleteBucket extends EndpointBase {
                          @PathParam("bucket") String bucketName)
       throws IOException, OS3Exception {
 
-    setRequestId(OzoneUtils.getRequestID());
-
     try {
       getVolume(volumeName).deleteBucket(bucketName);
     } catch (IOException ex) {
       if (ex.getMessage().contains("BUCKET_NOT_EMPTY")) {
         OS3Exception os3Exception = S3ErrorTable.newError(S3ErrorTable
-            .BUCKET_NOT_EMPTY, getRequestId(), S3ErrorTable.Resource.BUCKET);
+            .BUCKET_NOT_EMPTY, S3ErrorTable.Resource.BUCKET);
         throw os3Exception;
       } else if (ex.getMessage().contains("BUCKET_NOT_FOUND")) {
         OS3Exception os3Exception = S3ErrorTable.newError(S3ErrorTable
-            .NO_SUCH_BUCKET, getRequestId(), S3ErrorTable.Resource.BUCKET);
+            .NO_SUCH_BUCKET, S3ErrorTable.Resource.BUCKET);
         throw os3Exception;
       } else {
         throw ex;
       }
     }
 
-    return Response.ok().status(HttpStatus.SC_NO_CONTENT).header(
-        "x-amz-request-id", getRequestId()).header("x-amz-id-2",
-        RandomStringUtils.randomAlphanumeric(8, 16)).build();
+    return Response
+        .status(HttpStatus.SC_NO_CONTENT)
+        .build();
 
   }
 }

+ 9 - 12
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/bucket/HeadBucket.java

@@ -18,21 +18,20 @@
 
 package org.apache.hadoop.ozone.s3.bucket;
 
+import javax.ws.rs.HEAD;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.core.Response;
+import java.io.IOException;
+
 import org.apache.hadoop.ozone.s3.EndpointBase;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable.Resource;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
-import org.apache.http.HttpStatus;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.HEAD;
-import javax.ws.rs.Path;
-import javax.ws.rs.PathParam;
-import javax.ws.rs.core.Response;
-import java.io.IOException;
-
 /**
  * Finds the bucket exists or not.
  */
@@ -46,7 +45,6 @@ public class HeadBucket extends EndpointBase {
   public Response head(@PathParam("volume") String volumeName,
                        @PathParam("bucket") String bucketName)
       throws Exception {
-    setRequestId(OzoneUtils.getRequestID());
     try {
       getVolume(volumeName).getBucket(bucketName);
       // Not sure what kind of error, we need to show for volume not exist
@@ -55,13 +53,12 @@ public class HeadBucket extends EndpointBase {
       LOG.error("Exception occurred in headBucket", ex);
       if (ex.getMessage().contains("NOT_FOUND")) {
         OS3Exception os3Exception = S3ErrorTable.newError(S3ErrorTable
-                .NO_SUCH_BUCKET, getRequestId(), Resource.BUCKET);
+                .NO_SUCH_BUCKET, Resource.BUCKET);
         throw os3Exception;
       } else {
         throw ex;
       }
     }
-    return Response.ok().status(HttpStatus.SC_OK).header("x-amz-request-id",
-        getRequestId()).build();
+    return Response.ok().build();
   }
 }

+ 9 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/OS3ExceptionMapper.java

@@ -20,21 +20,30 @@ package org.apache.hadoop.ozone.s3.exception;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.inject.Inject;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.ext.ExceptionMapper;
 import javax.ws.rs.ext.Provider;
 
+import org.apache.hadoop.ozone.s3.RequestIdentifier;
+
 /**
  *  Class the represents various errors returned by the
  *  Ozone S3 service.
  */
 @Provider
 public class OS3ExceptionMapper implements ExceptionMapper<OS3Exception> {
+
   private static final Logger LOG =
       LoggerFactory.getLogger(OS3ExceptionMapper.class);
+
+  @Inject
+  private RequestIdentifier requestIdentifier;
+
   @Override
   public Response toResponse(OS3Exception exception) {
     LOG.debug("Returning exception. ex: {}", exception.toString());
+    exception.setRequestId(requestIdentifier.getRequestId());
     return Response.status(exception.getHttpCode())
         .entity(exception.toXml()).build();
   }

+ 1 - 4
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java

@@ -46,15 +46,12 @@ public final class S3ErrorTable {
   /**
    * Create a new instance of Error.
    * @param e Error Template
-   * @param requestID
    * @param resource Resource associated with this exception
    * @return creates a new instance of error based on the template
    */
-  public static OS3Exception newError(OS3Exception e, String requestID,
-                                         Resource resource){
+  public static OS3Exception newError(OS3Exception e, Resource resource) {
     OS3Exception err =  new OS3Exception(e.getCode(), e.getErrorMessage(),
         e.getHttpCode());
-    err.setRequestId(requestID);
     err.setResource(resource.getResource());
     return err;
   }

+ 1 - 4
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/TestDeleteBucket.java

@@ -70,8 +70,7 @@ public class TestDeleteBucket {
   public void testDeleteBucket() throws Exception {
     Response response = deleteBucket.delete(volumeName, bucketName);
     assertEquals(response.getStatus(), HttpStatus.SC_NO_CONTENT);
-    assertEquals(deleteBucket.getRequestId(), response.getHeaderString(
-        "x-amz-request-id"));
+
   }
 
   @Test
@@ -82,7 +81,6 @@ public class TestDeleteBucket {
       assertEquals(S3ErrorTable.NO_SUCH_BUCKET.getCode(), ex.getCode());
       assertEquals(S3ErrorTable.NO_SUCH_BUCKET.getErrorMessage(),
           ex.getErrorMessage());
-      assertEquals(deleteBucket.getRequestId(), ex.getRequestId());
       return;
     }
     fail("testDeleteWithNoSuchBucket failed");
@@ -101,7 +99,6 @@ public class TestDeleteBucket {
       assertEquals(S3ErrorTable.BUCKET_NOT_EMPTY.getCode(), ex.getCode());
       assertEquals(S3ErrorTable.BUCKET_NOT_EMPTY.getErrorMessage(),
           ex.getErrorMessage());
-      assertEquals(deleteBucket.getRequestId(), ex.getRequestId());
       return;
     }
     fail("testDeleteWithBucketNotEmpty failed");

+ 1 - 4
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/bucket/TestHeadBucket.java

@@ -69,8 +69,7 @@ public class TestHeadBucket {
 
     Response response = headBucket.head(volumeName, bucketName);
     assertEquals(200, response.getStatus());
-    assertEquals(headBucket.getRequestId(), response.getHeaderString(
-        "x-amz-request-id"));
+
   }
 
   @Test
@@ -83,8 +82,6 @@ public class TestHeadBucket {
             ((OS3Exception) ex).getCode());
         assertEquals(S3ErrorTable.NO_SUCH_BUCKET.getErrorMessage(), (
             (OS3Exception) ex).getErrorMessage());
-        assertEquals(headBucket.getRequestId(), (
-            (OS3Exception) ex).getRequestId());
       } else {
         fail("testHeadFail failed");
       }

+ 2 - 1
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/exception/TestOS3Exception.java

@@ -32,7 +32,8 @@ public class TestOS3Exception {
     OS3Exception ex = new OS3Exception("AccessDenied", "Access Denied",
         403);
     String requestId = OzoneUtils.getRequestID();
-    ex = S3ErrorTable.newError(ex, requestId, S3ErrorTable.Resource.BUCKET);
+    ex = S3ErrorTable.newError(ex, S3ErrorTable.Resource.BUCKET);
+    ex.setRequestId(requestId);
     String val = ex.toXml();
     String formatString = "<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n" +
         "<Error>\n" +