Browse Source

HDFS-10179. Ozone: Adding logging support. Contributed by Anu Engineer.

Chris Nauroth 9 years ago
parent
commit
fd2fe47b32

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/exceptions/OzoneExceptionMapper.java

@@ -22,14 +22,22 @@ package org.apache.hadoop.ozone.web.exceptions;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.ext.ExceptionMapper;
 import javax.ws.rs.ext.ExceptionMapper;
 
 
+import org.apache.log4j.MDC;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
 /**
  *  Class the represents various errors returned by the
  *  Class the represents various errors returned by the
  *  Object Layer.
  *  Object Layer.
  */
  */
 public class OzoneExceptionMapper implements ExceptionMapper<OzoneException> {
 public class OzoneExceptionMapper implements ExceptionMapper<OzoneException> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OzoneExceptionMapper.class);
 
 
   @Override
   @Override
   public Response toResponse(OzoneException exception) {
   public Response toResponse(OzoneException exception) {
+    LOG.info("Returning exception. ex: {}", exception.toJsonString());
+    MDC.clear();
     return Response.status((int)exception.getHttpCode())
     return Response.status((int)exception.getHttpCode())
       .entity(exception.toJsonString()).build();
       .entity(exception.toJsonString()).build();
   }
   }

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketHandler.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.ozone.web.interfaces.Bucket;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.utils.OzoneConsts;
 import org.apache.hadoop.ozone.web.utils.OzoneConsts;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.slf4j.MDC;
 
 
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Request;
 import javax.ws.rs.core.Request;
