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

(cherry picked from commit 453d48bdfb)
This commit is contained in:
Akira Ajisaka 2017-10-03 12:14:54 +09:00
parent dcc454b4a0
commit 47aa044fcc
No known key found for this signature in database
GPG Key ID: C1EDBB9CA400FD50
35 changed files with 142 additions and 138 deletions

View File

@ -37,8 +37,6 @@ import java.util.concurrent.LinkedBlockingQueue;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FSError;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileStatus;
@ -70,6 +68,8 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Runs the container task locally in a thread.
@ -80,7 +80,8 @@ public class LocalContainerLauncher extends AbstractService implements
ContainerLauncher {
private static final File curDir = new File(".");
private static final Log LOG = LogFactory.getLog(LocalContainerLauncher.class);
private static final Logger LOG =
LoggerFactory.getLogger(LocalContainerLauncher.class);
private FileContext curFC = null;
private Set<File> localizedFiles = new HashSet<File>();
@ -376,7 +377,7 @@ public class LocalContainerLauncher extends AbstractService implements
// if umbilical itself barfs (in error-handler of runSubMap()),
// we're pretty much hosed, so do what YarnChild main() does
// (i.e., exit clumsily--but can never happen, so no worries!)
LOG.fatal("oopsie... this can never happen: "
LOG.error("oopsie... this can never happen: "
+ StringUtils.stringifyException(ioe));
ExitUtil.terminate(-1);
} finally {
@ -477,7 +478,7 @@ public class LocalContainerLauncher extends AbstractService implements
}
} catch (FSError e) {
LOG.fatal("FSError from child", e);
LOG.error("FSError from child", e);
// umbilical: MRAppMaster creates (taskAttemptListener), passes to us
if (!ShutdownHookManager.get().isShutdownInProgress()) {
umbilical.fsError(classicAttemptID, e.getMessage());
@ -502,7 +503,7 @@ public class LocalContainerLauncher extends AbstractService implements
throw new RuntimeException();
} catch (Throwable throwable) {
LOG.fatal("Error running local (uberized) 'child' : "
LOG.error("Error running local (uberized) 'child' : "
+ StringUtils.stringifyException(throwable));
if (!ShutdownHookManager.get().isShutdownInProgress()) {
Throwable tCause = throwable.getCause();

View File

@ -26,8 +26,6 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.ipc.ProtocolSignature;
@ -57,6 +55,8 @@ import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.util.StringInterner;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class is responsible for talking to the task umblical.
@ -72,7 +72,8 @@ public class TaskAttemptListenerImpl extends CompositeService
private static final JvmTask TASK_FOR_INVALID_JVM = new JvmTask(null, true);
private static final Log LOG = LogFactory.getLog(TaskAttemptListenerImpl.class);
private static final Logger LOG =
LoggerFactory.getLogger(TaskAttemptListenerImpl.class);
private AppContext context;
private Server server;
@ -274,7 +275,7 @@ public class TaskAttemptListenerImpl extends CompositeService
public void fatalError(TaskAttemptID taskAttemptID, String msg)
throws IOException {
// This happens only in Child and in the Task.
LOG.fatal("Task: " + taskAttemptID + " - exited : " + msg);
LOG.error("Task: " + taskAttemptID + " - exited : " + msg);
reportDiagnosticInfo(taskAttemptID, "Error: " + msg);
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
@ -291,7 +292,7 @@ public class TaskAttemptListenerImpl extends CompositeService
public void fsError(TaskAttemptID taskAttemptID, String message)
throws IOException {
// This happens only in Child.
LOG.fatal("Task: " + taskAttemptID + " - failed due to FSError: "
LOG.error("Task: " + taskAttemptID + " - failed due to FSError: "
+ message);
reportDiagnosticInfo(taskAttemptID, "FSError: " + message);

View File

@ -26,8 +26,6 @@ import java.net.InetSocketAddress;
import java.security.PrivilegedExceptionAction;
import java.util.concurrent.ScheduledExecutorService;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FSError;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalDirAllocator;
@ -60,13 +58,15 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The main() for MapReduce task processes.
*/
class YarnChild {
private static final Log LOG = LogFactory.getLog(YarnChild.class);
private static final Logger LOG = LoggerFactory.getLogger(YarnChild.class);
static volatile TaskAttemptID taskid = null;
@ -100,10 +100,7 @@ class YarnChild {
// Security framework already loaded the tokens into current ugi
Credentials credentials =
UserGroupInformation.getCurrentUser().getCredentials();
LOG.info("Executing with tokens:");
for (Token<?> token: credentials.getAllTokens()) {
LOG.info(token);
}
LOG.info("Executing with tokens: {}", credentials.getAllTokens());
// Create TaskUmbilicalProtocol as actual task owner.
UserGroupInformation taskOwner =
@ -129,7 +126,7 @@ class YarnChild {
try {
int idleLoopCount = 0;
JvmTask myTask = null;;
JvmTask myTask = null;
// poll for new task
for (int idle = 0; null == myTask; ++idle) {
long sleepTimeMilliSecs = Math.min(idle * 500, 1500);
@ -179,7 +176,7 @@ class YarnChild {
}
});
} catch (FSError e) {
LOG.fatal("FSError from child", e);
LOG.error("FSError from child", e);
if (!ShutdownHookManager.get().isShutdownInProgress()) {
umbilical.fsError(taskid, e.getMessage());
}
@ -213,7 +210,7 @@ class YarnChild {
}
}
} catch (Throwable throwable) {
LOG.fatal("Error running child : "
LOG.error("Error running child : "
+ StringUtils.stringifyException(throwable));
if (taskid != null) {
if (!ShutdownHookManager.get().isShutdownInProgress()) {
@ -352,7 +349,7 @@ class YarnChild {
out = FileSystem.create(localFs, jobFile, urw_gr);
conf.writeXml(out);
} finally {
IOUtils.cleanup(LOG, out);
IOUtils.cleanupWithLogger(LOG, out);
}
}

View File

@ -20,8 +20,6 @@ package org.apache.hadoop.mapred;
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;

View File

@ -20,8 +20,6 @@ package org.apache.hadoop.mapreduce.jobhistory;
import java.io.IOException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileContext;
@ -34,6 +32,8 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Reads in history events from the JobHistoryFile and sends them out again
@ -41,7 +41,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
*/
public class JobHistoryCopyService extends CompositeService implements HistoryEventHandler {
private static final Log LOG = LogFactory.getLog(JobHistoryCopyService.class);
private static final Logger LOG =
LoggerFactory.getLogger(JobHistoryCopyService.class);
private final ApplicationAttemptId applicationAttemptId;
private final EventHandler<Event> handler;

View File

@ -31,8 +31,6 @@ import java.util.TimerTask;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -82,6 +80,8 @@ import org.apache.hadoop.yarn.util.TimelineServiceHelper;
import com.google.common.annotations.VisibleForTesting;
import com.sun.jersey.api.client.ClientHandlerException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The job history events get routed to this class. This class writes the Job
@ -123,7 +123,7 @@ public class JobHistoryEventHandler extends AbstractService
private volatile boolean stopped;
private final Object lock = new Object();
private static final Log LOG = LogFactory.getLog(
private static final Logger LOG = LoggerFactory.getLogger(
JobHistoryEventHandler.class);
protected static final Map<JobId, MetaInfo> fileMap =

View File

@ -41,8 +41,6 @@ import java.util.regex.Pattern;
import javax.crypto.KeyGenerator;
import org.apache.commons.io.IOUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem;
@ -155,6 +153,8 @@ import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.SystemClock;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* The Map-Reduce Application Master.
@ -177,7 +177,7 @@ import com.google.common.annotations.VisibleForTesting;
@SuppressWarnings("rawtypes")
public class MRAppMaster extends CompositeService {
private static final Log LOG = LogFactory.getLog(MRAppMaster.class);
private static final Logger LOG = LoggerFactory.getLogger(MRAppMaster.class);
/**
* Priority of the MRAppMaster shutdown hook.
@ -327,7 +327,7 @@ public class MRAppMaster extends CompositeService {
errorHappenedShutDown = true;
forcedState = JobStateInternal.ERROR;
shutDownMessage = "Staging dir does not exist " + stagingDir;
LOG.fatal(shutDownMessage);
LOG.error(shutDownMessage);
} else if (commitStarted) {
//A commit was started so this is the last time, we just need to know
// what result we will use to notify, and how we will unregister
@ -1667,7 +1667,7 @@ public class MRAppMaster extends CompositeService {
conf.set(MRJobConfig.USER_NAME, jobUserName);
initAndStartAppMaster(appMaster, conf, jobUserName);
} catch (Throwable t) {
LOG.fatal("Error starting MRAppMaster", t);
LOG.error("Error starting MRAppMaster", t);
ExitUtil.terminate(1, t);
}
}
@ -1716,10 +1716,7 @@ public class MRAppMaster extends CompositeService {
// them
Credentials credentials =
UserGroupInformation.getCurrentUser().getCredentials();
LOG.info("Executing with tokens:");
for (Token<?> token : credentials.getAllTokens()) {
LOG.info(token);
}
LOG.info("Executing with tokens: {}", credentials.getAllTokens());
UserGroupInformation appMasterUgi = UserGroupInformation
.createRemoteUser(jobUserName);

View File

@ -24,8 +24,6 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.util.MRJobConfUtil;
@ -36,6 +34,8 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.Clock;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
@ -63,7 +63,8 @@ public class TaskHeartbeatHandler extends AbstractService {
}
}
private static final Log LOG = LogFactory.getLog(TaskHeartbeatHandler.class);
private static final Logger LOG =
LoggerFactory.getLogger(TaskHeartbeatHandler.class);
//thread which runs periodically to see the last time since a heartbeat is
//received from a task.

View File

@ -23,8 +23,6 @@ import java.net.InetSocketAddress;
import java.util.Arrays;
import java.util.Collection;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.http.HttpConfig.Policy;
@ -90,6 +88,8 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.webapp.WebApp;
import org.apache.hadoop.yarn.webapp.WebApps;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This module is responsible for talking to the
@ -98,7 +98,7 @@ import org.apache.hadoop.yarn.webapp.WebApps;
*/
public class MRClientService extends AbstractService implements ClientService {
static final Log LOG = LogFactory.getLog(MRClientService.class);
static final Logger LOG = LoggerFactory.getLogger(MRClientService.class);
private MRClientProtocol protocolHandler;
private Server server;

View File

@ -26,8 +26,6 @@ import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@ -54,12 +52,14 @@ import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class CommitterEventHandler extends AbstractService
implements EventHandler<CommitterEvent> {
private static final Log LOG =
LogFactory.getLog(CommitterEventHandler.class);
private static final Logger LOG =
LoggerFactory.getLogger(CommitterEventHandler.class);
private final AppContext context;
private final OutputCommitter committer;

View File

@ -38,8 +38,6 @@ import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
@ -133,6 +131,8 @@ import org.apache.hadoop.yarn.util.Clock;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/** Implementation of Job interface. Maintains the state machines of Job.
* The read and write calls use ReadWriteLock for concurrency.
@ -147,7 +147,7 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
private static final TaskCompletionEvent[]
EMPTY_TASK_COMPLETION_EVENTS = new TaskCompletionEvent[0];
private static final Log LOG = LogFactory.getLog(JobImpl.class);
private static final Logger LOG = LoggerFactory.getLogger(JobImpl.class);
//The maximum fraction of fetch failures allowed for a map
private float maxAllowedFetchFailuresFraction;

View File

@ -38,8 +38,6 @@ import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.regex.Pattern;
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;
@ -139,6 +137,8 @@ import org.apache.hadoop.yarn.util.RackResolver;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Implementation of TaskAttempt interface.
@ -149,7 +149,8 @@ public abstract class TaskAttemptImpl implements
EventHandler<TaskAttemptEvent> {
static final Counters EMPTY_COUNTERS = new Counters();
private static final Log LOG = LogFactory.getLog(TaskAttemptImpl.class);
private static final Logger LOG =
LoggerFactory.getLogger(TaskAttemptImpl.class);
private static final long MEMORY_SPLITS_RESOLUTION = 1024; //TODO Make configurable?
private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);

View File

@ -32,8 +32,6 @@ import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Counters;
@ -92,6 +90,8 @@ import org.apache.hadoop.yarn.state.SingleArcTransition;
import org.apache.hadoop.yarn.state.StateMachine;
import org.apache.hadoop.yarn.state.StateMachineFactory;
import org.apache.hadoop.yarn.util.Clock;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.annotations.VisibleForTesting;
@ -101,7 +101,7 @@ import com.google.common.annotations.VisibleForTesting;
@SuppressWarnings({ "rawtypes", "unchecked" })
public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
private static final Log LOG = LogFactory.getLog(TaskImpl.class);
private static final Logger LOG = LoggerFactory.getLogger(TaskImpl.class);
private static final String SPECULATION = "Speculation: ";
protected final JobConf conf;

View File

@ -32,8 +32,6 @@ import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.ShuffleHandler;
import org.apache.hadoop.mapreduce.MRJobConfig;
@ -60,6 +58,8 @@ import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.C
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class is responsible for launching of containers.
@ -67,7 +67,8 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder;
public class ContainerLauncherImpl extends AbstractService implements
ContainerLauncher {
static final Log LOG = LogFactory.getLog(ContainerLauncherImpl.class);
static final Logger LOG =
LoggerFactory.getLogger(ContainerLauncherImpl.class);
private ConcurrentHashMap<ContainerId, Container> containers =
new ConcurrentHashMap<ContainerId, Container>();

View File

@ -21,8 +21,6 @@ package org.apache.hadoop.mapreduce.v2.app.local;
import java.io.IOException;
import java.util.ArrayList;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.JobCounter;
@ -55,6 +53,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Allocates containers locally. Doesn't allocate a real container;
@ -63,8 +63,8 @@ import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
public class LocalContainerAllocator extends RMCommunicator
implements ContainerAllocator {
private static final Log LOG =
LogFactory.getLog(LocalContainerAllocator.class);
private static final Logger LOG =
LoggerFactory.getLogger(LocalContainerAllocator.class);
@SuppressWarnings("rawtypes")
private final EventHandler eventHandler;

View File

@ -26,8 +26,6 @@ import java.util.Map;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.MRJobConfig;
@ -61,13 +59,16 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Registers/unregisters to RM and sends heartbeats to RM.
*/
public abstract class RMCommunicator extends AbstractService
implements RMHeartbeatHandler {
private static final Log LOG = LogFactory.getLog(RMCommunicator.class);
private static final Logger LOG =
LoggerFactory.getLogger(RMCommunicator.class);
private int rmPollInterval;//millis
protected ApplicationId applicationId;
private final AtomicBoolean stopped;

View File

@ -35,8 +35,6 @@ import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
@ -90,6 +88,8 @@ import org.apache.hadoop.yarn.util.RackResolver;
import org.apache.hadoop.yarn.util.resource.Resources;
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Allocates the container from the ResourceManager scheduler.
@ -97,7 +97,7 @@ import com.google.common.annotations.VisibleForTesting;
public class RMContainerAllocator extends RMContainerRequestor
implements ContainerAllocator {
static final Log LOG = LogFactory.getLog(RMContainerAllocator.class);
static final Logger LOG = LoggerFactory.getLogger(RMContainerAllocator.class);
public static final
float DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART = 0.05f;

View File

@ -30,8 +30,6 @@ import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.MRJobConfig;
@ -55,6 +53,8 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.yarn.util.resource.Resources;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
@ -62,7 +62,8 @@ import org.apache.hadoop.yarn.util.resource.Resources;
*/
public abstract class RMContainerRequestor extends RMCommunicator {
private static final Log LOG = LogFactory.getLog(RMContainerRequestor.class);
private static final Logger LOG =
LoggerFactory.getLogger(RMContainerRequestor.class);
private static final ResourceRequestComparator RESOURCE_REQUEST_COMPARATOR =
new ResourceRequestComparator();

View File

@ -25,8 +25,6 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
@ -44,6 +42,8 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
import org.apache.hadoop.yarn.event.EventHandler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This policy works in combination with an implementation of task
@ -75,8 +75,8 @@ public class CheckpointAMPreemptionPolicy implements AMPreemptionPolicy {
@SuppressWarnings("rawtypes")
private EventHandler eventHandler;
static final Log LOG = LogFactory
.getLog(CheckpointAMPreemptionPolicy.class);
static final Logger LOG = LoggerFactory
.getLogger(CheckpointAMPreemptionPolicy.class);
public CheckpointAMPreemptionPolicy() {
this(Collections.synchronizedSet(new HashSet<TaskAttemptId>()),

View File

@ -17,8 +17,6 @@
*/
package org.apache.hadoop.mapreduce.v2.app.rm.preemption;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
@ -33,14 +31,16 @@ import org.apache.hadoop.yarn.api.records.PreemptionContract;
import org.apache.hadoop.yarn.api.records.PreemptionMessage;
import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
import org.apache.hadoop.yarn.event.EventHandler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Sample policy that aggressively kills tasks when requested.
*/
public class KillAMPreemptionPolicy implements AMPreemptionPolicy {
private static final Log LOG =
LogFactory.getLog(KillAMPreemptionPolicy.class);
private static final Logger LOG =
LoggerFactory.getLogger(KillAMPreemptionPolicy.class);
@SuppressWarnings("rawtypes")
private EventHandler dispatcher = null;

View File

@ -30,8 +30,6 @@ import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@ -53,6 +51,8 @@ import org.apache.hadoop.yarn.util.Clock;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.yarn.event.Event;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class DefaultSpeculator extends AbstractService implements
Speculator {
@ -70,7 +70,8 @@ public class DefaultSpeculator extends AbstractService implements
private double proportionTotalTasksSpeculatable;
private int minimumAllowedSpeculativeTasks;
private static final Log LOG = LogFactory.getLog(DefaultSpeculator.class);
private static final Logger LOG =
LoggerFactory.getLogger(DefaultSpeculator.class);
private final ConcurrentMap<TaskId, Boolean> runningTasks
= new ConcurrentHashMap<TaskId, Boolean>();

View File

@ -26,8 +26,6 @@ import java.net.URLDecoder;
import javax.servlet.http.HttpServletResponse;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@ -45,12 +43,15 @@ import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
import com.google.common.base.Joiner;
import com.google.inject.Inject;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* This class renders the various pages that the web app supports.
*/
public class AppController extends Controller implements AMParams {
private static final Log LOG = LogFactory.getLog(AppController.class);
private static final Logger LOG =
LoggerFactory.getLogger(AppController.class);
private static final Joiner JOINER = Joiner.on("");
protected final App app;

View File

@ -31,8 +31,6 @@ import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileSystem;
@ -61,10 +59,12 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TestLocalContainerLauncher {
private static final Log LOG =
LogFactory.getLog(TestLocalContainerLauncher.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestLocalContainerLauncher.class);
private static File testWorkDir;
private static final String[] localDirs = new String[2];

View File

@ -36,8 +36,6 @@ import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileContext;
@ -87,12 +85,14 @@ import org.mockito.Mockito;
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TestJobHistoryEventHandler {
private static final Log LOG = LogFactory
.getLog(TestJobHistoryEventHandler.class);
private static final Logger LOG = LoggerFactory
.getLogger(TestJobHistoryEventHandler.class);
private static MiniDFSCluster dfsCluster = null;
private static String coreSitePath;

View File

@ -18,19 +18,20 @@
package org.apache.hadoop.mapreduce.jobhistory;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class TestJobSummary {
private static final Log LOG = LogFactory.getLog(TestJobSummary.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestJobSummary.class);
private JobSummary summary = new JobSummary();
@Before

View File

@ -24,8 +24,6 @@ import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.EnumSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileSystem;
@ -101,6 +99,8 @@ import org.apache.hadoop.yarn.state.StateMachineFactory;
import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.SystemClock;
import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
@ -109,7 +109,7 @@ import org.junit.Assert;
*/
@SuppressWarnings("unchecked")
public class MRApp extends MRAppMaster {
private static final Log LOG = LogFactory.getLog(MRApp.class);
private static final Logger LOG = LoggerFactory.getLogger(MRApp.class);
/**
* The available resource of each container allocated.

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator;
import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy;
import org.apache.hadoop.mapreduce.v2.app.rm.preemption.NoopAMPreemptionPolicy;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
@ -52,10 +53,8 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.util.Records;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.Test;
import org.slf4j.event.Level;
public class MRAppBenchmark {
@ -67,8 +66,7 @@ public class MRAppBenchmark {
* @throws Exception On application failure
*/
public void run(MRApp app) throws Exception {
Logger rootLogger = LogManager.getRootLogger();
rootLogger.setLevel(Level.WARN);
GenericTestUtils.setRootLogLevel(Level.WARN);
long startTime = System.currentTimeMillis();
Job job = app.submit(new Configuration());
while (!job.getReport().getJobState().equals(JobState.SUCCEEDED)) {

View File

@ -41,8 +41,6 @@ import java.util.Map;
import org.junit.Assert;
import org.apache.commons.io.FileUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.FileSystem;
@ -77,6 +75,7 @@ import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.ExitUtil;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@ -84,18 +83,19 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.log4j.Level;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import org.slf4j.event.Level;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TestMRAppMaster {
private static final Log LOG = LogFactory.getLog(TestMRAppMaster.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestMRAppMaster.class);
private static final Path TEST_ROOT_DIR =
new Path(System.getProperty("test.build.data", "target/test-dir"));
private static final Path testDir = new Path(TEST_ROOT_DIR,
@ -408,8 +408,7 @@ public class TestMRAppMaster {
@Test
public void testMRAppMasterCredentials() throws Exception {
Logger rootLogger = LogManager.getRootLogger();
rootLogger.setLevel(Level.DEBUG);
GenericTestUtils.setRootLogLevel(Level.DEBUG);
// Simulate credentials passed to AM via client->RM->NM
Credentials credentials = new Credentials();

View File

@ -38,8 +38,6 @@ import java.util.Map;
import java.util.concurrent.TimeoutException;
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.Path;
import org.apache.hadoop.io.NullWritable;
@ -103,11 +101,13 @@ import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.util.SystemClock;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"unchecked", "rawtypes"})
public class TestRecovery {
private static final Log LOG = LogFactory.getLog(TestRecovery.class);
private static final Logger LOG = LoggerFactory.getLogger(TestRecovery.class);
private static Path outputDir = new Path(new File("target",
TestRecovery.class.getName()).getAbsolutePath() +
Path.SEPARATOR + "out");

View File

@ -29,8 +29,6 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.TaskCompletionEvent;
@ -81,6 +79,8 @@ import org.apache.hadoop.yarn.util.ControlledClock;
import org.apache.hadoop.yarn.util.SystemClock;
import org.junit.Assert;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings({"unchecked", "rawtypes"})
public class TestRuntimeEstimators {
@ -98,7 +98,8 @@ public class TestRuntimeEstimators {
AppContext myAppContext;
private static final Log LOG = LogFactory.getLog(TestRuntimeEstimators.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestRuntimeEstimators.class);
private final AtomicInteger slotsInUse = new AtomicInteger(0);

View File

@ -24,8 +24,6 @@ import java.util.Map;
import org.apache.hadoop.mapreduce.TaskType;
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.mapred.JobConf;
import org.apache.hadoop.mapreduce.MRConfig;
@ -40,10 +38,13 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TestMapReduceChildJVM {
private static final Log LOG = LogFactory.getLog(TestMapReduceChildJVM.class);
private static final Logger LOG =
LoggerFactory.getLogger(TestMapReduceChildJVM.class);
@Test (timeout = 30000)
public void testCommandLine() throws Exception {

View File

@ -28,8 +28,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.Task;
@ -69,11 +67,13 @@ import org.apache.hadoop.yarn.util.SystemClock;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings("rawtypes")
public class TestTaskImpl {
private static final Log LOG = LogFactory.getLog(TestTaskImpl.class);
private static final Logger LOG = LoggerFactory.getLogger(TestTaskImpl.class);
private JobConf conf;
private TaskAttemptListener taskAttemptListener;

View File

@ -42,8 +42,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RestartContainerResponse;
import org.apache.hadoop.yarn.api.protocolrecords.RollbackResponse;
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.CommonConfigurationKeysPublic;
import org.apache.hadoop.ipc.Server;
@ -93,6 +91,8 @@ import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
import org.apache.hadoop.yarn.util.Records;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TestContainerLauncher {
@ -101,7 +101,8 @@ public class TestContainerLauncher {
Configuration conf;
Server server;
static final Log LOG = LogFactory.getLog(TestContainerLauncher.class);
static final Logger LOG =
LoggerFactory.getLogger(TestContainerLauncher.class);
@Test (timeout = 10000)
public void testPoolSize() throws InterruptedException {
@ -446,7 +447,7 @@ public class TestContainerLauncher {
// make the thread sleep to look like its not going to respond
Thread.sleep(15000);
} catch (Exception e) {
LOG.error(e);
LOG.error("Setup thread sleep interrupted: ", e);
throw new UndeclaredThrowableException(e);
}
status.setState(ContainerState.RUNNING);

View File

@ -32,8 +32,6 @@ import java.util.Map;
import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.CyclicBarrier;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.ShuffleHandler;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@ -82,9 +80,12 @@ import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TestContainerLauncherImpl {
static final Log LOG = LogFactory.getLog(TestContainerLauncherImpl.class);
static final Logger LOG =
LoggerFactory.getLogger(TestContainerLauncherImpl.class);
private static final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);

View File

@ -48,8 +48,6 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.MRJobConfig;
@ -154,12 +152,14 @@ import org.junit.Test;
import com.google.common.base.Supplier;
import org.mockito.InOrder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@SuppressWarnings("unchecked")
public class TestRMContainerAllocator {
static final Log LOG = LogFactory
.getLog(TestRMContainerAllocator.class);
static final Logger LOG = LoggerFactory
.getLogger(TestRMContainerAllocator.class);
static final RecordFactory recordFactory = RecordFactoryProvider
.getRecordFactory(null);