浏览代码

HADOOP-211. Switch logging use the Jakarta Commons logging API, configured to use log4j by default.

git-svn-id: https://svn.apache.org/repos/asf/lucene/hadoop/trunk@411254 13f79535-47bb-0310-9956-ffa450edef68
Doug Cutting 19 年之前
父节点
当前提交
9ea3823cec
共有 55 个文件被更改,包括 403 次插入365 次删除
  1. 3 0
      CHANGES.txt
  2. 9 0
      bin/hadoop
  3. 1 0
      bin/hadoop-daemon.sh
  4. 1 1
      build.xml
  5. 7 0
      conf/commons-logging.properties
  6. 52 0
      conf/log4j.properties
  7. 二进制
      lib/commons-logging-1.0.4.jar
  8. 二进制
      lib/log4j-1.2.13.jar
  9. 0 1
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java
  10. 0 1
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamInputFormat.java
  11. 0 1
      src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
  12. 14 13
      src/java/org/apache/hadoop/conf/Configuration.java
  13. 6 5
      src/java/org/apache/hadoop/dfs/DFSClient.java
  14. 3 2
      src/java/org/apache/hadoop/dfs/DFSck.java
  15. 14 15
      src/java/org/apache/hadoop/dfs/DataNode.java
  16. 11 11
      src/java/org/apache/hadoop/dfs/FSDirectory.java
  17. 51 50
      src/java/org/apache/hadoop/dfs/FSNamesystem.java
  18. 17 29
      src/java/org/apache/hadoop/dfs/NameNode.java
  19. 8 7
      src/java/org/apache/hadoop/fs/FSDataInputStream.java
  20. 3 3
      src/java/org/apache/hadoop/fs/FileSystem.java
  21. 2 2
      src/java/org/apache/hadoop/fs/LocalFileSystem.java
  22. 1 1
      src/java/org/apache/hadoop/io/MapFile.java
  23. 9 10
      src/java/org/apache/hadoop/io/SequenceFile.java
  24. 5 5
      src/java/org/apache/hadoop/io/UTF8.java
  25. 9 10
      src/java/org/apache/hadoop/ipc/Client.java
  26. 6 6
      src/java/org/apache/hadoop/ipc/RPC.java
  27. 13 15
      src/java/org/apache/hadoop/ipc/Server.java
  28. 4 4
      src/java/org/apache/hadoop/mapred/InputFormatBase.java
  29. 4 3
      src/java/org/apache/hadoop/mapred/JobClient.java
  30. 5 5
      src/java/org/apache/hadoop/mapred/JobInProgress.java
  31. 8 9
      src/java/org/apache/hadoop/mapred/JobTracker.java
  32. 7 7
      src/java/org/apache/hadoop/mapred/LocalJobRunner.java
  33. 9 9
      src/java/org/apache/hadoop/mapred/ReduceTaskRunner.java
  34. 5 4
      src/java/org/apache/hadoop/mapred/Task.java
  35. 2 3
      src/java/org/apache/hadoop/mapred/TaskInProgress.java
  36. 11 11
      src/java/org/apache/hadoop/mapred/TaskRunner.java
  37. 19 19
      src/java/org/apache/hadoop/mapred/TaskTracker.java
  38. 1 2
      src/java/org/apache/hadoop/util/LogFormatter.java
  39. 7 0
      src/test/log4j.properties
  40. 4 4
      src/test/org/apache/hadoop/dfs/ClusterTestDFS.java
  41. 5 10
      src/test/org/apache/hadoop/dfs/ClusterTestDFSNamespaceLogging.java
  42. 3 2
      src/test/org/apache/hadoop/fs/DFSCIOTest.java
  43. 3 2
      src/test/org/apache/hadoop/fs/DistributedFSCheck.java
  44. 3 2
      src/test/org/apache/hadoop/fs/TestDFSIO.java
  45. 3 2
      src/test/org/apache/hadoop/fs/TestFileSystem.java
  46. 7 8
      src/test/org/apache/hadoop/io/TestArrayFile.java
  47. 13 14
      src/test/org/apache/hadoop/io/TestSequenceFile.java
  48. 8 9
      src/test/org/apache/hadoop/io/TestSetFile.java
  49. 8 22
      src/test/org/apache/hadoop/ipc/TestIPC.java
  50. 5 14
      src/test/org/apache/hadoop/ipc/TestRPC.java
  51. 3 3
      src/test/org/apache/hadoop/mapred/TestMapRed.java
  52. 3 2
      src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java
  53. 3 2
      src/test/org/apache/hadoop/mapred/TestTextInputFormat.java
  54. 3 3
      src/test/org/apache/hadoop/record/test/TestMapRed.java
  55. 2 2
      src/test/org/apache/hadoop/record/test/TestWritable.java

+ 3 - 0
CHANGES.txt

@@ -92,6 +92,9 @@ Trunk (unreleased)
 
 24. HADOOP-256.  Add a C API for DFS.  (Arun C Murthy via cutting)
 
+25. HADOOP-211.  Switch to use the Jakarta Commons logging internally,
+    configured to use log4j by default.  (Arun C Murthy and cutting)
+
 
 Release 0.2.1 - 2006-05-12
 

+ 9 - 0
bin/hadoop

@@ -120,6 +120,14 @@ done
 # restore ordinary behaviour
 unset IFS
 
+# default log directory & file
+if [ "HADOOP_LOG_DIR" = "" ]; then
+  HADOOP_LOG_DIR="$HADOOP_HOME/logs"
+fi
+if [ "HADOOP_LOGFILE" = "" ]; then
+  HADOOP_LOGFILE='hadoop.log'
+fi
+
 # figure out which class to run
 if [ "$COMMAND" = "namenode" ] ; then
   CLASS='org.apache.hadoop.dfs.NameNode'
@@ -151,6 +159,7 @@ if expr `uname` : 'CYGWIN*' > /dev/null; then
 fi
 
 HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.log.dir=$HADOOP_LOG_DIR"
+HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.log.file=$HADOOP_LOGFILE"
 HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.home.dir=$HADOOP_HOME"
 HADOOP_OPTS="$HADOOP_OPTS -Dhadoop.id.str=$HADOOP_IDENT_STRING"
 

+ 1 - 0
bin/hadoop-daemon.sh

@@ -62,6 +62,7 @@ if [ "$HADOOP_IDENT_STRING" = "" ]; then
 fi
 
 # some variables
+export HADOOP_LOGFILE=hadoop-$HADOOP_IDENT_STRING-$command-`hostname`.log
 log=$HADOOP_LOG_DIR/hadoop-$HADOOP_IDENT_STRING-$command-`hostname`.out
 pid=$HADOOP_PID_DIR/hadoop-$HADOOP_IDENT_STRING-$command.pid
 

+ 1 - 1
build.xml

@@ -111,7 +111,7 @@
   <!-- Compile the Java files                                 -->
   <!-- ====================================================== -->
   <taskdef classname="org.apache.jasper.JspC" name="jsp-compile" >
-      <classpath refid="classpath"/>
+      <classpath refid="test.classpath"/>
   </taskdef>
 
   <target name="record-parser" depends="init" if="javacc.home">

+ 7 - 0
conf/commons-logging.properties

@@ -0,0 +1,7 @@
+#Logging Implementation
+
+#Log4J
+org.apache.commons.logging.Log=org.apache.commons.logging.impl.Log4JLogger
+
+#JDK Logger
+#org.apache.commons.logging.Log=org.apache.commons.logging.impl.Jdk14Logger

+ 52 - 0
conf/log4j.properties

@@ -0,0 +1,52 @@
+# RootLogger - DailyRollingFileAppender
+log4j.rootLogger=INFO,DRFA
+
+# Logging Threshold
+log4j.threshhold=ALL
+
+
+#
+# Daily Rolling File Appender
+#
+
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} - %m%n
+# Debugging Pattern format: Date LogLevel LoggerName (FileName:MethodName:LineNo) LogMessage
+#log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# stdout
+# Add *stdout* to rootlogger above if you want to use this 
+#
+
+#log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+#log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+#log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+#
+# Rolling File Appender
+#
+
+#log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+#log4j.appender.RFA.File=${hadoop.log.dir}/${hadoop.log.file}
+
+# Logfile size and and 30-day backups
+#log4j.appender.RFA.MaxFileSize=1MB
+#log4j.appender.RFA.MaxBackupIndex=30
+
+#log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} - %m%n
+#log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
+

二进制
lib/commons-logging-1.0.4.jar


二进制
lib/log4j-1.2.13.jar


+ 0 - 1
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java

@@ -17,7 +17,6 @@
 package org.apache.hadoop.streaming;
 
 import java.io.*;
-import java.util.logging.Logger;
 
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;

+ 0 - 1
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamInputFormat.java

@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
-import java.util.logging.*;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;

+ 0 - 1
src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.streaming;
 import java.io.File;
 import java.io.IOException;
 import java.net.URL;
-import java.util.logging.*;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Iterator;

+ 14 - 13
src/java/org/apache/hadoop/conf/Configuration.java

@@ -19,7 +19,6 @@ package org.apache.hadoop.conf;
 import java.util.*;
 import java.net.URL;
 import java.io.*;
-import java.util.logging.*;
 
 import javax.xml.parsers.*;
 
