소스 검색

HADOOP-17482: Remove Commons Logger from FileSystem Class (#2633)

belugabehr 4 년 전
부모
커밋
21a3fc3d2d

+ 2 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java

@@ -223,7 +223,7 @@ public class DelegationTokenRenewer
     if (action.token != null) {
       queue.add(action);
     } else {
-      fs.LOG.error("does not have a token for renewal");
+      FileSystem.LOG.error("does not have a token for renewal");
     }
     return action;
   }
@@ -247,7 +247,6 @@ public class DelegationTokenRenewer
     }
   }
 
-  @SuppressWarnings("static-access")
   @Override
   public void run() {
     for(;;) {
@@ -260,8 +259,7 @@ public class DelegationTokenRenewer
       } catch (InterruptedException ie) {
         return;
       } catch (Exception ie) {
-        action.weakFs.get().LOG.warn("Failed to renew token, action=" + action,
-            ie);
+        FileSystem.LOG.warn("Failed to renew token, action=" + action, ie);
       }
     }
   }

+ 2 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -48,8 +48,6 @@ import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.atomic.AtomicLong;
 
-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;
@@ -183,7 +181,7 @@ public abstract class FileSystem extends Configured
    * so must be considered something to only be changed with care.
    */
   @InterfaceAudience.Private
-  public static final Log LOG = LogFactory.getLog(FileSystem.class);
+  public static final Logger LOG = LoggerFactory.getLogger(FileSystem.class);
 
   /**
    * The SLF4J logger to use in logging within the FileSystem class itself.
@@ -3391,15 +3389,7 @@ public abstract class FileSystem extends Configured
               LOGGER.info("Full exception loading: {}", fs, e);
             }
           } catch (ServiceConfigurationError ee) {
-            LOG.warn("Cannot load filesystem: " + ee);
-            Throwable cause = ee.getCause();
-            // print all the nested exception messages
-            while (cause != null) {
-              LOG.warn(cause.toString());
-              cause = cause.getCause();
-            }
-            // and at debug: the full stack
-            LOG.debug("Stack Trace", ee);
+            LOGGER.warn("Cannot load filesystem", ee);
           }
         }
         FILE_SYSTEMS_LOADED = true;

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -318,7 +318,7 @@ public class RawLocalFileSystem extends FileSystem {
             success = true;
           } finally {
             if (!success) {
-              IOUtils.cleanup(LOG, this.fos);
+              IOUtils.cleanupWithLogger(LOG, this.fos);
             }
           }
         }

+ 3 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java

@@ -630,10 +630,9 @@ abstract class InodeTree<T> {
             .append(theUri.getScheme()).append("://").append(mountTableName)
             .append("/").toString());
       }
-      StringBuilder msg =
-          new StringBuilder("Empty mount table detected for ").append(theUri)
-              .append(" and considering itself as a linkFallback.");
-      FileSystem.LOG.info(msg.toString());
+      FileSystem.LOG
+          .info("Empty mount table detected for {} and considering itself "
+              + "as a linkFallback.", theUri);
       rootFallbackLink =
           new INodeLink<T>(mountTableName, ugi, getTargetFileSystem(theUri),
               theUri);

+ 4 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java

@@ -1289,10 +1289,8 @@ public class ViewFileSystem extends FileSystem {
               .create(fileToCreate, permission, overwrite, bufferSize,
                   replication, blockSize, progress);
         } catch (IOException e) {
-          StringBuilder msg =
-              new StringBuilder("Failed to create file:").append(fileToCreate)
-                  .append(" at fallback : ").append(linkedFallbackFs.getUri());
-          LOG.error(msg.toString(), e);
+          LOG.error("Failed to create file: {} at fallback: {}", fileToCreate,
+              linkedFallbackFs.getUri(), e);
           throw e;
         }
       }
@@ -1523,11 +1521,8 @@ public class ViewFileSystem extends FileSystem {
           return linkedFallbackFs.mkdirs(dirToCreate, permission);
         } catch (IOException e) {
           if (LOG.isDebugEnabled()) {
-            StringBuilder msg =
-                new StringBuilder("Failed to create ").append(dirToCreate)
-                    .append(" at fallback : ")
-                    .append(linkedFallbackFs.getUri());
-            LOG.debug(msg.toString(), e);
+            LOG.debug("Failed to create: {} at fallback: {}", dirToCreate,
+                linkedFallbackFs.getUri(), e);
           }
           throw e;
         }

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFileSystem.java

@@ -28,7 +28,6 @@ import java.net.URI;
 import java.util.EnumSet;
 import java.util.Iterator;
 
-import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.Options.CreateOpts;
@@ -39,10 +38,11 @@ import org.apache.hadoop.security.token.DelegationTokenIssuer;
 import org.apache.hadoop.util.Progressable;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.slf4j.Logger;
 
 public class TestFilterFileSystem {
 
-  private static final Log LOG = FileSystem.LOG;
+  private static final Logger LOG = FileSystem.LOG;
   private static final Configuration conf = new Configuration();
 
   @BeforeClass

+ 2 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFilterFs.java

@@ -23,14 +23,14 @@ import java.lang.reflect.Modifier;
 import java.net.URI;
 import java.util.Iterator;
 
-import org.apache.commons.logging.Log;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
 import org.junit.Test;
+import org.slf4j.Logger;
 
 public class TestFilterFs {
 
-  private static final Log LOG = FileSystem.LOG;
+  private static final Logger LOG = FileSystem.LOG;
 
   public static class DontCheck {
     public void checkScheme(URI uri, String supportedScheme) { }

+ 0 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSetUMask.java

@@ -29,10 +29,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.util.StringUtils;
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
-import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -100,16 +97,6 @@ public class TestFcHdfsSetUMask {
       cluster.shutdown();
     }
   }
-  
-  {
-    try {
-      GenericTestUtils.setLogLevel(FileSystem.LOG, Level.DEBUG);
-    }
-    catch(Exception e) {
-      System.out.println("Cannot change log level\n"
-          + StringUtils.stringifyException(e));
-    }
-  }
 
   @Before
   public void setUp() throws Exception {

+ 0 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestListFilesInDFS.java

@@ -17,11 +17,8 @@
  */
 package org.apache.hadoop.hdfs;
 
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.TestListFiles;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
@@ -29,10 +26,6 @@ import org.junit.BeforeClass;
  * This class tests the FileStatus API.
  */
 public class TestListFilesInDFS extends TestListFiles {
-  {
-    GenericTestUtils.setLogLevel(FileSystem.LOG, Level.ALL);
-  }
-
 
   private static MiniDFSCluster cluster;
 

+ 0 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestListFilesInFileContext.java

@@ -29,14 +29,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -46,9 +43,6 @@ import org.junit.Test;
  * This class tests the FileStatus API.
  */
 public class TestListFilesInFileContext {
-  {
-    GenericTestUtils.setLogLevel(FileSystem.LOG, Level.ALL);
-  }
 
   static final long seed = 0xDEADBEEFL;
 

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java

@@ -374,7 +374,7 @@ public class WindowsSecureContainerExecutor extends DefaultContainerExecutor {
           return os;
         } finally {
           if (!success) {
-            IOUtils.cleanup(LOG, os);
+            IOUtils.cleanupWithLogger(LOG, os);
           }
         }
       }