Browse Source

HADOOP-12350. WASB Logging: Improve WASB Logging around deletes, reads and writes. Contributed by Dushyanth.

(cherry picked from commit 5f6edb30c2bb648d5564c951edc25645e17e6636)
cnauroth 9 years ago
parent
commit
535762fadb

+ 3 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -288,6 +288,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12452. Fix tracing documention reflecting the update to htrace-4
     (Masatake Iwasaki via Colin P. McCabe)
 
+    HADOOP-12350. WASB Logging: Improve WASB Logging around deletes, reads and
+    writes (Dushyanth via cnauroth)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

+ 25 - 41
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/AzureNativeFileSystemStore.java

@@ -45,8 +45,6 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -63,7 +61,8 @@ import org.apache.hadoop.fs.azure.metrics.ResponseReceivedMetricUpdater;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.mortbay.util.ajax.JSON;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.microsoft.azure.storage.CloudStorageAccount;
 import com.microsoft.azure.storage.OperationContext;
@@ -104,8 +103,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   static final String DEFAULT_STORAGE_EMULATOR_ACCOUNT_NAME = "storageemulator";
   static final String STORAGE_EMULATOR_ACCOUNT_NAME_PROPERTY_NAME = "fs.azure.storage.emulator.account.name";
 
-  public static final Log LOG = LogFactory
-      .getLog(AzureNativeFileSystemStore.class);
+  public static final Logger LOG = LoggerFactory.getLogger(AzureNativeFileSystemStore.class);
 
   private StorageInterface storageInteractionLayer;
   private CloudBlobDirectoryWrapper rootDirectory;
@@ -441,7 +439,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
 
     // Extract the directories that should contain page blobs
     pageBlobDirs = getDirectorySet(KEY_PAGE_BLOB_DIRECTORIES);
-    LOG.debug("Page blob directories:  " + setToString(pageBlobDirs));
+    LOG.debug("Page blob directories:  {}", setToString(pageBlobDirs));
 
     // Extract directories that should have atomic rename applied.
     atomicRenameDirs = getDirectorySet(KEY_ATOMIC_RENAME_DIRECTORIES);
@@ -455,7 +453,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     } catch (URISyntaxException e) {
       LOG.warn("Unable to initialize HBase root as an atomic rename directory.");
     }
