Jelajahi Sumber

HDFS-12552. Use slf4j instead of log4j in FSNamesystem. Contributed by Ajay Kumar.

Chen Liang 7 tahun lalu
induk
melakukan
5a68c2cef3

+ 6 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -310,6 +310,7 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.LoggerFactory;
 
 /**
  * FSNamesystem is a container of both transient
@@ -342,7 +343,9 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
 @Metrics(context="dfs")
 public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     NameNodeMXBean, ReplicatedBlocksMBean, ECBlockGroupsMBean {
-  public static final Log LOG = LogFactory.getLog(FSNamesystem.class);
+
+  public static final org.slf4j.Logger LOG = LoggerFactory
+      .getLogger(FSNamesystem.class.getName());
   private final MetricsRegistry registry = new MetricsRegistry("FSNamesystem");
   @Metric final MutableRatesWithAggregation detailedLockHoldTimeMetrics =
       registry.newRatesWithAggregation("detailedLockHoldTimeMetrics");
@@ -1671,8 +1674,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         stopStandbyServices();
       } catch (IOException ie) {
       } finally {
-        IOUtils.cleanup(LOG, dir);
-        IOUtils.cleanup(LOG, fsImage);
+        IOUtils.cleanupWithLogger(LOG, dir);
+        IOUtils.cleanupWithLogger(LOG, fsImage);
       }
     }
   }

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileConcurrentReader.java

@@ -38,12 +38,12 @@ import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.slf4j.event.Level;
 
 
 /**
@@ -62,9 +62,9 @@ public class TestFileConcurrentReader {
     Logger.getLogger(TestFileConcurrentReader.class);
 
   {
-    GenericTestUtils.setLogLevel(LeaseManager.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(LeaseManager.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(DFSClient.LOG, Level.TRACE);
   }
 
   static final long seed = 0xDEADBEEFL;

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java

@@ -86,9 +86,9 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Time;
-import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Test;
+import org.slf4j.event.Level;
 
 /**
  * This class tests various cases during file creation.
@@ -97,9 +97,9 @@ public class TestFileCreation {
   static final String DIR = "/" + TestFileCreation.class.getSimpleName() + "/";
 
   {
-    GenericTestUtils.setLogLevel(LeaseManager.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(LeaseManager.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(DFSClient.LOG, Level.TRACE);
   }
   private static final String RPC_DETAILED_METRICS =
       "RpcDetailedActivityForPort";

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreationClient.java

@@ -30,8 +30,8 @@ import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
 import org.junit.Test;
+import org.slf4j.event.Level;
 
 /**
  * This class tests client lease recovery.
@@ -40,10 +40,10 @@ public class TestFileCreationClient {
   static final String DIR = "/" + TestFileCreationClient.class.getSimpleName() + "/";
 
   {
-    GenericTestUtils.setLogLevel(DataNode.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(LeaseManager.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(InterDatanodeProtocol.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(DataNode.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(LeaseManager.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(InterDatanodeProtocol.LOG, Level.TRACE);
   }
 
   /** Test lease recovery Triggered by DFSClient. */

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java

@@ -38,18 +38,18 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.event.Level;
 
 /**
  * This class tests the FileStatus API.
  */
 public class TestFileStatus {
   {
-    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(FileSystem.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(FileSystem.LOG, Level.TRACE);
   }
 
   static final long seed = 0xDEADBEEFL;

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java

@@ -52,21 +52,21 @@ import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
+import org.slf4j.event.Level;
 
 public class TestLeaseRecovery2 {
   
   public static final Log LOG = LogFactory.getLog(TestLeaseRecovery2.class);
   
   {
-    GenericTestUtils.setLogLevel(DataNode.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(LeaseManager.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(DataNode.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(LeaseManager.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.TRACE);
   }
 
   static final private long BLOCK_SIZE = 1024;

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java

@@ -43,8 +43,8 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
-import org.apache.log4j.Level;
 import org.junit.Test;
+import org.slf4j.event.Level;
 
 /**
  * A JUnit test for checking if restarting DFS preserves the
@@ -52,8 +52,8 @@ import org.junit.Test;
  */
 public class TestPersistBlocks {
   static {
-    GenericTestUtils.setLogLevel(FSImage.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(FSImage.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.TRACE);
   }
   
   private static final int BLOCK_SIZE = 4096;

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadWhileWriting.java

@@ -32,15 +32,15 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
 import org.junit.Assert;
 import org.junit.Test;
+import org.slf4j.event.Level;
 
 /** Test reading from hdfs while a file is being written. */
 public class TestReadWhileWriting {
   {
-    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(DFSClient.LOG, Level.TRACE);
   }
 
   private static final String DIR = "/"

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlocksWithNotEnoughRacks.java

@@ -40,14 +40,14 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.util.HostsFileWriter;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
 import org.junit.Test;
+import org.slf4j.event.Level;
 
 public class TestBlocksWithNotEnoughRacks {
   public static final Log LOG = LogFactory.getLog(TestBlocksWithNotEnoughRacks.class);
   static {
-    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.ALL);
-    GenericTestUtils.setLogLevel(LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(FSNamesystem.LOG, Level.TRACE);
+    GenericTestUtils.setLogLevel(LOG, Level.TRACE);
   }
 
   /*

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java

@@ -28,7 +28,6 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
 import org.apache.hadoop.test.MetricsAsserts;
 import org.apache.hadoop.util.FakeTimer;
-import org.apache.log4j.Level;
 import org.junit.Test;
 
 import java.io.IOException;
@@ -39,6 +38,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.regex.Pattern;
 import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
 
 import static org.junit.Assert.*;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_FSLOCK_FAIR_KEY;