@@ -36,6 +37,7 @@ import java.io.IOException;
 
 
 import static java.net.HttpURLConnection.HTTP_CREATED;
 import static java.net.HttpURLConnection.HTTP_CREATED;
 import static java.net.HttpURLConnection.HTTP_OK;
 import static java.net.HttpURLConnection.HTTP_OK;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_FUNCTION;
 
 
 
 
 /**
 /**
@@ -59,6 +61,7 @@ public class BucketHandler implements Bucket {
   public Response createBucket(String volume, String bucket, Request req,
   public Response createBucket(String volume, String bucket, Request req,
                                UriInfo info, HttpHeaders headers)
                                UriInfo info, HttpHeaders headers)
       throws OzoneException {
       throws OzoneException {
+    MDC.put(OZONE_FUNCTION, "createBucket");
     return new BucketProcessTemplate() {
     return new BucketProcessTemplate() {
       @Override
       @Override
       public Response doProcess(BucketArgs args)
       public Response doProcess(BucketArgs args)
@@ -94,6 +97,7 @@ public class BucketHandler implements Bucket {
   public Response updateBucket(String volume, String bucket, Request req,
   public Response updateBucket(String volume, String bucket, Request req,
                                UriInfo info, HttpHeaders headers)
                                UriInfo info, HttpHeaders headers)
       throws OzoneException {
       throws OzoneException {
+    MDC.put(OZONE_FUNCTION, "updateBucket");
     return new BucketProcessTemplate() {
     return new BucketProcessTemplate() {
       @Override
       @Override
       public Response doProcess(BucketArgs args)
       public Response doProcess(BucketArgs args)
@@ -136,6 +140,7 @@ public class BucketHandler implements Bucket {
   public Response deleteBucket(String volume, String bucket, Request req,
   public Response deleteBucket(String volume, String bucket, Request req,
                                UriInfo info, HttpHeaders headers)
                                UriInfo info, HttpHeaders headers)
       throws OzoneException {
       throws OzoneException {
+    MDC.put(OZONE_FUNCTION, "deleteBucket");
     return new BucketProcessTemplate() {
     return new BucketProcessTemplate() {
       @Override
       @Override
       public Response doProcess(BucketArgs args)
       public Response doProcess(BucketArgs args)
@@ -169,6 +174,7 @@ public class BucketHandler implements Bucket {
                              final String startPage, Request req,
                              final String startPage, Request req,
                              UriInfo uriInfo, HttpHeaders headers)
                              UriInfo uriInfo, HttpHeaders headers)
       throws OzoneException {
       throws OzoneException {
+    MDC.put(OZONE_FUNCTION, "listBucket");
     return new BucketProcessTemplate() {
     return new BucketProcessTemplate() {
       @Override
       @Override
       public Response doProcess(BucketArgs args)
       public Response doProcess(BucketArgs args)

+ 37 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/BucketProcessTemplate.java

@@ -30,6 +30,11 @@ import org.apache.hadoop.ozone.web.response.ListKeys;
 import org.apache.hadoop.ozone.web.utils.OzoneConsts;
 import org.apache.hadoop.ozone.web.utils.OzoneConsts;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.MDC;
+
+
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Request;
 import javax.ws.rs.core.Request;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response;
@@ -42,6 +47,11 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 
 
 import static java.net.HttpURLConnection.HTTP_OK;
 import static java.net.HttpURLConnection.HTTP_OK;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_COMPONENT;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_RESOURCE;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_REQUEST;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_USER;
+
 
 
 
 
 /**
 /**
@@ -49,6 +59,8 @@ import static java.net.HttpURLConnection.HTTP_OK;
  * Bucket handling code.
  * Bucket handling code.
  */
  */
 public abstract class BucketProcessTemplate {
 public abstract class BucketProcessTemplate {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(BucketProcessTemplate.class);
 
 
   /**
   /**
    * This function serves as the common error handling function
    * This function serves as the common error handling function
@@ -70,22 +82,30 @@ public abstract class BucketProcessTemplate {
     // TODO : Add logging
     // TODO : Add logging
     String reqID = OzoneUtils.getRequestID();
     String reqID = OzoneUtils.getRequestID();
     String hostName = OzoneUtils.getHostName();
     String hostName = OzoneUtils.getHostName();
+    MDC.put(OZONE_COMPONENT, "ozone");
+    MDC.put(OZONE_REQUEST, reqID);
+    UserArgs userArgs = null;
     try {
     try {
+      userArgs = new UserArgs(reqID, hostName, request, uriInfo, headers);
+
       OzoneUtils.validate(request, headers, reqID, bucket, hostName);
       OzoneUtils.validate(request, headers, reqID, bucket, hostName);
       OzoneUtils.verifyBucketName(bucket);
       OzoneUtils.verifyBucketName(bucket);
 
 
       UserAuth auth = UserHandlerBuilder.getAuthHandler();
       UserAuth auth = UserHandlerBuilder.getAuthHandler();
-      UserArgs userArgs =
-          new UserArgs(reqID, hostName, request, uriInfo, headers);
       userArgs.setUserName(auth.getUser(userArgs));
       userArgs.setUserName(auth.getUser(userArgs));
+      MDC.put(OZONE_USER, userArgs.getUserName());
 
 
       BucketArgs args = new BucketArgs(volume, bucket, userArgs);
       BucketArgs args = new BucketArgs(volume, bucket, userArgs);
-      return doProcess(args);
-    } catch (IllegalArgumentException argExp) {
-      OzoneException ex = ErrorTable
-          .newError(ErrorTable.INVALID_BUCKET_NAME, reqID, bucket, hostName);
-      ex.setMessage(argExp.getMessage());
-      throw ex;
+      MDC.put(OZONE_RESOURCE, args.getResourceName());
+      Response response =  doProcess(args);
+      LOG.info("Success");
+      MDC.clear();
+      return response;
+
+    } catch (IllegalArgumentException argEx) {
+      LOG.debug("Invalid bucket. ex:{}", argEx);
+      throw ErrorTable.newError(ErrorTable.INVALID_BUCKET_NAME, userArgs,
+          argEx);
     } catch (IOException fsExp) {
     } catch (IOException fsExp) {
       handleIOException(bucket, reqID, hostName, fsExp);
       handleIOException(bucket, reqID, hostName, fsExp);
     }
     }
@@ -133,6 +153,7 @@ public abstract class BucketProcessTemplate {
    */
    */
   void handleIOException(String bucket, String reqID, String hostName,
   void handleIOException(String bucket, String reqID, String hostName,
                          IOException fsExp) throws OzoneException {
                          IOException fsExp) throws OzoneException {
+    LOG.debug("IOException: {}", fsExp);
 
 
     if (fsExp instanceof FileAlreadyExistsException) {
     if (fsExp instanceof FileAlreadyExistsException) {
       throw ErrorTable
       throw ErrorTable
@@ -224,6 +245,7 @@ public abstract class BucketProcessTemplate {
     try {
     try {
       return OzoneConsts.Versioning.valueOf(version);
       return OzoneConsts.Versioning.valueOf(version);
     } catch (IllegalArgumentException ex) {
     } catch (IllegalArgumentException ex) {
+      LOG.debug("Malformed Version. version: {}", version);
       throw ErrorTable.newError(ErrorTable.MALFORMED_BUCKET_VERSION, args, ex);
       throw ErrorTable.newError(ErrorTable.MALFORMED_BUCKET_VERSION, args, ex);
     }
     }
   }
   }
@@ -239,10 +261,11 @@ public abstract class BucketProcessTemplate {
    * @throws OzoneException
    * @throws OzoneException
    */
    */
   StorageType getStorageType(BucketArgs args) throws OzoneException {
   StorageType getStorageType(BucketArgs args) throws OzoneException {
-
+    List<String> storageClassString = null;
     try {
     try {
-      List<String> storageClassString =
+      storageClassString =
           args.getHeaders().getRequestHeader(Header.OZONE_STORAGE_TYPE);
           args.getHeaders().getRequestHeader(Header.OZONE_STORAGE_TYPE);
+
       if (storageClassString == null) {
       if (storageClassString == null) {
         return null;
         return null;
       }
       }
@@ -254,6 +277,10 @@ public abstract class BucketProcessTemplate {
       }
       }
       return StorageType.valueOf(storageClassString.get(0).toUpperCase());
       return StorageType.valueOf(storageClassString.get(0).toUpperCase());
     } catch (IllegalArgumentException ex) {
     } catch (IllegalArgumentException ex) {
+      if(storageClassString != null) {
+        LOG.debug("Malformed storage type. Type: {}",
+            storageClassString.get(0).toUpperCase());
+      }
       throw ErrorTable.newError(ErrorTable.MALFORMED_STORAGE_TYPE, args, ex);
       throw ErrorTable.newError(ErrorTable.MALFORMED_STORAGE_TYPE, args, ex);
     }
     }
   }
   }

+ 35 - 18
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/KeyProcessTemplate.java

@@ -25,6 +25,9 @@ import org.apache.hadoop.ozone.web.headers.Header;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.interfaces.UserAuth;
 import org.apache.hadoop.ozone.web.interfaces.UserAuth;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.MDC;
 
 
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Request;
 import javax.ws.rs.core.Request;
@@ -41,11 +44,17 @@ import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.INVALID_BUCKET_N
 import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.INVALID_REQUEST;
 import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.INVALID_REQUEST;
 import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.SERVER_ERROR;
 import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.SERVER_ERROR;
 import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.newError;
 import static org.apache.hadoop.ozone.web.exceptions.ErrorTable.newError;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_COMPONENT;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_REQUEST;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_RESOURCE;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_USER;
 
 
 /**
 /**
  * This class abstracts way the repetitive tasks in  Key handling code.
  * This class abstracts way the repetitive tasks in  Key handling code.
  */
  */
 public abstract class KeyProcessTemplate {
 public abstract class KeyProcessTemplate {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(KeyProcessTemplate.class);
 
 
   /**
   /**
    * This function serves as the common error handling function for all Key
    * This function serves as the common error handling function for all Key
@@ -63,31 +72,37 @@ public abstract class KeyProcessTemplate {
 
 
     String reqID = OzoneUtils.getRequestID();
     String reqID = OzoneUtils.getRequestID();
     String hostName = OzoneUtils.getHostName();
     String hostName = OzoneUtils.getHostName();
+    MDC.put(OZONE_COMPONENT, "ozone");
+    MDC.put(OZONE_REQUEST, reqID);
     UserArgs userArgs = null;
     UserArgs userArgs = null;
     try {
     try {
+      userArgs = new UserArgs(reqID, hostName, request, info, headers);
       OzoneUtils.validate(request, headers, reqID, bucket, hostName);
       OzoneUtils.validate(request, headers, reqID, bucket, hostName);
       OzoneUtils.verifyBucketName(bucket);
       OzoneUtils.verifyBucketName(bucket);
 
 
       UserAuth auth = UserHandlerBuilder.getAuthHandler();
       UserAuth auth = UserHandlerBuilder.getAuthHandler();
-      userArgs = new UserArgs(reqID, hostName, request, info, headers);
       userArgs.setUserName(auth.getUser(userArgs));
       userArgs.setUserName(auth.getUser(userArgs));
+      MDC.put(OZONE_USER, userArgs.getUserName());
 
 
       KeyArgs args = new KeyArgs(volume, bucket, key, userArgs);
       KeyArgs args = new KeyArgs(volume, bucket, key, userArgs);
-      return doProcess(args, is, request, headers, info);
+      MDC.put(OZONE_RESOURCE, args.getResourceName());
+      Response response =  doProcess(args, is, request, headers, info);
+      LOG.info("Success");
+      MDC.clear();
+      return response;
+
     } catch (IllegalArgumentException argExp) {
     } catch (IllegalArgumentException argExp) {
-      OzoneException ex =
-          newError(INVALID_BUCKET_NAME, reqID, bucket, hostName);
-      ex.setMessage(argExp.getMessage());
-      throw ex;
+      LOG.debug("Invalid bucket in key call. ex:{}", argExp);
+      throw newError(INVALID_BUCKET_NAME, userArgs, argExp);
     } catch (IOException fsExp) {
     } catch (IOException fsExp) {
       // TODO : Handle errors from the FileSystem , let us map to server error
       // TODO : Handle errors from the FileSystem , let us map to server error
       // for now.
       // for now.
+      LOG.debug("IOException. ex : {}", fsExp);
       throw ErrorTable.newError(ErrorTable.SERVER_ERROR, userArgs, fsExp);
       throw ErrorTable.newError(ErrorTable.SERVER_ERROR, userArgs, fsExp);
     } catch (NoSuchAlgorithmException algoEx) {
     } catch (NoSuchAlgorithmException algoEx) {
-      OzoneException ex =
-          ErrorTable.newError(SERVER_ERROR, reqID, key, hostName);
-      ex.setMessage(algoEx.getMessage());
-      throw ex;
+      LOG.debug("NoSuchAlgorithmException. Probably indicates an unusual java "
+          + "installation.  ex : {}", algoEx);
+      throw ErrorTable.newError(SERVER_ERROR, userArgs, algoEx);
     }
     }
   }
   }
 
 
@@ -131,10 +146,11 @@ public abstract class KeyProcessTemplate {
 
 
       if (!contentString.equals(computedString)) {
       if (!contentString.equals(computedString)) {
         fs.deleteKey(args);
         fs.deleteKey(args);
-        OzoneException ex = ErrorTable.newError(BAD_DIGEST, args.getRequestID(),
-            args.getKeyName(), args.getHostName());
-        ex.setMessage(String.format("MD5 Digest mismatch. Expected %s Found " +
-            "%s", contentString, computedString));
+        OzoneException ex = ErrorTable.newError(BAD_DIGEST, args);
+        String msg = String.format("MD5 Digest mismatch. Expected %s Found " +
+            "%s", contentString, computedString);
+        ex.setMessage(msg);
+        LOG.debug(msg);
         throw ex;
         throw ex;
       }
       }
     }
     }
@@ -158,10 +174,11 @@ public abstract class KeyProcessTemplate {
       throws IOException, OzoneException {
       throws IOException, OzoneException {
     if (bytesRead != contentLen) {
     if (bytesRead != contentLen) {
       fs.deleteKey(args);
       fs.deleteKey(args);
-      OzoneException ex = ErrorTable.newError(INCOMPLETE_BODY,
-          args.getRequestID(), args.getKeyName(), args.getHostName());
-      ex.setMessage(String.format("Body length mismatch. Expected length : %d" +
-          " Found %d", contentLen, bytesRead));
+      OzoneException ex = ErrorTable.newError(INCOMPLETE_BODY, args);
+      String msg = String.format("Body length mismatch. Expected length : %d" +
+          " Found %d", contentLen, bytesRead);
+      ex.setMessage(msg);
+      LOG.debug(msg);
       throw ex;
       throw ex;
     }
     }
   }
   }

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeHandler.java

@@ -26,6 +26,9 @@ import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
 import org.apache.hadoop.ozone.web.interfaces.UserAuth;
 import org.apache.hadoop.ozone.web.interfaces.UserAuth;
 import org.apache.hadoop.ozone.web.interfaces.Volume;
 import org.apache.hadoop.ozone.web.interfaces.Volume;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.MDC;
 
 
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Request;
 import javax.ws.rs.core.Request;
@@ -35,6 +38,7 @@ import java.io.IOException;
 
 
 import static java.net.HttpURLConnection.HTTP_CREATED;
 import static java.net.HttpURLConnection.HTTP_CREATED;
 import static java.net.HttpURLConnection.HTTP_OK;
 import static java.net.HttpURLConnection.HTTP_OK;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_FUNCTION;
 
 
 /**
 /**
  * VolumeHandler handles volume specific HTTP calls.
  * VolumeHandler handles volume specific HTTP calls.
@@ -50,6 +54,8 @@ import static java.net.HttpURLConnection.HTTP_OK;
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class VolumeHandler implements Volume {
 public class VolumeHandler implements Volume {
+  private static final Logger LOG = LoggerFactory.getLogger(VolumeHandler
+      .class);
   /**
   /**
    * Creates a volume.
    * Creates a volume.
    *
    *
@@ -67,6 +73,7 @@ public class VolumeHandler implements Volume {
   public Response createVolume(String volume, final String quota, Request req,
   public Response createVolume(String volume, final String quota, Request req,
                                UriInfo uriInfo, HttpHeaders headers)
                                UriInfo uriInfo, HttpHeaders headers)
       throws OzoneException {
       throws OzoneException {
+    MDC.put(OZONE_FUNCTION, "createVolume");
     return new VolumeProcessTemplate() {
     return new VolumeProcessTemplate() {
       @Override
       @Override
       public Response doProcess(VolumeArgs args)
       public Response doProcess(VolumeArgs args)
@@ -119,6 +126,7 @@ public class VolumeHandler implements Volume {
   public Response updateVolume(String volume, final String quota, Request req,
   public Response updateVolume(String volume, final String quota, Request req,
                                UriInfo uriInfo, HttpHeaders headers)
                                UriInfo uriInfo, HttpHeaders headers)
       throws OzoneException {
       throws OzoneException {
+    MDC.put(OZONE_FUNCTION, "updateVolume");
     return new VolumeProcessTemplate() {
     return new VolumeProcessTemplate() {
       @Override
       @Override
       public Response doProcess(VolumeArgs args)
       public Response doProcess(VolumeArgs args)
@@ -171,6 +179,8 @@ public class VolumeHandler implements Volume {
   @Override
   @Override
   public Response deleteVolume(String volume, Request req, UriInfo uriInfo,
   public Response deleteVolume(String volume, Request req, UriInfo uriInfo,
                                HttpHeaders headers) throws OzoneException {
                                HttpHeaders headers) throws OzoneException {
+    MDC.put(OZONE_FUNCTION, "deleteVolume");
+
     return new VolumeProcessTemplate() {
     return new VolumeProcessTemplate() {
       @Override
       @Override
       public Response doProcess(VolumeArgs args)
       public Response doProcess(VolumeArgs args)
@@ -202,6 +212,7 @@ public class VolumeHandler implements Volume {
   public Response getVolumeInfo(String volume, final String info, Request req,
   public Response getVolumeInfo(String volume, final String info, Request req,
                                 final UriInfo uriInfo, HttpHeaders headers)
                                 final UriInfo uriInfo, HttpHeaders headers)
       throws OzoneException {
       throws OzoneException {
+    MDC.put(OZONE_FUNCTION, "getVolumeInfo");
     return new VolumeProcessTemplate() {
     return new VolumeProcessTemplate() {
       @Override
       @Override
       public Response doProcess(VolumeArgs args)
       public Response doProcess(VolumeArgs args)
@@ -215,6 +226,7 @@ public class VolumeHandler implements Volume {
           case Header.OZONE_LIST_QUERY_SERVICE:
           case Header.OZONE_LIST_QUERY_SERVICE:
             return getVolumesByUser(args); // Return list of volumes
             return getVolumesByUser(args); // Return list of volumes
           default:
           default:
+            LOG.debug("Unrecognized query param : {} ", info);
             OzoneException ozoneException =
             OzoneException ozoneException =
                 ErrorTable.newError(ErrorTable.INVALID_QUERY_PARAM, args);
                 ErrorTable.newError(ErrorTable.INVALID_QUERY_PARAM, args);
             ozoneException.setMessage("Unrecognized query param : " + info);
             ozoneException.setMessage("Unrecognized query param : " + info);

+ 33 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/handlers/VolumeProcessTemplate.java

@@ -27,6 +27,9 @@ import org.apache.hadoop.ozone.web.response.ListBuckets;
 import org.apache.hadoop.ozone.web.response.ListVolumes;
 import org.apache.hadoop.ozone.web.response.ListVolumes;
 import org.apache.hadoop.ozone.web.response.VolumeInfo;
 import org.apache.hadoop.ozone.web.response.VolumeInfo;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.MDC;
 
 
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.HttpHeaders;
 import javax.ws.rs.core.Request;
 import javax.ws.rs.core.Request;
@@ -38,6 +41,11 @@ import java.nio.file.FileAlreadyExistsException;
 import java.nio.file.NoSuchFileException;
 import java.nio.file.NoSuchFileException;
 
 
 import static java.net.HttpURLConnection.HTTP_OK;
 import static java.net.HttpURLConnection.HTTP_OK;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_COMPONENT;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_RESOURCE;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_REQUEST;
+import static org.apache.hadoop.ozone.web.utils.OzoneConsts.OZONE_USER;
+
 
 
 /**
 /**
  * This class abstracts way the repetitive tasks in
  * This class abstracts way the repetitive tasks in
@@ -45,6 +53,8 @@ import static java.net.HttpURLConnection.HTTP_OK;
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public abstract class VolumeProcessTemplate {
 public abstract class VolumeProcessTemplate {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(VolumeProcessTemplate.class);
 
 
 
 
   /**
   /**
@@ -64,24 +74,30 @@ public abstract class VolumeProcessTemplate {
                              HttpHeaders headers) throws OzoneException {
                              HttpHeaders headers) throws OzoneException {
     String reqID = OzoneUtils.getRequestID();
     String reqID = OzoneUtils.getRequestID();
     String hostName = OzoneUtils.getHostName();
     String hostName = OzoneUtils.getHostName();
+    MDC.put(OZONE_COMPONENT, "ozone");
+    MDC.put(OZONE_REQUEST, reqID);
+    UserArgs userArgs  = null;
     try {
     try {
-
+      userArgs = new UserArgs(reqID, hostName, request, info, headers);
       OzoneUtils.validate(request, headers, reqID, volume, hostName);
       OzoneUtils.validate(request, headers, reqID, volume, hostName);
 
 
       // we use the same logic for both bucket and volume names
       // we use the same logic for both bucket and volume names
       OzoneUtils.verifyBucketName(volume);
       OzoneUtils.verifyBucketName(volume);
       UserAuth auth = UserHandlerBuilder.getAuthHandler();
       UserAuth auth = UserHandlerBuilder.getAuthHandler();
-      UserArgs userArgs = new UserArgs(reqID, hostName, request, info, headers);
 
 
       userArgs.setUserName(auth.getUser(userArgs));
       userArgs.setUserName(auth.getUser(userArgs));
+      MDC.put(OZONE_USER, userArgs.getUserName());
       VolumeArgs args = new VolumeArgs(volume, userArgs);
       VolumeArgs args = new VolumeArgs(volume, userArgs);
 
 
-      return doProcess(args);
+      MDC.put(OZONE_RESOURCE, args.getResourceName());
+      Response response =  doProcess(args);
+      LOG.info("Success");
+      MDC.clear();
+      return response;
+
     } catch (IllegalArgumentException ex) {
     } catch (IllegalArgumentException ex) {
-      OzoneException exp = ErrorTable
-          .newError(ErrorTable.INVALID_VOLUME_NAME, reqID, volume, hostName);
-      exp.setMessage(ex.getMessage());
-      throw exp;
+      LOG.debug("illegal argument. {}", ex);
+      throw ErrorTable.newError(ErrorTable.INVALID_VOLUME_NAME, userArgs, ex);
     } catch (IOException ex) {
     } catch (IOException ex) {
       handleIOException(volume, reqID, hostName, ex);
       handleIOException(volume, reqID, hostName, ex);
     }
     }
@@ -142,6 +158,7 @@ public abstract class VolumeProcessTemplate {
         exp.setMessage(fsExp.getMessage());
         exp.setMessage(fsExp.getMessage());
       }
       }
     }
     }
+    LOG.debug("IOException: {}", exp);
     throw exp;
     throw exp;
   }
   }
 
 
@@ -158,6 +175,7 @@ public abstract class VolumeProcessTemplate {
     try {
     try {
       args.setQuota(quota);
       args.setQuota(quota);
     } catch (IllegalArgumentException ex) {
     } catch (IllegalArgumentException ex) {
+      LOG.debug("Malformed Quota: {}", ex);
       throw ErrorTable.newError(ErrorTable.MALFORMED_QUOTA, args, ex);
       throw ErrorTable.newError(ErrorTable.MALFORMED_QUOTA, args, ex);
     }
     }
   }
   }
@@ -227,7 +245,9 @@ public abstract class VolumeProcessTemplate {
                        args.getRequest(), args.getUri(), args.getHeaders());
                        args.getRequest(), args.getUri(), args.getHeaders());
       return getVolumesByUser(user);
       return getVolumesByUser(user);
     } catch (IOException ex) {
     } catch (IOException ex) {
-      OzoneException exp = ErrorTable.newError(ErrorTable.SERVER_ERROR, args);
+      LOG.debug("unable to get the volume list for the user. Ex: {}", ex);
+      OzoneException exp = ErrorTable.newError(ErrorTable.SERVER_ERROR,
+          args, ex);
       exp.setMessage("unable to get the volume list for the user");
       exp.setMessage("unable to get the volume list for the user");
       throw exp;
       throw exp;
     }
     }
@@ -242,20 +262,19 @@ public abstract class VolumeProcessTemplate {
    * @throws OzoneException
    * @throws OzoneException
    */
    */
   Response getBucketsInVolume(VolumeArgs args) throws OzoneException {
   Response getBucketsInVolume(VolumeArgs args) throws OzoneException {
-    String requestID = OzoneUtils.getRequestID();
-    String hostName = OzoneUtils.getHostName();
     try {
     try {
-      UserAuth auth = UserHandlerBuilder.getAuthHandler();
-      // TODO : Check for ACLS access.
+      // UserAuth auth = UserHandlerBuilder.getAuthHandler();
+      // TODO : Check ACLS.
       StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
       StorageHandler fs = StorageHandlerBuilder.getStorageHandler();
       ListBuckets bucketList = fs.listBuckets(args);
       ListBuckets bucketList = fs.listBuckets(args);
       return OzoneUtils.getResponse(args, HTTP_OK, bucketList.toJsonString());
       return OzoneUtils.getResponse(args, HTTP_OK, bucketList.toJsonString());
     } catch (IOException ex) {
     } catch (IOException ex) {
+      LOG.debug("unable to get the bucket list for the specified volume." +
+          " Ex: {}", ex);
       OzoneException exp =
       OzoneException exp =
-          ErrorTable.newError(ErrorTable.SERVER_ERROR, requestID, "", hostName);
+          ErrorTable.newError(ErrorTable.SERVER_ERROR, args, ex);
       exp.setMessage("unable to get the bucket list for the specified volume.");
       exp.setMessage("unable to get the bucket list for the specified volume.");
       throw exp;
       throw exp;
-
     }
     }
   }
   }
 }
 }

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/ozone/web/utils/OzoneConsts.java

@@ -47,6 +47,13 @@ public final class OzoneConsts {
       "EEE, dd MMM yyyy HH:mm:ss zzz";
       "EEE, dd MMM yyyy HH:mm:ss zzz";
   public static final String OZONE_TIME_ZONE = "GMT";
   public static final String OZONE_TIME_ZONE = "GMT";
 
 
+  public static final String OZONE_COMPONENT = "component";
+  public static final String OZONE_FUNCTION  = "function";
+  public static final String OZONE_RESOURCE = "resource";
+  public static final String OZONE_USER = "user";
+  public static final String OZONE_REQUEST = "request";
+
+
   /**
   /**
    * Supports Bucket Versioning.
    * Supports Bucket Versioning.
    */
    */

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/log4j.properties

@@ -47,3 +47,27 @@ log4j.appender.DNMETRICSRFA.layout=org.apache.log4j.PatternLayout
 log4j.appender.DNMETRICSRFA.layout.ConversionPattern=%d{ISO8601} %m%n
 log4j.appender.DNMETRICSRFA.layout.ConversionPattern=%d{ISO8601} %m%n
 log4j.appender.DNMETRICSRFA.MaxBackupIndex=1
 log4j.appender.DNMETRICSRFA.MaxBackupIndex=1
 log4j.appender.DNMETRICSRFA.MaxFileSize=64MB
 log4j.appender.DNMETRICSRFA.MaxFileSize=64MB
+
+#
+# Add a logger for ozone that is separate from the Datanode.
+#
+log4j.logger.org.apache.hadoop.ozone=DEBUG,OZONE,FILE
+
+# Do not log into datanode logs. Remove this line to have single log.
+log4j.additivity.org.apache.hadoop.ozone=false
+
+# For development purposes, log both to console and log file.
+log4j.appender.OZONE=org.apache.log4j.ConsoleAppender
+log4j.appender.OZONE.Threshold=info
+log4j.appender.OZONE.layout=org.apache.log4j.PatternLayout
+log4j.appender.OZONE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p \
+ %X{component} %X{function} %X{resource} %X{user} %X{request} - %m%n
+
+# Real ozone logger that writes to ozone.log
+log4j.appender.FILE=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.FILE.File=${hadoop.log.dir}/ozone.log
+log4j.appender.FILE.Threshold=debug
+log4j.appender.FILE.layout=org.apache.log4j.PatternLayout
+log4j.appender.FILE.layout.ConversionPattern=%d{ISO8601} [%t] %-5p \
+  (%F:%L) %X{function} %X{resource} %X{user} %X{request} - \
+  %m%n