@@ -29,6 +28,8 @@ import javax.xml.transform.Transformer;
 import javax.xml.transform.dom.DOMSource;
 import javax.xml.transform.stream.StreamResult;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.util.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -50,8 +51,8 @@ import org.apache.hadoop.fs.Path;
  * may specify additional resources.
  */
 public class Configuration {
-  private static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.conf.Configuration");
+  private static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.conf.Configuration");
 
   private ArrayList defaultResources = new ArrayList();
   private ArrayList finalResources = new ArrayList();
@@ -62,8 +63,8 @@ public class Configuration {
 
   /** A new configuration. */
   public Configuration() {
-    if (LOG.isLoggable(Level.FINE)) {
-      LOG.fine(StringUtils.stringifyException(new IOException("config()")));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(StringUtils.stringifyException(new IOException("config()")));
     }
     defaultResources.add("hadoop-default.xml");
     finalResources.add("hadoop-site.xml");
@@ -71,9 +72,9 @@ public class Configuration {
 
   /** A new configuration with the same settings cloned from another. */
   public Configuration(Configuration other) {
-    if (LOG.isLoggable(Level.FINE)) {
-      LOG.fine(StringUtils.stringifyException
-                 (new IOException("config(config)")));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug(StringUtils.stringifyException
+                (new IOException("config(config)")));
     }
     this.defaultResources = (ArrayList)other.defaultResources.clone();
     this.finalResources = (ArrayList)other.finalResources.clone();
@@ -290,11 +291,11 @@ public class Configuration {
         return file;
       }
     }
-    LOG.warning("Could not make " + path + 
+    LOG.warn("Could not make " + path + 
                 " in local directories from " + dirsProp);
     for(int i=0; i < dirs.length; i++) {
       int index = (hashCode+i & Integer.MAX_VALUE) % dirs.length;
-      LOG.warning(dirsProp + "[" + index + "]=" + dirs[index]);
+      LOG.warn(dirsProp + "[" + index + "]=" + dirs[index]);
     }
     throw new IOException("No valid local directories in property: "+dirsProp);
   }
@@ -418,7 +419,7 @@ public class Configuration {
 
       Element root = doc.getDocumentElement();
       if (!"configuration".equals(root.getTagName()))
-        LOG.severe("bad conf file: top-level element not <configuration>");
+        LOG.fatal("bad conf file: top-level element not <configuration>");
       NodeList props = root.getChildNodes();
       for (int i = 0; i < props.getLength(); i++) {
         Node propNode = props.item(i);
@@ -426,7 +427,7 @@ public class Configuration {
           continue;
         Element prop = (Element)propNode;
         if (!"property".equals(prop.getTagName()))
-          LOG.warning("bad conf file: element not <property>");
+          LOG.warn("bad conf file: element not <property>");
         NodeList fields = prop.getChildNodes();
         String attr = null;
         String value = null;
@@ -445,7 +446,7 @@ public class Configuration {
       }
         
     } catch (Exception e) {
-      LOG.severe("error parsing conf file: " + e);
+      LOG.fatal("error parsing conf file: " + e);
       throw new RuntimeException(e);
     }
     

+ 6 - 5
src/java/org/apache/hadoop/dfs/DFSClient.java

@@ -21,10 +21,11 @@ import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.*;
 
+import org.apache.commons.logging.*;
+
 import java.io.*;
 import java.net.*;
 import java.util.*;
-import java.util.logging.*;
 
 /********************************************************
  * DFSClient can connect to a Hadoop Filesystem and 
@@ -39,7 +40,7 @@ import java.util.logging.*;
  * @author Mike Cafarella, Tessa MacDuff
  ********************************************************/
 class DFSClient implements FSConstants {
-    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.fs.DFSClient");
+    public static final Log LOG = LogFactory.getLog("org.apache.hadoop.fs.DFSClient");
     static int MAX_BLOCK_ACQUIRE_FAILURES = 3;
     private static final long DEFAULT_BLOCK_SIZE = 64 * 1024 * 1024;
     ClientProtocol namenode;
@@ -405,7 +406,7 @@ class DFSClient implements FSConstants {
                         lastRenewed = System.currentTimeMillis();
                     } catch (IOException ie) {
                       String err = StringUtils.stringifyException(ie);
-                      LOG.warning("Problem renewing lease for " + clientName +
+                      LOG.warn("Problem renewing lease for " + clientName +
                                   ": " + err);
                     }
                 }
@@ -1024,14 +1025,14 @@ class DFSClient implements FSConstants {
         }
 
         private void handleSocketException(IOException ie) throws IOException {
-          LOG.log(Level.WARNING, "Error while writing.", ie);
+          LOG.warn("Error while writing.", ie);
           try {
             if (s != null) {
               s.close();
               s = null;
             }
           } catch (IOException ie2) {
-            LOG.log(Level.WARNING, "Error closing socket.", ie2);
+            LOG.warn("Error closing socket.", ie2);
           }
           namenode.abandonBlock(block, src.toString());
         }

+ 3 - 2
src/java/org/apache/hadoop/dfs/DFSck.java

@@ -25,7 +25,8 @@ import java.net.Socket;
 import java.util.ArrayList;
 import java.util.Random;
 import java.util.TreeSet;
-import java.util.logging.Logger;
+
+import org.apache.commons.logging.*;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSOutputStream;
@@ -56,7 +57,7 @@ import org.apache.hadoop.io.UTF8;
  * @author Andrzej Bialecki
  */
 public class DFSck {
-  private static final Logger LOG = Logger.getLogger(DFSck.class.getName());
+  private static final Log LOG = LogFactory.getLog(DFSck.class.getName());
 
   /** Don't attempt any fixing . */
   public static final int FIXING_NONE = 0;

+ 14 - 15
src/java/org/apache/hadoop/dfs/DataNode.java

@@ -15,6 +15,8 @@
  */
 package org.apache.hadoop.dfs;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.*;
@@ -24,7 +26,6 @@ import java.io.*;
 import java.net.*;
 import java.nio.channels.FileLock;
 import java.util.*;
-import java.util.logging.*;
 
 /**********************************************************
  * DataNode is a class (and program) that stores a set of
@@ -59,7 +60,7 @@ import java.util.logging.*;
  * @author Mike Cafarella
  **********************************************************/
 public class DataNode implements FSConstants, Runnable {
-    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.dfs.DataNode");
+    public static final Log LOG = LogFactory.getLog("org.apache.hadoop.dfs.DataNode");
     //
     // REMIND - mjc - I might bring "maxgigs" back so user can place 
     // artificial  limit on space
@@ -198,7 +199,7 @@ public class DataNode implements FSConstants, Runnable {
     }
 
     void handleDiskError( String errMsgr ) {
-        LOG.warning( "Shuting down DataNode because "+errMsgr );
+        LOG.warn( "Shuting down DataNode because "+errMsgr );
         try {
             namenode.errorReport(
                     dnRegistration, DatanodeProtocol.DISK_ERROR, errMsgr);
@@ -356,7 +357,7 @@ public class DataNode implements FSConstants, Runnable {
                 ss.close();
             } catch (DiskErrorException de ) {
                 String errMsgr = de.getMessage();
-                LOG.warning("Exiting DataXceiveServer due to "+ errMsgr );
+                LOG.warn("Exiting DataXceiveServer due to "+ errMsgr );
                 handleDiskError(errMsgr);
             } catch (IOException ie) {
                 LOG.info("Exiting DataXceiveServer due to " + ie.toString());
@@ -403,7 +404,7 @@ public class DataNode implements FSConstants, Runnable {
                     in.close();
                 }
             } catch (IOException ie) {
-              LOG.log(Level.WARNING, "DataXCeiver", ie);
+              LOG.warn("DataXCeiver", ie);
             } finally {
                 try {
                     s.close();
@@ -797,7 +798,7 @@ public class DataNode implements FSConstants, Runnable {
                 }
                 LOG.info("Transmitted block " + b + " to " + curTarget);
             } catch (IOException ie) {
-              LOG.log(Level.WARNING, "Failed to transfer "+b+" to "+curTarget, ie);
+              LOG.warn("Failed to transfer "+b+" to "+curTarget, ie);
             } finally {
     xmitsInProgress--;
       }
@@ -887,7 +888,7 @@ public class DataNode implements FSConstants, Runnable {
         dn = new DataNode(conf, dataDir);
         return dn;
     } catch( DiskErrorException e ) {
-        LOG.warning("Can't start DataNode because " + e.getMessage() );
+        LOG.warn("Can't start DataNode because " + e.getMessage() );
         return null;
     }
   }
@@ -901,12 +902,10 @@ public class DataNode implements FSConstants, Runnable {
         "}";
   }
 
-  /**
-   */
-  public static void main(String args[]) throws IOException {
-    Configuration conf = new Configuration();
-    LogFormatter.setShowThreadIDs(true);
-    LogFormatter.initFileHandler(conf, "datanode");
-    runAndWait(conf);
-  }
+    /**
+     */
+    public static void main(String args[]) throws IOException {
+        Configuration conf = new Configuration();
+        runAndWait(conf);
+    }
 }

+ 11 - 11
src/java/org/apache/hadoop/dfs/FSDirectory.java

@@ -671,7 +671,7 @@ class FSDirectory implements FSConstants {
         logEdit(OP_ADD,
                 new ArrayWritable( UTF8.class, nameReplicationPair ), 
                 new ArrayWritable( Block.class, newNode.blocks ));
-        NameNode.stateChangeLog.fine("DIR* FSDirectory.addFile: "
+        NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
                 +path+" with "+blocks.length+" blocks is added to the file system" );
         return true;
     }
@@ -700,7 +700,7 @@ class FSDirectory implements FSConstants {
      * Change the filename
      */
     public boolean renameTo(UTF8 src, UTF8 dst) {
-        NameNode.stateChangeLog.fine("DIR* FSDirectory.renameTo: "
+        NameNode.stateChangeLog.debug("DIR* FSDirectory.renameTo: "
                 +src+" to "+dst );
         waitForReady();
         if (unprotectedRenameTo(src, dst)) {
@@ -719,7 +719,7 @@ class FSDirectory implements FSConstants {
           String dstStr = dst.toString();
             INode renamedNode = rootDir.getNode(srcStr);
             if (renamedNode == null) {
-                NameNode.stateChangeLog.warning("DIR* FSDirectory.unprotectedRenameTo: "
+                NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
                         +"failed to rename "+src+" to "+dst+ " because "+ src+" does not exist" );
                 return false;
             }
@@ -729,12 +729,12 @@ class FSDirectory implements FSConstants {
             }
             // the renamed node can be reused now
             if( rootDir.addNode(dstStr, renamedNode ) == null ) {
-                NameNode.stateChangeLog.warning("DIR* FSDirectory.unprotectedRenameTo: "
+                NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedRenameTo: "
                         +"failed to rename "+src+" to "+dst );
               rootDir.addNode(srcStr, renamedNode); // put it back
               return false;
             }
-            NameNode.stateChangeLog.fine("DIR* FSDirectory.unprotectedRenameTo: "
+            NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedRenameTo: "
                      +src+" is renamed to "+dst );
             return true;
         }
@@ -808,7 +808,7 @@ class FSDirectory implements FSConstants {
      * Remove the file from management, return blocks
      */
     public Block[] delete(UTF8 src) {
-        NameNode.stateChangeLog.fine("DIR* FSDirectory.delete: "
+        NameNode.stateChangeLog.debug("DIR* FSDirectory.delete: "
                 +src );
         waitForReady();
         Block[] blocks = unprotectedDelete(src); 
@@ -823,7 +823,7 @@ class FSDirectory implements FSConstants {
         synchronized (rootDir) {
             INode targetNode = rootDir.getNode(src.toString());
             if (targetNode == null) {
-                NameNode.stateChangeLog.warning("DIR* FSDirectory.unprotectedDelete: "
+                NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedDelete: "
                         +"failed to remove "+src+" because it does not exist" );
                 return null;
             } else {
@@ -832,11 +832,11 @@ class FSDirectory implements FSConstants {
                 // the blocks underneath the node.
                 //
                 if (! targetNode.removeNode()) {
-                    NameNode.stateChangeLog.warning("DIR* FSDirectory.unprotectedDelete: "
+                    NameNode.stateChangeLog.warn("DIR* FSDirectory.unprotectedDelete: "
                             +"failed to remove "+src+" because it does not have a parent" );
                     return null;
                 } else {
-                    NameNode.stateChangeLog.fine("DIR* FSDirectory.unprotectedDelete: "
+                    NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
                             +src+" is removed" );
                     Vector v = new Vector();
                     targetNode.collectSubtreeBlocks(v);
@@ -983,7 +983,7 @@ class FSDirectory implements FSConstants {
             String cur = (String) v.elementAt(i);
             INode inserted = unprotectedMkdir(cur);
             if (inserted != null) {
-                NameNode.stateChangeLog.fine("DIR* FSDirectory.mkdirs: "
+                NameNode.stateChangeLog.debug("DIR* FSDirectory.mkdirs: "
                         +"created directory "+cur );
                 logEdit(OP_MKDIR, new UTF8(inserted.computeName()), null);
                 lastSuccess = true;
@@ -992,7 +992,7 @@ class FSDirectory implements FSConstants {
             }
         }
 /*        if( !lastSuccess )
-            NameNode.stateChangeLog.warning("DIR* FSDirectory.mkdirs: "
+            NameNode.stateChangeLog.warn("DIR* FSDirectory.mkdirs: "
                     +"failed to create directory "+src );*/
         return lastSuccess;
     }

+ 51 - 50
src/java/org/apache/hadoop/dfs/FSNamesystem.java

@@ -15,13 +15,14 @@
  */
 package org.apache.hadoop.dfs;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.*;
 
 import java.io.*;
 import java.util.*;
-import java.util.logging.*;
 
 /***************************************************
  * FSNamesystem does the actual bookkeeping work for the
@@ -36,7 +37,7 @@ import java.util.logging.*;
  * 5)  LRU cache of updated-heartbeat machines
  ***************************************************/
 class FSNamesystem implements FSConstants {
-    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.fs.FSNamesystem");
+    public static final Log LOG = LogFactory.getLog("org.apache.hadoop.fs.FSNamesystem");
 
     //
     // Stores the correct file name hierarchy
@@ -321,7 +322,7 @@ class FSNamesystem implements FSConstants {
                                             short replication,
                                             long blockSize
                                           ) throws IOException {
-      NameNode.stateChangeLog.fine("DIR* NameSystem.startFile: file "
+      NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: file "
             +src+" for "+holder+" at "+clientMachine);
       try {
         if (pendingCreates.get(src) != null) {
@@ -362,7 +363,7 @@ class FSNamesystem implements FSConstants {
                                                      blockSize,
                                                      holder,
                                                      clientMachine));
-        NameNode.stateChangeLog.finer( "DIR* NameSystem.startFile: "
+        NameNode.stateChangeLog.debug( "DIR* NameSystem.startFile: "
                    +"add "+src+" to pendingCreates for "+holder );
         synchronized (leases) {
             Lease lease = (Lease) leases.get(holder);
@@ -384,7 +385,7 @@ class FSNamesystem implements FSConstants {
         results[1] = targets;
         return results;
       } catch (IOException ie) {
-          NameNode.stateChangeLog.warning("DIR* NameSystem.startFile: "
+          NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: "
                   +ie.getMessage());
         throw ie;
       }
@@ -404,7 +405,7 @@ class FSNamesystem implements FSConstants {
     public synchronized Object[] getAdditionalBlock(UTF8 src, 
                                                     UTF8 clientName
                                                     ) throws IOException {
-        NameNode.stateChangeLog.fine("BLOCK* NameSystem.getAdditionalBlock: file "
+        NameNode.stateChangeLog.debug("BLOCK* NameSystem.getAdditionalBlock: file "
             +src+" for "+clientName);
         FileUnderConstruction pendingFile = 
           (FileUnderConstruction) pendingCreates.get(src);
@@ -448,7 +449,7 @@ class FSNamesystem implements FSConstants {
         //
         // Remove the block from the pending creates list
         //
-        NameNode.stateChangeLog.fine("BLOCK* NameSystem.abandonBlock: "
+        NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
                 +b.getBlockName()+"of file "+src );
         FileUnderConstruction pendingFile = 
           (FileUnderConstruction) pendingCreates.get(src);
@@ -459,7 +460,7 @@ class FSNamesystem implements FSConstants {
                 if (cur.compareTo(b) == 0) {
                     pendingCreateBlocks.remove(cur);
                     it.remove();
-                    NameNode.stateChangeLog.finer(
+                    NameNode.stateChangeLog.debug(
                              "BLOCK* NameSystem.abandonBlock: "
                             +b.getBlockName()
                             +" is removed from pendingCreateBlock and pendingCreates");
@@ -476,7 +477,7 @@ class FSNamesystem implements FSConstants {
     public synchronized void abandonFileInProgress(UTF8 src, 
                                                    UTF8 holder
                                                    ) throws IOException {
-      NameNode.stateChangeLog.fine("DIR* NameSystem.abandonFileInProgress:" + src );
+      NameNode.stateChangeLog.debug("DIR* NameSystem.abandonFileInProgress:" + src );
       synchronized (leases) {
         // find the lease
         Lease lease = (Lease) leases.get(holder);
@@ -504,9 +505,9 @@ class FSNamesystem implements FSConstants {
      * been reported by datanodes and are replicated correctly.
      */
     public synchronized int completeFile(UTF8 src, UTF8 holder) {
-        NameNode.stateChangeLog.fine("DIR* NameSystem.completeFile: " + src + " for " + holder );
+        NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src + " for " + holder );
         if (dir.getFile(src) != null || pendingCreates.get(src) == null) {
-            NameNode.stateChangeLog.warning( "DIR* NameSystem.completeFile: "
+            NameNode.stateChangeLog.warn( "DIR* NameSystem.completeFile: "
                     + "failed to complete " + src
                     + " because dir.getFile()==" + dir.getFile(src) 
                     + " and " + pendingCreates.get(src));
@@ -552,7 +553,7 @@ class FSNamesystem implements FSConstants {
 
         // The file is no longer pending
         pendingCreates.remove(src);
-        NameNode.stateChangeLog.finer(
+        NameNode.stateChangeLog.debug(
              "DIR* NameSystem.completeFile: " + src
            + " is removed from pendingCreates");
         for (int i = 0; i < nrBlocks; i++) {
@@ -582,7 +583,7 @@ class FSNamesystem implements FSConstants {
         for (int i = 0; i < nrBlocks; i++) {
             TreeSet containingNodes = (TreeSet) blocksMap.get(pendingBlocks[i]);
             if (containingNodes.size() < pendingFile.getReplication()) {
-                   NameNode.stateChangeLog.finer(
+                   NameNode.stateChangeLog.debug(
                           "DIR* NameSystem.completeFile:"
                         + pendingBlocks[i].getBlockName()+" has only "+containingNodes.size()
                         +" replicas so is added to neededReplications");           
@@ -608,7 +609,7 @@ class FSNamesystem implements FSConstants {
           (FileUnderConstruction) pendingCreates.get(src);
         v.getBlocks().add(b);
         pendingCreateBlocks.add(b);
-        NameNode.stateChangeLog.finer("BLOCK* NameSystem.allocateBlock: "
+        NameNode.stateChangeLog.debug("BLOCK* NameSystem.allocateBlock: "
             +src+ ". "+b.getBlockName()+
             " is created and added to pendingCreates and pendingCreateBlocks" );      
         return b;
@@ -647,7 +648,7 @@ class FSNamesystem implements FSConstants {
      * Change the indicated filename.
      */
     public boolean renameTo(UTF8 src, UTF8 dst) {
-        NameNode.stateChangeLog.fine("DIR* NameSystem.renameTo: " + src + " to " + dst );
+        NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src + " to " + dst );
         return dir.renameTo(src, dst);
     }
 
@@ -656,7 +657,7 @@ class FSNamesystem implements FSConstants {
      * invalidate some blocks that make up the file.
      */
     public synchronized boolean delete(UTF8 src) {
-        NameNode.stateChangeLog.fine("DIR* NameSystem.delete: " + src );
+        NameNode.stateChangeLog.debug("DIR* NameSystem.delete: " + src );
         Block deletedBlocks[] = (Block[]) dir.delete(src);
         if (deletedBlocks != null) {
             for (int i = 0; i < deletedBlocks.length; i++) {
@@ -672,7 +673,7 @@ class FSNamesystem implements FSConstants {
                             recentInvalidateSets.put(node.getStorageID(), invalidateSet);
                         }
                         invalidateSet.add(b);
-                        NameNode.stateChangeLog.finer("BLOCK* NameSystem.delete: "
+                        NameNode.stateChangeLog.debug("BLOCK* NameSystem.delete: "
                             + b.getBlockName() + " is added to invalidSet of " + node.getName() );
                     }
                 }
@@ -704,7 +705,7 @@ class FSNamesystem implements FSConstants {
      * Create all the necessary directories
      */
     public boolean mkdirs(UTF8 src) {
-        NameNode.stateChangeLog.fine("DIR* NameSystem.mkdirs: " + src );
+        NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src );
         return dir.mkdirs(src);
     }
 
@@ -936,7 +937,7 @@ class FSNamesystem implements FSConstants {
       FileUnderConstruction v = 
         (FileUnderConstruction) pendingCreates.remove(src);
       if (v != null) {
-         NameNode.stateChangeLog.finer(
+         NameNode.stateChangeLog.debug(
                       "DIR* NameSystem.internalReleaseCreate: " + src
                     + " is removed from pendingCreates for "
                     + holder + " (failure)");
@@ -945,7 +946,7 @@ class FSNamesystem implements FSConstants {
           pendingCreateBlocks.remove(b);
         }
       } else {
-          NameNode.stateChangeLog.warning("DIR* NameSystem.internalReleaseCreate: "
+          NameNode.stateChangeLog.warn("DIR* NameSystem.internalReleaseCreate: "
                  + "attempt to release a create lock on "+ src.toString()
                  + " that was not in pedingCreates");
       }
@@ -1004,7 +1005,7 @@ class FSNamesystem implements FSConstants {
      */
     public synchronized void registerDatanode( DatanodeRegistration nodeReg 
                                               ) throws IOException {
-      NameNode.stateChangeLog.fine(
+      NameNode.stateChangeLog.debug(
           "BLOCK* NameSystem.registerDatanode: "
           + "node registration from " + nodeReg.getName()
           + " storage " + nodeReg.getStorageID() );
@@ -1017,7 +1018,7 @@ class FSNamesystem implements FSConstants {
         // The same datanode has been just restarted to serve the same data 
         // storage. We do not need to remove old data blocks, the delta will  
         // be calculated on the next block report from the datanode
-        NameNode.stateChangeLog.fine(
+        NameNode.stateChangeLog.debug(
             "BLOCK* NameSystem.registerDatanode: "
             + "node restarted." );
         return;
@@ -1037,14 +1038,14 @@ class FSNamesystem implements FSConstants {
           // this data storage has never registered
           // it is either empty or was created by previous version of DFS
           nodeReg.storageID = newStorageID();
-          NameNode.stateChangeLog.fine(
+          NameNode.stateChangeLog.debug(
               "BLOCK* NameSystem.registerDatanode: "
               + "new storageID " + nodeReg.getStorageID() + " assigned." );
         }
         // register new datanode
         datanodeMap.put(nodeReg.getStorageID(), 
                         new DatanodeInfo( nodeReg ) );
-        NameNode.stateChangeLog.fine(
+        NameNode.stateChangeLog.debug(
             "BLOCK* NameSystem.registerDatanode: "
             + "node registered." );
         return;
@@ -1053,7 +1054,7 @@ class FSNamesystem implements FSConstants {
       // nodeS is found
       // The registering datanode is a replacement node for the existing 
       // data storage, which from now on will be served by a new node.
-      NameNode.stateChangeLog.fine(
+      NameNode.stateChangeLog.debug(
           "BLOCK* NameSystem.registerDatanode: "
           + "node " + nodeS.name
           + " is replaced by " + nodeReg.getName() + "." );
@@ -1105,7 +1106,7 @@ class FSNamesystem implements FSConstants {
           DatanodeInfo nodeinfo = getDatanode( nodeID );
 
           if (nodeinfo == null) {
-            NameNode.stateChangeLog.fine("BLOCK* NameSystem.gotHeartbeat: "
+            NameNode.stateChangeLog.debug("BLOCK* NameSystem.gotHeartbeat: "
                     +"brand-new heartbeat from "+nodeID.getName() );
             nodeinfo = new DatanodeInfo(nodeID, capacity, remaining);
             datanodeMap.put(nodeinfo.getStorageID(), nodeinfo);
@@ -1152,7 +1153,7 @@ class FSNamesystem implements FSConstants {
       if (nodeInfo != null) {
         removeDatanode( nodeInfo );
       } else {
-          NameNode.stateChangeLog.warning("BLOCK* NameSystem.removeDatanode: "
+          NameNode.stateChangeLog.warn("BLOCK* NameSystem.removeDatanode: "
                   + nodeInfo.getName() + " does not exist");
       }
   }
@@ -1165,7 +1166,7 @@ class FSNamesystem implements FSConstants {
     private void removeDatanode( DatanodeInfo nodeInfo ) {
       heartbeats.remove(nodeInfo);
       datanodeMap.remove(nodeInfo.getStorageID());
-      NameNode.stateChangeLog.finer("BLOCK* NameSystem.removeDatanode: "
+      NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeDatanode: "
               + nodeInfo.getName() + " is removed from datanodeMap");
       totalCapacity -= nodeInfo.getCapacity();
       totalRemaining -= nodeInfo.getRemaining();
@@ -1204,7 +1205,7 @@ class FSNamesystem implements FSConstants {
     public synchronized Block[] processReport(DatanodeID nodeID, 
                                               Block newReport[]
                                             ) throws IOException {
-        NameNode.stateChangeLog.fine("BLOCK* NameSystem.processReport: "
+        NameNode.stateChangeLog.debug("BLOCK* NameSystem.processReport: "
           +"from "+nodeID.getName()+" "+newReport.length+" blocks" );
         DatanodeInfo node = getDatanode( nodeID );
 
@@ -1286,15 +1287,15 @@ class FSNamesystem implements FSConstants {
             containingNodes.add(node);
             // 
             // Hairong: I would prefer to set the level of next logrecord
-            // to be finer.
+            // to be debug.
             // But at startup time, because too many new blocks come in
             // they simply take up all the space in the log file 
-            // So I set the level to be finest
+            // So I set the level to be trace
             //
-            NameNode.stateChangeLog.finest("BLOCK* NameSystem.addStoredBlock: "
+            NameNode.stateChangeLog.trace("BLOCK* NameSystem.addStoredBlock: "
                     +"blockMap updated: "+node.getName()+" is added to "+block.getBlockName() );
         } else {
-            NameNode.stateChangeLog.warning("BLOCK* NameSystem.addStoredBlock: "
+            NameNode.stateChangeLog.warn("BLOCK* NameSystem.addStoredBlock: "
                     + "Redundant addStoredBlock request received for " 
                     + block.getBlockName() + " on " + node.getName());
         }
@@ -1307,12 +1308,12 @@ class FSNamesystem implements FSConstants {
             if (containingNodes.size() >= fileReplication ) {
                 neededReplications.remove(block);
                 pendingReplications.remove(block);
-                NameNode.stateChangeLog.finest("BLOCK* NameSystem.addStoredBlock: "
+                NameNode.stateChangeLog.trace("BLOCK* NameSystem.addStoredBlock: "
                         +block.getBlockName()+" has "+containingNodes.size()
                         +" replicas so is removed from neededReplications and pendingReplications" );
             } else {// containingNodes.size() < fileReplication
                 neededReplications.add(block);
-                NameNode.stateChangeLog.finer("BLOCK* NameSystem.addStoredBlock: "
+                NameNode.stateChangeLog.debug("BLOCK* NameSystem.addStoredBlock: "
                     +block.getBlockName()+" has only "+containingNodes.size()
                     +" replicas so is added to neededReplications" );
             }
@@ -1360,7 +1361,7 @@ class FSNamesystem implements FSConstants {
                 excessReplicateMap.put(cur.getStorageID(), excessBlocks);
             }
             excessBlocks.add(b);
-            NameNode.stateChangeLog.finer("BLOCK* NameSystem.chooseExcessReplicates: "
+            NameNode.stateChangeLog.debug("BLOCK* NameSystem.chooseExcessReplicates: "
                     +"("+cur.getName()+", "+b.getBlockName()+") is added to excessReplicateMap" );
 
             //
@@ -1378,7 +1379,7 @@ class FSNamesystem implements FSConstants {
                 recentInvalidateSets.put(cur.getStorageID(), invalidateSet);
             }
             invalidateSet.add(b);
-            NameNode.stateChangeLog.finer("BLOCK* NameSystem.chooseExcessReplicates: "
+            NameNode.stateChangeLog.debug("BLOCK* NameSystem.chooseExcessReplicates: "
                     +"("+cur.getName()+", "+b.getBlockName()+") is added to recentInvalidateSets" );
         }
     }
@@ -1388,7 +1389,7 @@ class FSNamesystem implements FSConstants {
      * replication tasks, if the removed block is still valid.
      */
     synchronized void removeStoredBlock(Block block, DatanodeInfo node) {
-        NameNode.stateChangeLog.fine("BLOCK* NameSystem.removeStoredBlock: "
+        NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
                 +block.getBlockName() + " from "+node.getName() );
         TreeSet containingNodes = (TreeSet) blocksMap.get(block);
         if (containingNodes == null || ! containingNodes.contains(node)) {
@@ -1406,7 +1407,7 @@ class FSNamesystem implements FSConstants {
             synchronized (neededReplications) {
                 neededReplications.add(block);
             }
-            NameNode.stateChangeLog.finer("BLOCK* NameSystem.removeStoredBlock: "
+            NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
                     +block.getBlockName()+" has only "+containingNodes.size()
                     +" replicas so is added to neededReplications" );
         }
@@ -1418,7 +1419,7 @@ class FSNamesystem implements FSConstants {
         TreeSet excessBlocks = (TreeSet) excessReplicateMap.get(node.getStorageID());
         if (excessBlocks != null) {
             excessBlocks.remove(block);
-            NameNode.stateChangeLog.finer("BLOCK* NameSystem.removeStoredBlock: "
+            NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
                     +block.getBlockName()+" is removed from excessBlocks" );
             if (excessBlocks.size() == 0) {
                 excessReplicateMap.remove(node.getStorageID());
@@ -1434,14 +1435,14 @@ class FSNamesystem implements FSConstants {
                                           ) throws IOException {
         DatanodeInfo node = getDatanode( nodeID );
         if (node == null) {
-            NameNode.stateChangeLog.warning("BLOCK* NameSystem.blockReceived: "
+            NameNode.stateChangeLog.warn("BLOCK* NameSystem.blockReceived: "
              + block.getBlockName() + " is received from an unrecorded node " 
              + nodeID.getName() );
             throw new IllegalArgumentException(
                 "Unexpected exception.  Got blockReceived message from node " 
                 + block.getBlockName() + ", but there is no info for it");
         }
-        NameNode.stateChangeLog.fine("BLOCK* NameSystem.blockReceived: "
+        NameNode.stateChangeLog.debug("BLOCK* NameSystem.blockReceived: "
                 +block.getBlockName()+" is received from " + nodeID.getName() );
         //
         // Modify the blocks->datanode map
@@ -1502,7 +1503,7 @@ class FSNamesystem implements FSConstants {
         if (invalidateSet == null ) 
             return null;
         
-        if(NameNode.stateChangeLog.isLoggable(Level.INFO)) {
+        if(NameNode.stateChangeLog.isInfoEnabled()) {
             StringBuffer blockList = new StringBuffer();
             for( int i=0; i<invalidateSet.size(); i++ ) {
                 blockList.append(' ');
@@ -1591,13 +1592,13 @@ class FSNamesystem implements FSConstants {
                     dir.getFileByBlock( block).getReplication() ) {
               neededReplications.remove(block);
               pendingReplications.add(block);
-              NameNode.stateChangeLog.finer(
+              NameNode.stateChangeLog.debug(
                 "BLOCK* NameSystem.pendingTransfer: "
                 + block.getBlockName()
                 + " is removed from neededReplications to pendingReplications");
             }
 
-            if (NameNode.stateChangeLog.isLoggable(Level.INFO)) {
+            if (NameNode.stateChangeLog.isInfoEnabled()) {
               StringBuffer targetList = new StringBuffer("datanode(s)");
               for (int k = 0; k < targets.length; k++) {
                 targetList.append(' ');
@@ -1640,7 +1641,7 @@ class FSNamesystem implements FSConstants {
     DatanodeInfo[] chooseTargets(int desiredReplicates, TreeSet forbiddenNodes,
                                  UTF8 clientMachine, long blockSize) {
         if (desiredReplicates > datanodeMap.size()) {
-          LOG.warning("Replication requested of "+desiredReplicates
+          LOG.warn("Replication requested of "+desiredReplicates
                       +" is larger than cluster size ("+datanodeMap.size()
                       +"). Using cluster size.");
           desiredReplicates  = datanodeMap.size();
@@ -1679,7 +1680,7 @@ class FSNamesystem implements FSConstants {
         //
         int totalMachines = datanodeMap.size();
         if (totalMachines == 0) {
-            LOG.warning("While choosing target, totalMachines is " + totalMachines);
+            LOG.warn("While choosing target, totalMachines is " + totalMachines);
             return null;
         }
 
@@ -1752,10 +1753,10 @@ class FSNamesystem implements FSConstants {
                     return node;
                 }
             }
-            LOG.warning("Could not find any nodes with sufficient capacity");
+            LOG.warn("Could not find any nodes with sufficient capacity");
             return null;
         } else {
-            LOG.warning("Zero targets found, forbidden1.size=" +
+            LOG.warn("Zero targets found, forbidden1.size=" +
                 ( forbidden1 != null ? forbidden1.size() : 0 ) +
                 " forbidden2.size()=" +
                 ( forbidden2 != null ? forbidden2.size() : 0 ));
@@ -1825,7 +1826,7 @@ class FSNamesystem implements FSConstants {
         return null;
       if (!node.getName().equals(nodeID.getName())) {
         e = new UnregisteredDatanodeException( nodeID, node );
-        NameNode.stateChangeLog.severe("BLOCK* NameSystem.getDatanode: "
+        NameNode.stateChangeLog.fatal("BLOCK* NameSystem.getDatanode: "
             + e.getLocalizedMessage() );
         throw e;
       }

+ 17 - 29
src/java/org/apache/hadoop/dfs/NameNode.java

@@ -15,13 +15,14 @@
  */
 package org.apache.hadoop.dfs;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.*;
 
 import java.io.*;
-import java.util.logging.*;
 
 /**********************************************************
  * NameNode serves as both directory namespace manager and
@@ -56,8 +57,8 @@ import java.util.logging.*;
  * @author Mike Cafarella
  **********************************************************/
 public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
-    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.dfs.NameNode");
-    public static final Logger stateChangeLog = LogFormatter.getLogger( "org.apache.hadoop.dfs.StateChange");
+    public static final Log LOG = LogFactory.getLog("org.apache.hadoop.dfs.NameNode");
+    public static final Log stateChangeLog = LogFactory.getLog( "org.apache.hadoop.dfs.StateChange");
 
     private FSNamesystem namesystem;
     private Server server;
@@ -158,7 +159,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
                                short replication,
                                long blockSize
     ) throws IOException {
-       stateChangeLog.fine("*DIR* NameNode.create: file "
+       stateChangeLog.debug("*DIR* NameNode.create: file "
             +src+" for "+clientName+" at "+clientMachine);
        Object results[] = namesystem.startFile(new UTF8(src), 
                                                 new UTF8(clientName), 
@@ -181,7 +182,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
      */
     public LocatedBlock addBlock(String src, 
                                  String clientName) throws IOException {
-        stateChangeLog.fine("*BLOCK* NameNode.addBlock: file "
+        stateChangeLog.debug("*BLOCK* NameNode.addBlock: file "
             +src+" for "+clientName);
         UTF8 src8 = new UTF8(src);
         UTF8 client8 = new UTF8(clientName);
@@ -199,7 +200,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     public void reportWrittenBlock(LocatedBlock lb) throws IOException {
         Block b = lb.getBlock();        
         DatanodeInfo targets[] = lb.getLocations();
-        stateChangeLog.fine("*BLOCK* NameNode.reportWrittenBlock"
+        stateChangeLog.debug("*BLOCK* NameNode.reportWrittenBlock"
                 +": " + b.getBlockName() +" is written to "
                 +targets.length + " locations" );
 
@@ -212,7 +213,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
      * The client needs to give up on the block.
      */
     public void abandonBlock(Block b, String src) throws IOException {
-        stateChangeLog.fine("*BLOCK* NameNode.abandonBlock: "
+        stateChangeLog.debug("*BLOCK* NameNode.abandonBlock: "
                 +b.getBlockName()+" of file "+src );
         if (! namesystem.abandonBlock(b, new UTF8(src))) {
             throw new IOException("Cannot abandon block during write to " + src);
@@ -222,13 +223,13 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
      */
     public void abandonFileInProgress(String src, 
                                       String holder) throws IOException {
-        stateChangeLog.fine("*DIR* NameNode.abandonFileInProgress:" + src );
+        stateChangeLog.debug("*DIR* NameNode.abandonFileInProgress:" + src );
         namesystem.abandonFileInProgress(new UTF8(src), new UTF8(holder));
     }
     /**
      */
     public boolean complete(String src, String clientName) throws IOException {
-        stateChangeLog.fine("*DIR* NameNode.complete: " + src + " for " + clientName );
+        stateChangeLog.debug("*DIR* NameNode.complete: " + src + " for " + clientName );
         int returnCode = namesystem.completeFile(new UTF8(src), new UTF8(clientName));
         if (returnCode == STILL_WAITING) {
             return false;
@@ -263,14 +264,14 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     /**
      */
     public boolean rename(String src, String dst) throws IOException {
-        stateChangeLog.fine("*DIR* NameNode.rename: " + src + " to " + dst );
+        stateChangeLog.debug("*DIR* NameNode.rename: " + src + " to " + dst );
         return namesystem.renameTo(new UTF8(src), new UTF8(dst));
     }
 
     /**
      */
     public boolean delete(String src) throws IOException {
-        stateChangeLog.fine("*DIR* NameNode.delete: " + src );
+        stateChangeLog.debug("*DIR* NameNode.delete: " + src );
         return namesystem.delete(new UTF8(src));
     }
 
@@ -289,7 +290,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     /**
      */
     public boolean mkdirs(String src) throws IOException {
-        stateChangeLog.fine("*DIR* NameNode.mkdirs: " + src );
+        stateChangeLog.debug("*DIR* NameNode.mkdirs: " + src );
         return namesystem.mkdirs(new UTF8(src));
     }
 
@@ -416,7 +417,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     public Block[] blockReport( DatanodeRegistration nodeReg,
                                 Block blocks[]) throws IOException {
         verifyRequest( nodeReg );
-        stateChangeLog.fine("*BLOCK* NameNode.blockReport: "
+        stateChangeLog.debug("*BLOCK* NameNode.blockReport: "
                 +"from "+nodeReg.getName()+" "+blocks.length+" blocks" );
         if( firstBlockReportTime==0)
               firstBlockReportTime=System.currentTimeMillis();
@@ -427,7 +428,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
     public void blockReceived(DatanodeRegistration nodeReg, 
                               Block blocks[]) throws IOException {
         verifyRequest( nodeReg );
-        stateChangeLog.fine("*BLOCK* NameNode.blockReceived: "
+        stateChangeLog.debug("*BLOCK* NameNode.blockReceived: "
                 +"from "+nodeReg.getName()+" "+blocks.length+" blocks." );
         for (int i = 0; i < blocks.length; i++) {
             namesystem.blockReceived( nodeReg, blocks[i] );
@@ -441,7 +442,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
                             String msg) throws IOException {
       // Log error message from datanode
       verifyRequest( nodeReg );
-      LOG.warning("Report from " + nodeReg.getName() + ": " + msg);
+      LOG.warn("Report from " + nodeReg.getName() + ": " + msg);
       if( errorCode == DatanodeProtocol.DISK_ERROR ) {
           namesystem.removeDatanode( nodeReg );            
       }
@@ -490,20 +491,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol, FSConstants {
           System.err.println("Formatted "+dir);
           System.exit(0);
         }
-            
-        LogFormatter.initFileHandler( conf, "namenode" );
-        LogFormatter.setShowThreadIDs(true);
-        String confLevel = conf.get("dfs.namenode.logging.level", "info");
-        Level level;
-        if( confLevel.equals( "dir"))
-                level=Level.FINE;
-        else if( confLevel.equals( "block"))
-                level=Level.FINER;
-        else if( confLevel.equals( "all"))
-                level=Level.FINEST;
-        else level=Level.INFO;
-        stateChangeLog.setLevel( level);
-
+        
         NameNode namenode = new NameNode(conf);
         namenode.join();
     }

+ 8 - 7
src/java/org/apache/hadoop/fs/FSDataInputStream.java

@@ -17,16 +17,17 @@ package org.apache.hadoop.fs;
 
 import java.io.*;
 import java.util.Arrays;
-import java.util.logging.*;
 import java.util.zip.*;
+
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
 
 /** Utility that wraps a {@link FSInputStream} in a {@link DataInputStream}
  * and buffers input through a {@link BufferedInputStream}. */
 public class FSDataInputStream extends DataInputStream {
-  private static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.fs.DataInputStream");
+  private static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.fs.DataInputStream");
 
   private static final byte[] VERSION = FSDataOutputStream.CHECKSUM_VERSION;
   private static final int HEADER_LENGTH = 8;
@@ -58,7 +59,7 @@ public class FSDataInputStream extends DataInputStream {
       } catch (FileNotFoundException e) {         // quietly ignore
         stopSumming();
       } catch (IOException e) {                   // loudly ignore
-        LOG.warning("Problem opening checksum file: "+ file + ".  Ignoring with exception " + e + ".");
+        LOG.warn("Problem opening checksum file: "+ file + ".  Ignoring with exception " + e + ".");
         stopSumming();
       }
     }
@@ -71,7 +72,7 @@ public class FSDataInputStream extends DataInputStream {
         try {
           sums.seek(HEADER_LENGTH + 4*(desired/bytesPerSum));
         } catch (IOException e) {
-          LOG.warning("Problem seeking checksum file: "+e+". Ignoring.");
+          LOG.warn("Problem seeking checksum file: "+e+". Ignoring.");
           stopSumming();
         }
         sum.reset();
@@ -108,7 +109,7 @@ public class FSDataInputStream extends DataInputStream {
       try {
         crc = sums.readInt();
       } catch (IOException e) {
-        LOG.warning("Problem reading checksum file: "+e+". Ignoring.");
+        LOG.warn("Problem reading checksum file: "+e+". Ignoring.");
         stopSumming();
         return;
       }

+ 3 - 3
src/java/org/apache/hadoop/fs/FileSystem.java

@@ -18,11 +18,11 @@ package org.apache.hadoop.fs;
 import java.io.*;
 import java.net.*;
 import java.util.*;
-import java.util.logging.*;
+
+import org.apache.commons.logging.*;
 
 import org.apache.hadoop.dfs.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
 
 /****************************************************************
  * An abstract base class for a fairly generic filesystem.  It
@@ -44,7 +44,7 @@ import org.apache.hadoop.util.LogFormatter;
  * @author Mike Cafarella
  *****************************************************************/
 public abstract class FileSystem extends Configured {
-    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.dfs.DistributedFileSystem");
+    public static final Log LOG = LogFactory.getLog("org.apache.hadoop.dfs.DistributedFileSystem");
 
     private static final HashMap NAME_TO_FS = new HashMap();
     /**

+ 2 - 2
src/java/org/apache/hadoop/fs/LocalFileSystem.java

@@ -344,7 +344,7 @@ public class LocalFileSystem extends FileSystem {
         badDir.mkdirs();
         String suffix = "." + new Random().nextInt();
         File badFile = new File(badDir,f.getName()+suffix);
-        LOG.warning("Moving bad file " + f + " to " + badFile);
+        LOG.warn("Moving bad file " + f + " to " + badFile);
         in.close();                               // close it first
         f.renameTo(badFile);                      // rename it
 
@@ -353,7 +353,7 @@ public class LocalFileSystem extends FileSystem {
         checkFile.renameTo(new File(badDir, checkFile.getName()+suffix));
 
       } catch (IOException e) {
-        LOG.warning("Error moving bad file " + p + ": " + e);
+        LOG.warn("Error moving bad file " + p + ": " + e);
       }
     }
 

+ 1 - 1
src/java/org/apache/hadoop/io/MapFile.java

@@ -258,7 +258,7 @@ public class MapFile {
           count++;
         }
       } catch (EOFException e) {
-        SequenceFile.LOG.warning("Unexpected EOF reading " + index +
+        SequenceFile.LOG.warn("Unexpected EOF reading " + index +
                                  " at entry #" + count + ".  Ignoring.");
       } finally {
 	indexClosed = true;

+ 9 - 10
src/java/org/apache/hadoop/io/SequenceFile.java

@@ -19,19 +19,18 @@ package org.apache.hadoop.io;
 import java.io.*;
 import java.util.*;
 import java.util.zip.*;
-import java.util.logging.*;
 import java.net.InetAddress;
 import java.rmi.server.UID;
 import java.security.MessageDigest;
 import org.apache.lucene.util.PriorityQueue;
+import org.apache.commons.logging.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
 
 /** Support for flat files of binary key/value pairs. */
 public class SequenceFile {
-  public static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.io.SequenceFile");
+  public static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.io.SequenceFile");
 
   private SequenceFile() {}                         // no public ctor
 
@@ -401,7 +400,7 @@ public class SequenceFile {
     private void handleChecksumException(ChecksumException e)
       throws IOException {
       if (this.conf.getBoolean("io.skip.checksum.errors", false)) {
-        LOG.warning("Bad checksum at "+getPosition()+". Skipping entries.");
+        LOG.warn("Bad checksum at "+getPosition()+". Skipping entries.");
         sync(getPosition()+this.conf.getInt("io.bytes.per.checksum", 512));
       } else {
         throw e;
@@ -527,7 +526,7 @@ public class SequenceFile {
     }
 
     private int sortPass() throws IOException {
-      LOG.fine("running sort pass");
+      LOG.debug("running sort pass");
       SortPass sortPass = new SortPass(this.conf);         // make the SortPass
       try {
         return sortPass.run();                    // run it
@@ -584,7 +583,7 @@ public class SequenceFile {
           }
 
           // buffer is full -- sort & flush it
-          LOG.finer("flushing segment " + segments);
+          LOG.info("flushing segment " + segments);
           rawBuffer = buffer.getData();
           sort(count);
           flush(count, segments==0 && atEof);
@@ -692,7 +691,7 @@ public class SequenceFile {
     }
 
     private int mergePass(int pass, boolean last) throws IOException {
-      LOG.fine("running merge pass=" + pass);
+      LOG.debug("running merge pass=" + pass);
       MergePass mergePass = new MergePass(pass, last);
       try {                                       // make a merge pass
         return mergePass.run();                  // run it
@@ -732,7 +731,7 @@ public class SequenceFile {
         long end = fs.getLength(inName);
 
         while (in.getPos() < end) {
-          LOG.finer("merging segment " + segments);
+          LOG.debug("merging segment " + segments);
           long totalLength = 0;
           long totalCount = 0;
           while (in.getPos() < end && queue.size() < factor) {
@@ -799,7 +798,7 @@ public class SequenceFile {
       }
 
       public void run() throws IOException {
-        LOG.finer("merging files=" + inFiles.length);
+        LOG.debug("merging files=" + inFiles.length);
         for (int i = 0; i < inFiles.length; i++) {
           Path inFile = inFiles[i];
           MergeStream ms =

+ 5 - 5
src/java/org/apache/hadoop/io/UTF8.java

@@ -20,8 +20,8 @@ import java.io.IOException;
 import java.io.DataInput;
 import java.io.DataOutput;
 
-import java.util.logging.Logger;
-import org.apache.hadoop.util.LogFormatter;
+
+import org.apache.commons.logging.*;
 
 /** A WritableComparable for strings that uses the UTF8 encoding.
  * 
@@ -30,7 +30,7 @@ import org.apache.hadoop.util.LogFormatter;
  * @author Doug Cutting
  */
 public class UTF8 implements WritableComparable {
-  private static final Logger LOG= LogFormatter.getLogger("org.apache.hadoop.io.UTF8");
+  private static final Log LOG= LogFactory.getLog("org.apache.hadoop.io.UTF8");
   private static final DataOutputBuffer OBUF = new DataOutputBuffer();
   private static final DataInputBuffer IBUF = new DataInputBuffer();
 
@@ -66,7 +66,7 @@ public class UTF8 implements WritableComparable {
   /** Set to contain the contents of a string. */
   public void set(String string) {
     if (string.length() > 0xffff/3) {             // maybe too long
-      LOG.warning("truncating long string: " + string.length()
+      LOG.warn("truncating long string: " + string.length()
                   + " chars, starting with " + string.substring(0, 20));
       string = string.substring(0, 0xffff/3);
     }
@@ -231,7 +231,7 @@ public class UTF8 implements WritableComparable {
    */
   public static int writeString(DataOutput out, String s) throws IOException {
     if (s.length() > 0xffff/3) {         // maybe too long
-      LOG.warning("truncating long string: " + s.length()
+      LOG.warn("truncating long string: " + s.length()
                   + " chars, starting with " + s.substring(0, 20));
       s = s.substring(0, 0xffff/3);
     }

+ 9 - 10
src/java/org/apache/hadoop/ipc/Client.java

@@ -30,10 +30,9 @@ import java.io.FilterInputStream;
 import java.io.FilterOutputStream;
 
 import java.util.Hashtable;
-import java.util.logging.Logger;
-import java.util.logging.Level;
 
-import org.apache.hadoop.util.LogFormatter;
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.io.Writable;
@@ -48,8 +47,8 @@ import org.apache.hadoop.io.UTF8;
  * @see Server
  */
 public class Client {
-  public static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.ipc.Client");
+  public static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.ipc.Client");
 
   private Hashtable connections = new Hashtable();
 
@@ -150,8 +149,8 @@ public class Client {
             continue;
           }
 
-          if (LOG.isLoggable(Level.FINE))
-            LOG.fine(getName() + " got value #" + id);
+          if (LOG.isDebugEnabled())
+            LOG.debug(getName() + " got value #" + id);
 
           Call call = (Call)calls.remove(new Integer(id));
           boolean isError = in.readBoolean();     // read if error
@@ -178,7 +177,7 @@ public class Client {
       } catch (EOFException eof) {
           // This is what happens when the remote side goes down
       } catch (Exception e) {
-        LOG.log(Level.INFO, getName() + " caught: " + e, e);
+        LOG.info(getName() + " caught: " + e, e);
       } finally {
         close();
       }
@@ -193,8 +192,8 @@ public class Client {
       try {
         calls.put(new Integer(call.id), call);
         synchronized (out) {
-          if (LOG.isLoggable(Level.FINE))
-            LOG.fine(getName() + " sending #" + call.id);
+          if (LOG.isDebugEnabled())
+            LOG.debug(getName() + " sending #" + call.id);
           try {
             writingCall = call;
             out.writeInt(call.id);

+ 6 - 6
src/java/org/apache/hadoop/ipc/RPC.java

@@ -23,12 +23,12 @@ import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 
 import java.net.InetSocketAddress;
-import java.util.logging.*;
 import java.io.*;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
 
 /** A simple RPC mechanism.
  *
@@ -49,8 +49,8 @@ import org.apache.hadoop.util.LogFormatter;
  * the protocol instance is transmitted.
  */
 public class RPC {
-  private static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.ipc.RPC");
+  private static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.ipc.RPC");
 
   private RPC() {}                                  // no public ctor
 
@@ -150,7 +150,7 @@ public class RPC {
       ObjectWritable value = (ObjectWritable)
         client.call(new Invocation(method, args), address);
       long callTime = System.currentTimeMillis() - startTime;
-      LOG.fine("Call: " + method.getName() + " " + callTime);
+      LOG.debug("Call: " + method.getName() + " " + callTime);
       return value.get();
     }
   }
@@ -242,7 +242,7 @@ public class RPC {
         long startTime = System.currentTimeMillis();
         Object value = method.invoke(instance, call.getParameters());
         long callTime = System.currentTimeMillis() - startTime;
-        LOG.fine("Served: " + call.getMethodName() + " " + callTime);
+        LOG.debug("Served: " + call.getMethodName() + " " + callTime);
         if (verbose) log("Return: "+value);
 
         return new ObjectWritable(method.getReturnType(), value);

+ 13 - 15
src/java/org/apache/hadoop/ipc/Server.java

@@ -31,10 +31,9 @@ import java.net.SocketException;
 import java.net.SocketTimeoutException;
 
 import java.util.LinkedList;
-import java.util.logging.Logger;
-import java.util.logging.Level;
 
-import org.apache.hadoop.util.LogFormatter;
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Writable;
@@ -49,8 +48,8 @@ import org.apache.hadoop.io.UTF8;
  * @see Client
  */
 public abstract class Server {
-  public static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.ipc.Server");
+  public static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.ipc.Server");
 
   private static final ThreadLocal SERVER = new ThreadLocal();
 
@@ -110,8 +109,7 @@ public abstract class Server {
           // we can run out of memory if we have too many threads
           // log the event and sleep for a minute and give 
           // some thread(s) a chance to finish
-          LOG.log(Level.WARNING,
-                  getName() + " out of memory, sleeping...", e);          
+          LOG.warn(getName() + " out of memory, sleeping...", e);          
           try {
             acceptedSock.close();
             Thread.sleep(60000);
@@ -120,7 +118,7 @@ public abstract class Server {
           }          
         }
         catch (Exception e) {           // log all other exceptions
-          LOG.log(Level.INFO, getName() + " caught: " + e, e);
+          LOG.info(getName() + " caught: " + e, e);
         }        
       }
       try {
@@ -162,8 +160,8 @@ public abstract class Server {
             continue;
           }
         
-          if (LOG.isLoggable(Level.FINE))
-            LOG.fine(getName() + " got #" + id);
+          if (LOG.isDebugEnabled())
+            LOG.debug(getName() + " got #" + id);
         
           Writable param = makeParam();           // read param
           param.readFields(in);        
@@ -186,7 +184,7 @@ public abstract class Server {
       } catch (SocketException eof) {
           // This is what happens on Win32 when the other side shuts down
       } catch (Exception e) {
-        LOG.log(Level.INFO, getName() + " caught: " + e, e);
+        LOG.info(getName() + " caught: " + e, e);
       } finally {
         try {
           socket.close();
@@ -222,8 +220,8 @@ public abstract class Server {
             callDequeued.notify();
           }
 
-          if (LOG.isLoggable(Level.FINE))
-            LOG.fine(getName() + ": has #" + call.id + " from " +
+          if (LOG.isDebugEnabled())
+            LOG.debug(getName() + ": has #" + call.id + " from " +
                      call.connection.socket.getInetAddress().getHostAddress());
           
           String errorClass = null;
@@ -232,7 +230,7 @@ public abstract class Server {
           try {
             value = call(call.param);             // make the call
           } catch (Throwable e) {
-            LOG.log(Level.INFO, getName() + " call error: " + e, e);
+            LOG.info(getName() + " call error: " + e, e);
             errorClass = e.getClass().getName();
             error = getStackTrace(e);
           }
@@ -251,7 +249,7 @@ public abstract class Server {
           }
 
         } catch (Exception e) {
-          LOG.log(Level.INFO, getName() + " caught: " + e, e);
+          LOG.info(getName() + " caught: " + e, e);
         }
       }
       LOG.info(getName() + ": exiting");

+ 4 - 4
src/java/org/apache/hadoop/mapred/InputFormatBase.java

@@ -20,17 +20,17 @@ import java.io.IOException;
 import java.io.File;                              // deprecated
 
 import java.util.ArrayList;
-import java.util.logging.Logger;
+
+import org.apache.commons.logging.*;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.LogFormatter;
 
 /** A base class for {@link InputFormat}. */
 public abstract class InputFormatBase implements InputFormat {
 
-  public static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.mapred.InputFormatBase");
+  public static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.mapred.InputFormatBase");
 
   private static final double SPLIT_SLOP = 1.1;   // 10% slop
 

+ 4 - 3
src/java/org/apache/hadoop/mapred/JobClient.java

@@ -15,6 +15,8 @@
  */
 package org.apache.hadoop.mapred;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
@@ -23,7 +25,6 @@ import org.apache.hadoop.util.*;
 import java.io.*;
 import java.net.*;
 import java.util.*;
-import java.util.logging.*;
 
 /*******************************************************
  * JobClient interacts with the JobTracker network interface.
@@ -34,7 +35,7 @@ import java.util.logging.*;
  * @author Mike Cafarella
  *******************************************************/
 public class JobClient implements MRConstants {
-    private static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.mapred.JobClient");
+    private static final Log LOG = LogFactory.getLog("org.apache.hadoop.mapred.JobClient");
 
     static long MAX_JOBPROFILE_AGE = 1000 * 2;
 
@@ -330,7 +331,7 @@ public class JobClient implements MRConstants {
             retries = MAX_RETRIES;
           } catch (IOException ie) {
             if (--retries == 0) {
-              LOG.warning("Final attempt failed, killing job.");
+              LOG.warn("Final attempt failed, killing job.");
               throw ie;
             }
             LOG.info("Communication problem with server: " +

+ 5 - 5
src/java/org/apache/hadoop/mapred/JobInProgress.java

@@ -15,14 +15,14 @@
  */
 package org.apache.hadoop.mapred;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
 
 import java.io.*;
 import java.net.*;
 import java.util.*;
-import java.util.logging.*;
 
 ///////////////////////////////////////////////////////
 // JobInProgress maintains all the info for keeping
@@ -31,7 +31,7 @@ import java.util.logging.*;
 // doing bookkeeping of its Tasks.
 ///////////////////////////////////////////////////////
 class JobInProgress {
-    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.mapred.JobInProgress");
+    public static final Log LOG = LogFactory.getLog("org.apache.hadoop.mapred.JobInProgress");
 
     JobProfile profile;
     JobStatus status;
@@ -261,7 +261,7 @@ class JobInProgress {
                                               TaskStatus status) {
         double oldProgress = tip.getProgress();   // save old progress
         tip.updateStatus(status);                 // update tip
-        LOG.fine("Taking progress for " + tip.getTIPId() + " from " + 
+        LOG.debug("Taking progress for " + tip.getTIPId() + " from " + 
                  oldProgress + " to " + tip.getProgress());
 
         //
@@ -632,7 +632,7 @@ class JobInProgress {
         fs.delete(new Path(profile.getJobFile()).getParent());
 
       } catch (IOException e) {
-        LOG.warning("Error cleaning up "+profile.getJobId()+": "+e);
+        LOG.warn("Error cleaning up "+profile.getJobId()+": "+e);
       }
     }
 

+ 8 - 9
src/java/org/apache/hadoop/mapred/JobTracker.java

@@ -16,16 +16,16 @@
 package org.apache.hadoop.mapred;
 
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
 
 import java.io.*;
 import java.net.*;
 import java.text.NumberFormat;
 import java.util.*;
-import java.util.logging.*;
 
 /*******************************************************
  * JobTracker is the central location for submitting and 
@@ -55,7 +55,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
 
     private int nextJobId = 1;
 
-    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.mapred.JobTracker");
+    public static final Log LOG = LogFactory.getLog("org.apache.hadoop.mapred.JobTracker");
 
     private static JobTracker tracker = null;
     public static void startTracker(Configuration conf) throws IOException {
@@ -66,7 +66,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
           tracker = new JobTracker(conf);
           break;
         } catch (IOException e) {
-          LOG.log(Level.WARNING, "Starting tracker", e);
+          LOG.warn("Starting tracker", e);
         }
         try {
           Thread.sleep(1000);
@@ -102,14 +102,14 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
             // Every 3 minutes check for any tasks that are overdue
             Thread.sleep(TASKTRACKER_EXPIRY_INTERVAL/3);
             long now = System.currentTimeMillis();
-            LOG.fine("Starting launching task sweep");
+            LOG.debug("Starting launching task sweep");
             synchronized (launchingTasks) {
               Iterator itr = launchingTasks.entrySet().iterator();
               while (itr.hasNext()) {
                 Map.Entry pair = (Map.Entry) itr.next();
                 String taskId = (String) pair.getKey();
                 long age = now - ((Long) pair.getValue()).longValue();
-                LOG.fine(taskId + " is " + age + " ms old.");
+                LOG.info(taskId + " is " + age + " ms debug.");
                 if (age > TASKTRACKER_EXPIRY_INTERVAL) {
                   LOG.info("Launching task " + taskId + " timed out.");
                   TaskInProgress tip = null;
@@ -293,7 +293,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
                         job.initTasks();
                     }
                 } catch (Exception e) {
-                    LOG.log(Level.WARNING, "job init failed", e);
+                    LOG.warn("job init failed", e);
                     job.kill();
                 }
             }
@@ -679,7 +679,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         // Get map + reduce counts for the current tracker.
         //
         if (tts == null) {
-          LOG.warning("Unknown task tracker polling; ignoring: " + taskTracker);
+          LOG.warn("Unknown task tracker polling; ignoring: " + taskTracker);
           return null;
         }
 
@@ -1041,7 +1041,6 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         }
 
         Configuration conf=new Configuration();
-        LogFormatter.initFileHandler( conf, "jobtracker" );
         startTracker(conf);
     }
 }

+ 7 - 7
src/java/org/apache/hadoop/mapred/LocalJobRunner.java

@@ -18,16 +18,16 @@ package org.apache.hadoop.mapred;
 
 import java.io.*;
 import java.util.*;
-import java.util.logging.*;
+
+import org.apache.commons.logging.*;
 
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
 
 /** Implements MapReduce locally, in-process, for debugging. */ 
 class LocalJobRunner implements JobSubmissionProtocol {
-  public static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.mapred.LocalJobRunner");
+  public static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.mapred.LocalJobRunner");
 
   private FileSystem fs;
   private HashMap jobs = new HashMap();
@@ -116,14 +116,14 @@ class LocalJobRunner implements JobSubmissionProtocol {
 
       } catch (Throwable t) {
         this.status.runState = JobStatus.FAILED;
-        LOG.log(Level.WARNING, id, t);
+        LOG.warn(id, t);
 
       } finally {
         try {
           fs.delete(new Path(file).getParent());  // delete submit dir
           localFs.delete(localFile);              // delete local copy
         } catch (IOException e) {
-          LOG.warning("Error cleaning up "+id+": "+e);
+          LOG.warn("Error cleaning up "+id+": "+e);
         }
       }
     }
@@ -165,7 +165,7 @@ class LocalJobRunner implements JobSubmissionProtocol {
     }
 
     public synchronized void fsError(String message) throws IOException {
-      LOG.severe("FSError: "+ message);
+      LOG.fatal("FSError: "+ message);
     }
 
   }

+ 9 - 9
src/java/org/apache/hadoop/mapred/ReduceTaskRunner.java

@@ -206,9 +206,9 @@ class ReduceTaskRunner extends TaskRunner {
             size = copyOutput(loc, pingTimer);
             pingTimer.reset();
           } catch (IOException e) {
-            LOG.warning(reduceTask.getTaskId() + " copy failed: " +
+            LOG.warn(reduceTask.getTaskId() + " copy failed: " +
                         loc.getMapTaskId() + " from " + loc.getHost());
-            LOG.warning(StringUtils.stringifyException(e));
+            LOG.warn(StringUtils.stringifyException(e));
           }
           finish(size);
         }
@@ -242,7 +242,7 @@ class ReduceTaskRunner extends TaskRunner {
         return bytes;
       }
       catch (IOException e) {
-        LOG.warning(reduceTask.getTaskId() + " failed to copy " + loc.getMapTaskId() +
+        LOG.warn(reduceTask.getTaskId() + " failed to copy " + loc.getMapTaskId() +
                     " output from " + loc.getHost() + ".");
         throw e;
       }
@@ -269,8 +269,8 @@ class ReduceTaskRunner extends TaskRunner {
                 long lastProgress = copiers[i].getLastProgressTime();
                 if (lastProgress != 0 && 
                     currentTime - lastProgress > STALLED_COPY_TIMEOUT)  {
-                  LOG.warning("Map output copy stalled on " + 
-                              copiers[i].getLocation());
+                  LOG.warn("Map output copy stalled on " +
+                           copiers[i].getLocation());
                   // mark the current file as failed
                   copiers[i].fail();
                   // tell the thread to stop
@@ -372,7 +372,7 @@ class ReduceTaskRunner extends TaskRunner {
                    " map outputs from jobtracker");
         }
         catch (IOException ie) {
-          LOG.warning(reduceTask.getTaskId() +
+          LOG.warn(reduceTask.getTaskId() +
                       " Problem locating map outputs: " +
                       StringUtils.stringifyException(ie));
         }
@@ -449,9 +449,9 @@ class ReduceTaskRunner extends TaskRunner {
             long nextContact = currentTime + 60 * 1000 +
                                backoff.nextInt(maxBackoff*1000);
             penaltyBox.put(cr.getHost(), new Long(nextContact));          
-            LOG.warning(reduceTask.getTaskId() + " adding host " +
-                        cr.getHost() + " to penalty box, next contact in " +
-                        ((nextContact-currentTime)/1000) + " seconds");
+            LOG.warn(reduceTask.getTaskId() + " adding host " +
+                     cr.getHost() + " to penalty box, next contact in " +
+                     ((nextContact-currentTime)/1000) + " seconds");
           }
           uniqueHosts.remove(cr.getHost());
           numInFlight--;

+ 5 - 4
src/java/org/apache/hadoop/mapred/Task.java

@@ -16,17 +16,18 @@
 
 package org.apache.hadoop.mapred;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.util.*;
 
 import java.io.*;
-import java.util.logging.Logger;
 
 /** Base class for tasks. */
 abstract class Task implements Writable, Configurable {
-  private static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.mapred.TaskRunner");
+  private static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.mapred.TaskRunner");
 
   ////////////////////////////////////////////
   // Fields
@@ -116,7 +117,7 @@ abstract class Task implements Writable, Configurable {
         try {
           umbilical.progress(getTaskId(), progress, status);
         } catch (IOException ie) {
-          LOG.warning(StringUtils.stringifyException(ie));
+          LOG.warn(StringUtils.stringifyException(ie));
         }
       }
     }

+ 2 - 3
src/java/org/apache/hadoop/mapred/TaskInProgress.java

@@ -15,11 +15,10 @@
  */
 package org.apache.hadoop.mapred;
 
-import org.apache.hadoop.util.LogFormatter;
+import org.apache.commons.logging.*;
 
 import java.text.NumberFormat;
 import java.util.*;
-import java.util.logging.*;
 
 
 ////////////////////////////////////////////////////////
@@ -47,7 +46,7 @@ class TaskInProgress {
       idFormat.setGroupingUsed(false);
     }
 
-    public static final Logger LOG = LogFormatter.getLogger("org.apache.hadoop.mapred.TaskInProgress");
+    public static final Log LOG = LogFactory.getLog("org.apache.hadoop.mapred.TaskInProgress");
 
     // Defines the TIP
     private String jobFile = null;

+ 11 - 11
src/java/org/apache/hadoop/mapred/TaskRunner.java

@@ -15,14 +15,14 @@
  */
 package org.apache.hadoop.mapred;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.conf.*;
-import org.apache.hadoop.util.LogFormatter;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.util.*;
 
 import java.io.*;
 import java.util.jar.*;
-import java.util.logging.*;
 import java.util.Vector;
 import java.util.Enumeration;
 
@@ -31,8 +31,8 @@ import java.util.Enumeration;
  * user supplied map and reduce functions.
  */
 abstract class TaskRunner extends Thread {
-  public static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.mapred.TaskRunner");
+  public static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.mapred.TaskRunner");
 
   boolean killed = false;
   private Process process;
@@ -144,20 +144,20 @@ abstract class TaskRunner extends Thread {
       // Run java
       runChild((String[])vargs.toArray(new String[0]), workDir);
     } catch (FSError e) {
-      LOG.log(Level.SEVERE, "FSError", e);
+      LOG.fatal("FSError", e);
       try {
         tracker.fsError(e.getMessage());
       } catch (IOException ie) {
-        LOG.log(Level.SEVERE, t.getTaskId()+" reporting FSError", ie);
+        LOG.fatal(t.getTaskId()+" reporting FSError", ie);
       }
     } catch (Throwable throwable) {
-      LOG.log(Level.WARNING, t.getTaskId()+" Child Error", throwable);
+      LOG.warn(t.getTaskId()+" Child Error", throwable);
       ByteArrayOutputStream baos = new ByteArrayOutputStream();
       throwable.printStackTrace(new PrintStream(baos));
       try {
         tracker.reportDiagnosticInfo(t.getTaskId(), baos.toString());
       } catch (IOException e) {
-        LOG.log(Level.WARNING, t.getTaskId()+" Reporting Diagnostics", e);
+        LOG.warn(t.getTaskId()+" Reporting Diagnostics", e);
       }
     } finally {
       tracker.reportTaskFinished(t.getTaskId());
@@ -187,7 +187,7 @@ abstract class TaskRunner extends Thread {
         javaOpts = javaOpts.substring(0, index + MX.length()) +
             heapSize + ((end < 0)? "": javaOpts.substring(end));
     }
-    LOG.warning("mapred.child.heap.size is deprecated. Use " +
+    LOG.warn("mapred.child.heap.size is deprecated. Use " +
         "mapred.child.java.opt instead. Meantime, mapred.child.heap.size " +
         "is interpolated into mapred.child.java.opt: " + javaOpts);
     return javaOpts;
@@ -270,12 +270,12 @@ abstract class TaskRunner extends Thread {
         LOG.info(t.getTaskId()+" "+line);
       }
     } catch (IOException e) {
-      LOG.log(Level.WARNING, t.getTaskId()+" Error reading child output", e);
+      LOG.warn(t.getTaskId()+" Error reading child output", e);
     } finally {
       try {
         output.close();
       } catch (IOException e) {
-        LOG.log(Level.WARNING, t.getTaskId()+" Error closing child output", e);
+        LOG.warn(t.getTaskId()+" Error closing child output", e);
       }
     }
   }

+ 19 - 19
src/java/org/apache/hadoop/mapred/TaskTracker.java

@@ -15,6 +15,8 @@
  */
  package org.apache.hadoop.mapred;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.ipc.*;
 import org.apache.hadoop.util.*;
@@ -22,7 +24,6 @@ import org.apache.hadoop.util.*;
 import java.io.*;
 import java.net.*;
 import java.util.*;
-import java.util.logging.*;
 
 /*******************************************************
  * TaskTracker is a process that starts and tracks MR Tasks
@@ -42,8 +43,8 @@ public class TaskTracker
     // required for unknown reason to make WritableFactories work distributed
     static { new MapTask(); new ReduceTask(); new MapOutputLocation(); }
 
-    public static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.mapred.TaskTracker");
+    public static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.mapred.TaskTracker");
 
     private boolean running = true;
 
@@ -90,7 +91,7 @@ public class TaskTracker
               TaskInProgress tip = (TaskInProgress) tasksToCleanup.take();
               tip.jobHasFinished();
             } catch (Throwable except) {
-              LOG.warning(StringUtils.stringifyException(except));
+              LOG.warn(StringUtils.stringifyException(except));
             }
           }
         }
@@ -407,7 +408,7 @@ public class TaskTracker
                                 staleState = true;
                             }
                         } catch (Exception ex) {
-                            LOG.log(Level.INFO, "Lost connection to JobTracker [" + jobTrackAddr + "].  Retrying...", ex);
+                            LOG.info("Lost connection to JobTracker [" + jobTrackAddr + "].  Retrying...", ex);
                             try {
                                 Thread.sleep(5000);
                             } catch (InterruptedException ie) {
@@ -689,7 +690,7 @@ public class TaskTracker
               runningTasks.put(task.getTaskId(), this);
               mapTotal++;
             } else {
-              LOG.warning("Output already reported lost:"+task.getTaskId());
+              LOG.warn("Output already reported lost:"+task.getTaskId());
             }
         }
 
@@ -700,7 +701,7 @@ public class TaskTracker
          */
         void cleanup() throws IOException {
             String taskId = task.getTaskId();
-            LOG.fine("Cleaning up " + taskId);
+            LOG.debug("Cleaning up " + taskId);
             synchronized (TaskTracker.this) {
                tasks.remove(taskId);
                synchronized (this) {
@@ -737,7 +738,7 @@ public class TaskTracker
         if (tip != null) {
           tip.reportProgress(progress, state);
         } else {
-          LOG.warning("Progress from unknown child task: "+taskid+". Ignored.");
+          LOG.warn("Progress from unknown child task: "+taskid+". Ignored.");
         }
     }
 
@@ -750,7 +751,7 @@ public class TaskTracker
         if (tip != null) {
           tip.reportDiagnosticInfo(info);
         } else {
-          LOG.warning("Error from unknown child task: "+taskid+". Ignored.");
+          LOG.warn("Error from unknown child task: "+taskid+". Ignored.");
         }
     }
 
@@ -767,14 +768,14 @@ public class TaskTracker
         if (tip != null) {
           tip.reportDone();
         } else {
-          LOG.warning("Unknown child task done: "+taskid+". Ignored.");
+          LOG.warn("Unknown child task done: "+taskid+". Ignored.");
         }
     }
 
     /** A child task had a local filesystem error.  Exit, so that no future
      * jobs are accepted. */
     public synchronized void fsError(String message) throws IOException {
-      LOG.severe("FSError, exiting: "+ message);
+      LOG.fatal("FSError, exiting: "+ message);
       running = false;
     }
 
@@ -792,7 +793,7 @@ public class TaskTracker
         if (tip != null) {
           tip.taskFinished();
         } else {
-          LOG.warning("Unknown child task finshed: "+taskid+". Ignored.");
+          LOG.warn("Unknown child task finshed: "+taskid+". Ignored.");
         }
     }
 
@@ -804,7 +805,7 @@ public class TaskTracker
         if (tip != null) {
           tip.mapOutputLost();
         } else {
-          LOG.warning("Unknown child with bad map output: "+taskid+". Ignored.");
+          LOG.warn("Unknown child with bad map output: "+taskid+". Ignored.");
         }
     }
 
@@ -813,7 +814,7 @@ public class TaskTracker
      */
     public static class Child {
         public static void main(String[] args) throws Throwable {
-          LogFormatter.showTime(false);
+          //LogFactory.showTime(false);
           LOG.info("Child starting");
 
           JobConf defaultConf = new JobConf();
@@ -836,10 +837,10 @@ public class TaskTracker
             FileSystem.get(job).setWorkingDirectory(job.getWorkingDirectory());
             task.run(job, umbilical);             // run the task
           } catch (FSError e) {
-            LOG.log(Level.SEVERE, "FSError from child", e);
+            LOG.fatal("FSError from child", e);
             umbilical.fsError(e.getMessage());
           } catch (Throwable throwable) {
-              LOG.log(Level.WARNING, "Error running child", throwable);
+              LOG.warn("Error running child", throwable);
               // Report back any failures, for diagnostic purposes
               ByteArrayOutputStream baos = new ByteArrayOutputStream();
               throwable.printStackTrace(new PrintStream(baos));
@@ -857,7 +858,7 @@ public class TaskTracker
                 while (true) {
                   try {
                     if (!umbilical.ping(taskid)) {
-                      LOG.log(Level.WARNING, "Parent died.  Exiting "+taskid);
+                      LOG.warn("Parent died.  Exiting "+taskid);
                       System.exit(66);
                     }
                     remainingRetries = MAX_RETRIES;
@@ -866,7 +867,7 @@ public class TaskTracker
                     LOG.info("Ping exception: " + msg);
                     remainingRetries -=1;
                     if (remainingRetries == 0) {
-                      LOG.log(Level.WARNING, "Last retry, killing "+taskid);
+                      LOG.warn("Last retry, killing "+taskid);
                       System.exit(65);
                     }
                   }
@@ -922,7 +923,6 @@ public class TaskTracker
         }
 
         JobConf conf=new JobConf();
-        LogFormatter.initFileHandler( conf, "tasktracker" );
         new TaskTracker(conf).run();
     }
 }

+ 1 - 2
src/java/org/apache/hadoop/util/LogFormatter.java

@@ -24,8 +24,7 @@ import java.util.Date;
 
 import org.apache.hadoop.conf.Configuration;
 
-/** Prints just the date and the log message. */
-
+/** @deprecated use {@link org.apache.commons.logging.LogFactory} instead. */
 public class LogFormatter extends Formatter {
   private static final String FORMAT = "yyMMdd HHmmss";
   private static final String NEWLINE = System.getProperty("line.separator");

+ 7 - 0
src/test/log4j.properties

@@ -0,0 +1,7 @@
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n

+ 4 - 4
src/test/org/apache/hadoop/dfs/ClusterTestDFS.java

@@ -18,11 +18,12 @@ package org.apache.hadoop.dfs;
 
 import junit.framework.TestCase;
 import junit.framework.AssertionFailedError;
+
+import org.apache.commons.logging.*;
 import org.apache.hadoop.fs.FSInputStream;
 import org.apache.hadoop.fs.FSOutputStream;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.io.UTF8;
-import org.apache.hadoop.util.LogFormatter;
 import org.apache.hadoop.conf.Configuration;
 
 import java.io.File;
@@ -30,7 +31,6 @@ import java.io.FilenameFilter;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.ListIterator;
-import java.util.logging.Logger;
 import java.util.Random;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
@@ -69,8 +69,8 @@ import java.lang.reflect.InvocationTargetException;
  * @author Paul Baclace
  */
 public class ClusterTestDFS extends TestCase implements FSConstants {
-  private static final Logger LOG =
-      LogFormatter.getLogger("org.apache.hadoop.dfs.ClusterTestDFS");
+  private static final Log LOG =
+      LogFactory.getLog("org.apache.hadoop.dfs.ClusterTestDFS");
 
   private static Configuration conf = new Configuration();
   private static int BUFFER_SIZE =

+ 5 - 10
src/test/org/apache/hadoop/dfs/ClusterTestDFSNamespaceLogging.java

@@ -18,10 +18,12 @@ package org.apache.hadoop.dfs;
 
 import junit.framework.TestCase;
 import junit.framework.AssertionFailedError;
+
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.fs.FSOutputStream;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.io.UTF8;
-import org.apache.hadoop.util.LogFormatter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.dfs.NameNode;
 
@@ -32,8 +34,6 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
-import java.util.logging.Level;
-import java.util.logging.Logger;
 
 /**
  * Test DFS logging
@@ -41,8 +41,8 @@ import java.util.logging.Logger;
  * @author Hairong Kuang
  */
 public class ClusterTestDFSNamespaceLogging extends TestCase implements FSConstants {
-  private static final Logger LOG =
-      LogFormatter.getLogger("org.apache.hadoop.dfs.ClusterTestDFS");
+  private static final Log LOG =
+      LogFactory.getLog("org.apache.hadoop.dfs.ClusterTestDFS");
 
   private static Configuration conf = new Configuration();
 
@@ -337,11 +337,6 @@ public class ClusterTestDFSNamespaceLogging extends TestCase implements FSConsta
 	System.setProperty("hadoop.log.dir", baseDirSpecified+"/logs");
 	conf.setInt("hadoop.logfile.count", 1);
 	conf.setInt("hadoop.logfile.size", 1000000000);
-
-	// logging configuration for namenode
-    logFile = LogFormatter.initFileHandler( conf, "namenode" );
-    LogFormatter.setShowThreadIDs(true);
-    NameNode.stateChangeLog.setLevel( Level.FINEST);
   }
   
   private void startDFS( int dataNodeNum) throws IOException {

+ 3 - 2
src/test/org/apache/hadoop/fs/DFSCIOTest.java

@@ -19,10 +19,11 @@ package org.apache.hadoop.fs;
 import java.io.*;
 
 import junit.framework.TestCase;
-import java.util.logging.*;
 import java.util.Date;
 import java.util.StringTokenizer;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
@@ -65,7 +66,7 @@ public class DFSCIOTest extends TestCase {
   private static final String BASE_FILE_NAME = "test_io_";
   private static final String DEFAULT_RES_FILE_NAME = "DFSCIOTest_results.log";
   
-  private static final Logger LOG = InputFormatBase.LOG;
+  private static final Log LOG = InputFormatBase.LOG;
   private static Configuration fsConfig = new Configuration();
   private static final long MEGA = 0x100000;
   private static String TEST_ROOT_DIR = System.getProperty("test.build.data","/benchmarks/DFSCIOTest");

+ 3 - 2
src/test/org/apache/hadoop/fs/DistributedFSCheck.java

@@ -19,12 +19,13 @@ package org.apache.hadoop.fs;
 import java.io.*;
 
 import junit.framework.TestCase;
-import java.util.logging.*;
 import java.util.Date;
 import java.util.StringTokenizer;
 import java.util.TreeSet;
 import java.util.Vector;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
@@ -49,7 +50,7 @@ public class DistributedFSCheck extends TestCase {
   private static final long MEGA = 0x100000;
   
   private static Configuration fsConfig = new Configuration();
-  private static final Logger LOG = InputFormatBase.LOG;
+  private static final Log LOG = InputFormatBase.LOG;
   private static Path TEST_ROOT_DIR = new Path(System.getProperty("test.build.data","/benchmarks/DistributedFSCheck"));
   private static Path MAP_INPUT_DIR = new Path(TEST_ROOT_DIR, "map_input");
   private static Path READ_DIR = new Path(TEST_ROOT_DIR, "io_read");

+ 3 - 2
src/test/org/apache/hadoop/fs/TestDFSIO.java

@@ -19,10 +19,11 @@ package org.apache.hadoop.fs;
 import java.io.*;
 
 import junit.framework.TestCase;
-import java.util.logging.*;
 import java.util.Date;
 import java.util.StringTokenizer;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
@@ -65,7 +66,7 @@ public class TestDFSIO extends TestCase {
   private static final String BASE_FILE_NAME = "test_io_";
   private static final String DEFAULT_RES_FILE_NAME = "TestDFSIO_results.log";
   
-  private static final Logger LOG = InputFormatBase.LOG;
+  private static final Log LOG = InputFormatBase.LOG;
   private static Configuration fsConfig = new Configuration();
   private static final long MEGA = 0x100000;
   private static String TEST_ROOT_DIR = System.getProperty("test.build.data","/benchmarks/TestDFSIO");

+ 3 - 2
src/test/org/apache/hadoop/fs/TestFileSystem.java

@@ -19,7 +19,8 @@ package org.apache.hadoop.fs;
 import java.io.*;
 import java.util.*;
 import junit.framework.TestCase;
-import java.util.logging.*;
+
+import org.apache.commons.logging.*;
 
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.mapred.lib.*;
@@ -27,7 +28,7 @@ import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
 
 public class TestFileSystem extends TestCase {
-  private static final Logger LOG = InputFormatBase.LOG;
+  private static final Log LOG = InputFormatBase.LOG;
 
   private static Configuration conf = new Configuration();
   private static int BUFFER_SIZE = conf.getInt("io.file.buffer.size", 4096);

+ 7 - 8
src/test/org/apache/hadoop/io/TestArrayFile.java

@@ -18,14 +18,15 @@ package org.apache.hadoop.io;
 
 import java.io.*;
 import junit.framework.TestCase;
-import java.util.logging.*;
+
+import org.apache.commons.logging.*;
 
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.*;
 
 /** Support for flat files of binary key/value pairs. */
 public class TestArrayFile extends TestCase {
-  private static Logger LOG = SequenceFile.LOG;
+  private static Log LOG = SequenceFile.LOG;
   private static String FILE =
     System.getProperty("test.build.data",".") + "/test.array";
 
@@ -51,7 +52,7 @@ public class TestArrayFile extends TestCase {
   }
 
   private static RandomDatum[] generate(int count) {
-    LOG.fine("generating " + count + " records in memory");
+    LOG.debug("generating " + count + " records in debug");
     RandomDatum[] data = new RandomDatum[count];
     RandomDatum.Generator generator = new RandomDatum.Generator();
     for (int i = 0; i < count; i++) {
@@ -64,7 +65,7 @@ public class TestArrayFile extends TestCase {
   private static void writeTest(FileSystem fs, RandomDatum[] data, String file)
     throws IOException {
     MapFile.delete(fs, file);
-    LOG.fine("creating with " + data.length + " records");
+    LOG.debug("creating with " + data.length + " debug");
     ArrayFile.Writer writer = new ArrayFile.Writer(fs, file, RandomDatum.class);
     writer.setIndexInterval(100);
     for (int i = 0; i < data.length; i++)
@@ -75,7 +76,7 @@ public class TestArrayFile extends TestCase {
   private static void readTest(FileSystem fs, RandomDatum[] data, String file, Configuration conf)
     throws IOException {
     RandomDatum v = new RandomDatum();
-    LOG.fine("reading " + data.length + " records");
+    LOG.debug("reading " + data.length + " debug");
     ArrayFile.Reader reader = new ArrayFile.Reader(fs, file, conf);
     for (int i = 0; i < data.length; i++) {       // try forwards
       reader.get(i, v);
@@ -90,7 +91,7 @@ public class TestArrayFile extends TestCase {
       }
     }
     reader.close();
-    LOG.fine("done reading " + data.length + " records");
+    LOG.debug("done reading " + data.length + " debug");
   }
 
 
@@ -131,8 +132,6 @@ public class TestArrayFile extends TestCase {
         LOG.info("check = " + check);
         LOG.info("file = " + file);
 
-        LOG.setLevel(Level.FINE);
-
         RandomDatum[] data = generate(count);
 
         if (create) {

+ 13 - 14
src/test/org/apache/hadoop/io/TestSequenceFile.java

@@ -19,7 +19,8 @@ package org.apache.hadoop.io;
 import java.io.*;
 import java.util.*;
 import junit.framework.TestCase;
-import java.util.logging.*;
+
+import org.apache.commons.logging.*;
 
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.*;
@@ -27,7 +28,7 @@ import org.apache.hadoop.conf.*;
 
 /** Support for flat files of binary key/value pairs. */
 public class TestSequenceFile extends TestCase {
-  private static Logger LOG = SequenceFile.LOG;
+  private static Log LOG = SequenceFile.LOG;
 
   private static Configuration conf = new Configuration();
   
@@ -68,7 +69,7 @@ public class TestSequenceFile extends TestCase {
                                 Path file, boolean compress)
     throws IOException {
     fs.delete(file);
-    LOG.fine("creating with " + count + " records");
+    LOG.debug("creating with " + count + " records");
     SequenceFile.Writer writer =
       new SequenceFile.Writer(fs, file, RandomDatum.class, RandomDatum.class,
                               compress);
@@ -87,7 +88,7 @@ public class TestSequenceFile extends TestCase {
     throws IOException {
     RandomDatum k = new RandomDatum();
     RandomDatum v = new RandomDatum();
-    LOG.fine("reading " + count + " records");
+    LOG.debug("reading " + count + " records");
     SequenceFile.Reader reader = new SequenceFile.Reader(fs, file, conf);
     RandomDatum.Generator generator = new RandomDatum.Generator(seed);
     for (int i = 0; i < count; i++) {
@@ -111,14 +112,14 @@ public class TestSequenceFile extends TestCase {
     throws IOException {
     fs.delete(new Path(file+".sorted"));
     SequenceFile.Sorter sorter = newSorter(fs, fast, megabytes, factor);
-    LOG.fine("sorting " + count + " records");
+    LOG.debug("sorting " + count + " records");
     sorter.sort(file, file.suffix(".sorted"));
-    LOG.fine("done sorting " + count + " records");
+    LOG.info("done sorting " + count + " debug");
   }
 
   private static void checkSort(FileSystem fs, int count, int seed, Path file)
     throws IOException {
-    LOG.fine("sorting " + count + " records in memory for check");
+    LOG.info("sorting " + count + " records in memory for debug");
     RandomDatum.Generator generator = new RandomDatum.Generator(seed);
     SortedMap map = new TreeMap();
     for (int i = 0; i < count; i++) {
@@ -128,7 +129,7 @@ public class TestSequenceFile extends TestCase {
       map.put(key, value);
     }
 
-    LOG.fine("checking order of " + count + " records");
+    LOG.debug("checking order of " + count + " records");
     RandomDatum k = new RandomDatum();
     RandomDatum v = new RandomDatum();
     Iterator iterator = map.entrySet().iterator();
@@ -148,7 +149,7 @@ public class TestSequenceFile extends TestCase {
     }
 
     reader.close();
-    LOG.fine("sucessfully checked " + count + " records");
+    LOG.debug("sucessfully checked " + count + " records");
   }
 
   private static void mergeTest(FileSystem fs, int count, int seed, 
@@ -156,7 +157,7 @@ public class TestSequenceFile extends TestCase {
                                 int megabytes)
     throws IOException {
 
-    LOG.fine("creating "+factor+" files with "+count/factor+" records");
+    LOG.debug("creating "+factor+" files with "+count/factor+" records");
 
     SequenceFile.Writer[] writers = new SequenceFile.Writer[factor];
     Path[] names = new Path[factor];
@@ -185,11 +186,11 @@ public class TestSequenceFile extends TestCase {
       writers[i].close();
 
     for (int i = 0; i < factor; i++) {
-      LOG.fine("sorting file " + i + " with " + count/factor + " records");
+      LOG.debug("sorting file " + i + " with " + count/factor + " records");
       newSorter(fs, fast, megabytes, factor).sort(names[i], sortedNames[i]);
     }
 
-    LOG.fine("merging " + factor + " files with " + count/factor + " records");
+    LOG.info("merging " + factor + " files with " + count/factor + " debug");
     fs.delete(new Path(file+".sorted"));
     newSorter(fs, fast, megabytes, factor)
       .merge(sortedNames, file.suffix(".sorted"));
@@ -264,8 +265,6 @@ public class TestSequenceFile extends TestCase {
 
         int seed = 0;
  
-        LOG.setLevel(Level.FINE);
-
         if (create && !merge) {
             writeTest(fs, count, seed, file, compress);
             readTest(fs, count, seed, file);

+ 8 - 9
src/test/org/apache/hadoop/io/TestSetFile.java

@@ -19,14 +19,15 @@ package org.apache.hadoop.io;
 import java.io.*;
 import java.util.*;
 import junit.framework.TestCase;
-import java.util.logging.*;
+
+import org.apache.commons.logging.*;
 
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.conf.*;
 
 /** Support for flat files of binary key/value pairs. */
 public class TestSetFile extends TestCase {
-  private static Logger LOG = SequenceFile.LOG;
+  private static Log LOG = SequenceFile.LOG;
   private static String FILE =
     System.getProperty("test.build.data",".") + "/test.set";
 
@@ -46,14 +47,14 @@ public class TestSetFile extends TestCase {
   }
 
   private static RandomDatum[] generate(int count) {
-    LOG.fine("generating " + count + " records in memory");
+    LOG.debug("generating " + count + " records in memory");
     RandomDatum[] data = new RandomDatum[count];
     RandomDatum.Generator generator = new RandomDatum.Generator();
     for (int i = 0; i < count; i++) {
       generator.next();
       data[i] = generator.getValue();
     }
-    LOG.fine("sorting " + count + " records in memory");
+    LOG.info("sorting " + count + " records in debug");
     Arrays.sort(data);
     return data;
   }
@@ -61,7 +62,7 @@ public class TestSetFile extends TestCase {
   private static void writeTest(FileSystem fs, RandomDatum[] data, String file)
     throws IOException {
     MapFile.delete(fs, file);
-    LOG.fine("creating with " + data.length + " records");
+    LOG.debug("creating with " + data.length + " records");
     SetFile.Writer writer = new SetFile.Writer(fs, file, RandomDatum.class);
     for (int i = 0; i < data.length; i++)
       writer.append(data[i]);
@@ -71,14 +72,14 @@ public class TestSetFile extends TestCase {
   private static void readTest(FileSystem fs, RandomDatum[] data, String file)
     throws IOException {
     RandomDatum v = new RandomDatum();
-    LOG.fine("reading " + data.length + " records");
+    LOG.debug("reading " + data.length + " records");
     SetFile.Reader reader = new SetFile.Reader(fs, file, conf);
     for (int i = 0; i < data.length; i++) {
       if (!reader.seek(data[i]))
         throw new RuntimeException("wrong value at " + i);
     }
     reader.close();
-    LOG.fine("done reading " + data.length + " records");
+    LOG.info("done reading " + data.length + " debug");
   }
 
 
@@ -117,8 +118,6 @@ public class TestSetFile extends TestCase {
         LOG.info("check = " + check);
         LOG.info("file = " + file);
 
-        LOG.setLevel(Level.FINE);
-
         RandomDatum[] data = generate(count);
 
         if (create) {

+ 8 - 22
src/test/org/apache/hadoop/ipc/TestIPC.java

@@ -16,6 +16,8 @@
 
 package org.apache.hadoop.ipc;
 
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.LongWritable;
 
@@ -25,26 +27,15 @@ import java.net.InetSocketAddress;
 
 import junit.framework.TestCase;
 
-import java.util.logging.Logger;
-import java.util.logging.Level;
-
-import org.apache.hadoop.util.LogFormatter;
 import org.apache.hadoop.conf.Configuration;
 
 /** Unit tests for IPC. */
 public class TestIPC extends TestCase {
-  public static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.ipc.TestIPC");
+  public static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.ipc.TestIPC");
 
   private static Configuration conf = new Configuration();
   
-  // quiet during testing, since output ends up on console
-  static {
-    LOG.setLevel(Level.WARNING);
-    Client.LOG.setLevel(Level.WARNING);
-    Server.LOG.setLevel(Level.WARNING);
-  }
-
   public TestIPC(String name) { super(name); }
 
   private static final Random RANDOM = new Random();
@@ -88,12 +79,12 @@ public class TestIPC extends TestCase {
           LongWritable value =
             (LongWritable)client.call(param, new InetSocketAddress(PORT));
           if (!param.equals(value)) {
-            LOG.severe("Call failed!");
+            LOG.fatal("Call failed!");
             failed = true;
             break;
           }
         } catch (Exception e) {
-          LOG.severe("Caught: " + e);
+          LOG.fatal("Caught: " + e);
           failed = true;
         }
       }
@@ -123,13 +114,13 @@ public class TestIPC extends TestCase {
           Writable[] values = client.call(params, addresses);
           for (int j = 0; j < addresses.length; j++) {
             if (!params[j].equals(values[j])) {
-              LOG.severe("Call failed!");
+              LOG.fatal("Call failed!");
               failed = true;
               break;
             }
           }
         } catch (Exception e) {
-          LOG.severe("Caught: " + e);
+          LOG.fatal("Caught: " + e);
           failed = true;
         }
       }
@@ -209,11 +200,6 @@ public class TestIPC extends TestCase {
   }
 	
   public static void main(String[] args) throws Exception {
-    // crank up the volume!
-    LOG.setLevel(Level.FINE);
-    Client.LOG.setLevel(Level.FINE);
-    Server.LOG.setLevel(Level.FINE);
-    LogFormatter.setShowThreadIDs(true);
 
     //new TestIPC("test").testSerial(5, false, 2, 10, 1000);
 

+ 5 - 14
src/test/org/apache/hadoop/ipc/TestRPC.java

@@ -22,11 +22,10 @@ import java.lang.reflect.Method;
 
 import junit.framework.TestCase;
 
-import java.util.logging.Logger;
-import java.util.logging.Level;
 import java.util.Arrays;
 
-import org.apache.hadoop.util.LogFormatter;
+import org.apache.commons.logging.*;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.Writable;
@@ -35,17 +34,14 @@ import org.apache.hadoop.io.Writable;
 public class TestRPC extends TestCase {
   private static final int PORT = 1234;
 
-  public static final Logger LOG =
-    LogFormatter.getLogger("org.apache.hadoop.ipc.TestRPC");
+  public static final Log LOG =
+    LogFactory.getLog("org.apache.hadoop.ipc.TestRPC");
   
   private static Configuration conf = new Configuration();
 
   // quiet during testing, since output ends up on console
   static {
     conf.setInt("ipc.client.timeout", 5000);
-    LOG.setLevel(Level.WARNING);
-    Client.LOG.setLevel(Level.WARNING);
-    Server.LOG.setLevel(Level.WARNING);
   }
 
   public TestRPC(String name) { super(name); }
@@ -134,7 +130,7 @@ public class TestRPC extends TestCase {
     try {
       proxy.error();
     } catch (IOException e) {
-      LOG.fine("Caught " + e);
+      LOG.debug("Caught " + e);
       caught = true;
     }
     assertTrue(caught);
@@ -156,11 +152,6 @@ public class TestRPC extends TestCase {
     server.stop();
   }
   public static void main(String[] args) throws Exception {
-    // crank up the volume!
-    LOG.setLevel(Level.FINE);
-    Client.LOG.setLevel(Level.FINE);
-    Server.LOG.setLevel(Level.FINE);
-    LogFormatter.setShowThreadIDs(true);
 
     new TestRPC("test").testCalls();
 

+ 3 - 3
src/test/org/apache/hadoop/mapred/TestMapRed.java

@@ -273,7 +273,7 @@ public class TestMapRed extends TestCase {
         // file of random numbers.
         //
         Path randomOuts = new Path(testdir, "genouts");
-        //fs.mkdirs(randomOuts);
+        fs.delete(randomOuts);
 
 
         JobConf genJob = new JobConf(conf);
@@ -322,7 +322,7 @@ public class TestMapRed extends TestCase {
         //
         int intermediateReduces = 10;
         Path intermediateOuts = new Path(testdir, "intermediateouts");
-        //fs.mkdirs(intermediateOuts);
+        fs.delete(intermediateOuts);
         JobConf checkJob = new JobConf(conf);
         checkJob.setInputPath(randomOuts);
         checkJob.setInputKeyClass(LongWritable.class);
@@ -347,7 +347,7 @@ public class TestMapRed extends TestCase {
         // all the files.
         //
         Path finalOuts = new Path(testdir, "finalouts");        
-        //fs.mkdirs(finalOuts);
+        fs.delete(finalOuts);
         JobConf mergeJob = new JobConf(conf);
         mergeJob.setInputPath(intermediateOuts);
         mergeJob.setInputKeyClass(IntWritable.class);

+ 3 - 2
src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java

@@ -19,14 +19,15 @@ package org.apache.hadoop.mapred;
 import java.io.*;
 import java.util.*;
 import junit.framework.TestCase;
-import java.util.logging.*;
+
+import org.apache.commons.logging.*;
 
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
 
 public class TestSequenceFileInputFormat extends TestCase {
-  private static final Logger LOG = InputFormatBase.LOG;
+  private static final Log LOG = InputFormatBase.LOG;
 
   private static int MAX_LENGTH = 10000;
   private static Configuration conf = new Configuration();

+ 3 - 2
src/test/org/apache/hadoop/mapred/TestTextInputFormat.java

@@ -19,14 +19,15 @@ package org.apache.hadoop.mapred;
 import java.io.*;
 import java.util.*;
 import junit.framework.TestCase;
-import java.util.logging.*;
+
+import org.apache.commons.logging.*;
 
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
 
 public class TestTextInputFormat extends TestCase {
-  private static final Logger LOG = InputFormatBase.LOG;
+  private static final Log LOG = InputFormatBase.LOG;
 
   private static int MAX_LENGTH = 10000;
   private static Configuration conf = new Configuration();

+ 3 - 3
src/test/org/apache/hadoop/record/test/TestMapRed.java

@@ -283,7 +283,7 @@ public class TestMapRed extends TestCase {
         // file of random numbers.
         //
         File randomOuts = new File(testdir, "genouts");
-        //fs.mkdirs(randomOuts);
+        fs.delete(randomOuts);
 
 
         JobConf genJob = new JobConf(conf);
@@ -332,7 +332,7 @@ public class TestMapRed extends TestCase {
         //
         int intermediateReduces = 10;
         File intermediateOuts = new File(testdir, "intermediateouts");
-        //fs.mkdirs(intermediateOuts);
+        fs.delete(intermediateOuts);
         JobConf checkJob = new JobConf(conf);
         checkJob.setInputDir(randomOuts);
         checkJob.setInputKeyClass(RecInt.class);
@@ -357,7 +357,7 @@ public class TestMapRed extends TestCase {
         // all the files.
         //
         File finalOuts = new File(testdir, "finalouts");        
-        //fs.mkdirs(finalOuts);
+        fs.delete(finalOuts);
         JobConf mergeJob = new JobConf(conf);
         mergeJob.setInputDir(intermediateOuts);
         mergeJob.setInputKeyClass(RecInt.class);

+ 2 - 2
src/test/org/apache/hadoop/record/test/TestWritable.java

@@ -19,11 +19,11 @@ package org.apache.hadoop.record.test;
 import java.io.*;
 import java.util.*;
 import junit.framework.TestCase;
-import java.util.logging.*;
 
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.conf.*;
+import org.apache.commons.logging.*;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputFormatBase;
@@ -33,7 +33,7 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 
 public class TestWritable extends TestCase {
-  private static final Logger LOG = InputFormatBase.LOG;
+  private static final Log LOG = InputFormatBase.LOG;
 
   private static int MAX_LENGTH = 10000;
   private static Configuration conf = new Configuration();