-    LOG.debug("Atomic rename directories:  " + setToString(atomicRenameDirs));
+    LOG.debug("Atomic rename directories: {} ", setToString(atomicRenameDirs));
   }
 
   /**
@@ -686,16 +684,13 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     OperationContext.setLoggingEnabledByDefault(sessionConfiguration.
         getBoolean(KEY_ENABLE_STORAGE_CLIENT_LOGGING, false));
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug(String
-          .format(
-              "AzureNativeFileSystemStore init. Settings=%d,%b,%d,{%d,%d,%d,%d},{%b,%f,%f}",
-              concurrentWrites, tolerateOobAppends,
-              ((storageConnectionTimeout > 0) ? storageConnectionTimeout
-                  : STORAGE_CONNECTION_TIMEOUT_DEFAULT), minBackoff,
-              deltaBackoff, maxBackoff, maxRetries, selfThrottlingEnabled,
-              selfThrottlingReadFactor, selfThrottlingWriteFactor));
-    }
+    LOG.debug(
+        "AzureNativeFileSystemStore init. Settings={},{},{},{{},{},{},{}},{{},{},{}}",
+        concurrentWrites, tolerateOobAppends,
+        ((storageConnectionTimeout > 0) ? storageConnectionTimeout
+          : STORAGE_CONNECTION_TIMEOUT_DEFAULT), minBackoff,
+        deltaBackoff, maxBackoff, maxRetries, selfThrottlingEnabled,
+        selfThrottlingReadFactor, selfThrottlingWriteFactor);
   }
 
   /**
@@ -1075,8 +1070,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
           }
         }
       } catch (URISyntaxException e) {
-        LOG.info(String.format(
-                   "URI syntax error creating URI for %s", dir));
+        LOG.info("URI syntax error creating URI for {}", dir);
       }
     }
     return false;
@@ -1843,9 +1837,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
       throw new AssertionError(errMsg);
     }
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Retrieving metadata for " + key);
-    }
+    LOG.debug("Retrieving metadata for {}", key);
 
     try {
       if (checkContainer(ContainerAccessType.PureRead) == ContainerState.DoesntExist) {
@@ -1869,10 +1861,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
       // exists.
       if (null != blob && blob.exists(getInstrumentedContext())) {
 
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Found " + key
-              + " as an explicit blob. Checking if it's a file or folder.");
-        }
+        LOG.debug("Found {} as an explicit blob. Checking if it's a file or folder.", key);
 
         // The blob exists, so capture the metadata from the blob
         // properties.
@@ -1880,15 +1869,12 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
         BlobProperties properties = blob.getProperties();
 
         if (retrieveFolderAttribute(blob)) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(key + " is a folder blob.");
-          }
+          LOG.debug("{} is a folder blob.", key);
           return new FileMetadata(key, properties.getLastModified().getTime(),
               getPermissionStatus(blob), BlobMaterialization.Explicit);
         } else {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(key + " is a normal blob.");
-          }
+
+          LOG.debug("{} is a normal blob.", key);
 
           return new FileMetadata(
               key, // Always return denormalized key with metadata.
@@ -1914,8 +1900,8 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
       for (ListBlobItem blobItem : objects) {
         if (blobItem instanceof CloudBlockBlobWrapper
             || blobItem instanceof CloudPageBlobWrapper) {
-          LOG.debug("Found blob as a directory-using this file under it to infer its properties "
-              + blobItem.getUri());
+          LOG.debug("Found blob as a directory-using this file under it to infer its properties {}",
+              blobItem.getUri());
 
           blob = (CloudBlobWrapper) blobItem;
           // The key specifies a directory. Create a FileMetadata object which
@@ -2326,6 +2312,8 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
     try {
       blob.delete(operationContext, lease);
     } catch (StorageException e) {
+      LOG.error("Encountered Storage Exception for delete on Blob: {}, Exception Details: {} Error Code: {}",
+          blob.getUri(), e.getMessage(), e.getErrorCode());
       // On exception, check that if:
       // 1. It's a BlobNotFound exception AND
       // 2. It got there after one-or-more retries THEN
@@ -2334,9 +2322,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
           e.getErrorCode().equals("BlobNotFound") &&
           operationContext.getRequestResults().size() > 1 &&
           operationContext.getRequestResults().get(0).getException() != null) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Swallowing delete exception on retry: " + e.getMessage());
-        }
+        LOG.debug("Swallowing delete exception on retry: {}", e.getMessage());
         return;
       } else {
         throw e;
@@ -2381,9 +2367,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
   public void rename(String srcKey, String dstKey, boolean acquireLease,
       SelfRenewingLease existingLease) throws IOException {
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Moving " + srcKey + " to " + dstKey);
-    }
+    LOG.debug("Moving {} to {}", srcKey, dstKey);
 
     if (acquireLease && existingLease != null) {
       throw new IOException("Cannot acquire new lease if one already exists.");
@@ -2559,7 +2543,7 @@ public class AzureNativeFileSystemStore implements NativeFileSystemStore {
    */
   @Override
   public SelfRenewingLease acquireLease(String key) throws AzureException {
-    LOG.debug("acquiring lease on " + key);
+    LOG.debug("acquiring lease on {}", key);
     try {
       checkContainer(ContainerAccessType.ReadThenWrite);
       CloudBlobWrapper blob = getBlobReference(key);

+ 161 - 144
hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azure/NativeAzureFileSystem.java

@@ -41,8 +41,6 @@ import java.util.regex.Pattern;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -61,12 +59,12 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.azure.AzureException;
 import org.apache.hadoop.fs.azure.StorageInterface.CloudBlobWrapper;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Progressable;
 
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.codehaus.jackson.JsonNode;
 import org.codehaus.jackson.JsonParseException;
 import org.codehaus.jackson.JsonParser;
@@ -174,8 +172,8 @@ public class NativeAzureFileSystem extends FileSystem {
       }
       
       if (!this.committed) {
-        LOG.error("Deleting corruped rename pending file "
-            + redoFile + "\n" + contents);
+        LOG.error("Deleting corruped rename pending file {} \n {}",
+            redoFile, contents);
 
         // delete the -RenamePending.json file
         fs.delete(redoFile, false);
@@ -244,9 +242,7 @@ public class NativeAzureFileSystem extends FileSystem {
      */
     public void writeFile(FileSystem fs) throws IOException {
       Path path = getRenamePendingFilePath();
-      if (LOG.isDebugEnabled()){
-        LOG.debug("Preparing to write atomic rename state to " + path.toString());
-      }
+      LOG.debug("Preparing to write atomic rename state to {}", path.toString());
       OutputStream output = null;
 
       String contents = makeRenamePendingFileContents();
@@ -259,7 +255,7 @@ public class NativeAzureFileSystem extends FileSystem {
         throw new IOException("Unable to write RenamePending file for folder rename from "
             + srcKey + " to " + dstKey, e);
       } finally {
-        IOUtils.cleanup(LOG, output);
+        NativeAzureFileSystem.cleanup(LOG, output);
       }
     }
 
@@ -289,8 +285,8 @@ public class NativeAzureFileSystem extends FileSystem {
             MAX_RENAME_PENDING_FILE_SIZE - FORMATTING_BUFFER) {
 
           // Give up now to avoid using too much memory.
-          LOG.error("Internal error: Exceeded maximum rename pending file size of "
-              + MAX_RENAME_PENDING_FILE_SIZE + " bytes.");
+          LOG.error("Internal error: Exceeded maximum rename pending file size of {} bytes.",
+              MAX_RENAME_PENDING_FILE_SIZE);
 
           // return some bad JSON with an error message to make it human readable
           return "exceeded maximum rename pending file size";
@@ -602,7 +598,7 @@ public class NativeAzureFileSystem extends FileSystem {
     }
   }
 
-  public static final Log LOG = LogFactory.getLog(NativeAzureFileSystem.class);
+  public static final Logger LOG = LoggerFactory.getLogger(NativeAzureFileSystem.class);
 
   static final String AZURE_BLOCK_SIZE_PROPERTY_NAME = "fs.azure.block.size";
   /**
@@ -695,18 +691,27 @@ public class NativeAzureFileSystem extends FileSystem {
      */
     @Override
     public synchronized int read() throws IOException {
-      int result = 0;
-      result = in.read();
-      if (result != -1) {
-        pos++;
-        if (statistics != null) {
-          statistics.incrementBytesRead(1);
+      try {
+        int result = 0;
+        result = in.read();
+        if (result != -1) {
+          pos++;
+          if (statistics != null) {
+            statistics.incrementBytesRead(1);
+          }
         }
-      }
-
       // Return to the caller with the result.
       //
-      return result;
+        return result;
+      } catch(IOException e) {
+        if (e.getCause() instanceof StorageException) {
+          StorageException storageExcp  = (StorageException) e.getCause();
+          LOG.error("Encountered Storage Exception for read on Blob : {}"
+              + " Exception details: {} Error Code : {}",
+              key, e.getMessage(), storageExcp.getErrorCode());
+        }
+        throw e;
+      }
     }
 
     /*
@@ -731,18 +736,28 @@ public class NativeAzureFileSystem extends FileSystem {
      */
     @Override
     public synchronized int read(byte[] b, int off, int len) throws IOException {
-      int result = 0;
-      result = in.read(b, off, len);
-      if (result > 0) {
-        pos += result;
-      }
+      try {
+        int result = 0;
+        result = in.read(b, off, len);
+        if (result > 0) {
+          pos += result;
+        }
 
-      if (null != statistics) {
-        statistics.incrementBytesRead(result);
-      }
+        if (null != statistics) {
+          statistics.incrementBytesRead(result);
+        }
 
-      // Return to the caller with the result.
-      return result;
+        // Return to the caller with the result.
+        return result;
+      } catch(IOException e) {
+        if (e.getCause() instanceof StorageException) {
+          StorageException storageExcp  = (StorageException) e.getCause();
+          LOG.error("Encountered Storage Exception for read on Blob : {}"
+              + " Exception details: {} Error Code : {}",
+              key, e.getMessage(), storageExcp.getErrorCode());
+        }
+        throw e;
+      }
     }
 
     @Override
@@ -756,10 +771,8 @@ public class NativeAzureFileSystem extends FileSystem {
      in.close();
      in = store.retrieve(key);
      this.pos = in.skip(pos);
-     if (LOG.isDebugEnabled()) {
-       LOG.debug(String.format("Seek to position %d. Bytes skipped %d", pos,
-         this.pos));
-     }
+     LOG.debug("Seek to position {}. Bytes skipped {}", pos,
+         this.pos);
     }
 
     @Override
@@ -832,7 +845,17 @@ public class NativeAzureFileSystem extends FileSystem {
      */
     @Override
     public void write(int b) throws IOException {
-      out.write(b);
+      try {
+        out.write(b);
+      } catch(IOException e) {
+        if (e.getCause() instanceof StorageException) {
+          StorageException storageExcp  = (StorageException) e.getCause();
+          LOG.error("Encountered Storage Exception for write on Blob : {}"
+              + " Exception details: {} Error Code : {}",
+              key, e.getMessage(), storageExcp.getErrorCode());
+        }
+        throw e;
+      }
     }
 
     /**
@@ -845,7 +868,17 @@ public class NativeAzureFileSystem extends FileSystem {
      */
     @Override
     public void write(byte[] b) throws IOException {
-      out.write(b);
+      try {
+        out.write(b);
+      } catch(IOException e) {
+        if (e.getCause() instanceof StorageException) {
+          StorageException storageExcp  = (StorageException) e.getCause();
+          LOG.error("Encountered Storage Exception for write on Blob : {}"
+              + " Exception details: {} Error Code : {}",
+              key, e.getMessage(), storageExcp.getErrorCode());
+        }
+        throw e;
+      }
     }
 
     /**
@@ -865,7 +898,17 @@ public class NativeAzureFileSystem extends FileSystem {
      */
     @Override
     public void write(byte[] b, int off, int len) throws IOException {
-      out.write(b, off, len);
+      try {
+        out.write(b, off, len);
+      } catch(IOException e) {
+        if (e.getCause() instanceof StorageException) {
+          StorageException storageExcp  = (StorageException) e.getCause();
+          LOG.error("Encountered Storage Exception for write on Blob : {}"
+              + " Exception details: {} Error Code : {}",
+              key, e.getMessage(), storageExcp.getErrorCode());
+        }
+        throw e;
+      }
     }
 
     /**
@@ -1053,11 +1096,11 @@ public class NativeAzureFileSystem extends FileSystem {
     this.blockSize = conf.getLong(AZURE_BLOCK_SIZE_PROPERTY_NAME,
         MAX_AZURE_BLOCK_SIZE);
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("NativeAzureFileSystem. Initializing.");
-      LOG.debug("  blockSize  = "
-          + conf.getLong(AZURE_BLOCK_SIZE_PROPERTY_NAME, MAX_AZURE_BLOCK_SIZE));
-    }
+
+    LOG.debug("NativeAzureFileSystem. Initializing.");
+    LOG.debug("  blockSize  = {}",
+        conf.getLong(AZURE_BLOCK_SIZE_PROPERTY_NAME, MAX_AZURE_BLOCK_SIZE));
+
   }
 
   private NativeFileSystemStore createDefaultStore(Configuration conf) {
@@ -1231,8 +1274,8 @@ public class NativeAzureFileSystem extends FileSystem {
               f.getName() + " because parent folder does not exist.");
         }
 
-        LOG.warn("Got unexpected exception trying to get lease on "
-          + pathToKey(parent) + ". " + e.getMessage());
+        LOG.warn("Got unexpected exception trying to get lease on {} . {}",
+          pathToKey(parent), e.getMessage());
         throw e;
       }
     }
@@ -1250,7 +1293,7 @@ public class NativeAzureFileSystem extends FileSystem {
         // This'll let the keep-alive thread exit as soon as it wakes up.
         lease.free();
       } catch (Exception e) {
-        LOG.warn("Unable to free lease because: " + e.getMessage());
+        LOG.warn("Unable to free lease because: {}", e.getMessage());
       }
       throw new FileNotFoundException("Cannot create file " +
           f.getName() + " because parent folder does not exist.");
@@ -1268,7 +1311,7 @@ public class NativeAzureFileSystem extends FileSystem {
           lease.free();
         }
       } catch (Exception e) {
-        IOUtils.cleanup(LOG, out);
+        NativeAzureFileSystem.cleanup(LOG, out);
         String msg = "Unable to free lease on " + parent.toUri();
         LOG.error(msg);
         throw new IOException(msg, e);
@@ -1327,9 +1370,7 @@ public class NativeAzureFileSystem extends FileSystem {
       SelfRenewingLease parentFolderLease)
           throws IOException {
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Creating file: " + f.toString());
-    }
+    LOG.debug("Creating file: {}", f.toString());
 
     if (containsColon(f)) {
       throw new IOException("Cannot create file " + f
@@ -1456,9 +1497,7 @@ public class NativeAzureFileSystem extends FileSystem {
   public boolean delete(Path f, boolean recursive,
       boolean skipParentFolderLastModifidedTimeUpdate) throws IOException {
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Deleting file: " + f.toString());
-    }
+    LOG.debug("Deleting file: {}", f.toString());
 
     Path absolutePath = makeAbsolute(f);
     String key = pathToKey(absolutePath);
@@ -1492,11 +1531,10 @@ public class NativeAzureFileSystem extends FileSystem {
               + parentPath + " which is also a file. Can't resolve.");
         }
         if (parentMetadata.getBlobMaterialization() == BlobMaterialization.Implicit) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Found an implicit parent directory while trying to"
-                + " delete the file " + f + ". Creating the directory blob for"
-                + " it in " + parentKey + ".");
-          }
+          LOG.debug("Found an implicit parent directory while trying to"
+              + " delete the file {}. Creating the directory blob for"
+              + " it in {}.", f, parentKey);
+
           store.storeEmptyFolder(parentKey,
               createPermissionStatus(FsPermission.getDefault()));
         } else {
@@ -1510,18 +1548,17 @@ public class NativeAzureFileSystem extends FileSystem {
     } else {
       // The path specifies a folder. Recursively delete all entries under the
       // folder.
+      LOG.debug("Directory Delete encountered: {}", f.toString());
       Path parentPath = absolutePath.getParent();
       if (parentPath.getParent() != null) {
         String parentKey = pathToKey(parentPath);
         FileMetadata parentMetadata = store.retrieveMetadata(parentKey);
 
         if (parentMetadata.getBlobMaterialization() == BlobMaterialization.Implicit) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Found an implicit parent directory while trying to"
-                + " delete the directory " + f
-                + ". Creating the directory blob for" + " it in " + parentKey
-                + ".");
-          }
+          LOG.debug("Found an implicit parent directory while trying to"
+              + " delete the directory {}. Creating the directory blob for"
+              + " it in {}. ", f, parentKey);
+
           store.storeEmptyFolder(parentKey,
               createPermissionStatus(FsPermission.getDefault()));
         }
@@ -1570,15 +1607,14 @@ public class NativeAzureFileSystem extends FileSystem {
     }
 
     // File or directory was successfully deleted.
+    LOG.debug("Delete Successful for : {}", f.toString());
     return true;
   }
 
   @Override
   public FileStatus getFileStatus(Path f) throws IOException {
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Getting the file status for " + f.toString());
-    }
+    LOG.debug("Getting the file status for {}", f.toString());
 
     // Capture the absolute path and the path to key.
     Path absolutePath = makeAbsolute(f);
@@ -1594,9 +1630,8 @@ public class NativeAzureFileSystem extends FileSystem {
       if (meta.isDir()) {
         // The path is a folder with files in it.
         //
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Path " + f.toString() + "is a folder.");
-        }
+
+        LOG.debug("Path {} is a folder.", f.toString());
 
         // If a rename operation for the folder was pending, redo it.
         // Then the file does not exist, so signal that.
@@ -1610,9 +1645,7 @@ public class NativeAzureFileSystem extends FileSystem {
       }
 
       // The path is a file.
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Found the path: " + f.toString() + " as a file.");
-      }
+      LOG.debug("Found the path: {} as a file.", f.toString());
 
       // Return with reference to a file object.
       return newFile(meta, absolutePath);
@@ -1665,9 +1698,7 @@ public class NativeAzureFileSystem extends FileSystem {
   @Override
   public FileStatus[] listStatus(Path f) throws IOException {
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Listing status for " + f.toString());
-    }
+    LOG.debug("Listing status for {}", f.toString());
 
     Path absolutePath = makeAbsolute(f);
     String key = pathToKey(absolutePath);
@@ -1676,9 +1707,9 @@ public class NativeAzureFileSystem extends FileSystem {
 
     if (meta != null) {
       if (!meta.isDir()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Found path as a file");
-        }
+
+        LOG.debug("Found path as a file");
+
         return new FileStatus[] { newFile(meta, absolutePath) };
       }
       String partialKey = null;
@@ -1714,15 +1745,13 @@ public class NativeAzureFileSystem extends FileSystem {
           status.add(newFile(fileMetadata, subpath));
         }
       }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Found path as a directory with " + status.size()
-            + " files in it.");
-      }
+
+      LOG.debug("Found path as a directory with {}"
+          + " files in it.", status.size());
+
     } else {
       // There is no metadata found for the path.
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Did not find any metadata for path: " + key);
-      }
+      LOG.debug("Did not find any metadata for path: {}", key);
 
       throw new FileNotFoundException("File" + f + " does not exist.");
     }
@@ -1835,9 +1864,9 @@ public class NativeAzureFileSystem extends FileSystem {
   }
 
   public boolean mkdirs(Path f, FsPermission permission, boolean noUmask) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Creating directory: " + f.toString());
-    }
+
+
+    LOG.debug("Creating directory: {}", f.toString());
 
     if (containsColon(f)) {
       throw new IOException("Cannot create directory " + f
@@ -1894,9 +1923,8 @@ public class NativeAzureFileSystem extends FileSystem {
 
   @Override
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Opening file: " + f.toString());
-    }
+
+    LOG.debug("Opening file: {}", f.toString());
 
     Path absolutePath = makeAbsolute(f);
     String key = pathToKey(absolutePath);
@@ -1918,9 +1946,7 @@ public class NativeAzureFileSystem extends FileSystem {
 
     FolderRenamePending renamePending = null;
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Moving " + src + " to " + dst);
-    }
+    LOG.debug("Moving {} to {}", src, dst);
 
     if (containsColon(dst)) {
       throw new IOException("Cannot rename to file " + dst
@@ -1941,46 +1967,34 @@ public class NativeAzureFileSystem extends FileSystem {
     if (dstMetadata != null && dstMetadata.isDir()) {
       // It's an existing directory.
       dstKey = pathToKey(makeAbsolute(new Path(dst, src.getName())));
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Destination " + dst
-            + " is a directory, adjusted the destination to be " + dstKey);
-      }
+      LOG.debug("Destination {} "
+          + " is a directory, adjusted the destination to be {}", dst, dstKey);
     } else if (dstMetadata != null) {
       // Attempting to overwrite a file using rename()
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Destination " + dst
-            + " is an already existing file, failing the rename.");
-      }
+      LOG.debug("Destination {}"
+          + " is an already existing file, failing the rename.", dst);
       return false;
     } else {
       // Check that the parent directory exists.
       FileMetadata parentOfDestMetadata =
           store.retrieveMetadata(pathToKey(absoluteDst.getParent()));
       if (parentOfDestMetadata == null) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Parent of the destination " + dst
-              + " doesn't exist, failing the rename.");
-        }
+        LOG.debug("Parent of the destination {}"
+            + " doesn't exist, failing the rename.", dst);
         return false;
       } else if (!parentOfDestMetadata.isDir()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Parent of the destination " + dst
-              + " is a file, failing the rename.");
-        }
+        LOG.debug("Parent of the destination {}"
+            + " is a file, failing the rename.", dst);
         return false;
       }
     }
     FileMetadata srcMetadata = store.retrieveMetadata(srcKey);
     if (srcMetadata == null) {
       // Source doesn't exist
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Source " + src + " doesn't exist, failing the rename.");
-      }
+      LOG.debug("Source {} doesn't exist, failing the rename.", src);
       return false;
     } else if (!srcMetadata.isDir()) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Source " + src + " found as a file, renaming.");
-      }
+      LOG.debug("Source {} found as a file, renaming.", src);
       store.rename(srcKey, dstKey);
     } else {
 
@@ -1995,9 +2009,8 @@ public class NativeAzureFileSystem extends FileSystem {
       // In the future, we could generalize it easily to all folders.
       renamePending = prepareAtomicFolderRename(srcKey, dstKey);
       renamePending.execute();
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Renamed " + src + " to " + dst + " successfully.");
-      }
+
+      LOG.debug("Renamed {} to {} successfully.", src, dst);
       renamePending.cleanup();
       return true;
     }
@@ -2007,9 +2020,7 @@ public class NativeAzureFileSystem extends FileSystem {
     updateParentFolderLastModifiedTime(srcKey);
     updateParentFolderLastModifiedTime(dstKey);
 
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Renamed " + src + " to " + dst + " successfully.");
-    }
+    LOG.debug("Renamed {} to {} successfully.", src, dst);
     return true;
   }
 
@@ -2053,8 +2064,8 @@ public class NativeAzureFileSystem extends FileSystem {
             if (errorCode.equals("BlobNotFound")) {
               throw new FileNotFoundException("Folder does not exist: " + parentKey);
             }
-            LOG.warn("Got unexpected exception trying to get lease on "
-                + parentKey + ". " + e.getMessage());
+            LOG.warn("Got unexpected exception trying to get lease on {}. {}",
+                parentKey, e.getMessage());
             throw e;
           } finally {
             try {
@@ -2062,7 +2073,7 @@ public class NativeAzureFileSystem extends FileSystem {
                 lease.free();
               }
             } catch (Exception e) {
-              LOG.error("Unable to free lease on " + parentKey, e);
+              LOG.error("Unable to free lease on {}", parentKey, e);
             }
           }
         } else {
@@ -2244,10 +2255,8 @@ public class NativeAzureFileSystem extends FileSystem {
       AzureFileSystemMetricsSystem.fileSystemClosed();
     }
 
-    if (LOG.isDebugEnabled()) {
-        LOG.debug("Submitting metrics when file system closed took "
-                + (System.currentTimeMillis() - startTime) + " ms.");
-    }
+    LOG.debug("Submitting metrics when file system closed took {} ms.",
+        (System.currentTimeMillis() - startTime));
     isClosed = true;
   }
 
@@ -2268,9 +2277,8 @@ public class NativeAzureFileSystem extends FileSystem {
     @Override
     void handleFile(FileMetadata file, FileMetadata tempFile)
         throws IOException {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Deleting dangling file " + file.getKey());
-      }
+
+      LOG.debug("Deleting dangling file {}", file.getKey());
       store.delete(file.getKey());
       store.delete(tempFile.getKey());
     }
@@ -2290,9 +2298,8 @@ public class NativeAzureFileSystem extends FileSystem {
     @Override
     void handleFile(FileMetadata file, FileMetadata tempFile)
         throws IOException {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Recovering " + file.getKey());
-      }
+
+      LOG.debug("Recovering {}", file.getKey());
       // Move to the final destination
       String finalDestinationKey =
           pathToKey(new Path(destination, file.getKey()));
@@ -2369,9 +2376,8 @@ public class NativeAzureFileSystem extends FileSystem {
    */
   public void recoverFilesWithDanglingTempData(Path root, Path destination)
       throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Recovering files with dangling temp data in " + root);
-    }
+
+    LOG.debug("Recovering files with dangling temp data in {}", root);
     handleFilesWithDanglingTempData(root,
         new DanglingFileRecoverer(destination));
   }
@@ -2387,9 +2393,8 @@ public class NativeAzureFileSystem extends FileSystem {
    * @throws IOException
    */
   public void deleteFilesWithDanglingTempData(Path root) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Deleting files with dangling temp data in " + root);
-    }
+
+    LOG.debug("Deleting files with dangling temp data in {}", root);
     handleFilesWithDanglingTempData(root, new DanglingFileDeleter());
   }
 
@@ -2425,4 +2430,16 @@ public class NativeAzureFileSystem extends FileSystem {
     // Return to the caller with the randomized key.
     return randomizedKey;
   }
-}
+
+  private static void cleanup(Logger log, java.io.Closeable closeable) {
+    if (closeable != null) {
+      try {
+        closeable.close();
+      } catch(IOException e) {
+        if (log != null) {
+          log.debug("Exception in closing {}", closeable, e);
+        }
+      }
+    }
+  }
+}