You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2006/06/02 21:14:24 UTC
svn commit: r411254 [1/2] - in /lucene/hadoop/trunk: ./ bin/ conf/ lib/
src/contrib/streaming/src/java/org/apache/hadoop/streaming/
src/java/org/apache/hadoop/conf/ src/java/org/apache/hadoop/dfs/
src/java/org/apache/hadoop/fs/ src/java/org/apache/hado...
Author: cutting
Date: Fri Jun 2 12:14:22 2006
New Revision: 411254
URL: http://svn.apache.org/viewvc?rev=411254&view=rev
Log:
HADOOP-211. Switch logging use the Jakarta Commons logging API, configured to use log4j by default.
Added:
lucene/hadoop/trunk/conf/commons-logging.properties
lucene/hadoop/trunk/conf/log4j.properties
lucene/hadoop/trunk/lib/commons-logging-1.0.4.jar (with props)
lucene/hadoop/trunk/lib/log4j-1.2.13.jar (with props)
lucene/hadoop/trunk/src/test/log4j.properties
Modified:
lucene/hadoop/trunk/CHANGES.txt
lucene/hadoop/trunk/bin/hadoop
lucene/hadoop/trunk/bin/hadoop-daemon.sh
lucene/hadoop/trunk/build.xml
lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java
lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamInputFormat.java
lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/conf/Configuration.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSck.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/LocalFileSystem.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/io/UTF8.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Client.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/RPC.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Server.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/InputFormatBase.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/ReduceTaskRunner.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/Task.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskInProgress.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskRunner.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/util/LogFormatter.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/ClusterTestDFS.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/dfs/ClusterTestDFSNamespaceLogging.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/DFSCIOTest.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/DistributedFSCheck.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestDFSIO.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestFileSystem.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestArrayFile.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestSequenceFile.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/io/TestSetFile.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/ipc/TestIPC.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/ipc/TestRPC.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestMapRed.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestSequenceFileInputFormat.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/mapred/TestTextInputFormat.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestMapRed.java
lucene/hadoop/trunk/src/test/org/apache/hadoop/record/test/TestWritable.java
Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Fri Jun 2 12:14:22 2006
@@ -92,6 +92,9 @@
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
Modified: lucene/hadoop/trunk/bin/hadoop
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/bin/hadoop?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/bin/hadoop (original)
+++ lucene/hadoop/trunk/bin/hadoop Fri Jun 2 12:14:22 2006
@@ -120,6 +120,14 @@
# 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 @@
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"
Modified: lucene/hadoop/trunk/bin/hadoop-daemon.sh
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/bin/hadoop-daemon.sh?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/bin/hadoop-daemon.sh (original)
+++ lucene/hadoop/trunk/bin/hadoop-daemon.sh Fri Jun 2 12:14:22 2006
@@ -62,6 +62,7 @@
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
Modified: lucene/hadoop/trunk/build.xml
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/build.xml?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/build.xml (original)
+++ lucene/hadoop/trunk/build.xml Fri Jun 2 12:14:22 2006
@@ -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">
Added: lucene/hadoop/trunk/conf/commons-logging.properties
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/conf/commons-logging.properties?rev=411254&view=auto
==============================================================================
--- lucene/hadoop/trunk/conf/commons-logging.properties (added)
+++ lucene/hadoop/trunk/conf/commons-logging.properties Fri Jun 2 12:14:22 2006
@@ -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
Added: lucene/hadoop/trunk/conf/log4j.properties
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/conf/log4j.properties?rev=411254&view=auto
==============================================================================
--- lucene/hadoop/trunk/conf/log4j.properties (added)
+++ lucene/hadoop/trunk/conf/log4j.properties Fri Jun 2 12:14:22 2006
@@ -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
+
Added: lucene/hadoop/trunk/lib/commons-logging-1.0.4.jar
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/lib/commons-logging-1.0.4.jar?rev=411254&view=auto
==============================================================================
Binary file - no diff available.
Propchange: lucene/hadoop/trunk/lib/commons-logging-1.0.4.jar
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Added: lucene/hadoop/trunk/lib/log4j-1.2.13.jar
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/lib/log4j-1.2.13.jar?rev=411254&view=auto
==============================================================================
Binary file - no diff available.
Propchange: lucene/hadoop/trunk/lib/log4j-1.2.13.jar
------------------------------------------------------------------------------
svn:mime-type = application/octet-stream
Modified: lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java (original)
+++ lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamBaseRecordReader.java Fri Jun 2 12:14:22 2006
@@ -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;
Modified: lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamInputFormat.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamInputFormat.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamInputFormat.java (original)
+++ lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamInputFormat.java Fri Jun 2 12:14:22 2006
@@ -22,7 +22,6 @@
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;
Modified: lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java (original)
+++ lucene/hadoop/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/StreamJob.java Fri Jun 2 12:14:22 2006
@@ -19,7 +19,6 @@
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;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/conf/Configuration.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/conf/Configuration.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/conf/Configuration.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/conf/Configuration.java Fri Jun 2 12:14:22 2006
@@ -19,7 +19,6 @@
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.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 @@
* 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 @@
/** 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 @@
/** 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 @@
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 @@
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 @@
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 @@
}
} catch (Exception e) {
- LOG.severe("error parsing conf file: " + e);
+ LOG.fatal("error parsing conf file: " + e);
throw new RuntimeException(e);
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java Fri Jun 2 12:14:22 2006
@@ -21,10 +21,11 @@
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 @@
* @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 @@
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 @@
}
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());
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSck.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSck.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSck.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSck.java Fri Jun 2 12:14:22 2006
@@ -25,7 +25,8 @@
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 @@
* @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;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java Fri Jun 2 12:14:22 2006
@@ -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.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 @@
* @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 @@
}
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 @@
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 @@
in.close();
}
} catch (IOException ie) {
- LOG.log(Level.WARNING, "DataXCeiver", ie);
+ LOG.warn("DataXCeiver", ie);
} finally {
try {
s.close();
@@ -797,7 +798,7 @@
}
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 @@
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 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);
+ }
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDirectory.java Fri Jun 2 12:14:22 2006
@@ -671,7 +671,7 @@
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 @@
* 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 @@
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 @@
}
// 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 @@
* 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 @@
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 @@
// 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 @@
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 @@
}
}
/* if( !lastSuccess )
- NameNode.stateChangeLog.warning("DIR* FSDirectory.mkdirs: "
+ NameNode.stateChangeLog.warn("DIR* FSDirectory.mkdirs: "
+"failed to create directory "+src );*/
return lastSuccess;
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java Fri Jun 2 12:14:22 2006
@@ -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 @@
* 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 @@
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 @@
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 @@
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 @@
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 @@
//
// 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 @@
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 @@
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 @@
* 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 @@
// 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 @@
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 @@
(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 @@
* 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 @@
* 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 @@
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 @@
* 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 @@
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 @@
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 @@
*/
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 @@
// 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 @@
// 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 @@
// 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 @@
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 @@
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 @@
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 @@
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 @@
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 @@
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 @@
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 @@
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 @@
* 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 @@
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 @@
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 @@
) 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 @@
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 @@
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 @@
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 @@
//
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 @@
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 @@
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;
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/NameNode.java Fri Jun 2 12:14:22 2006
@@ -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 @@
* @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 @@
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 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 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 @@
* 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 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 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 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 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 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 @@
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 @@
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();
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java Fri Jun 2 12:14:22 2006
@@ -17,16 +17,17 @@
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 @@
} 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 @@
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 @@
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;
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java Fri Jun 2 12:14:22 2006
@@ -18,11 +18,11 @@
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 @@
* @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();
/**
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/LocalFileSystem.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/LocalFileSystem.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/LocalFileSystem.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/LocalFileSystem.java Fri Jun 2 12:14:22 2006
@@ -344,7 +344,7 @@
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 @@
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);
}
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/MapFile.java Fri Jun 2 12:14:22 2006
@@ -258,7 +258,7 @@
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;
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/SequenceFile.java Fri Jun 2 12:14:22 2006
@@ -19,19 +19,18 @@
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 @@
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 @@
}
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 @@
}
// 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 @@
}
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 @@
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 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 =
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/UTF8.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/UTF8.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/UTF8.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/UTF8.java Fri Jun 2 12:14:22 2006
@@ -20,8 +20,8 @@
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 @@
* @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 @@
/** 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 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);
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Client.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Client.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Client.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Client.java Fri Jun 2 12:14:22 2006
@@ -30,10 +30,9 @@
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 @@
* @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 @@
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 @@
} 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 @@
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);
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/RPC.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/RPC.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/RPC.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/RPC.java Fri Jun 2 12:14:22 2006
@@ -23,12 +23,12 @@
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 @@
* 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 @@
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 @@
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);
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Server.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Server.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Server.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Server.java Fri Jun 2 12:14:22 2006
@@ -31,10 +31,9 @@
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 @@
* @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 @@
// 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 @@
}
}
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 @@
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 @@
} 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 @@
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 @@
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 @@
}
} catch (Exception e) {
- LOG.log(Level.INFO, getName() + " caught: " + e, e);
+ LOG.info(getName() + " caught: " + e, e);
}
}
LOG.info(getName() + ": exiting");
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/InputFormatBase.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/InputFormatBase.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/InputFormatBase.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/InputFormatBase.java Fri Jun 2 12:14:22 2006
@@ -20,17 +20,17 @@
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
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobClient.java Fri Jun 2 12:14:22 2006
@@ -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 java.io.*;
import java.net.*;
import java.util.*;
-import java.util.logging.*;
/*******************************************************
* JobClient interacts with the JobTracker network interface.
@@ -34,7 +35,7 @@
* @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 @@
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: " +
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java Fri Jun 2 12:14:22 2006
@@ -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 @@
// 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 @@
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 @@
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);
}
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java Fri Jun 2 12:14:22 2006
@@ -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 @@
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 @@
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 @@
// 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 @@
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 @@
// 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 @@
}
Configuration conf=new Configuration();
- LogFormatter.initFileHandler( conf, "jobtracker" );
startTracker(conf);
}
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LocalJobRunner.java Fri Jun 2 12:14:22 2006
@@ -18,16 +18,16 @@
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 @@
} 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 @@
}
public synchronized void fsError(String message) throws IOException {
- LOG.severe("FSError: "+ message);
+ LOG.fatal("FSError: "+ message);
}
}
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/ReduceTaskRunner.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/ReduceTaskRunner.java?rev=411254&r1=411253&r2=411254&view=diff
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/ReduceTaskRunner.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/ReduceTaskRunner.java Fri Jun 2 12:14:22 2006
@@ -206,9 +206,9 @@
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 @@
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 @@
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 @@
" 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 @@
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--;