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:
parent
dcc454b4a0
commit
47aa044fcc
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 =
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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>();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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>()),
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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>();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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];
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
Loading…
Reference in New Issue