MAPREDUCE-6983. Moving logging APIs over to slf4j in hadoop-mapreduce-client-core. Contributed by Jinjiang Ling.

This commit is contained in:
Akira Ajisaka 2017-11-02 17:42:52 +09:00
parent 940ffe3f9c
commit 178751ed8c
No known key found for this signature in database
GPG Key ID: C1EDBB9CA400FD50
94 changed files with 352 additions and 318 deletions

View File

@ -26,8 +26,6 @@
import java.util.List;
import java.util.NoSuchElementException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -45,6 +43,8 @@
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.CryptoUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* <code>BackupStore</code> is an utility class that is used to support
@ -60,7 +60,8 @@
@InterfaceStability.Unstable
public class BackupStore<K,V> {
private static final Log LOG = LogFactory.getLog(BackupStore.class.getName());
private static final Logger LOG =
LoggerFactory.getLogger(BackupStore.class.getName());
private static final int MAX_VINT_SIZE = 9;
private static final int EOF_MARKER_SIZE = 2 * MAX_VINT_SIZE;
private final TaskAttemptID tid;

View File

@ -21,16 +21,16 @@
import java.io.IOException;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
class CleanupQueue {
public static final Log LOG =
LogFactory.getLog(CleanupQueue.class);
public static final Logger LOG =
LoggerFactory.getLogger(CleanupQueue.class);
private static PathCleanupThread cleanupThread;

View File

@ -30,7 +30,6 @@
import java.util.Iterator;
import org.apache.commons.collections.IteratorUtils;
import org.apache.commons.logging.Log;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapreduce.FileSystemCounter;
@ -44,6 +43,7 @@
import org.apache.hadoop.mapreduce.counters.Limits;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
import org.slf4j.Logger;
import com.google.common.collect.Iterators;
@ -596,7 +596,7 @@ public static Counters sum(Counters a, Counters b) {
* Logs the current counter values.
* @param log The log to use.
*/
public void log(Log log) {
public void log(Logger log) {
log.info("Counters: " + size());
for(Group group: this) {
log.info(" " + group.getDisplayName());

View File

@ -23,8 +23,8 @@
import org.apache.hadoop.mapreduce.QueueState;
import org.apache.hadoop.security.authorize.AccessControlList;
import static org.apache.hadoop.mapred.QueueManager.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.List;
import java.util.Map;
@ -37,8 +37,8 @@
*
*/
class DeprecatedQueueConfigurationParser extends QueueConfigurationParser {
private static final Log LOG =
LogFactory.getLog(DeprecatedQueueConfigurationParser.class);
private static final Logger LOG =
LoggerFactory.getLogger(DeprecatedQueueConfigurationParser.class);
static final String MAPRED_QUEUE_NAMES_KEY = "mapred.queue.names";
DeprecatedQueueConfigurationParser(Configuration conf) {

View File

@ -30,8 +30,6 @@
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.BlockLocation;
@ -50,6 +48,8 @@
import org.apache.hadoop.util.StringUtils;
import com.google.common.collect.Iterables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A base class for file-based {@link InputFormat}.
@ -68,8 +68,8 @@
@InterfaceStability.Stable
public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
public static final Log LOG =
LogFactory.getLog(FileInputFormat.class);
public static final Logger LOG =
LoggerFactory.getLogger(FileInputFormat.class);
@Deprecated
public enum Counter {

View File

@ -20,12 +20,12 @@
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** An {@link OutputCommitter} that commits files specified
* in job output directory i.e. ${mapreduce.output.fileoutputformat.outputdir}.
@ -34,7 +34,7 @@
@InterfaceStability.Stable
public class FileOutputCommitter extends OutputCommitter {
public static final Log LOG = LogFactory.getLog(
public static final Logger LOG = LoggerFactory.getLogger(
"org.apache.hadoop.mapred.FileOutputCommitter");
/**

View File

@ -43,8 +43,8 @@
import org.apache.hadoop.io.serializer.SerializationFactory;
import org.apache.hadoop.io.serializer.Serializer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* <code>IFile</code> is the simple &lt;key-len, value-len, key, value&gt; format
@ -56,7 +56,7 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class IFile {
private static final Log LOG = LogFactory.getLog(IFile.class);
private static final Logger LOG = LoggerFactory.getLogger(IFile.class);
public static final int EOF_MARKER = -1; // End of File Marker
/**

View File

@ -25,8 +25,6 @@
import java.io.InputStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.ChecksumException;
@ -36,6 +34,8 @@
import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.util.DataChecksum;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A checksum input stream, used for IFiles.
* Used to validate the checksum of files created by {@link IFileOutputStream}.
@ -59,7 +59,8 @@ public class IFileInputStream extends InputStream {
private boolean readahead;
private int readaheadLength;
public static final Log LOG = LogFactory.getLog(IFileInputStream.class);
public static final Logger LOG =
LoggerFactory.getLogger(IFileInputStream.class);
private boolean disableChecksumValidation = false;

View File

@ -22,17 +22,17 @@
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
class IndexCache {
private final JobConf conf;
private final int totalMemoryAllowed;
private AtomicInteger totalMemoryUsed = new AtomicInteger();
private static final Log LOG = LogFactory.getLog(IndexCache.class);
private static final Logger LOG = LoggerFactory.getLogger(IndexCache.class);
private final ConcurrentHashMap<String,IndexInformation> cache =
new ConcurrentHashMap<String,IndexInformation>();

View File

@ -20,8 +20,6 @@
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobACL;
@ -29,11 +27,13 @@
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
public class JobACLsManager {
static final Log LOG = LogFactory.getLog(JobACLsManager.class);
static final Logger LOG = LoggerFactory.getLogger(JobACLsManager.class);
Configuration conf;
private final AccessControlList adminAcl;

View File

@ -24,8 +24,6 @@
import java.util.regex.Pattern;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability;
@ -53,6 +51,8 @@
import org.apache.hadoop.util.ClassUtil;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Tool;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A map/reduce job configuration.
@ -115,7 +115,7 @@
@InterfaceStability.Stable
public class JobConf extends Configuration {
private static final Log LOG = LogFactory.getLog(JobConf.class);
private static final Logger LOG = LoggerFactory.getLogger(JobConf.class);
private static final Pattern JAVA_OPTS_XMX_PATTERN =
Pattern.compile(".*(?:^|\\s)-Xmx(\\d+)([gGmMkK]?)(?:$|\\s).*");

View File

@ -24,20 +24,20 @@
import java.util.concurrent.Delayed;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.http.client.methods.HttpGet;
import org.apache.http.client.params.ClientPNames;
import org.apache.http.impl.client.DefaultHttpClient;
import org.apache.http.params.CoreConnectionPNames;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class JobEndNotifier {
private static final Log LOG =
LogFactory.getLog(JobEndNotifier.class.getName());
private static final Logger LOG =
LoggerFactory.getLogger(JobEndNotifier.class.getName());

View File

@ -24,13 +24,13 @@
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
class JvmContext implements Writable {
public static final Log LOG =
LogFactory.getLog(JvmContext.class);
public static final Logger LOG =
LoggerFactory.getLogger(JvmContext.class);
JVMId jvmId;
String pid;

View File

@ -39,8 +39,8 @@
import org.apache.hadoop.mapreduce.lib.input.CompressedSplitLineReader;
import org.apache.hadoop.mapreduce.lib.input.SplitLineReader;
import org.apache.hadoop.mapreduce.lib.input.UncompressedSplitLineReader;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.Log;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Treats keys as offset in file and value as line.
@ -48,8 +48,8 @@
@InterfaceAudience.LimitedPrivate({"MapReduce", "Pig"})
@InterfaceStability.Unstable
public class LineRecordReader implements RecordReader<LongWritable, Text> {
private static final Log LOG
= LogFactory.getLog(LineRecordReader.class.getName());
private static final Logger LOG =
LoggerFactory.getLogger(LineRecordReader.class.getName());
private CompressionCodecFactory compressionCodecs = null;
private long start;

View File

@ -32,8 +32,6 @@
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FSDataInputStream;
@ -74,6 +72,8 @@
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringInterner;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** A Map task. */
@InterfaceAudience.LimitedPrivate({"MapReduce"})
@ -87,7 +87,8 @@ public class MapTask extends Task {
private TaskSplitIndex splitMetaInfo = new TaskSplitIndex();
private final static int APPROX_HEADER_LENGTH = 150;
private static final Log LOG = LogFactory.getLog(MapTask.class.getName());
private static final Logger LOG =
LoggerFactory.getLogger(MapTask.class.getName());
private Progress mapPhase;
private Progress sortPhase;

View File

@ -23,8 +23,6 @@
import java.util.Comparator;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -45,6 +43,8 @@
import org.apache.hadoop.util.PriorityQueue;
import org.apache.hadoop.util.Progress;
import org.apache.hadoop.util.Progressable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Merger is an utility class used by the Map and Reduce tasks for merging
@ -53,7 +53,7 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class Merger {
private static final Log LOG = LogFactory.getLog(Merger.class);
private static final Logger LOG = LoggerFactory.getLogger(Merger.class);
// Local directories
private static LocalDirAllocator lDirAlloc =

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.mapred;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapreduce.QueueState;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.HashMap;
@ -36,7 +36,7 @@
*/
class Queue implements Comparable<Queue>{
private static final Log LOG = LogFactory.getLog(Queue.class);
private static final Logger LOG = LoggerFactory.getLogger(Queue.class);
//Queue name
private String name = null;
@ -348,14 +348,14 @@ boolean isHierarchySameAs(Queue newState) {
//check for the individual children and then see if all of them
//are updated.
if (newState.getChildren() == null) {
LOG.fatal("In the current state, queue " + getName() + " has "
LOG.error("In the current state, queue " + getName() + " has "
+ children.size() + " but the new state has none!");
return false;
}
int childrenSize = children.size();
int newChildrenSize = newState.getChildren().size();
if (childrenSize != newChildrenSize) {
LOG.fatal("Number of children for queue " + newState.getName()
LOG.error("Number of children for queue " + newState.getName()
+ " in newState is " + newChildrenSize + " which is not equal to "
+ childrenSize + " in the current state.");
return false;

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.mapred;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.QueueState;
@ -31,6 +29,8 @@
import org.w3c.dom.Node;
import org.w3c.dom.NodeList;
import org.w3c.dom.DOMException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import javax.xml.parsers.ParserConfigurationException;
import javax.xml.parsers.DocumentBuilderFactory;
@ -59,8 +59,8 @@
* Creates the complete queue hieararchy
*/
class QueueConfigurationParser {
private static final Log LOG =
LogFactory.getLog(QueueConfigurationParser.class);
private static final Logger LOG =
LoggerFactory.getLogger(QueueConfigurationParser.class);
private boolean aclsEnabled = false;

View File

@ -21,8 +21,6 @@
import com.fasterxml.jackson.core.JsonFactory;
import com.fasterxml.jackson.core.JsonGenerationException;
import com.fasterxml.jackson.core.JsonGenerator;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.IOUtils;
@ -31,6 +29,8 @@
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AccessControlList;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.BufferedInputStream;
import java.io.InputStream;
@ -82,7 +82,7 @@
@InterfaceAudience.Private
public class QueueManager {
private static final Log LOG = LogFactory.getLog(QueueManager.class);
private static final Logger LOG = LoggerFactory.getLogger(QueueManager.class);
// Map of a queue name and Queue object
private Map<String, Queue> leafQueues = new HashMap<String,Queue>();

View File

@ -28,8 +28,6 @@
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -56,6 +54,8 @@
import org.apache.hadoop.util.Progress;
import org.apache.hadoop.util.Progressable;
import org.apache.hadoop.util.ReflectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** A Reduce task. */
@InterfaceAudience.Private
@ -70,7 +70,8 @@ public class ReduceTask extends Task {
});
}
private static final Log LOG = LogFactory.getLog(ReduceTask.class.getName());
private static final Logger LOG =
LoggerFactory.getLogger(ReduceTask.class.getName());
private int numMaps;
private CompressionCodec codec;
@ -456,7 +457,7 @@ public void collect(OUTKEY key, OUTVALUE value)
out.close(reporter);
out = null;
} finally {
IOUtils.cleanup(LOG, reducer);
IOUtils.cleanupWithLogger(LOG, reducer);
closeQuietly(out, reporter);
}
}

View File

@ -25,9 +25,9 @@
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.Writable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Keeps the Ranges sorted by startIndex.
@ -37,8 +37,8 @@
*/
class SortedRanges implements Writable{
private static final Log LOG =
LogFactory.getLog(SortedRanges.class);
private static final Logger LOG =
LoggerFactory.getLogger(SortedRanges.class);
private TreeSet<Range> ranges = new TreeSet<Range>();
private long indicesCount;

View File

@ -35,8 +35,6 @@
import javax.crypto.SecretKey;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
@ -73,6 +71,8 @@
import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.hadoop.util.StringInterner;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Base class for tasks.
@ -80,8 +80,8 @@
@InterfaceAudience.LimitedPrivate({"MapReduce"})
@InterfaceStability.Unstable
abstract public class Task implements Writable, Configurable {
private static final Log LOG =
LogFactory.getLog(Task.class);
private static final Logger LOG =
LoggerFactory.getLogger(Task.class);
public static String MERGED_OUTPUT_PREFIX = ".merged";
public static final long DEFAULT_COMBINE_RECORDS_BEFORE_PROGRESS = 10000;
@ -355,7 +355,7 @@ protected void setWriteSkipRecs(boolean writeSkipRecs) {
*/
protected void reportFatalError(TaskAttemptID id, Throwable throwable,
String logMsg) {
LOG.fatal(logMsg);
LOG.error(logMsg);
if (ShutdownHookManager.get().isShutdownInProgress()) {
return;
@ -368,7 +368,7 @@ protected void reportFatalError(TaskAttemptID id, Throwable throwable,
try {
umbilical.fatalError(id, cause);
} catch (IOException ioe) {
LOG.fatal("Failed to contact the tasktracker", ioe);
LOG.error("Failed to contact the tasktracker", ioe);
System.exit(-1);
}
}
@ -849,13 +849,13 @@ public void run() {
} catch (TaskLimitException e) {
String errMsg = "Task exceeded the limits: " +
StringUtils.stringifyException(e);
LOG.fatal(errMsg);
LOG.error(errMsg);
try {
umbilical.fatalError(taskId, errMsg);
} catch (IOException ioe) {
LOG.fatal("Failed to update failure diagnosis", ioe);
LOG.error("Failed to update failure diagnosis", ioe);
}
LOG.fatal("Killing " + taskId);
LOG.error("Killing " + taskId);
resetDoneFlag();
ExitUtil.terminate(69);
} catch (Throwable t) {

View File

@ -35,8 +35,6 @@
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@ -56,6 +54,7 @@
import org.apache.log4j.Appender;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.base.Charsets;
@ -66,8 +65,8 @@
*/
@InterfaceAudience.Private
public class TaskLog {
private static final Log LOG =
LogFactory.getLog(TaskLog.class);
private static final org.slf4j.Logger LOG =
LoggerFactory.getLogger(TaskLog.class);
static final String USERLOGS_DIR_NAME = "userlogs";
@ -156,7 +155,7 @@ private static LogFileDetail getLogFileDetail(TaskAttemptID taskid,
fis.close();
fis = null;
} finally {
IOUtils.cleanup(LOG, fis);
IOUtils.cleanupWithLogger(LOG, fis);
}
return l;
}
@ -231,7 +230,7 @@ void writeToIndexFile(String logLocation,
bos.close();
bos = null;
} finally {
IOUtils.cleanup(LOG, dos, bos);
IOUtils.cleanupWithLogger(LOG, dos, bos);
}
File indexFile = getIndexFile(currentTaskid, isCleanup);

View File

@ -22,8 +22,6 @@
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Text;
@ -31,6 +29,8 @@
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.util.StringInterner;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**************************************************
* Describes the current status of a task. This is
* not intended to be a comprehensive piece of data.
@ -39,8 +39,8 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
public abstract class TaskStatus implements Writable, Cloneable {
static final Log LOG =
LogFactory.getLog(TaskStatus.class.getName());
static final Logger LOG =
LoggerFactory.getLogger(TaskStatus.class.getName());
//enumeration for reporting current phase of a task.
@InterfaceAudience.Private

View File

@ -23,19 +23,19 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.JobClient;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.JobID;
import org.apache.hadoop.mapreduce.lib.jobcontrol.ControlledJob;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Public
@InterfaceStability.Stable
public class Job extends ControlledJob {
static final Log LOG = LogFactory.getLog(Job.class);
static final Logger LOG = LoggerFactory.getLogger(Job.class);
final public static int SUCCESS = 0;
final public static int WAITING = 1;

View File

@ -23,8 +23,6 @@
import java.util.Iterator;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Text;
@ -35,6 +33,8 @@
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.TextInputFormat;
import org.apache.hadoop.mapreduce.lib.fieldsel.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class implements a mapper/reducer class that can be used to perform
@ -92,7 +92,8 @@ public class FieldSelectionMapReduce<K, V>
private int allReduceValueFieldsFrom = -1;
public static final Log LOG = LogFactory.getLog("FieldSelectionMapReduce");
public static final Logger LOG =
LoggerFactory.getLogger("FieldSelectionMapReduce");
private String specToString() {
StringBuffer sb = new StringBuffer();

View File

@ -22,8 +22,6 @@
import java.util.ArrayList;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.InputFormat;
@ -32,13 +30,15 @@
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapreduce.Job;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Public
@InterfaceStability.Stable
public class InputSampler<K,V> extends
org.apache.hadoop.mapreduce.lib.partition.InputSampler<K, V> {
private static final Log LOG = LogFactory.getLog(InputSampler.class);
private static final Logger LOG = LoggerFactory.getLogger(InputSampler.class);
public InputSampler(JobConf conf) {
super(conf);

View File

@ -29,9 +29,9 @@
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.SkipBadRecords;
import org.apache.hadoop.mapreduce.lib.map.MultithreadedMapper;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.util.concurrent.HadoopThreadPoolExecutor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.concurrent.*;
@ -57,8 +57,8 @@
public class MultithreadedMapRunner<K1, V1, K2, V2>
implements MapRunnable<K1, V1, K2, V2> {
private static final Log LOG =
LogFactory.getLog(MultithreadedMapRunner.class.getName());
private static final Logger LOG =
LoggerFactory.getLogger(MultithreadedMapRunner.class.getName());
private JobConf job;
private Mapper<K1, V1, K2, V2> mapper;

View File

@ -30,8 +30,6 @@
import javax.crypto.SecretKey;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
@ -56,6 +54,8 @@
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class is responsible for launching and communicating with the child
@ -63,7 +63,8 @@
*/
class Application<K1 extends WritableComparable, V1 extends Writable,
K2 extends WritableComparable, V2 extends Writable> {
private static final Log LOG = LogFactory.getLog(Application.class.getName());
private static final Logger LOG =
LoggerFactory.getLogger(Application.class.getName());
private ServerSocket serverSocket;
private Process process;
private Socket clientSocket;

View File

@ -32,8 +32,6 @@
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.IOUtils;
@ -44,6 +42,8 @@
import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This protocol is a binary implementation of the Pipes protocol.
@ -60,8 +60,8 @@ class BinaryProtocol<K1 extends WritableComparable, V1 extends Writable,
private DataOutputStream stream;
private DataOutputBuffer buffer = new DataOutputBuffer();
private static final Log LOG =
LogFactory.getLog(BinaryProtocol.class.getName());
private static final Logger LOG =
LoggerFactory.getLogger(BinaryProtocol.class.getName());
private UplinkReaderThread uplink;
/**

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.mapred.pipes;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.mapred.JobConf;
@ -28,6 +26,8 @@
import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.SkipBadRecords;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Iterator;
@ -38,7 +38,8 @@
class PipesReducer<K2 extends WritableComparable, V2 extends Writable,
K3 extends WritableComparable, V3 extends Writable>
implements Reducer<K2, V2, K3, V3> {
private static final Log LOG= LogFactory.getLog(PipesReducer.class.getName());
private static final Logger LOG =
LoggerFactory.getLogger(PipesReducer.class.getName());
private JobConf job;
private Application<K2, V2, K3, V3> application = null;
private DownwardProtocol<K2, V2> downlink = null;

View File

@ -34,8 +34,6 @@
import org.apache.commons.cli.Options;
import org.apache.commons.cli.ParseException;
import org.apache.commons.cli.Parser;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -61,6 +59,8 @@
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.util.GenericOptionsParser;
import org.apache.hadoop.util.Tool;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The main entry point and job submitter. It may either be used as a command
@ -70,7 +70,7 @@
@InterfaceStability.Stable
public class Submitter extends Configured implements Tool {
protected static final Log LOG = LogFactory.getLog(Submitter.class);
protected static final Logger LOG = LoggerFactory.getLogger(Submitter.class);
public static final String PRESERVE_COMMANDFILE =
"mapreduce.pipes.commandfile.preserve";
public static final String EXECUTABLE = "mapreduce.pipes.executable";

View File

@ -28,8 +28,6 @@
import java.util.ServiceLoader;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -45,6 +43,8 @@
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Provides a way to access information about the map/reduce cluster.
@ -64,7 +64,8 @@ public enum JobTrackerStatus {INITIALIZING, RUNNING};
private Path sysDir = null;
private Path stagingAreaDir = null;
private Path jobHistoryDir = null;
private static final Log LOG = LogFactory.getLog(Cluster.class);
private static final Logger LOG =
LoggerFactory.getLogger(Cluster.class);
@VisibleForTesting
static Iterable<ClientProtocolProvider> frameworkLoader =

View File

@ -22,8 +22,6 @@
import java.nio.ByteBuffer;
import org.apache.commons.codec.binary.Base64;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -37,6 +35,8 @@
import org.apache.hadoop.mapreduce.security.TokenCache;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.LimitInputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class provides utilities to make it easier to work with Cryptographic
@ -47,7 +47,7 @@
@InterfaceStability.Unstable
public class CryptoUtils {
private static final Log LOG = LogFactory.getLog(CryptoUtils.class);
private static final Logger LOG = LoggerFactory.getLogger(CryptoUtils.class);
public static boolean isEncryptedSpillEnabled(Configuration conf) {
return conf.getBoolean(MRJobConfig.MR_ENCRYPTED_INTERMEDIATE_DATA,

View File

@ -26,8 +26,6 @@
import java.util.LinkedHashMap;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@ -44,6 +42,8 @@
import org.apache.hadoop.mapreduce.util.ConfigUtil;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ReservationId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The job submitter's view of the Job.
@ -81,7 +81,7 @@
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class Job extends JobContextImpl implements JobContext, AutoCloseable {
private static final Log LOG = LogFactory.getLog(Job.class);
private static final Logger LOG = LoggerFactory.getLogger(Job.class);
@InterfaceStability.Evolving
public enum JobState {DEFINE, RUNNING};

View File

@ -27,8 +27,6 @@
import java.util.LinkedHashMap;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.conf.Configuration;
@ -45,6 +43,8 @@
import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.client.api.SharedCacheClient;
import org.apache.hadoop.yarn.exceptions.YarnException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
@ -55,7 +55,8 @@
@Private
@Unstable
class JobResourceUploader {
protected static final Log LOG = LogFactory.getLog(JobResourceUploader.class);
protected static final Logger LOG =
LoggerFactory.getLogger(JobResourceUploader.class);
private final boolean useWildcard;
private final FileSystem jtFs;
private SharedCacheClient scClient = null;

View File

@ -30,8 +30,8 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A utility to manage job submission files.
@ -39,7 +39,8 @@
@InterfaceAudience.Private
public class JobSubmissionFiles {
private final static Log LOG = LogFactory.getLog(JobSubmissionFiles.class);
private final static Logger LOG =
LoggerFactory.getLogger(JobSubmissionFiles.class);
// job submission directory is private!
final public static FsPermission JOB_DIR_PERMISSION =

View File

@ -36,8 +36,6 @@
import com.fasterxml.jackson.databind.JsonMappingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectReader;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -49,6 +47,8 @@
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.QueueACL;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.apache.hadoop.mapred.QueueManager.toFullPropertyName;
@ -69,7 +69,8 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
class JobSubmitter {
protected static final Log LOG = LogFactory.getLog(JobSubmitter.class);
protected static final Logger LOG =
LoggerFactory.getLogger(JobSubmitter.class);
private static final ObjectReader READER =
new ObjectMapper().readerFor(Map.class);
private static final String SHUFFLE_KEYGEN_ALGORITHM = "HmacSHA1";
@ -298,9 +299,7 @@ private void writeConf(Configuration conf, Path jobFile)
private void printTokens(JobID jobId,
Credentials credentials) throws IOException {
LOG.info("Submitting tokens for job: " + jobId);
for (Token<?> token: credentials.getAllTokens()) {
LOG.info(token);
}
LOG.info("Executing with tokens: {}", credentials.getAllTokens());
}
@SuppressWarnings("unchecked")

View File

@ -29,8 +29,6 @@
import java.util.Map;
import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.Text;
@ -41,6 +39,8 @@
import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.TaskCounter;
import org.apache.hadoop.util.StringInterner;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
@ -59,7 +59,8 @@ public abstract class AbstractCounters<C extends Counter,
G extends CounterGroupBase<C>>
implements Writable, Iterable<G> {
protected static final Log LOG = LogFactory.getLog("mapreduce.Counters");
protected static final Logger LOG =
LoggerFactory.getLogger("mapreduce.Counters");
/**
* A cache from enum values to the associated counter.

View File

@ -33,8 +33,6 @@
import com.google.common.collect.Iterators;
import com.google.common.collect.Maps;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.io.WritableUtils;
@ -42,6 +40,8 @@
import org.apache.hadoop.mapreduce.FileSystemCounter;
import org.apache.hadoop.mapreduce.util.ResourceBundles;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* An abstract class to provide common implementation of the filesystem
@ -56,7 +56,8 @@ public abstract class FileSystemCounterGroup<C extends Counter>
static final int MAX_NUM_SCHEMES = 100; // intern/sanity check
static final ConcurrentMap<String, String> schemes = Maps.newConcurrentMap();
private static final Log LOG = LogFactory.getLog(FileSystemCounterGroup.class);
private static final Logger LOG =
LoggerFactory.getLogger(FileSystemCounterGroup.class);
// C[] would need Array.newInstance which requires a Class<C> reference.
// Just a few local casts probably worth not having to carry it around.

View File

@ -26,13 +26,13 @@
import java.util.Arrays;
import java.util.Iterator;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.io.WritableUtils;
import org.apache.hadoop.mapreduce.Counter;
import org.apache.hadoop.mapreduce.util.ResourceBundles;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.AbstractIterator;
import com.google.common.collect.Iterators;
@ -47,7 +47,8 @@
@InterfaceAudience.Private
public abstract class FrameworkCounterGroup<T extends Enum<T>,
C extends Counter> implements CounterGroupBase<C> {
private static final Log LOG = LogFactory.getLog(FrameworkCounterGroup.class);
private static final Logger LOG =
LoggerFactory.getLogger(FrameworkCounterGroup.class);
private final Class<T> enumClass; // for Enum.valueOf
private final Object[] counters; // local casts are OK and save a class ref

View File

@ -27,8 +27,6 @@
import org.apache.avro.io.EncoderFactory;
import org.apache.avro.specific.SpecificDatumWriter;
import org.apache.avro.util.Utf8;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -36,6 +34,8 @@
import org.apache.hadoop.mapreduce.Counter;
import org.apache.hadoop.mapreduce.CounterGroup;
import org.apache.hadoop.mapreduce.Counters;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
@ -55,7 +55,7 @@ public class EventWriter {
private DatumWriter<Event> writer =
new SpecificDatumWriter<Event>(Event.class);
private Encoder encoder;
private static final Log LOG = LogFactory.getLog(EventWriter.class);
private static final Logger LOG = LoggerFactory.getLogger(EventWriter.class);
/**
* avro encoding format supported by EventWriter.
@ -112,7 +112,7 @@ public void close() throws IOException {
out.close();
out = null;
} finally {
IOUtils.cleanup(LOG, out);
IOUtils.cleanupWithLogger(LOG, out);
}
}

View File

@ -24,8 +24,6 @@
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@ -45,6 +43,8 @@
import org.apache.hadoop.util.StringInterner;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Default Parser for the JobHistory files. Typical usage is
@ -56,7 +56,8 @@
@InterfaceStability.Unstable
public class JobHistoryParser implements HistoryEventHandler {
private static final Log LOG = LogFactory.getLog(JobHistoryParser.class);
private static final Logger LOG =
LoggerFactory.getLogger(JobHistoryParser.class);
private final FSDataInputStream in;
private JobInfo info = null;

View File

@ -24,8 +24,8 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -39,7 +39,8 @@
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class BigDecimalSplitter implements DBSplitter {
private static final Log LOG = LogFactory.getLog(BigDecimalSplitter.class);
private static final Logger LOG =
LoggerFactory.getLogger(BigDecimalSplitter.class);
public List<InputSplit> split(Configuration conf, ResultSet results, String colName)
throws SQLException {

View File

@ -30,8 +30,6 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
@ -46,6 +44,8 @@
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A InputFormat that reads input data from an SQL table.
@ -61,7 +61,8 @@
public class DBInputFormat<T extends DBWritable>
extends InputFormat<LongWritable, T> implements Configurable {
private static final Log LOG = LogFactory.getLog(DBInputFormat.class);
private static final Logger LOG =
LoggerFactory.getLogger(DBInputFormat.class);
protected String dbProductName = "DEFAULT";

View File

@ -24,8 +24,6 @@
import java.sql.PreparedStatement;
import java.sql.SQLException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapreduce.Job;
@ -37,6 +35,8 @@
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A OutputFormat that sends the reduce output to a SQL table.
@ -51,7 +51,8 @@
public class DBOutputFormat<K extends DBWritable, V>
extends OutputFormat<K,V> {
private static final Log LOG = LogFactory.getLog(DBOutputFormat.class);
private static final Logger LOG =
LoggerFactory.getLogger(DBOutputFormat.class);
public String dbProductName = "DEFAULT";
public void checkOutputSpecs(JobContext context)

View File

@ -30,8 +30,6 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapreduce.InputFormat;
@ -45,6 +43,8 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A RecordReader that reads records from a SQL table.
@ -56,7 +56,8 @@
public class DBRecordReader<T extends DBWritable> extends
RecordReader<LongWritable, T> {
private static final Log LOG = LogFactory.getLog(DBRecordReader.class);
private static final Logger LOG =
LoggerFactory.getLogger(DBRecordReader.class);
private ResultSet results = null;

View File

@ -31,8 +31,8 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
@ -61,7 +61,8 @@
public class DataDrivenDBInputFormat<T extends DBWritable>
extends DBInputFormat<T> implements Configurable {
private static final Log LOG = LogFactory.getLog(DataDrivenDBInputFormat.class);
private static final Logger LOG =
LoggerFactory.getLogger(DataDrivenDBInputFormat.class);
/** If users are providing their own query, the following string is expected to
appear in the WHERE clause, which will be substituted with a pair of conditions

View File

@ -30,8 +30,6 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapreduce.InputFormat;
@ -45,6 +43,8 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A RecordReader that reads records from a SQL table,
@ -56,7 +56,8 @@
@InterfaceStability.Evolving
public class DataDrivenDBRecordReader<T extends DBWritable> extends DBRecordReader<T> {
private static final Log LOG = LogFactory.getLog(DataDrivenDBRecordReader.class);
private static final Logger LOG =
LoggerFactory.getLogger(DataDrivenDBRecordReader.class);
private String dbProductName; // database manufacturer string.

View File

@ -27,8 +27,8 @@
import java.util.Date;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -45,7 +45,7 @@
@InterfaceStability.Evolving
public class DateSplitter extends IntegerSplitter {
private static final Log LOG = LogFactory.getLog(DateSplitter.class);
private static final Logger LOG = LoggerFactory.getLogger(DateSplitter.class);
public List<InputSplit> split(Configuration conf, ResultSet results, String colName)
throws SQLException {

View File

@ -23,8 +23,8 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -39,7 +39,8 @@
@InterfaceStability.Evolving
public class FloatSplitter implements DBSplitter {
private static final Log LOG = LogFactory.getLog(FloatSplitter.class);
private static final Logger LOG =
LoggerFactory.getLogger(FloatSplitter.class);
private static final double MIN_INCREMENT = 10000 * Double.MIN_VALUE;

View File

@ -26,8 +26,8 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A RecordReader that reads records from an Oracle SQL table.
@ -39,7 +39,8 @@ public class OracleDBRecordReader<T extends DBWritable> extends DBRecordReader<T
/** Configuration key to set to a timezone string. */
public static final String SESSION_TIMEZONE_KEY = "oracle.sessionTimeZone";
private static final Log LOG = LogFactory.getLog(OracleDBRecordReader.class);
private static final Logger LOG =
LoggerFactory.getLogger(OracleDBRecordReader.class);
public OracleDBRecordReader(DBInputFormat.DBInputSplit split,
Class<T> inputClass, Configuration conf, Connection conn, DBConfiguration dbConfig,

View File

@ -31,9 +31,6 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.Writable;

View File

@ -25,8 +25,8 @@
import java.util.Iterator;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -41,7 +41,7 @@
@InterfaceStability.Evolving
public class TextSplitter extends BigDecimalSplitter {
private static final Log LOG = LogFactory.getLog(TextSplitter.class);
private static final Logger LOG = LoggerFactory.getLogger(TextSplitter.class);
/**
* This method needs to determine the splits between two user-provided strings.

View File

@ -22,14 +22,14 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class implements a mapper class that can be used to perform
@ -73,7 +73,8 @@ public class FieldSelectionMapper<K, V>
private int allMapValueFieldsFrom = -1;
public static final Log LOG = LogFactory.getLog("FieldSelectionMapReduce");
public static final Logger LOG =
LoggerFactory.getLogger("FieldSelectionMapReduce");
public void setup(Context context)
throws IOException, InterruptedException {

View File

@ -22,13 +22,13 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Reducer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class implements a reducer class that can be used to perform field
@ -70,7 +70,8 @@ public class FieldSelectionReducer<K, V>
private int allReduceValueFieldsFrom = -1;
public static final Log LOG = LogFactory.getLog("FieldSelectionMapReduce");
public static final Logger LOG =
LoggerFactory.getLogger("FieldSelectionMapReduce");
public void setup(Context context)
throws IOException, InterruptedException {

View File

@ -30,8 +30,6 @@
import java.util.Iterator;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -51,6 +49,8 @@
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.net.NodeBase;
import org.apache.hadoop.net.NetworkTopology;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.HashMultiset;
@ -84,7 +84,8 @@
public abstract class CombineFileInputFormat<K, V>
extends FileInputFormat<K, V> {
private static final Log LOG = LogFactory.getLog(CombineFileInputFormat.class);
private static final Logger LOG =
LoggerFactory.getLogger(CombineFileInputFormat.class);
public static final String SPLIT_MINSIZE_PERNODE =
"mapreduce.input.fileinputformat.split.minsize.per.node";

View File

@ -23,8 +23,6 @@
import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -45,6 +43,8 @@
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StopWatch;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.Lists;
@ -80,7 +80,8 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
"mapreduce.input.fileinputformat.list-status.num-threads";
public static final int DEFAULT_LIST_STATUS_NUM_THREADS = 1;
private static final Log LOG = LogFactory.getLog(FileInputFormat.class);
private static final Logger LOG =
LoggerFactory.getLogger(FileInputFormat.class);
private static final double SPLIT_SLOP = 1.1; // 10% slop

View File

@ -38,8 +38,8 @@
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.Log;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A reader to read fixed length records from a split. Record offset is
@ -49,8 +49,8 @@
@InterfaceStability.Evolving
public class FixedLengthRecordReader
extends RecordReader<LongWritable, BytesWritable> {
private static final Log LOG
= LogFactory.getLog(FixedLengthRecordReader.class);
private static final Logger LOG
= LoggerFactory.getLogger(FixedLengthRecordReader.class);
private int recordLength;
private long start;

View File

@ -38,8 +38,8 @@
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.Log;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Treats keys as offset in file and value as line.
@ -47,7 +47,8 @@
@InterfaceAudience.LimitedPrivate({"MapReduce", "Pig"})
@InterfaceStability.Evolving
public class LineRecordReader extends RecordReader<LongWritable, Text> {
private static final Log LOG = LogFactory.getLog(LineRecordReader.class);
private static final Logger LOG =
LoggerFactory.getLogger(LineRecordReader.class);
public static final String MAX_LINE_LENGTH =
"mapreduce.input.linerecordreader.line.maxlength";

View File

@ -26,8 +26,6 @@
import java.util.regex.Pattern;
import java.util.regex.PatternSyntaxException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
@ -39,6 +37,8 @@
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.util.ReflectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A class that allows a map/red job to work on a sample of sequence files.
@ -48,7 +48,8 @@
@InterfaceStability.Stable
public class SequenceFileInputFilter<K, V>
extends SequenceFileInputFormat<K, V> {
public static final Log LOG = LogFactory.getLog(FileInputFormat.class);
public static final Logger LOG =
LoggerFactory.getLogger(FileInputFormat.class);
final public static String FILTER_CLASS =
"mapreduce.input.sequencefileinputfilter.class";
@ -260,7 +261,7 @@ public boolean accept(Object key) {
if (hashcode / frequency * frequency == hashcode)
return true;
} catch(Exception e) {
LOG.warn(e);
LOG.warn(e.toString());
throw new RuntimeException(e);
}
return false;

View File

@ -23,8 +23,6 @@
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -34,6 +32,8 @@
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class encapsulates a MapReduce job and its dependency. It monitors
@ -49,7 +49,8 @@
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class ControlledJob {
private static final Log LOG = LogFactory.getLog(ControlledJob.class);
private static final Logger LOG =
LoggerFactory.getLogger(ControlledJob.class);
// A job will be in one of the following states
public enum State {SUCCESS, WAITING, RUNNING, READY, FAILED,

View File

@ -27,13 +27,13 @@
import java.util.HashMap;
import java.util.HashSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.mapred.jobcontrol.Job;
import org.apache.hadoop.mapreduce.lib.jobcontrol.ControlledJob.State;
import org.apache.hadoop.util.StringUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class encapsulates a set of MapReduce jobs and its dependency.
@ -55,7 +55,7 @@
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class JobControl implements Runnable {
private static final Log LOG = LogFactory.getLog(JobControl.class);
private static final Logger LOG = LoggerFactory.getLogger(JobControl.class);
// The thread can be in one of the following state
public enum ThreadState {RUNNING, SUSPENDED,STOPPED, STOPPING, READY};

View File

@ -33,8 +33,8 @@
import org.apache.hadoop.mapreduce.StatusReporter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.task.MapContextImpl;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
@ -61,7 +61,8 @@
public class MultithreadedMapper<K1, V1, K2, V2>
extends Mapper<K1, V1, K2, V2> {
private static final Log LOG = LogFactory.getLog(MultithreadedMapper.class);
private static final Logger LOG =
LoggerFactory.getLogger(MultithreadedMapper.class);
public static String NUM_THREADS = "mapreduce.mapper.multithreadedmapper.threads";
public static String MAP_CLASS = "mapreduce.mapper.multithreadedmapper.mapclass";

View File

@ -21,8 +21,6 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability;
@ -40,6 +38,8 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** An {@link OutputCommitter} that commits files specified
* in job output directory i.e. ${mapreduce.output.fileoutputformat.outputdir}.
@ -47,7 +47,8 @@
@InterfaceAudience.Public
@InterfaceStability.Stable
public class FileOutputCommitter extends PathOutputCommitter {
private static final Log LOG = LogFactory.getLog(FileOutputCommitter.class);
private static final Logger LOG =
LoggerFactory.getLogger(FileOutputCommitter.class);
/**
* Name of directory where pending data is placed. Data that has not been

View File

@ -20,8 +20,6 @@
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -33,6 +31,8 @@
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.TaskID;
import org.apache.hadoop.mapreduce.task.annotation.Checkpointable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
@ -45,8 +45,8 @@
public class PartialFileOutputCommitter
extends FileOutputCommitter implements PartialOutputCommitter {
private static final Log LOG =
LogFactory.getLog(PartialFileOutputCommitter.class);
private static final Logger LOG =
LoggerFactory.getLogger(PartialFileOutputCommitter.class);
public PartialFileOutputCommitter(Path outputPath,

View File

@ -24,8 +24,6 @@
import java.util.List;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -47,6 +45,8 @@
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Utility for collecting samples and writing a partition file for
@ -56,7 +56,7 @@
@InterfaceStability.Stable
public class InputSampler<K,V> extends Configured implements Tool {
private static final Log LOG = LogFactory.getLog(InputSampler.class);
private static final Logger LOG = LoggerFactory.getLogger(InputSampler.class);
static int printUsage() {
System.out.println("sampler -r <reduces>\n" +

View File

@ -21,8 +21,6 @@
import java.io.UnsupportedEncodingException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
@ -32,6 +30,8 @@
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.Partitioner;
import org.apache.hadoop.mapreduce.lib.partition.KeyFieldHelper.KeyDescription;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Defines a way to partition keys based on certain key fields (also see
@ -51,7 +51,7 @@
public class KeyFieldBasedPartitioner<K2, V2> extends Partitioner<K2, V2>
implements Configurable {
private static final Log LOG = LogFactory.getLog(
private static final Logger LOG = LoggerFactory.getLogger(
KeyFieldBasedPartitioner.class.getName());
public static String PARTITIONER_OPTIONS =
"mapreduce.partition.keypartitioner.options";

View File

@ -23,8 +23,6 @@
import java.util.ArrayList;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configurable;
@ -40,6 +38,8 @@
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.Partitioner;
import org.apache.hadoop.util.ReflectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Partitioner effecting a total order by reading split points from
@ -59,7 +59,8 @@ public class TotalOrderPartitioner<K,V>
public static final String NATURAL_ORDER =
"mapreduce.totalorderpartitioner.naturalorder";
Configuration conf;
private static final Log LOG = LogFactory.getLog(TotalOrderPartitioner.class);
private static final Logger LOG =
LoggerFactory.getLogger(TotalOrderPartitioner.class);
public TotalOrderPartitioner() { }
@ -311,7 +312,7 @@ private K[] readPartitions(FileSystem fs, Path p, Class<K> keyClass,
reader.close();
reader = null;
} finally {
IOUtils.cleanup(LOG, reader);
IOUtils.cleanupWithLogger(LOG, reader);
}
return parts.toArray((K[])Array.newInstance(keyClass, parts.size()));
}

View File

@ -27,12 +27,12 @@
import javax.servlet.http.HttpServletRequest;
import org.apache.commons.codec.binary.Base64;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.WritableComparator;
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.base.Charsets;
@ -44,7 +44,8 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class SecureShuffleUtils {
private static final Log LOG = LogFactory.getLog(SecureShuffleUtils.class);
private static final Logger LOG =
LoggerFactory.getLogger(SecureShuffleUtils.class);
public static final String HTTP_HEADER_URL_HASH = "UrlHash";
public static final String HTTP_HEADER_REPLY_URL_HASH = "ReplyHash";

View File

@ -22,8 +22,6 @@
import java.util.HashSet;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@ -38,6 +36,8 @@
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
@ -50,7 +50,7 @@
@InterfaceStability.Evolving
public class TokenCache {
private static final Log LOG = LogFactory.getLog(TokenCache.class);
private static final Logger LOG = LoggerFactory.getLogger(TokenCache.class);
/**

View File

@ -40,8 +40,8 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The class that is used by the Job clients to write splits (both the meta
@ -51,7 +51,8 @@
@InterfaceStability.Unstable
public class JobSplitWriter {
private static final Log LOG = LogFactory.getLog(JobSplitWriter.class);
private static final Logger LOG =
LoggerFactory.getLogger(JobSplitWriter.class);
private static final int splitVersion = JobSplit.META_SPLIT_VERSION;
private static final byte[] SPLIT_FILE_HEADER;

View File

@ -19,18 +19,18 @@
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapred.MapTaskCompletionEventsUpdate;
import org.apache.hadoop.mapred.TaskCompletionEvent;
import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
class EventFetcher<K,V> extends Thread {
private static final long SLEEP_TIME = 1000;
private static final int MAX_RETRIES = 10;
private static final int RETRY_PERIOD = 5000;
private static final Log LOG = LogFactory.getLog(EventFetcher.class);
private static final Logger LOG = LoggerFactory.getLogger(EventFetcher.class);
private final TaskAttemptID reduce;
private final TaskUmbilicalProtocol umbilical;

View File

@ -35,8 +35,6 @@
import javax.crypto.SecretKey;
import javax.net.ssl.HttpsURLConnection;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.mapred.Counters;
import org.apache.hadoop.mapred.JobConf;
@ -49,12 +47,14 @@
import org.apache.hadoop.security.ssl.SSLFactory;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
class Fetcher<K,V> extends Thread {
private static final Log LOG = LogFactory.getLog(Fetcher.class);
private static final Logger LOG = LoggerFactory.getLogger(Fetcher.class);
/** Number of ms before timing out a copy */
private static final int DEFAULT_STALLED_COPY_TIMEOUT = 3 * 60 * 1000;
@ -341,7 +341,7 @@ protected void copyFromHost(MapHost host) throws IOException {
try {
failedTasks = copyMapOutput(host, input, remaining, fetchRetryEnabled);
} catch (IOException e) {
IOUtils.cleanup(LOG, input);
IOUtils.cleanupWithLogger(LOG, input);
//
// Setup connection again if disconnected by NM
connection.disconnect();
@ -371,7 +371,7 @@ protected void copyFromHost(MapHost host) throws IOException {
input = null;
} finally {
if (input != null) {
IOUtils.cleanup(LOG, input);
IOUtils.cleanupWithLogger(LOG, input);
input = null;
}
for (TaskAttemptID left : remaining) {

View File

@ -23,9 +23,6 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.BoundedByteArrayOutputStream;
@ -40,10 +37,14 @@
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
@InterfaceStability.Unstable
class InMemoryMapOutput<K, V> extends IFileWrappedMapOutput<K, V> {
private static final Log LOG = LogFactory.getLog(InMemoryMapOutput.class);
private static final Logger LOG =
LoggerFactory.getLogger(InMemoryMapOutput.class);
private final byte[] memory;
private BoundedByteArrayOutputStream byteStream;
// Decompression of map-outputs

View File

@ -25,8 +25,6 @@
import javax.crypto.SecretKey;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -38,6 +36,8 @@
import org.apache.hadoop.mapred.SpillRecord;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.CryptoUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* LocalFetcher is used by LocalJobRunner to perform a local filesystem
@ -45,7 +45,7 @@
*/
class LocalFetcher<K,V> extends Fetcher<K, V> {
private static final Log LOG = LogFactory.getLog(LocalFetcher.class);
private static final Logger LOG = LoggerFactory.getLogger(LocalFetcher.class);
private static final MapHost LOCALHOST = new MapHost("local", "local");
@ -156,7 +156,7 @@ private boolean copyMapOutput(TaskAttemptID mapTaskId) throws IOException {
mapOutput.shuffle(LOCALHOST, inStream, compressedLength,
decompressedLength, metrics, reporter);
} finally {
IOUtils.cleanup(LOG, inStream);
IOUtils.cleanupWithLogger(LOG, inStream);
}
scheduler.copySucceeded(mapTaskId, LOCALHOST, compressedLength, 0, 0,

View File

@ -25,8 +25,6 @@
import java.util.Set;
import java.util.TreeSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.fs.ChecksumFileSystem;
@ -59,6 +57,8 @@
import org.apache.hadoop.mapreduce.task.reduce.MapOutput.MapOutputComparator;
import org.apache.hadoop.util.Progress;
import org.apache.hadoop.util.ReflectionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
@ -67,7 +67,8 @@
@InterfaceStability.Unstable
public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
private static final Log LOG = LogFactory.getLog(MergeManagerImpl.class);
private static final Logger LOG =
LoggerFactory.getLogger(MergeManagerImpl.class);
/* Maximum percentage of the in-memory limit that a single shuffle can
* consume*/

View File

@ -25,12 +25,12 @@
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
abstract class MergeThread<T,K,V> extends Thread {
private static final Log LOG = LogFactory.getLog(MergeThread.class);
private static final Logger LOG = LoggerFactory.getLogger(MergeThread.class);
private AtomicInteger numPending = new AtomicInteger(0);
private LinkedList<List<T>> pendingToBeMerged;

View File

@ -23,8 +23,8 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -45,7 +45,8 @@
@InterfaceAudience.Private
@InterfaceStability.Unstable
class OnDiskMapOutput<K, V> extends IFileWrappedMapOutput<K, V> {
private static final Log LOG = LogFactory.getLog(OnDiskMapOutput.class);
private static final Logger LOG =
LoggerFactory.getLogger(OnDiskMapOutput.class);
private final FileSystem fs;
private final Path tmpOutputPath;
private final Path outputPath;
@ -120,7 +121,7 @@ protected void doShuffle(MapHost host, IFileInputStream input,
disk.close();
} catch (IOException ioe) {
// Close the streams
IOUtils.cleanup(LOG, disk);
IOUtils.cleanupWithLogger(LOG, disk);
// Re-throw
throw ioe;

View File

@ -36,8 +36,6 @@
import java.util.concurrent.TimeUnit;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.io.IntWritable;
@ -51,6 +49,8 @@
import org.apache.hadoop.mapreduce.task.reduce.MapHost.State;
import org.apache.hadoop.util.Progress;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
@InterfaceStability.Unstable
@ -62,7 +62,8 @@ protected Long initialValue() {
}
};
private static final Log LOG = LogFactory.getLog(ShuffleSchedulerImpl.class);
private static final Logger LOG =
LoggerFactory.getLogger(ShuffleSchedulerImpl.class);
private static final int MAX_MAPS_AT_ONCE = 20;
private static final long INITIAL_PENALTY = 10000;
private static final float PENALTY_GROWTH_RATE = 1.3f;
@ -389,7 +390,7 @@ private void checkReducerHealth() {
failureCounts.size() == (totalMaps - doneMaps))
&& !reducerHealthy
&& (!reducerProgressedEnough || reducerStalled)) {
LOG.fatal("Shuffle failed with too many fetch failures " +
LOG.error("Shuffle failed with too many fetch failures " +
"and insufficient progress!");
String errorMsg = "Exceeded MAX_FAILED_UNIQUE_FETCHES; bailing-out.";
reporter.reportException(new IOException(errorMsg));

View File

@ -32,8 +32,6 @@
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@ -63,6 +61,8 @@
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.hadoop.yarn.logaggregation.LogCLIHelpers;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.base.Charsets;
@ -72,7 +72,7 @@
@InterfaceAudience.Public
@InterfaceStability.Stable
public class CLI extends Configured implements Tool {
private static final Log LOG = LogFactory.getLog(CLI.class);
private static final Logger LOG = LoggerFactory.getLogger(CLI.class);
protected Cluster cluster;
private final Set<String> taskStates = new HashSet<String>(
Arrays.asList("pending", "running", "completed", "failed", "killed"));
@ -167,7 +167,7 @@ public int run(String[] argv) throws Exception {
try {
jpvalue = Integer.parseInt(argv[2]);
} catch (NumberFormatException ne) {
LOG.info(ne);
LOG.info("Error number format: ", ne);
displayUsage(cmd);
return exitCode;
}

View File

@ -21,13 +21,12 @@
import java.io.IOException;
import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.util.Shell.ExitCodeException;
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Process tree related operations
@ -36,7 +35,7 @@
@InterfaceStability.Unstable
public class ProcessTree {
private static final Log LOG = LogFactory.getLog(ProcessTree.class);
private static final Logger LOG = LoggerFactory.getLogger(ProcessTree.class);
public static final long DEFAULT_SLEEPTIME_BEFORE_SIGKILL = 5000L;

View File

@ -29,8 +29,6 @@
import org.junit.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapred.Counters.Counter;
import org.apache.hadoop.mapred.Counters.CountersExceededException;
import org.apache.hadoop.mapred.Counters.Group;
@ -41,6 +39,8 @@
import org.apache.hadoop.mapreduce.counters.FrameworkCounterGroup;
import org.apache.hadoop.mapreduce.counters.CounterGroupFactory.FrameworkGroupFactory;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* TestCounters checks the sanity and recoverability of {@code Counters}
@ -48,7 +48,7 @@
public class TestCounters {
enum myCounters {TEST1, TEST2};
private static final long MAX_VALUE = 10;
private static final Log LOG = LogFactory.getLog(TestCounters.class);
private static final Logger LOG = LoggerFactory.getLogger(TestCounters.class);
static final Enum<?> FRAMEWORK_COUNTER = TaskCounter.CPU_MILLISECONDS;
static final long FRAMEWORK_COUNTER_VALUE = 8;

View File

@ -23,8 +23,6 @@
import java.util.Collection;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.FileStatus;
@ -42,13 +40,16 @@
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.Lists;
@RunWith(value = Parameterized.class)
public class TestFileInputFormat {
private static final Log LOG = LogFactory.getLog(TestFileInputFormat.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestFileInputFormat.class);
private static String testTmpDir = System.getProperty("test.build.data", "/tmp");
private static final Path TEST_ROOT_DIR = new Path(testTmpDir, "TestFIF");

View File

@ -24,8 +24,6 @@
import java.util.Map;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -42,10 +40,12 @@
import static org.junit.Assert.assertTrue;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TestClientDistributedCacheManager {
private static final Log LOG = LogFactory.getLog(
TestClientDistributedCacheManager.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestClientDistributedCacheManager.class);
private static final Path TEST_ROOT_DIR = new Path(
System.getProperty("test.build.data",

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.mapreduce.jobhistory;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapred.TaskAttemptID;
import org.apache.hadoop.mapred.TaskID;
import org.apache.hadoop.mapred.TaskStatus;
@ -30,6 +28,8 @@
import org.junit.Test;
import org.skyscreamer.jsonassert.JSONAssert;
import org.skyscreamer.jsonassert.JSONCompareMode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.ByteArrayOutputStream;
import java.io.PrintStream;
@ -38,8 +38,8 @@
public class TestHistoryViewerPrinter {
private static final Log LOG = LogFactory.getLog(
TestHistoryViewerPrinter.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestHistoryViewerPrinter.class);
@Test
public void testHumanPrinter() throws Exception {

View File

@ -28,8 +28,6 @@
import org.junit.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.FileStatus;
@ -48,6 +46,8 @@
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
@ -57,7 +57,8 @@
@RunWith(value = Parameterized.class)
public class TestFileInputFormat {
private static final Log LOG = LogFactory.getLog(TestFileInputFormat.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestFileInputFormat.class);
private static String testTmpDir = System.getProperty("test.build.data", "/tmp");
private static final Path TEST_ROOT_DIR = new Path(testTmpDir, "TestFIF");

View File

@ -34,8 +34,6 @@
import org.apache.hadoop.util.concurrent.HadoopExecutors;
import org.junit.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@ -56,6 +54,8 @@
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.task.JobContextImpl;
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings("unchecked")
public class TestFileOutputCommitter {
@ -67,8 +67,8 @@ public class TestFileOutputCommitter {
private final static String SUB_DIR = "SUB_DIR";
private final static Path OUT_SUB_DIR = new Path(outDir, SUB_DIR);
private static final Log LOG =
LogFactory.getLog(TestFileOutputCommitter.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestFileOutputCommitter.class);
// A random task attempt id for testing.
private static final String attempt = "attempt_200707121733_0001_m_000000_0";

View File

@ -47,8 +47,6 @@
import javax.crypto.SecretKey;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapred.Counters;
import org.apache.hadoop.mapred.IFileInputStream;
@ -63,12 +61,14 @@
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Test that the Fetcher does what we expect it to.
*/
public class TestFetcher {
private static final Log LOG = LogFactory.getLog(TestFetcher.class);
private static final Logger LOG = LoggerFactory.getLogger(TestFetcher.class);
JobConf job = null;
JobConf jobWithRetry = null;
TaskAttemptID id = null;

View File

@ -32,12 +32,12 @@
import org.apache.hadoop.io.compress.DefaultCodec;
import org.apache.hadoop.mapred.*;
import org.apache.commons.logging.*;
import org.slf4j.Logger;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
public class TestSequenceFileMergeProgress {
private static final Log LOG = FileInputFormat.LOG;
private static final Logger LOG = FileInputFormat.LOG;
private static final int RECORDS = 10000;
@Test

View File

@ -18,13 +18,13 @@
package org.apache.hadoop.mapred;
import org.apache.commons.logging.Log;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.slf4j.Logger;
import org.junit.Test;
import java.io.IOException;
@ -34,7 +34,7 @@
import static org.junit.Assert.assertTrue;
public class TestSequenceFileAsBinaryInputFormat {
private static final Log LOG = FileInputFormat.LOG;
private static final Logger LOG = FileInputFormat.LOG;
private static final int RECORDS = 10000;
@Test

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.mapred;
import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -26,6 +25,7 @@
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.slf4j.Logger;
import org.junit.Test;
import java.util.BitSet;
@ -35,7 +35,7 @@
import static org.junit.Assert.assertFalse;
public class TestSequenceFileAsTextInputFormat {
private static final Log LOG = FileInputFormat.LOG;
private static final Logger LOG = FileInputFormat.LOG;
private static int MAX_LENGTH = 10000;
private static Configuration conf = new Configuration();

View File

@ -18,13 +18,13 @@
package org.apache.hadoop.mapred;
import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.slf4j.Logger;
import org.junit.Test;
import java.io.IOException;
@ -33,7 +33,7 @@
import static org.junit.Assert.assertEquals;
public class TestSequenceFileInputFilter {
private static final Log LOG = FileInputFormat.LOG;
private static final Logger LOG = FileInputFormat.LOG;
private static final int MAX_LENGTH = 15000;
private static final Configuration conf = new Configuration();

View File

@ -18,13 +18,13 @@
package org.apache.hadoop.mapred;
import org.apache.commons.logging.Log;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.SequenceFile;
import org.slf4j.Logger;
import org.junit.Test;
import java.util.BitSet;
@ -34,7 +34,7 @@
import static org.junit.Assert.assertFalse;
public class TestSequenceFileInputFormat {
private static final Log LOG = FileInputFormat.LOG;
private static final Logger LOG = FileInputFormat.LOG;
private static int MAX_LENGTH = 10000;
private static Configuration conf = new Configuration();