Browse Source

HDFS-12650. Use slf4j instead of log4j in LeaseManager. Contributed by Ajay Kumar.

Arpit Agarwal 7 years ago
parent
commit
96be795656

+ 26 - 31
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -37,8 +37,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 import com.google.common.collect.Lists;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -50,6 +48,8 @@ import org.apache.hadoop.util.Daemon;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * LeaseManager does the lease housekeeping for writing on files.   
@@ -75,7 +75,8 @@ import org.apache.hadoop.util.Time;
  */
 @InterfaceAudience.Private
 public class LeaseManager {
-  public static final Log LOG = LogFactory.getLog(LeaseManager.class);
+  public static final Logger LOG = LoggerFactory.getLogger(LeaseManager.class
+      .getName());
   private final FSNamesystem fsnamesystem;
   private long softLimit = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
   private long hardLimit = HdfsConstants.LEASE_HARDLIMIT_PERIOD;
@@ -142,8 +143,8 @@ public class LeaseManager {
     for (Long id : getINodeIdWithLeases()) {
       final INodeFile cons = fsnamesystem.getFSDirectory().getInode(id).asFile();
       if (!cons.isUnderConstruction()) {
-        LOG.warn("The file " + cons.getFullPathName()
-            + " is not under construction but has lease.");
+        LOG.warn("The file {} is not under construction but has lease.",
+            cons.getFullPathName());
         continue;
       }
       BlockInfo[] blocks = cons.getBlocks();
@@ -155,7 +156,7 @@ public class LeaseManager {
           numUCBlocks++;
         }
       }
-    LOG.info("Number of blocks under construction: " + numUCBlocks);
+    LOG.info("Number of blocks under construction: {}", numUCBlocks);
     return numUCBlocks;
   }
 
@@ -250,9 +251,8 @@ public class LeaseManager {
     }
     final long endTimeMs = Time.monotonicNow();
     if ((endTimeMs - startTimeMs) > 1000) {
-      LOG.info("Took " + (endTimeMs - startTimeMs) + " ms to collect "
-          + iipSet.size() + " open files with leases" +
-          ((ancestorDir != null) ?
+      LOG.info("Took {} ms to collect {} open files with leases {}",
+          (endTimeMs - startTimeMs), iipSet.size(), ((ancestorDir != null) ?
               " under " + ancestorDir.getFullPathName() : "."));
     }
     return iipSet;
@@ -287,8 +287,8 @@ public class LeaseManager {
       final INodeFile inodeFile =
           fsnamesystem.getFSDirectory().getInode(inodeId).asFile();
       if (!inodeFile.isUnderConstruction()) {
-        LOG.warn("The file " + inodeFile.getFullPathName()
-            + " is not under construction but has lease.");
+        LOG.warn("The file {} is not under construction but has lease.",
+            inodeFile.getFullPathName());
         continue;
       }
       openFileEntries.add(new OpenFileEntry(
@@ -348,16 +348,13 @@ public class LeaseManager {
   private synchronized void removeLease(Lease lease, long inodeId) {
     leasesById.remove(inodeId);
     if (!lease.removeFile(inodeId)) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("inode " + inodeId + " not found in lease.files (=" + lease
-                + ")");
-      }
+      LOG.debug("inode {} not found in lease.files (={})", inodeId, lease);
     }
 
     if (!lease.hasFiles()) {
       leases.remove(lease.holder);
       if (!sortedLeases.remove(lease)) {
-        LOG.error(lease + " not found in sortedLeases");
+        LOG.error("{} not found in sortedLeases", lease);
       }
     }
   }
@@ -370,8 +367,8 @@ public class LeaseManager {
     if (lease != null) {
       removeLease(lease, src.getId());
     } else {
-      LOG.warn("Removing non-existent lease! holder=" + holder +
-          " src=" + src.getFullPathName());
+      LOG.warn("Removing non-existent lease! holder={} src={}", holder, src
+          .getFullPathName());
     }
   }
 
@@ -513,9 +510,7 @@ public class LeaseManager {
   
           Thread.sleep(fsnamesystem.getLeaseRecheckIntervalMs());
         } catch(InterruptedException ie) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug(name + " is interrupted", ie);
-          }
+          LOG.debug("{} is interrupted", name, ie);
         } catch(Throwable e) {
           LOG.warn("Unexpected throwable: ", e);
         }
@@ -537,7 +532,7 @@ public class LeaseManager {
         sortedLeases.first().expiredHardLimit()
         && !isMaxLockHoldToReleaseLease(start)) {
       Lease leaseToCheck = sortedLeases.first();
-      LOG.info(leaseToCheck + " has expired hard limit");
+      LOG.info("{} has expired hard limit", leaseToCheck);
 
       final List<Long> removing = new ArrayList<>();
       // need to create a copy of the oldest lease files, because
@@ -568,16 +563,16 @@ public class LeaseManager {
             completed = fsnamesystem.internalReleaseLease(
                 leaseToCheck, p, iip, newHolder);
           } catch (IOException e) {
-            LOG.warn("Cannot release the path " + p + " in the lease "
-                + leaseToCheck + ". It will be retried.", e);
+            LOG.warn("Cannot release the path {} in the lease {}. It will be "
+                + "retried.", p, leaseToCheck, e);
             continue;
           }
           if (LOG.isDebugEnabled()) {
             if (completed) {
-              LOG.debug("Lease recovery for inode " + id + " is complete. " +
-                            "File closed.");
+              LOG.debug("Lease recovery for inode {} is complete. File closed"
+                  + ".", id);
             } else {
-              LOG.debug("Started block recovery " + p + " lease " + leaseToCheck);
+              LOG.debug("Started block recovery {} lease {}", p, leaseToCheck);
             }
           }
           // If a lease recovery happened, we need to sync later.
@@ -585,13 +580,13 @@ public class LeaseManager {
             needSync = true;
           }
         } catch (IOException e) {
-          LOG.warn("Removing lease with an invalid path: " + p + ","
-              + leaseToCheck, e);
+          LOG.warn("Removing lease with an invalid path: {},{}", p,
+              leaseToCheck, e);
           removing.add(id);
         }
         if (isMaxLockHoldToReleaseLease(start)) {
-          LOG.debug("Breaking out of checkLeases after " +
-              fsnamesystem.getMaxLockHoldToReleaseLeaseMs() + "ms.");
+          LOG.debug("Breaking out of checkLeases after {} ms.",
+              fsnamesystem.getMaxLockHoldToReleaseLeaseMs());
           break;
         }
       }

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/InterDatanodeProtocol.java

@@ -19,14 +19,13 @@
 package org.apache.hadoop.hdfs.server.protocol;
 
 import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
 import org.apache.hadoop.security.KerberosInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /** An inter-datanode protocol for updating generation stamp
  */
@@ -35,7 +34,7 @@ import org.apache.hadoop.security.KerberosInfo;
     clientPrincipal = DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY)
 @InterfaceAudience.Private
 public interface InterDatanodeProtocol {
-  public static final Log LOG = LogFactory.getLog(InterDatanodeProtocol.class);
+  Logger LOG = LoggerFactory.getLogger(InterDatanodeProtocol.class.getName());
 
   /**
    * Until version 9, this class InterDatanodeProtocol served as both

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeDeath.java

@@ -45,7 +45,8 @@ public class TestDatanodeDeath {
     DFSTestUtil.setNameNodeLogLevel(Level.ALL);
     GenericTestUtils.setLogLevel(DataNode.LOG, Level.ALL);
     GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(InterDatanodeProtocol.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(InterDatanodeProtocol.LOG, org.slf4j
+        .event.Level.TRACE);
   }
 
   static final int blockSize = 8192;

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java

@@ -58,7 +58,8 @@ public class TestFileAppend3  {
     DFSTestUtil.setNameNodeLogLevel(Level.ALL);
     GenericTestUtils.setLogLevel(DataNode.LOG, Level.ALL);
     GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(InterDatanodeProtocol.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(InterDatanodeProtocol.LOG, org.slf4j
+        .event.Level.TRACE);
   }
 
   static final long BLOCK_SIZE = 64 * 1024;