MAPREDUCE-279. MapReduce 2.0. Merging MR-279 branch into trunk. Contributed by Arun C Murthy, Christopher Douglas, Devaraj Das, Greg Roelofs, Jeffrey Naisbitt, Josh Wills, Jonathan Eagles, Krishna Ramachandran, Luke Lu, Mahadev Konar, Robert Evans, Sharad Agarwal, Siddharth Seth, Thomas Graves, and Vinod Kumar Vavilapalli.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1159166 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2011-08-18 11:07:10 +00:00
parent dd86860633
commit dbecbe5dfe
2874 changed files with 138864 additions and 208 deletions

89
hadoop-mapreduce/INSTALL Normal file
View File

@ -0,0 +1,89 @@
To compile Hadoop Mapreduce next following, do the following:
Step 1) Install dependencies for yarn
See http://svn.apache.org/repos/asf/hadoop/common/branches/MR-279/mapreduce/yarn/README
Make sure protbuf library is in your library path or set: export LD_LIBRARY_PATH=/usr/local/lib
Step 2) Checkout
svn checkout http://svn.apache.org/repos/asf/hadoop/common/branches/MR-279/
Step 3) Build common
Go to common directory
ant veryclean mvn-install
Step 4) Build HDFS
Go to hdfs directory
ant veryclean mvn-install -Dresolvers=internal
Step 5) Build yarn and mapreduce
Go to mapreduce directory
export MAVEN_OPTS=-Xmx512m
mvn clean install assembly:assembly
ant veryclean jar jar-test -Dresolvers=internal
In case you want to skip the tests run:
mvn clean install assembly:assembly -DskipTests
ant veryclean jar jar-test -Dresolvers=internal
You will see a tarball in
ls target/hadoop-mapreduce-1.0-SNAPSHOT-bin.tar.gz
Step 6) Untar the tarball in a clean and different directory.
say HADOOP_YARN_INSTALL
To run Hadoop Mapreduce next applications :
Step 7) cd $HADOOP_YARN_INSTALL
Step 8) export the following variables:
HADOOP_MAPRED_HOME=
HADOOP_COMMON_HOME=
HADOOP_HDFS_HOME=
YARN_HOME=directory where you untarred yarn
HADOOP_CONF_DIR=
YARN_CONF_DIR=$HADOOP_CONF_DIR
Step 9) bin/yarn-daemon.sh start resourcemanager
Step 10) bin/yarn-daemon.sh start nodemanager
Step 11) bin/yarn-daemon.sh start historyserver
Step 12) Create the following symlinks in hadoop-common/lib
ln -s $HADOOP_YARN_INSTALL/modules/hadoop-mapreduce-client-app-1.0-SNAPSHOT.jar .
ln -s $HADOOP_YARN_INSTALL/modules/yarn-api-1.0-SNAPSHOT.jar .
ln -s $HADOOP_YARN_INSTALL/modules/hadoop-mapreduce-client-common-1.0-SNAPSHOT.jar .
ln -s $HADOOP_YARN_INSTALL/modules/yarn-common-1.0-SNAPSHOT.jar .
ln -s $HADOOP_YARN_INSTALL/modules/hadoop-mapreduce-client-core-1.0-SNAPSHOT.jar .
ln -s $HADOOP_YARN_INSTALL/modules/yarn-server-common-1.0-SNAPSHOT.jar .
ln -s $HADOOP_YARN_INSTALL/modules/hadoop-mapreduce-client-jobclient-1.0-SNAPSHOT.jar .
ln -s $HADOOP_YARN_INSTALL/lib/protobuf-java-2.4.0a.jar .
Step 13) Yarn daemons are up! But for running mapreduce applications, which now are in user land, you need to setup nodemanager with the following configuration in your yarn-site.xml before you start the nodemanager.
<property>
<name>nodemanager.auxiluary.services</name>
<value>mapreduce.shuffle</value>
</property>
<property>
<name>nodemanager.aux.service.mapreduce.shuffle.class</name>
<value>org.apache.hadoop.mapred.ShuffleHandler</value>
</property>
Step 14) You are all set, an example on how to run a mapreduce job is:
cd $HADOOP_MAPRED_HOME
ant examples -Dresolvers=internal
$HADOOP_COMMON_HOME/bin/hadoop jar $HADOOP_MAPRED_HOME/build/hadoop-mapred-examples-0.22.0-SNAPSHOT.jar randomwriter -Dmapreduce.job.user.name=$USER -Dmapreduce.clientfactory.class.name=org.apache.hadoop.mapred.YarnClientFactory -Dmapreduce.randomwriter.bytespermap=10000 -Ddfs.blocksize=536870912 -Ddfs.block.size=536870912 -libjars $HADOOP_YARN_INSTALL/hadoop-mapreduce-1.0-SNAPSHOT/modules/hadoop-mapreduce-client-jobclient-1.0-SNAPSHOT.jar output
The output on the command line should be almost similar to what you see in the JT/TT setup (Hadoop 0.20/0.21)

View File

@ -0,0 +1,101 @@
<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
<id>all</id>
<formats>
<format>tar.gz</format>
</formats>
<includeBaseDirectory>true</includeBaseDirectory>
<!-- TODO: this layout is wrong. We need module specific bin files in module specific dirs -->
<fileSets>
<fileSet>
<directory>hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/target/classes/bin</directory>
<outputDirectory>bin</outputDirectory>
<includes>
<include>container-executor</include>
</includes>
<fileMode>0755</fileMode>
</fileSet>
<fileSet>
<directory>hadoop-yarn/bin</directory>
<outputDirectory>bin</outputDirectory>
<includes>
<include>*</include>
</includes>
<fileMode>0755</fileMode>
</fileSet>
<fileSet>
<directory>bin</directory>
<outputDirectory>bin</outputDirectory>
<includes>
<include>*</include>
</includes>
<fileMode>0755</fileMode>
</fileSet>
<fileSet>
<directory>hadoop-yarn/conf</directory>
<outputDirectory>conf</outputDirectory>
<includes>
<include>**/*</include>
</includes>
</fileSet>
<fileSet>
<outputDirectory>sources</outputDirectory>
<excludes>
<exclude>**/*.jar</exclude>
<exclude>**/target/**</exclude>
<!-- scripts to include later for setting fileMode -->
<exclude>**/bin/*</exclude>
<exclude>**/scripts/*</exclude>
<!-- images that we don't need (and cause problems for our tools) -->
<exclude>**/dt-*/images/**</exclude>
<!-- until the code that does this is fixed -->
<exclude>**/file:/**</exclude>
<exclude>**/SecurityAuth.audit*</exclude>
</excludes>
<includes>
<include>assembly/**</include>
<include>pom.xml</include>
<include>build*.xml</include>
<include>ivy.xml</include>
<include>ivy/**</include>
<include>INSTALL</include>
<include>LICENSE.txt</include>
<include>mr-client/**</include>
<include>hadoop-yarn/**</include>
<include>src/**</include>
</includes>
</fileSet>
<fileSet>
<outputDirectory>sources</outputDirectory>
<includes>
<include>**/bin/*</include>
<include>**/scripts/*</include>
</includes>
<fileMode>0755</fileMode>
</fileSet>
</fileSets>
<moduleSets>
<moduleSet>
<excludes>
<exclude>org.apache.hadoop:hadoop-yarn-server-tests</exclude>
</excludes>
<binaries>
<outputDirectory>modules</outputDirectory>
<includeDependencies>false</includeDependencies>
<unpack>false</unpack>
</binaries>
</moduleSet>
</moduleSets>
<dependencySets>
<dependencySet>
<useProjectArtifact>false</useProjectArtifact>
<outputDirectory>/lib</outputDirectory>
<!-- Exclude hadoop artifacts. They will be found via HADOOP* env -->
<excludes>
<exclude>org.apache.hadoop:hadoop-common</exclude>
<exclude>org.apache.hadoop:hadoop-hdfs</exclude>
</excludes>
</dependencySet>
</dependencySets>
</assembly>

View File

@ -0,0 +1,103 @@
<?xml version="1.0"?>
<project>
<parent>
<artifactId>hadoop-mapreduce-client</artifactId>
<groupId>org.apache.hadoop</groupId>
<version>${hadoop-mapreduce.version}</version>
</parent>
<modelVersion>4.0.0</modelVersion>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-mapreduce-client-app</artifactId>
<name>hadoop-mapreduce-client-app</name>
<properties>
<install.file>${project.artifact.file}</install.file>
<applink.base>${project.build.directory}/${project.name}</applink.base>
<mr.basedir>${project.parent.parent.basedir}</mr.basedir>
</properties>
<dependencies>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-common</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-mapreduce-client-common</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-server-common</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-server-nodemanager</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-yarn-server-resourcemanager</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-mapreduce-client-shuffle</artifactId>
</dependency>
</dependencies>
<build>
<!-- local name for links -->
<finalName>mr-app</finalName>
<plugins>
<plugin>
<artifactId>maven-jar-plugin</artifactId>
<executions>
<execution>
<goals>
<goal>test-jar</goal>
</goals>
<phase>test-compile</phase>
</execution>
</executions>
</plugin>
<plugin>
<artifactId>maven-dependency-plugin</artifactId>
<executions>
<execution>
<id>build-classpath</id>
<phase>generate-sources</phase>
<goals>
<goal>build-classpath</goal>
</goals>
<configuration>
<outputFile>target/classes/mrapp-generated-classpath</outputFile>
</configuration>
</execution>
</executions>
</plugin>
<plugin>
<artifactId>maven-antrun-plugin</artifactId>
<executions>
<execution>
<id>create-mr-app-symlinks</id>
<phase>package</phase>
<configuration>
<target>
<symlink link="${applink.base}.jar"
resource="mr-app.jar" failonerror="false"/>
<symlink link="${applink.base}-1.0-SNAPSHOT.jar"
resource="mr-app.jar" failonerror="false"/>
</target>
</configuration>
<goals>
<goal>run</goal>
</goals>
</execution>
</executions>
</plugin>
</plugins>
</build>
</project>

View File

@ -0,0 +1,67 @@
package org.apache.hadoop;
import java.io.IOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.FileOutputFormat;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.JobContext;
import org.apache.hadoop.mapred.OutputCommitter;
import org.apache.hadoop.mapred.TaskAttemptContext;
public class CustomOutputCommitter extends OutputCommitter {
public static final String JOB_SETUP_FILE_NAME = "_job_setup";
public static final String JOB_COMMIT_FILE_NAME = "_job_commit";
public static final String JOB_ABORT_FILE_NAME = "_job_abort";
public static final String TASK_SETUP_FILE_NAME = "_task_setup";
public static final String TASK_ABORT_FILE_NAME = "_task_abort";
public static final String TASK_COMMIT_FILE_NAME = "_task_commit";
@Override
public void setupJob(JobContext jobContext) throws IOException {
writeFile(jobContext.getJobConf(), JOB_SETUP_FILE_NAME);
}
@Override
public void commitJob(JobContext jobContext) throws IOException {
super.commitJob(jobContext);
writeFile(jobContext.getJobConf(), JOB_COMMIT_FILE_NAME);
}
@Override
public void abortJob(JobContext jobContext, int status)
throws IOException {
super.abortJob(jobContext, status);
writeFile(jobContext.getJobConf(), JOB_ABORT_FILE_NAME);
}
@Override
public void setupTask(TaskAttemptContext taskContext) throws IOException {
writeFile(taskContext.getJobConf(), TASK_SETUP_FILE_NAME);
}
@Override
public boolean needsTaskCommit(TaskAttemptContext taskContext)
throws IOException {
return true;
}
@Override
public void commitTask(TaskAttemptContext taskContext) throws IOException {
writeFile(taskContext.getJobConf(), TASK_COMMIT_FILE_NAME);
}
@Override
public void abortTask(TaskAttemptContext taskContext) throws IOException {
writeFile(taskContext.getJobConf(), TASK_ABORT_FILE_NAME);
}
private void writeFile(JobConf conf , String filename) throws IOException {
System.out.println("writing file ----" + filename);
Path outputPath = FileOutputFormat.getOutputPath(conf);
FileSystem fs = outputPath.getFileSystem(conf);
fs.create(new Path(outputPath, filename)).close();
}
}

View File

@ -0,0 +1,455 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapred;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.PrintStream;
import java.net.URI;
import java.util.HashSet;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
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.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.service.AbstractService;
/**
* Runs the container task locally in a thread.
* Since all (sub)tasks share the same local directory, they must be executed
* sequentially in order to avoid creating/deleting the same files/dirs.
*/
public class LocalContainerLauncher extends AbstractService implements
ContainerLauncher {
private static final File curDir = new File(".");
private static final Log LOG = LogFactory.getLog(LocalContainerLauncher.class);
private FileContext curFC = null;
private final HashSet<File> localizedFiles;
private final AppContext context;
private final TaskUmbilicalProtocol umbilical;
private Thread eventHandlingThread;
private BlockingQueue<ContainerLauncherEvent> eventQueue =
new LinkedBlockingQueue<ContainerLauncherEvent>();
public LocalContainerLauncher(AppContext context,
TaskUmbilicalProtocol umbilical) {
super(LocalContainerLauncher.class.getName());
this.context = context;
this.umbilical = umbilical;
// umbilical: MRAppMaster creates (taskAttemptListener), passes to us (TODO/FIXME: pointless to use RPC to talk to self; should create LocalTaskAttemptListener or similar: implement umbilical protocol but skip RPC stuff)
try {
curFC = FileContext.getFileContext(curDir.toURI());
} catch (UnsupportedFileSystemException ufse) {
LOG.error("Local filesystem " + curDir.toURI().toString()
+ " is unsupported?? (should never happen)");
}
// Save list of files/dirs that are supposed to be present so can delete
// any extras created by one task before starting subsequent task. Note
// that there's no protection against deleted or renamed localization;
// users who do that get what they deserve (and will have to disable
// uberization in order to run correctly).
File[] curLocalFiles = curDir.listFiles();
localizedFiles = new HashSet<File>(curLocalFiles.length);
for (int j = 0; j < curLocalFiles.length; ++j) {
localizedFiles.add(curLocalFiles[j]);
}
// Relocalization note/future FIXME (per chrisdo, 20110315): At moment,
// full localization info is in AppSubmissionContext passed from client to
// RM and then to NM for AM-container launch: no difference between AM-
// localization and MapTask- or ReduceTask-localization, so can assume all
// OK. Longer-term, will need to override uber-AM container-localization
// request ("needed resources") with union of regular-AM-resources + task-
// resources (and, if maps and reduces ever differ, then union of all three
// types), OR will need localizer service/API that uber-AM can request
// after running (e.g., "localizeForTask()" or "localizeForMapTask()").
}
public void start() {
eventHandlingThread = new Thread(new SubtaskRunner(), "uber-SubtaskRunner");
eventHandlingThread.start();
super.start();
}
public void stop() {
eventHandlingThread.interrupt();
super.stop();
}
@Override
public void handle(ContainerLauncherEvent event) {
try {
eventQueue.put(event);
} catch (InterruptedException e) {
throw new YarnException(e); // FIXME? YarnException is "for runtime exceptions only"
}
}
/*
* Uber-AM lifecycle/ordering ("normal" case):
*
* - [somebody] sends TA_ASSIGNED
* - handled by ContainerAssignedTransition (TaskAttemptImpl.java)
* - creates "remoteTask" for us == real Task
* - sends CONTAINER_REMOTE_LAUNCH
* - TA: UNASSIGNED -> ASSIGNED
* - CONTAINER_REMOTE_LAUNCH handled by LocalContainerLauncher (us)
* - sucks "remoteTask" out of TaskAttemptImpl via getRemoteTask()
* - sends TA_CONTAINER_LAUNCHED
* [[ elsewhere...
* - TA_CONTAINER_LAUNCHED handled by LaunchedContainerTransition
* - registers "remoteTask" with TaskAttemptListener (== umbilical)
* - NUKES "remoteTask"
* - sends T_ATTEMPT_LAUNCHED (Task: SCHEDULED -> RUNNING)
* - TA: ASSIGNED -> RUNNING
* ]]
* - runs Task (runSubMap() or runSubReduce())
* - TA can safely send TA_UPDATE since in RUNNING state
* [modulo possible TA-state-machine race noted below: CHECK (TODO)]
*/
private class SubtaskRunner implements Runnable {
private boolean doneWithMaps = false;
private int finishedSubMaps = 0;
SubtaskRunner() {
}
@Override
public void run() {
ContainerLauncherEvent event = null;
// _must_ either run subtasks sequentially or accept expense of new JVMs
// (i.e., fork()), else will get weird failures when maps try to create/
// write same dirname or filename: no chdir() in Java
while (!Thread.currentThread().isInterrupted()) {
try {
event = eventQueue.take();
} catch (InterruptedException e) { // mostly via T_KILL? JOB_KILL?
LOG.error("Returning, interrupted : " + e);
return;
}
LOG.info("Processing the event " + event.toString());
if (event.getType() == EventType.CONTAINER_REMOTE_LAUNCH) {
ContainerRemoteLaunchEvent launchEv =
(ContainerRemoteLaunchEvent)event;
TaskAttemptId attemptID = launchEv.getTaskAttemptID(); //FIXME: can attemptID ever be null? (only if retrieved over umbilical?)
Job job = context.getAllJobs().get(attemptID.getTaskId().getJobId());
int numMapTasks = job.getTotalMaps();
int numReduceTasks = job.getTotalReduces();
// YARN (tracking) Task:
org.apache.hadoop.mapreduce.v2.app.job.Task ytask =
job.getTask(attemptID.getTaskId());
// classic mapred Task:
org.apache.hadoop.mapred.Task remoteTask = launchEv.getRemoteTask();
// after "launching," send launched event to task attempt to move
// state from ASSIGNED to RUNNING (also nukes "remoteTask", so must
// do getRemoteTask() call first)
context.getEventHandler().handle(
new TaskAttemptEvent(attemptID,
TaskAttemptEventType.TA_CONTAINER_LAUNCHED)); //FIXME: race condition here? or do we have same kind of lock on TA handler => MapTask can't send TA_UPDATE before TA_CONTAINER_LAUNCHED moves TA to RUNNING state? (probably latter)
if (numMapTasks == 0) {
doneWithMaps = true;
}
try {
if (remoteTask.isMapOrReduce()) {
JobCounterUpdateEvent jce = new JobCounterUpdateEvent(attemptID.getTaskId().getJobId());
jce.addCounterUpdate(JobCounter.TOTAL_LAUNCHED_UBERTASKS, 1);
if (remoteTask.isMapTask()) {
jce.addCounterUpdate(JobCounter.NUM_UBER_SUBMAPS, 1);
} else {
jce.addCounterUpdate(JobCounter.NUM_UBER_SUBREDUCES, 1);
}
context.getEventHandler().handle(jce);
}
runSubtask(remoteTask, ytask.getType(), attemptID, numMapTasks,
(numReduceTasks > 0));
} catch (RuntimeException re) {
JobCounterUpdateEvent jce = new JobCounterUpdateEvent(attemptID.getTaskId().getJobId());
jce.addCounterUpdate(JobCounter.NUM_FAILED_UBERTASKS, 1);
context.getEventHandler().handle(jce);
// this is our signal that the subtask failed in some way, so
// simulate a failed JVM/container and send a container-completed
// event to task attempt (i.e., move state machine from RUNNING
// to FAIL_CONTAINER_CLEANUP [and ultimately to FAILED])
context.getEventHandler().handle(new TaskAttemptEvent(attemptID,
TaskAttemptEventType.TA_CONTAINER_COMPLETED));
} catch (IOException ioe) {
// 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: "
+ StringUtils.stringifyException(ioe));
System.exit(-1);
}
} else if (event.getType() == EventType.CONTAINER_REMOTE_CLEANUP) {
// no container to kill, so just send "cleaned" event to task attempt
// to move us from SUCCESS_CONTAINER_CLEANUP to SUCCEEDED state
// (or {FAIL|KILL}_CONTAINER_CLEANUP to {FAIL|KILL}_TASK_CLEANUP)
context.getEventHandler().handle(
new TaskAttemptEvent(event.getTaskAttemptID(),
TaskAttemptEventType.TA_CONTAINER_CLEANED));
} else {
LOG.warn("Ignoring unexpected event " + event.toString());
}
}
}
private void runSubtask(org.apache.hadoop.mapred.Task task,
final TaskType taskType,
TaskAttemptId attemptID,
final int numMapTasks,
boolean renameOutputs)
throws RuntimeException, IOException {
org.apache.hadoop.mapred.TaskAttemptID classicAttemptID =
TypeConverter.fromYarn(attemptID);
try {
JobConf conf = new JobConf(getConfig());
// mark this as an uberized subtask so it can set task counter
// (longer-term/FIXME: could redefine as job counter and send
// "JobCounterEvent" to JobImpl on [successful] completion of subtask;
// will need new Job state-machine transition and JobImpl jobCounters
// map to handle)
conf.setBoolean("mapreduce.task.uberized", true);
// META-FIXME: do we want the extra sanity-checking (doneWithMaps,
// etc.), or just assume/hope the state machine(s) and uber-AM work
// as expected?
if (taskType == TaskType.MAP) {
if (doneWithMaps) {
LOG.error("CONTAINER_REMOTE_LAUNCH contains a map task ("
+ attemptID + "), but should be finished with maps");
// throw new RuntimeException() (FIXME: what's appropriate here?)
}
MapTask map = (MapTask)task;
//CODE-REVIEWER QUESTION: why not task.getConf() or map.getConf() instead of conf? do we need Task's localizeConfiguration() run on this first?
map.run(conf, umbilical);
if (renameOutputs) {
renameMapOutputForReduce(conf, attemptID, map.getMapOutputFile());
}
relocalize();
if (++finishedSubMaps == numMapTasks) {
doneWithMaps = true;
}
} else /* TaskType.REDUCE */ {
if (!doneWithMaps) {
//check if event-queue empty? whole idea of counting maps vs. checking event queue is a tad wacky...but could enforce ordering (assuming no "lost events") at LocalMRAppMaster [CURRENT BUG(?): doesn't send reduce event until maps all done]
LOG.error("CONTAINER_REMOTE_LAUNCH contains a reduce task ("
+ attemptID + "), but not yet finished with maps");
// throw new RuntimeException() (FIXME) // or push reduce event back onto end of queue? (probably former)
}
ReduceTask reduce = (ReduceTask)task;
// a.k.a. "mapreduce.jobtracker.address" in LocalJobRunner:
conf.set(MRConfig.MASTER_ADDRESS, "local"); // bypass shuffle
reduce.run(conf, umbilical);
//relocalize(); // needed only if more than one reducer supported (is MAPREDUCE-434 fixed yet?)
}
} catch (FSError e) {
LOG.fatal("FSError from child", e);
// umbilical: MRAppMaster creates (taskAttemptListener), passes to us
umbilical.fsError(classicAttemptID, e.getMessage());
throw new RuntimeException();
} catch (Exception exception) {
LOG.warn("Exception running local (uberized) 'child' : "
+ StringUtils.stringifyException(exception));
try {
if (task != null) {
// do cleanup for the task
// if (childUGI == null) { // no need to job into doAs block
task.taskCleanup(umbilical);
// } else {
// final Task taskFinal = task;
// childUGI.doAs(new PrivilegedExceptionAction<Object>() {
// @Override
// public Object run() throws Exception {
// taskFinal.taskCleanup(umbilical);
// return null;
// }
// });
// }
}
} catch (Exception e) {
LOG.info("Exception cleaning up: "
+ StringUtils.stringifyException(e));
}
// Report back any failures, for diagnostic purposes
ByteArrayOutputStream baos = new ByteArrayOutputStream();
exception.printStackTrace(new PrintStream(baos));
// if (classicAttemptID != null) {
umbilical.reportDiagnosticInfo(classicAttemptID, baos.toString());
// }
throw new RuntimeException();
} catch (Throwable throwable) {
LOG.fatal("Error running local (uberized) 'child' : "
+ StringUtils.stringifyException(throwable));
// if (classicAttemptID != null) {
Throwable tCause = throwable.getCause();
String cause = (tCause == null)
? throwable.getMessage()
: StringUtils.stringifyException(tCause);
umbilical.fatalError(classicAttemptID, cause);
// }
throw new RuntimeException();
} finally {
/*
FIXME: do we need to do any of this stuff? (guessing not since not in own JVM)
RPC.stopProxy(umbilical);
DefaultMetricsSystem.shutdown();
// Shutting down log4j of the child-vm...
// This assumes that on return from Task.run()
// there is no more logging done.
LogManager.shutdown();
*/
}
}
/* FIXME: may not need renameMapOutputForReduce() anymore? TEST!
${local.dir}/usercache/$user/appcache/$appId/$contId/ == $cwd for tasks;
contains task.sh script, which, when executed, creates symlinks and sets up env
"$local.dir"/usercache/$user/appcache/$appId/$contId/file.out
"$local.dir"/usercache/$user/appcache/$appId/$contId/file.out.idx (?)
"$local.dir"/usercache/$user/appcache/$appId/output/$taskId/ is where file.out* is moved after MapTask done
OHO! no further need for this at all? $taskId is unique per subtask
now => should work fine to leave alone. TODO: test with teragen or
similar
*/
/**
* Within the _local_ filesystem (not HDFS), all activity takes place within
* a single subdir (${local.dir}/usercache/$user/appcache/$appId/$contId/),
* and all sub-MapTasks create the same filename ("file.out"). Rename that
* to something unique (e.g., "map_0.out") to avoid collisions.
*
* Longer-term, we'll modify [something] to use TaskAttemptID-based
* filenames instead of "file.out". (All of this is entirely internal,
* so there are no particular compatibility issues.)
*/
private void renameMapOutputForReduce(JobConf conf, TaskAttemptId mapId,
MapOutputFile subMapOutputFile)
throws IOException {
FileSystem localFs = FileSystem.getLocal(conf);
// move map output to reduce input
Path mapOut = subMapOutputFile.getOutputFile();
Path reduceIn = subMapOutputFile.getInputFileForWrite(
TypeConverter.fromYarn(mapId).getTaskID(), localFs.getLength(mapOut));
if (!localFs.mkdirs(reduceIn.getParent())) {
throw new IOException("Mkdirs failed to create "
+ reduceIn.getParent().toString());
}
if (!localFs.rename(mapOut, reduceIn))
throw new IOException("Couldn't rename " + mapOut);
}
/**
* Also within the local filesystem, we need to restore the initial state
* of the directory as much as possible. Compare current contents against
* the saved original state and nuke everything that doesn't belong, with
* the exception of the renamed map outputs (see above).
FIXME: do we really need to worry about renamed map outputs, or already moved to output dir on commit? if latter, fix comment
*
* Any jobs that go out of their way to rename or delete things from the
* local directory are considered broken and deserve what they get...
*/
private void relocalize() {
File[] curLocalFiles = curDir.listFiles();
for (int j = 0; j < curLocalFiles.length; ++j) {
if (!localizedFiles.contains(curLocalFiles[j])) {
// found one that wasn't there before: delete it
boolean deleted = false;
try {
if (curFC != null) {
// this is recursive, unlike File delete():
deleted = curFC.delete(new Path(curLocalFiles[j].getName()),true);
}
} catch (IOException e) {
deleted = false;
}
if (!deleted) {
LOG.warn("Unable to delete unexpected local file/dir "
+ curLocalFiles[j].getName() + ": insufficient permissions?");
}
}
}
}
} // end SubtaskRunner
}

View File

@ -0,0 +1,264 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapred;
import java.io.File;
import java.net.InetSocketAddress;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Vector;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapred.TaskLog.LogName;
import org.apache.hadoop.mapreduce.ID;
import org.apache.hadoop.util.StringUtils;
public class MapReduceChildJVM {
private static final String SYSTEM_PATH_SEPARATOR =
System.getProperty("path.separator");
private static final Log LOG = LogFactory.getLog(MapReduceChildJVM.class);
private static File getTaskLogFile(String logDir, LogName filter) {
return new File(logDir, filter.toString());
}
private static String getChildEnv(JobConf jobConf, boolean isMap) {
if (isMap) {
return jobConf.get(JobConf.MAPRED_MAP_TASK_ENV,
jobConf.get(JobConf.MAPRED_TASK_ENV));
}
return jobConf.get(JobConf.MAPRED_REDUCE_TASK_ENV,
jobConf.get(jobConf.MAPRED_TASK_ENV));
}
public static void setVMEnv(Map<String, String> env,
List<String> classPaths, String pwd, String containerLogDir,
String nmLdLibraryPath, Task task, CharSequence applicationTokensFile) {
JobConf conf = task.conf;
// Add classpath.
CharSequence cp = env.get("CLASSPATH");
String classpath = StringUtils.join(SYSTEM_PATH_SEPARATOR, classPaths);
if (null == cp) {
env.put("CLASSPATH", classpath);
} else {
env.put("CLASSPATH", classpath + SYSTEM_PATH_SEPARATOR + cp);
}
/////// Environmental variable LD_LIBRARY_PATH
StringBuilder ldLibraryPath = new StringBuilder();
ldLibraryPath.append(nmLdLibraryPath);
ldLibraryPath.append(SYSTEM_PATH_SEPARATOR);
ldLibraryPath.append(pwd);
env.put("LD_LIBRARY_PATH", ldLibraryPath.toString());
/////// Environmental variable LD_LIBRARY_PATH
// for the child of task jvm, set hadoop.root.logger
env.put("HADOOP_ROOT_LOGGER", "DEBUG,CLA"); // TODO: Debug
// TODO: The following is useful for instance in streaming tasks. Should be
// set in ApplicationMaster's env by the RM.
String hadoopClientOpts = System.getenv("HADOOP_CLIENT_OPTS");
if (hadoopClientOpts == null) {
hadoopClientOpts = "";
} else {
hadoopClientOpts = hadoopClientOpts + " ";
}
// FIXME: don't think this is also needed given we already set java
// properties.
long logSize = TaskLog.getTaskLogLength(conf);
Vector<String> logProps = new Vector<String>(4);
setupLog4jProperties(logProps, logSize, containerLogDir);
Iterator<String> it = logProps.iterator();
StringBuffer buffer = new StringBuffer();
while (it.hasNext()) {
buffer.append(" " + it.next());
}
hadoopClientOpts = hadoopClientOpts + buffer.toString();
env.put("HADOOP_CLIENT_OPTS", hadoopClientOpts);
// add the env variables passed by the user
String mapredChildEnv = getChildEnv(conf, task.isMapTask());
if (mapredChildEnv != null && mapredChildEnv.length() > 0) {
String childEnvs[] = mapredChildEnv.split(",");
for (String cEnv : childEnvs) {
String[] parts = cEnv.split("="); // split on '='
String value = (String) env.get(parts[0]);
if (value != null) {
// replace $env with the child's env constructed by tt's
// example LD_LIBRARY_PATH=$LD_LIBRARY_PATH:/tmp
value = parts[1].replace("$" + parts[0], value);
} else {
// this key is not configured by the tt for the child .. get it
// from the tt's env
// example PATH=$PATH:/tmp
value = System.getenv(parts[0]); // Get from NM?
if (value != null) {
// the env key is present in the tt's env
value = parts[1].replace("$" + parts[0], value);
} else {
// the env key is note present anywhere .. simply set it
// example X=$X:/tmp or X=/tmp
value = parts[1].replace("$" + parts[0], "");
}
}
env.put(parts[0], value);
}
}
//This should not be set here (If an OS check is requied. moved to ContainerLuanch)
// env.put("JVM_PID", "`echo $$`");
env.put(Constants.STDOUT_LOGFILE_ENV,
getTaskLogFile(containerLogDir, TaskLog.LogName.STDOUT).toString());
env.put(Constants.STDERR_LOGFILE_ENV,
getTaskLogFile(containerLogDir, TaskLog.LogName.STDERR).toString());
}
private static String getChildJavaOpts(JobConf jobConf, boolean isMapTask) {
if (isMapTask) {
return jobConf.get(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, jobConf.get(
JobConf.MAPRED_TASK_JAVA_OPTS,
JobConf.DEFAULT_MAPRED_TASK_JAVA_OPTS));
}
return jobConf
.get(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, jobConf.get(
JobConf.MAPRED_TASK_JAVA_OPTS,
JobConf.DEFAULT_MAPRED_TASK_JAVA_OPTS));
}
private static void setupLog4jProperties(Vector<String> vargs,
long logSize, String containerLogDir) {
vargs.add("-Dlog4j.configuration=container-log4j.properties");
vargs.add("-Dhadoop.yarn.mr.containerLogDir=" + containerLogDir);
vargs.add("-Dhadoop.yarn.mr.totalLogFileSize=" + logSize);
}
public static List<String> getVMCommand(
InetSocketAddress taskAttemptListenerAddr, Task task, String javaHome,
String workDir, String logDir, String childTmpDir, ID jvmID) {
TaskAttemptID attemptID = task.getTaskID();
JobConf conf = task.conf;
Vector<String> vargs = new Vector<String>(8);
vargs.add("exec");
vargs.add(javaHome + "/bin/java");
// Add child (task) java-vm options.
//
// The following symbols if present in mapred.{map|reduce}.child.java.opts
// value are replaced:
// + @taskid@ is interpolated with value of TaskID.
// Other occurrences of @ will not be altered.
//
// Example with multiple arguments and substitutions, showing
// jvm GC logging, and start of a passwordless JVM JMX agent so can
// connect with jconsole and the likes to watch child memory, threads
// and get thread dumps.
//
// <property>
// <name>mapred.map.child.java.opts</name>
// <value>-Xmx 512M -verbose:gc -Xloggc:/tmp/@taskid@.gc \
// -Dcom.sun.management.jmxremote.authenticate=false \
// -Dcom.sun.management.jmxremote.ssl=false \
// </value>
// </property>
//
// <property>
// <name>mapred.reduce.child.java.opts</name>
// <value>-Xmx 1024M -verbose:gc -Xloggc:/tmp/@taskid@.gc \
// -Dcom.sun.management.jmxremote.authenticate=false \
// -Dcom.sun.management.jmxremote.ssl=false \
// </value>
// </property>
//
String javaOpts = getChildJavaOpts(conf, task.isMapTask());
javaOpts = javaOpts.replace("@taskid@", attemptID.toString());
String [] javaOptsSplit = javaOpts.split(" ");
// Add java.library.path; necessary for loading native libraries.
//
// 1. We add the 'cwd' of the task to it's java.library.path to help
// users distribute native libraries via the DistributedCache.
// 2. The user can also specify extra paths to be added to the
// java.library.path via mapred.{map|reduce}.child.java.opts.
//
String libraryPath = workDir;
boolean hasUserLDPath = false;
for(int i=0; i<javaOptsSplit.length ;i++) {
if(javaOptsSplit[i].startsWith("-Djava.library.path=")) {
// TODO: Does the above take care of escaped space chars
javaOptsSplit[i] += SYSTEM_PATH_SEPARATOR + libraryPath;
hasUserLDPath = true;
break;
}
}
if(!hasUserLDPath) {
vargs.add("-Djava.library.path=" + libraryPath);
}
for (int i = 0; i < javaOptsSplit.length; i++) {
vargs.add(javaOptsSplit[i]);
}
if (childTmpDir != null) {
vargs.add("-Djava.io.tmpdir=" + childTmpDir);
}
// Setup the log4j prop
long logSize = TaskLog.getTaskLogLength(conf);
setupLog4jProperties(vargs, logSize, logDir);
if (conf.getProfileEnabled()) {
if (conf.getProfileTaskRange(task.isMapTask()
).isIncluded(task.getPartition())) {
File prof = getTaskLogFile(logDir, TaskLog.LogName.PROFILE);
vargs.add(String.format(conf.getProfileParams(), prof.toString()));
}
}
// Add main class and its arguments
vargs.add(YarnChild.class.getName()); // main of Child
// pass TaskAttemptListener's address
vargs.add(taskAttemptListenerAddr.getAddress().getHostAddress());
vargs.add(Integer.toString(taskAttemptListenerAddr.getPort()));
vargs.add(attemptID.toString()); // pass task identifier
// Finally add the jvmID
vargs.add(String.valueOf(jvmID.getId()));
vargs.add("1>" + getTaskLogFile(logDir, TaskLog.LogName.STDERR));
vargs.add("2>" + getTaskLogFile(logDir, TaskLog.LogName.STDOUT));
// Final commmand
StringBuilder mergedCommand = new StringBuilder();
for (CharSequence str : vargs) {
mergedCommand.append(str).append(" ");
}
Vector<String> vargsFinal = new Vector<String>(1);
vargsFinal.add(mergedCommand.toString());
return vargsFinal;
}
}

View File

@ -0,0 +1,65 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapred;
import java.util.Collection;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
import org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.event.EventHandler;
public class MapTaskAttemptImpl extends TaskAttemptImpl {
private final TaskSplitMetaInfo splitInfo;
public MapTaskAttemptImpl(TaskId taskId, int attempt,
EventHandler eventHandler, Path jobFile,
int partition, TaskSplitMetaInfo splitInfo, Configuration conf,
TaskAttemptListener taskAttemptListener,
OutputCommitter committer, Token<JobTokenIdentifier> jobToken,
Collection<Token<? extends TokenIdentifier>> fsTokens, Clock clock) {
super(taskId, attempt, eventHandler,
taskAttemptListener, jobFile, partition, conf, splitInfo.getLocations(),
committer, jobToken, fsTokens, clock);
this.splitInfo = splitInfo;
}
@Override
public Task createRemoteTask() {
//job file name is set in TaskAttempt, setting it null here
MapTask mapTask =
new MapTask("", TypeConverter.fromYarn(getID()), partition,
splitInfo.getSplitIndex(), 1); // YARN doesn't have the concept of slots per task, set it as 1.
mapTask.setUser(conf.get(MRJobConfig.USER_NAME));
mapTask.setConf(conf);
return mapTask;
}
}

View File

@ -0,0 +1,64 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapred;
import java.util.Collection;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
import org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.event.EventHandler;
public class ReduceTaskAttemptImpl extends TaskAttemptImpl {
private final int numMapTasks;
public ReduceTaskAttemptImpl(TaskId id, int attempt,
EventHandler eventHandler, Path jobFile, int partition,
int numMapTasks, Configuration conf,
TaskAttemptListener taskAttemptListener, OutputCommitter committer,
Token<JobTokenIdentifier> jobToken,
Collection<Token<? extends TokenIdentifier>> fsTokens, Clock clock) {
super(id, attempt, eventHandler, taskAttemptListener, jobFile, partition,
conf, new String[] {}, committer, jobToken, fsTokens, clock);
this.numMapTasks = numMapTasks;
}
@Override
public Task createRemoteTask() {
//job file name is set in TaskAttempt, setting it null here
ReduceTask reduceTask =
new ReduceTask("", TypeConverter.fromYarn(getID()), partition,
numMapTasks, 1); // YARN doesn't have the concept of slots per task, set it as 1.
reduceTask.setUser(conf.get(MRJobConfig.USER_NAME));
reduceTask.setConf(conf);
return reduceTask;
}
}

View File

@ -0,0 +1,434 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapred;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RPC.Server;
import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.mapred.SortedRanges.Range;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
import org.apache.hadoop.mapreduce.v2.app.TaskHeartbeatHandler;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.service.CompositeService;
/**
* This class is responsible for talking to the task umblical.
* It also converts all the old data structures
* to yarn data structures.
*
* This class HAS to be in this package to access package private
* methods/classes.
*/
public class TaskAttemptListenerImpl extends CompositeService
implements TaskUmbilicalProtocol, TaskAttemptListener {
private static final Log LOG = LogFactory.getLog(TaskAttemptListenerImpl.class);
private AppContext context;
private Server server;
private TaskHeartbeatHandler taskHeartbeatHandler;
private InetSocketAddress address;
private Map<WrappedJvmID, org.apache.hadoop.mapred.Task> jvmIDToAttemptMap =
Collections.synchronizedMap(new HashMap<WrappedJvmID,
org.apache.hadoop.mapred.Task>());
private JobTokenSecretManager jobTokenSecretManager = null;
public TaskAttemptListenerImpl(AppContext context,
JobTokenSecretManager jobTokenSecretManager) {
super(TaskAttemptListenerImpl.class.getName());
this.context = context;
this.jobTokenSecretManager = jobTokenSecretManager;
}
@Override
public void init(Configuration conf) {
registerHeartbeatHandler();
super.init(conf);
}
@Override
public void start() {
startRpcServer();
super.start();
}
protected void registerHeartbeatHandler() {
taskHeartbeatHandler = new TaskHeartbeatHandler(context.getEventHandler(),
context.getClock());
addService(taskHeartbeatHandler);
}
protected void startRpcServer() {
Configuration conf = getConfig();
try {
server =
RPC.getServer(TaskUmbilicalProtocol.class, this, "0.0.0.0", 0,
conf.getInt(AMConstants.AM_TASK_LISTENER_THREADS,
AMConstants.DEFAULT_AM_TASK_LISTENER_THREADS),
false, conf, jobTokenSecretManager);
server.start();
InetSocketAddress listenerAddress = server.getListenerAddress();
this.address =
NetUtils.createSocketAddr(listenerAddress.getAddress()
.getLocalHost().getCanonicalHostName()
+ ":" + listenerAddress.getPort());
} catch (IOException e) {
throw new YarnException(e);
}
}
@Override
public void stop() {
stopRpcServer();
super.stop();
}
protected void stopRpcServer() {
server.stop();
}
@Override
public InetSocketAddress getAddress() {
return address;
}
/**
* Child checking whether it can commit.
*
* <br/>
* Commit is a two-phased protocol. First the attempt informs the
* ApplicationMaster that it is
* {@link #commitPending(TaskAttemptID, TaskStatus)}. Then it repeatedly polls
* the ApplicationMaster whether it {@link #canCommit(TaskAttemptID)} This is
* a legacy from the centralized commit protocol handling by the JobTracker.
*/
@Override
public boolean canCommit(TaskAttemptID taskAttemptID) throws IOException {
LOG.info("Commit go/no-go request from " + taskAttemptID.toString());
// An attempt is asking if it can commit its output. This can be decided
// only by the task which is managing the multiple attempts. So redirect the
// request there.
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
taskHeartbeatHandler.receivedPing(attemptID);
Job job = context.getJob(attemptID.getTaskId().getJobId());
Task task = job.getTask(attemptID.getTaskId());
return task.canCommit(attemptID);
}
/**
* TaskAttempt is reporting that it is in commit_pending and it is waiting for
* the commit Response
*
* <br/>
* Commit it a two-phased protocol. First the attempt informs the
* ApplicationMaster that it is
* {@link #commitPending(TaskAttemptID, TaskStatus)}. Then it repeatedly polls
* the ApplicationMaster whether it {@link #canCommit(TaskAttemptID)} This is
* a legacy from the centralized commit protocol handling by the JobTracker.
*/
@Override
public void commitPending(TaskAttemptID taskAttemptID, TaskStatus taskStatsu)
throws IOException, InterruptedException {
LOG.info("Commit-pending state update from " + taskAttemptID.toString());
// An attempt is asking if it can commit its output. This can be decided
// only by the task which is managing the multiple attempts. So redirect the
// request there.
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
taskHeartbeatHandler.receivedPing(attemptID);
//Ignorable TaskStatus? - since a task will send a LastStatusUpdate
context.getEventHandler().handle(
new TaskAttemptEvent(attemptID,
TaskAttemptEventType.TA_COMMIT_PENDING));
}
@Override
public void done(TaskAttemptID taskAttemptID) throws IOException {
LOG.info("Done acknowledgement from " + taskAttemptID.toString());
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
taskHeartbeatHandler.receivedPing(attemptID);
context.getEventHandler().handle(
new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_DONE));
}
@Override
public void fatalError(TaskAttemptID taskAttemptID, String msg)
throws IOException {
// This happens only in Child and in the Task.
LOG.fatal("Task: " + taskAttemptID + " - exited : " + msg);
reportDiagnosticInfo(taskAttemptID, "Error: " + msg);
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
context.getEventHandler().handle(
new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_FAILMSG));
}
@Override
public void fsError(TaskAttemptID taskAttemptID, String message)
throws IOException {
// This happens only in Child.
LOG.fatal("Task: " + taskAttemptID + " - failed due to FSError: "
+ message);
reportDiagnosticInfo(taskAttemptID, "FSError: " + message);
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
context.getEventHandler().handle(
new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_FAILMSG));
}
@Override
public void shuffleError(TaskAttemptID taskAttemptID, String message) throws IOException {
// TODO: This isn't really used in any MR code. Ask for removal.
}
@Override
public MapTaskCompletionEventsUpdate getMapCompletionEvents(
JobID jobIdentifier, int fromEventId, int maxEvents,
TaskAttemptID taskAttemptID) throws IOException {
LOG.info("MapCompletionEvents request from " + taskAttemptID.toString()
+ ". fromEventID " + fromEventId + " maxEvents " + maxEvents);
// TODO: shouldReset is never used. See TT. Ask for Removal.
boolean shouldReset = false;
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent[] events =
context.getJob(attemptID.getTaskId().getJobId()).getTaskAttemptCompletionEvents(
fromEventId, maxEvents);
taskHeartbeatHandler.receivedPing(attemptID);
// filter the events to return only map completion events in old format
List<TaskCompletionEvent> mapEvents = new ArrayList<TaskCompletionEvent>();
for (org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent event : events) {
if (TaskType.MAP.equals(event.getAttemptId().getTaskId().getTaskType())) {
mapEvents.add(TypeConverter.fromYarn(event));
}
}
return new MapTaskCompletionEventsUpdate(
mapEvents.toArray(new TaskCompletionEvent[0]), shouldReset);
}
@Override
public boolean ping(TaskAttemptID taskAttemptID) throws IOException {
LOG.info("Ping from " + taskAttemptID.toString());
taskHeartbeatHandler.receivedPing(TypeConverter.toYarn(taskAttemptID));
return true;
}
@Override
public void reportDiagnosticInfo(TaskAttemptID taskAttemptID, String diagnosticInfo)
throws IOException {
LOG.info("Diagnostics report from " + taskAttemptID.toString() + ": "
+ diagnosticInfo);
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID =
TypeConverter.toYarn(taskAttemptID);
taskHeartbeatHandler.receivedPing(attemptID);
// This is mainly used for cases where we want to propagate exception traces
// of tasks that fail.
// This call exists as a hadoop mapreduce legacy wherein all changes in
// counters/progress/phase/output-size are reported through statusUpdate()
// call but not diagnosticInformation.
context.getEventHandler().handle(
new TaskAttemptDiagnosticsUpdateEvent(attemptID, diagnosticInfo));
}
@Override
public boolean statusUpdate(TaskAttemptID taskAttemptID,
TaskStatus taskStatus) throws IOException, InterruptedException {
LOG.info("Status update from " + taskAttemptID.toString());
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId yarnAttemptID =
TypeConverter.toYarn(taskAttemptID);
taskHeartbeatHandler.receivedPing(yarnAttemptID);
TaskAttemptStatus taskAttemptStatus =
new TaskAttemptStatus();
taskAttemptStatus.id = yarnAttemptID;
// Task sends the updated progress to the TT.
taskAttemptStatus.progress = taskStatus.getProgress();
LOG.info("Progress of TaskAttempt " + taskAttemptID + " is : "
+ taskStatus.getProgress());
// Task sends the diagnostic information to the TT
taskAttemptStatus.diagnosticInfo = taskStatus.getDiagnosticInfo();
// Task sends the updated state-string to the TT.
taskAttemptStatus.stateString = taskStatus.getStateString();
// Set the output-size when map-task finishes. Set by the task itself.
taskAttemptStatus.outputSize = taskStatus.getOutputSize();
// Task sends the updated phase to the TT.
taskAttemptStatus.phase = TypeConverter.toYarn(taskStatus.getPhase());
// Counters are updated by the task.
taskAttemptStatus.counters =
TypeConverter.toYarn(taskStatus.getCounters());
// Map Finish time set by the task (map only)
if (taskStatus.getIsMap() && taskStatus.getMapFinishTime() != 0) {
taskAttemptStatus.mapFinishTime = taskStatus.getMapFinishTime();
}
// Shuffle Finish time set by the task (reduce only).
if (!taskStatus.getIsMap() && taskStatus.getShuffleFinishTime() != 0) {
taskAttemptStatus.shuffleFinishTime = taskStatus.getShuffleFinishTime();
}
// Sort finish time set by the task (reduce only).
if (!taskStatus.getIsMap() && taskStatus.getSortFinishTime() != 0) {
taskAttemptStatus.sortFinishTime = taskStatus.getSortFinishTime();
}
// Not Setting the task state. Used by speculation - will be set in TaskAttemptImpl
//taskAttemptStatus.taskState = TypeConverter.toYarn(taskStatus.getRunState());
//set the fetch failures
if (taskStatus.getFetchFailedMaps() != null
&& taskStatus.getFetchFailedMaps().size() > 0) {
taskAttemptStatus.fetchFailedMaps =
new ArrayList<org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId>();
for (TaskAttemptID failedMapId : taskStatus.getFetchFailedMaps()) {
taskAttemptStatus.fetchFailedMaps.add(
TypeConverter.toYarn(failedMapId));
}
}
// Task sends the information about the nextRecordRange to the TT
// TODO: The following are not needed here, but needed to be set somewhere inside AppMaster.
// taskStatus.getRunState(); // Set by the TT/JT. Transform into a state TODO
// taskStatus.getStartTime(); // Used to be set by the TaskTracker. This should be set by getTask().
// taskStatus.getFinishTime(); // Used to be set by TT/JT. Should be set when task finishes
// // This was used by TT to do counter updates only once every minute. So this
// // isn't ever changed by the Task itself.
// taskStatus.getIncludeCounters();
context.getEventHandler().handle(
new TaskAttemptStatusUpdateEvent(taskAttemptStatus.id,
taskAttemptStatus));
return true;
}
@Override
public long getProtocolVersion(String arg0, long arg1) throws IOException {
return TaskUmbilicalProtocol.versionID;
}
@Override
public void reportNextRecordRange(TaskAttemptID taskAttemptID, Range range)
throws IOException {
// This is used when the feature of skipping records is enabled.
// This call exists as a hadoop mapreduce legacy wherein all changes in
// counters/progress/phase/output-size are reported through statusUpdate()
// call but not the next record range information.
throw new IOException("Not yet implemented.");
}
@Override
public JvmTask getTask(JvmContext context) throws IOException {
// A rough imitation of code from TaskTracker.
JVMId jvmId = context.jvmId;
LOG.info("JVM with ID : " + jvmId + " asked for a task");
// TODO: Is it an authorised container to get a task? Otherwise return null.
// TODO: Is the request for task-launch still valid?
// TODO: Child.java's firstTaskID isn't really firstTaskID. Ask for update
// to jobId and task-type.
WrappedJvmID wJvmID = new WrappedJvmID(jvmId.getJobId(), jvmId.isMap,
jvmId.getId());
org.apache.hadoop.mapred.Task task = jvmIDToAttemptMap.get(wJvmID);
if (task != null) { //there may be lag in the attempt getting added here
LOG.info("JVM with ID: " + jvmId + " given task: " + task.getTaskID());
JvmTask jvmTask = new JvmTask(task, false);
//remove the task as it is no more needed and free up the memory
jvmIDToAttemptMap.remove(wJvmID);
return jvmTask;
}
return null;
}
@Override
public void register(org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID,
org.apache.hadoop.mapred.Task task, WrappedJvmID jvmID) {
//create the mapping so that it is easy to look up
//when it comes back to ask for Task.
jvmIDToAttemptMap.put(jvmID, task);
//register this attempt
taskHeartbeatHandler.register(attemptID);
}
@Override
public void unregister(org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID,
WrappedJvmID jvmID) {
//remove the mapping if not already removed
jvmIDToAttemptMap.remove(jvmID);
//unregister this attempt
taskHeartbeatHandler.unregister(attemptID);
}
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
return ProtocolSignature.getProtocolSignature(this,
protocol, clientVersion, clientMethodsHash);
}
}

View File

@ -0,0 +1,30 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapred;
/**
* A simple wrapper for increasing the visibility.
*/
public class WrappedJvmID extends JVMId {
public WrappedJvmID(JobID jobID, boolean mapTask, int nextInt) {
super(jobID, mapTask, nextInt);
}
}

View File

@ -0,0 +1,15 @@
package org.apache.hadoop.mapred;
//Workaround for PeriodicStateAccumulator being package access
public class WrappedPeriodicStatsAccumulator {
private PeriodicStatsAccumulator real;
public WrappedPeriodicStatsAccumulator(PeriodicStatsAccumulator real) {
this.real = real;
}
public void extend(double newProgress, int newValue) {
real.extend(newProgress, newValue);
}
}

View File

@ -0,0 +1,52 @@
package org.apache.hadoop.mapred;
// Workaround for ProgressSplitBlock being package access
public class WrappedProgressSplitsBlock extends ProgressSplitsBlock {
public static final int DEFAULT_NUMBER_PROGRESS_SPLITS = 12;
private WrappedPeriodicStatsAccumulator wrappedProgressWallclockTime;
private WrappedPeriodicStatsAccumulator wrappedProgressCPUTime;
private WrappedPeriodicStatsAccumulator wrappedProgressVirtualMemoryKbytes;
private WrappedPeriodicStatsAccumulator wrappedProgressPhysicalMemoryKbytes;
public WrappedProgressSplitsBlock(int numberSplits) {
super(numberSplits);
}
public int[][] burst() {
return super.burst();
}
public WrappedPeriodicStatsAccumulator getProgressWallclockTime() {
if (wrappedProgressWallclockTime == null) {
wrappedProgressWallclockTime = new WrappedPeriodicStatsAccumulator(
progressWallclockTime);
}
return wrappedProgressWallclockTime;
}
public WrappedPeriodicStatsAccumulator getProgressCPUTime() {
if (wrappedProgressCPUTime == null) {
wrappedProgressCPUTime = new WrappedPeriodicStatsAccumulator(
progressCPUTime);
}
return wrappedProgressCPUTime;
}
public WrappedPeriodicStatsAccumulator getProgressVirtualMemoryKbytes() {
if (wrappedProgressVirtualMemoryKbytes == null) {
wrappedProgressVirtualMemoryKbytes = new WrappedPeriodicStatsAccumulator(
progressVirtualMemoryKbytes);
}
return wrappedProgressVirtualMemoryKbytes;
}
public WrappedPeriodicStatsAccumulator getProgressPhysicalMemoryKbytes() {
if (wrappedProgressPhysicalMemoryKbytes == null) {
wrappedProgressPhysicalMemoryKbytes = new WrappedPeriodicStatsAccumulator(
progressPhysicalMemoryKbytes);
}
return wrappedProgressPhysicalMemoryKbytes;
}
}

View File

@ -0,0 +1,346 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapred;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.io.PrintStream;
import java.net.InetSocketAddress;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
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.FSError;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.mapreduce.MRConfig;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.TaskType;
import org.apache.hadoop.mapreduce.filecache.DistributedCache;
import org.apache.hadoop.mapreduce.security.TokenCache;
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
import org.apache.hadoop.mapreduce.v2.MRConstants;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.source.JvmMetrics;
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.util.StringUtils;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.log4j.LogManager;
/**
* The main() for MapReduce task processes.
*/
class YarnChild {
private static final Log LOG = LogFactory.getLog(YarnChild.class);
static volatile TaskAttemptID taskid = null;
public static void main(String[] args) throws Throwable {
LOG.debug("Child starting");
final JobConf defaultConf = new JobConf();
defaultConf.addResource(MRConstants.JOB_CONF_FILE);
UserGroupInformation.setConfiguration(defaultConf);
String host = args[0];
int port = Integer.parseInt(args[1]);
final InetSocketAddress address = new InetSocketAddress(host, port);
final TaskAttemptID firstTaskid = TaskAttemptID.forName(args[2]);
int jvmIdInt = Integer.parseInt(args[3]);
JVMId jvmId = new JVMId(firstTaskid.getJobID(),
firstTaskid.getTaskType() == TaskType.MAP, jvmIdInt);
// initialize metrics
DefaultMetricsSystem.initialize(
StringUtils.camelize(firstTaskid.getTaskType().name()) +"Task");
Token<JobTokenIdentifier> jt = loadCredentials(defaultConf, address);
// Create TaskUmbilicalProtocol as actual task owner.
UserGroupInformation taskOwner =
UserGroupInformation.createRemoteUser(firstTaskid.getJobID().toString());
taskOwner.addToken(jt);
final TaskUmbilicalProtocol umbilical =
taskOwner.doAs(new PrivilegedExceptionAction<TaskUmbilicalProtocol>() {
@Override
public TaskUmbilicalProtocol run() throws Exception {
return (TaskUmbilicalProtocol)RPC.getProxy(TaskUmbilicalProtocol.class,
TaskUmbilicalProtocol.versionID, address, defaultConf);
}
});
// report non-pid to application master
JvmContext context = new JvmContext(jvmId, "-1000");
LOG.debug("PID: " + System.getenv().get("JVM_PID"));
Task task = null;
UserGroupInformation childUGI = null;
try {
int idleLoopCount = 0;
JvmTask myTask = null;;
// poll for new task
for (int idle = 0; null == myTask; ++idle) {
long sleepTimeMilliSecs = Math.min(idle * 500, 1500);
LOG.info("Sleeping for " + sleepTimeMilliSecs
+ "ms before retrying again. Got null now.");
MILLISECONDS.sleep(sleepTimeMilliSecs);
myTask = umbilical.getTask(context);
}
if (myTask.shouldDie()) {
return;
}
task = myTask.getTask();
YarnChild.taskid = task.getTaskID();
// Create the job-conf and set credentials
final JobConf job =
configureTask(task, defaultConf.getCredentials(), jt);
// Initiate Java VM metrics
JvmMetrics.initSingleton(jvmId.toString(), job.getSessionId());
LOG.debug("Remote user: " + job.get("user.name"));
childUGI = UserGroupInformation.createRemoteUser(job.get("user.name"));
// Add tokens to new user so that it may execute its task correctly.
for(Token<?> token : UserGroupInformation.getCurrentUser().getTokens()) {
childUGI.addToken(token);
}
// Create a final reference to the task for the doAs block
final Task taskFinal = task;
childUGI.doAs(new PrivilegedExceptionAction<Object>() {
@Override
public Object run() throws Exception {
// use job-specified working directory
FileSystem.get(job).setWorkingDirectory(job.getWorkingDirectory());
taskFinal.run(job, umbilical); // run the task
return null;
}
});
} catch (FSError e) {
LOG.fatal("FSError from child", e);
umbilical.fsError(taskid, e.getMessage());
} catch (Exception exception) {
LOG.warn("Exception running child : "
+ StringUtils.stringifyException(exception));
try {
if (task != null) {
// do cleanup for the task
if (childUGI == null) { // no need to job into doAs block
task.taskCleanup(umbilical);
} else {
final Task taskFinal = task;
childUGI.doAs(new PrivilegedExceptionAction<Object>() {
@Override
public Object run() throws Exception {
taskFinal.taskCleanup(umbilical);
return null;
}
});
}
}
} catch (Exception e) {
LOG.info("Exception cleaning up: " + StringUtils.stringifyException(e));
}
// Report back any failures, for diagnostic purposes
ByteArrayOutputStream baos = new ByteArrayOutputStream();
exception.printStackTrace(new PrintStream(baos));
if (taskid != null) {
umbilical.reportDiagnosticInfo(taskid, baos.toString());
}
} catch (Throwable throwable) {
LOG.fatal("Error running child : "
+ StringUtils.stringifyException(throwable));
if (taskid != null) {
Throwable tCause = throwable.getCause();
String cause = tCause == null
? throwable.getMessage()
: StringUtils.stringifyException(tCause);
umbilical.fatalError(taskid, cause);
}
} finally {
RPC.stopProxy(umbilical);
DefaultMetricsSystem.shutdown();
// Shutting down log4j of the child-vm...
// This assumes that on return from Task.run()
// there is no more logging done.
LogManager.shutdown();
}
}
private static Token<JobTokenIdentifier> loadCredentials(JobConf conf,
InetSocketAddress address) throws IOException {
//load token cache storage
String tokenFileLocation =
System.getenv(ApplicationConstants.CONTAINER_TOKEN_FILE_ENV_NAME);
String jobTokenFile =
new Path(tokenFileLocation).makeQualified(FileSystem.getLocal(conf))
.toUri().getPath();
Credentials credentials =
TokenCache.loadTokens(jobTokenFile, conf);
LOG.debug("loading token. # keys =" +credentials.numberOfSecretKeys() +
"; from file=" + jobTokenFile);
Token<JobTokenIdentifier> jt = TokenCache.getJobToken(credentials);
jt.setService(new Text(address.getAddress().getHostAddress() + ":"
+ address.getPort()));
UserGroupInformation current = UserGroupInformation.getCurrentUser();
current.addToken(jt);
for (Token<? extends TokenIdentifier> tok : credentials.getAllTokens()) {
current.addToken(tok);
}
// Set the credentials
conf.setCredentials(credentials);
return jt;
}
/**
* Configure mapred-local dirs. This config is used by the task for finding
* out an output directory.
*/
private static void configureLocalDirs(Task task, JobConf job) {
String[] localSysDirs = StringUtils.getTrimmedStrings(
System.getenv(ApplicationConstants.LOCAL_DIR_ENV));
job.setStrings(MRConfig.LOCAL_DIR, localSysDirs);
LOG.info(MRConfig.LOCAL_DIR + " for child: " + job.get(MRConfig.LOCAL_DIR));
}
private static JobConf configureTask(Task task, Credentials credentials,
Token<JobTokenIdentifier> jt) throws IOException {
final JobConf job = new JobConf(MRConstants.JOB_CONF_FILE);
job.setCredentials(credentials);
// set tcp nodelay
job.setBoolean("ipc.client.tcpnodelay", true);
job.setClass(MRConfig.TASK_LOCAL_OUTPUT_CLASS,
YarnOutputFiles.class, MapOutputFile.class);
// set the jobTokenFile into task
task.setJobTokenSecret(
JobTokenSecretManager.createSecretKey(jt.getPassword()));
// setup the child's MRConfig.LOCAL_DIR.
configureLocalDirs(task, job);
// setup the child's attempt directories
// Do the task-type specific localization
task.localizeConfiguration(job);
// Set up the DistributedCache related configs
setupDistributedCacheConfig(job);
// Overwrite the localized task jobconf which is linked to in the current
// work-dir.
Path localTaskFile = new Path(Constants.JOBFILE);
writeLocalJobFile(localTaskFile, job);
task.setJobFile(localTaskFile.toString());
task.setConf(job);
return job;
}
/**
* Set up the DistributedCache related configs to make
* {@link DistributedCache#getLocalCacheFiles(Configuration)}
* and
* {@link DistributedCache#getLocalCacheArchives(Configuration)}
* working.
* @param job
* @throws IOException
*/
private static void setupDistributedCacheConfig(final JobConf job)
throws IOException {
String localWorkDir = System.getenv("PWD");
// ^ ^ all symlinks are created in the current work-dir
// Update the configuration object with localized archives.
URI[] cacheArchives = DistributedCache.getCacheArchives(job);
if (cacheArchives != null) {
List<String> localArchives = new ArrayList<String>();
for (int i = 0; i < cacheArchives.length; ++i) {
URI u = cacheArchives[i];
Path p = new Path(u);
Path name =
new Path((null == u.getFragment()) ? p.getName()
: u.getFragment());
String linkName = name.toUri().getPath();
localArchives.add(new Path(localWorkDir, linkName).toUri().getPath());
}
if (!localArchives.isEmpty()) {
job.set(MRJobConfig.CACHE_LOCALARCHIVES, StringUtils
.arrayToString(localArchives.toArray(new String[localArchives
.size()])));
}
}
// Update the configuration object with localized files.
URI[] cacheFiles = DistributedCache.getCacheFiles(job);
if (cacheFiles != null) {
List<String> localFiles = new ArrayList<String>();
for (int i = 0; i < cacheFiles.length; ++i) {
URI u = cacheFiles[i];
Path p = new Path(u);
Path name =
new Path((null == u.getFragment()) ? p.getName()
: u.getFragment());
String linkName = name.toUri().getPath();
localFiles.add(new Path(localWorkDir, linkName).toUri().getPath());
}
if (!localFiles.isEmpty()) {
job.set(MRJobConfig.CACHE_LOCALFILES,
StringUtils.arrayToString(localFiles
.toArray(new String[localFiles.size()])));
}
}
}
private static final FsPermission urw_gr =
FsPermission.createImmutable((short) 0640);
/**
* Write the task specific job-configuration file.
* @throws IOException
*/
private static void writeLocalJobFile(Path jobFile, JobConf conf)
throws IOException {
FileSystem localFs = FileSystem.getLocal(conf);
localFs.delete(jobFile);
OutputStream out = null;
try {
out = FileSystem.create(localFs, jobFile, urw_gr);
conf.writeXml(out);
} finally {
IOUtils.cleanup(LOG, out);
}
}
}

View File

@ -0,0 +1,238 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
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;
import org.apache.hadoop.fs.LocalDirAllocator;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.JobContext;
import org.apache.hadoop.mapreduce.MRConfig;
/**
* Manipulate the working area for the transient store for maps and reduces.
*
* This class is used by map and reduce tasks to identify the directories that
* they need to write to/read from for intermediate files. The callers of
* these methods are from child space.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class YarnOutputFiles extends MapOutputFile {
private JobConf conf;
private static final String JOB_OUTPUT_DIR = "output";
private static final String SPILL_FILE_PATTERN = "%s_spill_%d.out";
private static final String SPILL_INDEX_FILE_PATTERN = SPILL_FILE_PATTERN
+ ".index";
public YarnOutputFiles() {
}
// assume configured to $localdir/usercache/$user/appcache/$appId
private LocalDirAllocator lDirAlloc =
new LocalDirAllocator(MRConfig.LOCAL_DIR);
private Path getAttemptOutputDir() {
return new Path(JOB_OUTPUT_DIR, conf.get(JobContext.TASK_ATTEMPT_ID));
}
/**
* Return the path to local map output file created earlier
*
* @return path
* @throws IOException
*/
public Path getOutputFile() throws IOException {
Path attemptOutput =
new Path(getAttemptOutputDir(), MAP_OUTPUT_FILENAME_STRING);
return lDirAlloc.getLocalPathToRead(attemptOutput.toString(), conf);
}
/**
* Create a local map output file name.
*
* @param size the size of the file
* @return path
* @throws IOException
*/
public Path getOutputFileForWrite(long size) throws IOException {
Path attemptOutput =
new Path(getAttemptOutputDir(), MAP_OUTPUT_FILENAME_STRING);
return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), size, conf);
}
/**
* Create a local map output file name on the same volume.
*/
public Path getOutputFileForWriteInVolume(Path existing) {
Path outputDir = new Path(existing.getParent(), JOB_OUTPUT_DIR);
Path attemptOutputDir = new Path(outputDir,
conf.get(JobContext.TASK_ATTEMPT_ID));
return new Path(attemptOutputDir, MAP_OUTPUT_FILENAME_STRING);
}
/**
* Return the path to a local map output index file created earlier
*
* @return path
* @throws IOException
*/
public Path getOutputIndexFile() throws IOException {
Path attemptIndexOutput =
new Path(getAttemptOutputDir(), MAP_OUTPUT_FILENAME_STRING +
MAP_OUTPUT_INDEX_SUFFIX_STRING);
return lDirAlloc.getLocalPathToRead(attemptIndexOutput.toString(), conf);
}
/**
* Create a local map output index file name.
*
* @param size the size of the file
* @return path
* @throws IOException
*/
public Path getOutputIndexFileForWrite(long size) throws IOException {
Path attemptIndexOutput =
new Path(getAttemptOutputDir(), MAP_OUTPUT_FILENAME_STRING +
MAP_OUTPUT_INDEX_SUFFIX_STRING);
return lDirAlloc.getLocalPathForWrite(attemptIndexOutput.toString(),
size, conf);
}
/**
* Create a local map output index file name on the same volume.
*/
public Path getOutputIndexFileForWriteInVolume(Path existing) {
Path outputDir = new Path(existing.getParent(), JOB_OUTPUT_DIR);
Path attemptOutputDir = new Path(outputDir,
conf.get(JobContext.TASK_ATTEMPT_ID));
return new Path(attemptOutputDir, MAP_OUTPUT_FILENAME_STRING +
MAP_OUTPUT_INDEX_SUFFIX_STRING);
}
/**
* Return a local map spill file created earlier.
*
* @param spillNumber the number
* @return path
* @throws IOException
*/
public Path getSpillFile(int spillNumber) throws IOException {
return lDirAlloc.getLocalPathToRead(
String.format(SPILL_FILE_PATTERN,
conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber), conf);
}
/**
* Create a local map spill file name.
*
* @param spillNumber the number
* @param size the size of the file
* @return path
* @throws IOException
*/
public Path getSpillFileForWrite(int spillNumber, long size)
throws IOException {
return lDirAlloc.getLocalPathForWrite(
String.format(String.format(SPILL_FILE_PATTERN,
conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber)), size, conf);
}
/**
* Return a local map spill index file created earlier
*
* @param spillNumber the number
* @return path
* @throws IOException
*/
public Path getSpillIndexFile(int spillNumber) throws IOException {
return lDirAlloc.getLocalPathToRead(
String.format(SPILL_INDEX_FILE_PATTERN,
conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber), conf);
}
/**
* Create a local map spill index file name.
*
* @param spillNumber the number
* @param size the size of the file
* @return path
* @throws IOException
*/
public Path getSpillIndexFileForWrite(int spillNumber, long size)
throws IOException {
return lDirAlloc.getLocalPathForWrite(
String.format(SPILL_INDEX_FILE_PATTERN,
conf.get(JobContext.TASK_ATTEMPT_ID), spillNumber), size, conf);
}
/**
* Return a local reduce input file created earlier
*
* @param mapId a map task id
* @return path
* @throws IOException
*/
public Path getInputFile(int mapId) throws IOException {
throw new UnsupportedOperationException("Incompatible with LocalRunner");
}
/**
* Create a local reduce input file name.
*
* @param mapId a map task id
* @param size the size of the file
* @return path
* @throws IOException
*/
public Path getInputFileForWrite(org.apache.hadoop.mapreduce.TaskID mapId,
long size) throws IOException {
return lDirAlloc.getLocalPathForWrite(String.format(
REDUCE_INPUT_FILE_FORMAT_STRING,
getAttemptOutputDir().toString(), mapId.getId()),
size, conf);
}
/** Removes all of the files related to a task. */
public void removeAll() throws IOException {
throw new UnsupportedOperationException("Incompatible with LocalRunner");
}
@Override
public void setConf(Configuration conf) {
if (conf instanceof JobConf) {
this.conf = (JobConf) conf;
} else {
this.conf = new JobConf(conf);
}
}
@Override
public Configuration getConf() {
return conf;
}
}

View File

@ -0,0 +1,42 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.jobhistory;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.yarn.event.AbstractEvent;
public class JobHistoryEvent extends AbstractEvent<EventType>{
private final JobId jobID;
private final HistoryEvent historyEvent;
public JobHistoryEvent(JobId jobID, HistoryEvent historyEvent) {
super(historyEvent.getEventType());
this.jobID = jobID;
this.historyEvent = historyEvent;
}
public JobId getJobID() {
return jobID;
}
public HistoryEvent getHistoryEvent() {
return historyEvent;
}
}

View File

@ -0,0 +1,675 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.jobhistory;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
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.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.v2.api.records.Counter;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.jobhistory.FileNameIndexUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.service.AbstractService;
/**
* The job history events get routed to this class. This class writes the Job
* history events to the DFS directly into a staging dir and then moved to a
* done-dir. JobHistory implementation is in this package to access package
* private classes.
*/
public class JobHistoryEventHandler extends AbstractService
implements EventHandler<JobHistoryEvent> {
private final AppContext context;
private final int startCount;
//TODO Does the FS object need to be different ?
private FileSystem stagingDirFS; // log Dir FileSystem
private FileSystem doneDirFS; // done Dir FileSystem
private Configuration conf;
private Path stagingDirPath = null;
private Path doneDirPrefixPath = null; // folder for completed jobs
private BlockingQueue<JobHistoryEvent> eventQueue =
new LinkedBlockingQueue<JobHistoryEvent>();
private Thread eventHandlingThread;
private volatile boolean stopped;
private final Object lock = new Object();
private static final Log LOG = LogFactory.getLog(
JobHistoryEventHandler.class);
private static final Map<JobId, MetaInfo> fileMap =
Collections.<JobId,MetaInfo>synchronizedMap(new HashMap<JobId,MetaInfo>());
public JobHistoryEventHandler(AppContext context, int startCount) {
super("JobHistoryEventHandler");
this.context = context;
this.startCount = startCount;
}
/* (non-Javadoc)
* @see org.apache.hadoop.yarn.service.AbstractService#init(org.apache.hadoop.conf.Configuration)
* Initializes the FileSystem and Path objects for the log and done directories.
* Creates these directories if they do not already exist.
*/
@Override
public void init(Configuration conf) {
this.conf = conf;
String stagingDirStr = null;
String doneDirStr = null;
String userDoneDirStr = null;
try {
stagingDirStr = JobHistoryUtils.getConfiguredHistoryStagingDirPrefix(conf);
doneDirStr =
JobHistoryUtils.getConfiguredHistoryIntermediateDoneDirPrefix(conf);
userDoneDirStr =
JobHistoryUtils.getHistoryIntermediateDoneDirForUser(conf);
} catch (IOException e) {
LOG.error("Failed while getting the configured log directories", e);
throw new YarnException(e);
}
//Check for the existence of the history staging dir. Maybe create it.
try {
stagingDirPath =
FileSystem.get(conf).makeQualified(new Path(stagingDirStr));
stagingDirFS = FileSystem.get(stagingDirPath.toUri(), conf);
mkdir(stagingDirFS, stagingDirPath, new FsPermission(
JobHistoryUtils.HISTORY_STAGING_DIR_PERMISSIONS));
} catch (IOException e) {
LOG.error("Failed while checking for/creating history staging path: ["
+ stagingDirPath + "]", e);
throw new YarnException(e);
}
//Check for the existence of intermediate done dir.
Path doneDirPath = null;
try {
doneDirPath = FileSystem.get(conf).makeQualified(new Path(doneDirStr));
doneDirFS = FileSystem.get(doneDirPath.toUri(), conf);
// This directory will be in a common location, or this may be a cluster
// meant for a single user. Creating based on the conf. Should ideally be
// created by the JobHistoryServer or as part of deployment.
if (!doneDirFS.exists(doneDirPath)) {
if (JobHistoryUtils.shouldCreateNonUserDirectory(conf)) {
LOG.info("Creating intermediate history logDir: ["
+ doneDirPath
+ "] + based on conf. Should ideally be created by the JobHistoryServer: "
+ JHConfig.CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY);
mkdir(
doneDirFS,
doneDirPath,
new FsPermission(
JobHistoryUtils.HISTORY_INTERMEDIATE_DONE_DIR_PERMISSIONS
.toShort()));
// TODO Temporary toShort till new FsPermission(FsPermissions)
// respects
// sticky
} else {
String message = "Not creating intermediate history logDir: ["
+ doneDirPath
+ "] based on conf: "
+ JHConfig.CREATE_HISTORY_INTERMEDIATE_BASE_DIR_KEY
+ ". Either set to true or pre-create this directory with appropriate permissions";
LOG.error(message);
throw new YarnException(message);
}
}
} catch (IOException e) {
LOG.error("Failed checking for the existance of history intermediate done directory: ["
+ doneDirPath + "]");
throw new YarnException(e);
}
//Check/create user directory under intermediate done dir.
try {
doneDirPrefixPath =
FileSystem.get(conf).makeQualified(new Path(userDoneDirStr));
mkdir(doneDirFS, doneDirPrefixPath, new FsPermission(
JobHistoryUtils.HISTORY_INTERMEDIATE_USER_DIR_PERMISSIONS));
} catch (IOException e) {
LOG.error("Error creating user intermediate history done directory: [ "
+ doneDirPrefixPath + "]", e);
throw new YarnException(e);
}
super.init(conf);
}
private void mkdir(FileSystem fs, Path path, FsPermission fsp)
throws IOException {
if (!fs.exists(path)) {
try {
fs.mkdirs(path, fsp);
FileStatus fsStatus = fs.getFileStatus(path);
LOG.info("Perms after creating " + fsStatus.getPermission().toShort()
+ ", Expected: " + fsp.toShort());
if (fsStatus.getPermission().toShort() != fsp.toShort()) {
LOG.info("Explicitly setting permissions to : " + fsp.toShort()
+ ", " + fsp);
fs.setPermission(path, fsp);
}
} catch (FileAlreadyExistsException e) {
LOG.info("Directory: [" + path + "] already exists.");
}
}
}
@Override
public void start() {
eventHandlingThread = new Thread(new Runnable() {
@Override
public void run() {
JobHistoryEvent event = null;
while (!stopped && !Thread.currentThread().isInterrupted()) {
try {
event = eventQueue.take();
} catch (InterruptedException e) {
LOG.info("EventQueue take interrupted. Returning");
return;
}
// If an event has been removed from the queue. Handle it.
// The rest of the queue is handled via stop()
// Clear the interrupt status if it's set before calling handleEvent
// and set it if it was set before calling handleEvent.
// Interrupts received from other threads during handleEvent cannot be
// dealth with - Shell.runCommand() ignores them.
synchronized (lock) {
boolean isInterrupted = Thread.interrupted();
handleEvent(event);
if (isInterrupted) {
Thread.currentThread().interrupt();
}
}
}
}
});
eventHandlingThread.start();
super.start();
}
@Override
public void stop() {
LOG.info("Stopping JobHistoryEventHandler");
stopped = true;
//do not interrupt while event handling is in progress
synchronized(lock) {
eventHandlingThread.interrupt();
}
try {
eventHandlingThread.join();
} catch (InterruptedException ie) {
LOG.info("Interruped Exception while stopping", ie);
}
//write all the events remaining in queue
Iterator<JobHistoryEvent> it = eventQueue.iterator();
while(it.hasNext()) {
JobHistoryEvent ev = it.next();
LOG.info("In stop, writing event " + ev.getType());
handleEvent(ev);
}
//close all file handles
for (MetaInfo mi : fileMap.values()) {
try {
mi.closeWriter();
} catch (IOException e) {
LOG.info("Exception while closing file " + e.getMessage());
}
}
LOG.info("Stopped JobHistoryEventHandler. super.stop()");
super.stop();
}
/**
* Create an event writer for the Job represented by the jobID.
* Writes out the job configuration to the log directory.
* This should be the first call to history for a job
*
* @param jobId the jobId.
* @throws IOException
*/
protected void setupEventWriter(JobId jobId, JobSubmittedEvent jse)
throws IOException {
if (stagingDirPath == null) {
LOG.error("Log Directory is null, returning");
throw new IOException("Missing Log Directory for History");
}
MetaInfo oldFi = fileMap.get(jobId);
Configuration conf = getConfig();
long submitTime = oldFi == null ? jse.getSubmitTime() : oldFi
.getJobIndexInfo().getSubmitTime();
// TODO Ideally this should be written out to the job dir
// (.staging/jobid/files - RecoveryService will need to be patched)
Path historyFile = JobHistoryUtils.getStagingJobHistoryFile(
stagingDirPath, jobId, startCount);
String user = UserGroupInformation.getCurrentUser().getShortUserName();
if (user == null) {
throw new IOException(
"User is null while setting up jobhistory eventwriter");
}
String jobName = context.getJob(jobId).getName();
EventWriter writer = (oldFi == null) ? null : oldFi.writer;
if (writer == null) {
try {
FSDataOutputStream out = stagingDirFS.create(historyFile, true);
writer = new EventWriter(out);
LOG.info("Event Writer setup for JobId: " + jobId + ", File: "
+ historyFile);
} catch (IOException ioe) {
LOG.info("Could not create log file: [" + historyFile + "] + for job "
+ "[" + jobName + "]");
throw ioe;
}
}
Path logDirConfPath = null;
if (conf != null) {
// TODO Ideally this should be written out to the job dir
// (.staging/jobid/files - RecoveryService will need to be patched)
logDirConfPath = JobHistoryUtils.getStagingConfFile(stagingDirPath, jobId,
startCount);
FSDataOutputStream jobFileOut = null;
try {
if (logDirConfPath != null) {
jobFileOut = stagingDirFS.create(logDirConfPath, true);
conf.writeXml(jobFileOut);
jobFileOut.close();
}
} catch (IOException e) {
LOG.info("Failed to write the job configuration file", e);
throw e;
}
}
MetaInfo fi = new MetaInfo(historyFile, logDirConfPath, writer, submitTime,
user, jobName, jobId);
fi.getJobSummary().setJobId(jobId);
fi.getJobSummary().setJobSubmitTime(submitTime);
fileMap.put(jobId, fi);
}
/** Close the event writer for this id
* @throws IOException */
public void closeWriter(JobId id) throws IOException {
try {
final MetaInfo mi = fileMap.get(id);
if (mi != null) {
mi.closeWriter();
}
} catch (IOException e) {
LOG.error("Error closing writer for JobID: " + id);
throw e;
}
}
@Override
public void handle(JobHistoryEvent event) {
try {
eventQueue.put(event);
} catch (InterruptedException e) {
throw new YarnException(e);
}
}
protected void handleEvent(JobHistoryEvent event) {
synchronized (lock) {
// If this is JobSubmitted Event, setup the writer
if (event.getHistoryEvent().getEventType() == EventType.JOB_SUBMITTED) {
try {
JobSubmittedEvent jobSubmittedEvent =
(JobSubmittedEvent) event.getHistoryEvent();
setupEventWriter(event.getJobID(), jobSubmittedEvent);
} catch (IOException ioe) {
LOG.error("Error JobHistoryEventHandler in handleEvent: " + event,
ioe);
throw new YarnException(ioe);
}
}
// For all events
// (1) Write it out
// (2) Process it for JobSummary
MetaInfo mi = fileMap.get(event.getJobID());
try {
HistoryEvent historyEvent = event.getHistoryEvent();
mi.writeEvent(historyEvent);
processEventForJobSummary(event.getHistoryEvent(), mi.getJobSummary(), event.getJobID());
LOG.info("In HistoryEventHandler "
+ event.getHistoryEvent().getEventType());
} catch (IOException e) {
LOG.error("Error writing History Event: " + event.getHistoryEvent(),
e);
throw new YarnException(e);
}
// If this is JobFinishedEvent, close the writer and setup the job-index
if (event.getHistoryEvent().getEventType() == EventType.JOB_FINISHED) {
try {
JobFinishedEvent jFinishedEvent =
(JobFinishedEvent) event.getHistoryEvent();
mi.getJobIndexInfo().setFinishTime(jFinishedEvent.getFinishTime());
mi.getJobIndexInfo().setNumMaps(jFinishedEvent.getFinishedMaps());
mi.getJobIndexInfo().setNumReduces(
jFinishedEvent.getFinishedReduces());
mi.getJobIndexInfo().setJobStatus(JobState.SUCCEEDED.toString());
closeEventWriter(event.getJobID());
} catch (IOException e) {
throw new YarnException(e);
}
}
if (event.getHistoryEvent().getEventType() == EventType.JOB_FAILED
|| event.getHistoryEvent().getEventType() == EventType.JOB_KILLED) {
try {
JobUnsuccessfulCompletionEvent jucEvent = (JobUnsuccessfulCompletionEvent) event
.getHistoryEvent();
mi.getJobIndexInfo().setFinishTime(jucEvent.getFinishTime());
mi.getJobIndexInfo().setNumMaps(jucEvent.getFinishedMaps());
mi.getJobIndexInfo().setNumReduces(jucEvent.getFinishedReduces());
mi.getJobIndexInfo().setJobStatus(jucEvent.getStatus());
closeEventWriter(event.getJobID());
} catch (IOException e) {
throw new YarnException(e);
}
}
}
}
private void processEventForJobSummary(HistoryEvent event, JobSummary summary, JobId jobId) {
// context.getJob could be used for some of this info as well.
switch (event.getEventType()) {
case JOB_SUBMITTED:
JobSubmittedEvent jse = (JobSubmittedEvent) event;
summary.setUser(jse.getUserName());
summary.setQueue(jse.getJobQueueName());
break;
case JOB_INITED:
JobInitedEvent jie = (JobInitedEvent) event;
summary.setJobLaunchTime(jie.getLaunchTime());
break;
case MAP_ATTEMPT_STARTED:
TaskAttemptStartedEvent mtase = (TaskAttemptStartedEvent) event;
if (summary.getFirstMapTaskLaunchTime() == 0)
summary.setFirstMapTaskLaunchTime(mtase.getStartTime());
break;
case REDUCE_ATTEMPT_STARTED:
TaskAttemptStartedEvent rtase = (TaskAttemptStartedEvent) event;
if (summary.getFirstReduceTaskLaunchTime() == 0)
summary.setFirstReduceTaskLaunchTime(rtase.getStartTime());
break;
case JOB_FINISHED:
JobFinishedEvent jfe = (JobFinishedEvent) event;
summary.setJobFinishTime(jfe.getFinishTime());
summary.setNumFinishedMaps(jfe.getFinishedMaps());
summary.setNumFailedMaps(jfe.getFailedMaps());
summary.setNumFinishedReduces(jfe.getFinishedReduces());
summary.setNumFailedReduces(jfe.getFailedReduces());
if (summary.getJobStatus() == null)
summary
.setJobStatus(org.apache.hadoop.mapreduce.JobStatus.State.SUCCEEDED
.toString());
// TODO JOB_FINISHED does not have state. Effectively job history does not
// have state about the finished job.
setSummarySlotSeconds(summary, jobId);
break;
case JOB_FAILED:
case JOB_KILLED:
JobUnsuccessfulCompletionEvent juce = (JobUnsuccessfulCompletionEvent) event;
summary.setJobStatus(juce.getStatus());
summary.setNumFinishedMaps(context.getJob(jobId).getTotalMaps());
summary.setNumFinishedReduces(context.getJob(jobId).getTotalReduces());
summary.setJobFinishTime(juce.getFinishTime());
setSummarySlotSeconds(summary, jobId);
break;
}
}
private void setSummarySlotSeconds(JobSummary summary, JobId jobId) {
Counter slotMillisMapCounter =
context.getJob(jobId).getCounters()
.getCounter(JobCounter.SLOTS_MILLIS_MAPS);
if (slotMillisMapCounter != null) {
summary.setMapSlotSeconds(slotMillisMapCounter.getValue());
}
Counter slotMillisReduceCounter =
context.getJob(jobId).getCounters()
.getCounter(JobCounter.SLOTS_MILLIS_REDUCES);
if (slotMillisReduceCounter != null) {
summary.setMapSlotSeconds(slotMillisReduceCounter.getValue());
}
}
protected void closeEventWriter(JobId jobId) throws IOException {
final MetaInfo mi = fileMap.get(jobId);
if (mi == null) {
throw new IOException("No MetaInfo found for JobId: [" + jobId + "]");
}
if (!mi.isWriterActive()) {
throw new IOException(
"Inactive Writer: Likely received multiple JobFinished / JobUnsuccessful events for JobId: ["
+ jobId + "]");
}
// Close the Writer
try {
mi.closeWriter();
} catch (IOException e) {
LOG.error("Error closing writer for JobID: " + jobId);
throw e;
}
if (mi.getHistoryFile() == null) {
LOG.warn("No file for job-history with " + jobId + " found in cache!");
}
if (mi.getConfFile() == null) {
LOG.warn("No file for jobconf with " + jobId + " found in cache!");
}
// Writing out the summary file.
// TODO JH enhancement - reuse this file to store additional indexing info
// like ACLs, etc. JHServer can use HDFS append to build an index file
// with more info than is available via the filename.
Path qualifiedSummaryDoneFile = null;
FSDataOutputStream summaryFileOut = null;
try {
String doneSummaryFileName = getTempFileName(JobHistoryUtils
.getIntermediateSummaryFileName(jobId));
qualifiedSummaryDoneFile = doneDirFS.makeQualified(new Path(
doneDirPrefixPath, doneSummaryFileName));
summaryFileOut = doneDirFS.create(qualifiedSummaryDoneFile, true);
summaryFileOut.writeUTF(mi.getJobSummary().getJobSummaryString());
summaryFileOut.close();
} catch (IOException e) {
LOG.info("Unable to write out JobSummaryInfo to ["
+ qualifiedSummaryDoneFile + "]", e);
throw e;
}
try {
// Move historyFile to Done Folder.
Path qualifiedDoneFile = null;
if (mi.getHistoryFile() != null) {
Path historyFile = mi.getHistoryFile();
Path qualifiedLogFile = stagingDirFS.makeQualified(historyFile);
String doneJobHistoryFileName =
getTempFileName(FileNameIndexUtils.getDoneFileName(mi
.getJobIndexInfo()));
qualifiedDoneFile =
doneDirFS.makeQualified(new Path(doneDirPrefixPath,
doneJobHistoryFileName));
moveToDoneNow(qualifiedLogFile, qualifiedDoneFile);
}
// Move confFile to Done Folder
Path qualifiedConfDoneFile = null;
if (mi.getConfFile() != null) {
Path confFile = mi.getConfFile();
Path qualifiedConfFile = stagingDirFS.makeQualified(confFile);
String doneConfFileName =
getTempFileName(JobHistoryUtils
.getIntermediateConfFileName(jobId));
qualifiedConfDoneFile =
doneDirFS.makeQualified(new Path(doneDirPrefixPath,
doneConfFileName));
moveToDoneNow(qualifiedConfFile, qualifiedConfDoneFile);
}
moveTmpToDone(qualifiedSummaryDoneFile);
moveTmpToDone(qualifiedConfDoneFile);
moveTmpToDone(qualifiedDoneFile);
} catch (IOException e) {
LOG.error("Error closing writer for JobID: " + jobId);
throw e;
}
}
private class MetaInfo {
private Path historyFile;
private Path confFile;
private EventWriter writer;
JobIndexInfo jobIndexInfo;
JobSummary jobSummary;
MetaInfo(Path historyFile, Path conf, EventWriter writer, long submitTime,
String user, String jobName, JobId jobId) {
this.historyFile = historyFile;
this.confFile = conf;
this.writer = writer;
this.jobIndexInfo = new JobIndexInfo(submitTime, -1, user, jobName, jobId, -1, -1, null);
this.jobSummary = new JobSummary();
}
Path getHistoryFile() { return historyFile; }
Path getConfFile() {return confFile; }
JobIndexInfo getJobIndexInfo() { return jobIndexInfo; }
JobSummary getJobSummary() { return jobSummary; }
boolean isWriterActive() {return writer != null ; }
void closeWriter() throws IOException {
synchronized (lock) {
if (writer != null) {
writer.close();
}
writer = null;
}
}
void writeEvent(HistoryEvent event) throws IOException {
synchronized (lock) {
if (writer != null) {
writer.write(event);
writer.flush();
}
}
}
}
private void moveTmpToDone(Path tmpPath) throws IOException {
if (tmpPath != null) {
String tmpFileName = tmpPath.getName();
String fileName = getFileNameFromTmpFN(tmpFileName);
Path path = new Path(tmpPath.getParent(), fileName);
doneDirFS.rename(tmpPath, path);
LOG.info("Moved tmp to done: " + tmpPath + " to " + path);
}
}
// TODO If the FS objects are the same, this should be a rename instead of a
// copy.
private void moveToDoneNow(Path fromPath, Path toPath) throws IOException {
// check if path exists, in case of retries it may not exist
if (stagingDirFS.exists(fromPath)) {
LOG.info("Moving " + fromPath.toString() + " to " + toPath.toString());
// TODO temporarily removing the existing dst
if (doneDirFS.exists(toPath)) {
doneDirFS.delete(toPath, true);
}
boolean copied = FileUtil.copy(stagingDirFS, fromPath, doneDirFS, toPath,
false, conf);
if (copied)
LOG.info("Copied to done location: " + toPath);
else
LOG.info("copy failed");
doneDirFS.setPermission(toPath, new FsPermission(
JobHistoryUtils.HISTORY_INTERMEDIATE_FILE_PERMISSIONS));
stagingDirFS.delete(fromPath, false);
}
}
boolean pathExists(FileSystem fileSys, Path path) throws IOException {
return fileSys.exists(path);
}
private String getTempFileName(String srcFile) {
return srcFile + "_tmp";
}
private String getFileNameFromTmpFN(String tmpFileName) {
//TODO. Some error checking here.
return tmpFileName.substring(0, tmpFileName.length()-4);
}
}

View File

@ -0,0 +1,231 @@
package org.apache.hadoop.mapreduce.jobhistory;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.util.StringUtils;
public class JobSummary {
private JobId jobId;
private long jobSubmitTime;
private long jobLaunchTime;
private long firstMapTaskLaunchTime; // MapAttempteStarted |
// TaskAttemptStartEvent
private long firstReduceTaskLaunchTime; // ReduceAttemptStarted |
// TaskAttemptStartEvent
private long jobFinishTime;
private int numFinishedMaps;
private int numFailedMaps;
private int numFinishedReduces;
private int numFailedReduces;
// private int numSlotsPerMap; | Doesn't make sense with potentially different
// resource models
// private int numSlotsPerReduce; | Doesn't make sense with potentially
// different resource models
private String user;
private String queue;
private String jobStatus;
private long mapSlotSeconds; // TODO Not generated yet in MRV2
private long reduceSlotSeconds; // TODO Not generated yet MRV2
// private int clusterSlotCapacity;
JobSummary() {
}
public JobId getJobId() {
return jobId;
}
public void setJobId(JobId jobId) {
this.jobId = jobId;
}
public long getJobSubmitTime() {
return jobSubmitTime;
}
public void setJobSubmitTime(long jobSubmitTime) {
this.jobSubmitTime = jobSubmitTime;
}
public long getJobLaunchTime() {
return jobLaunchTime;
}
public void setJobLaunchTime(long jobLaunchTime) {
this.jobLaunchTime = jobLaunchTime;
}
public long getFirstMapTaskLaunchTime() {
return firstMapTaskLaunchTime;
}
public void setFirstMapTaskLaunchTime(long firstMapTaskLaunchTime) {
this.firstMapTaskLaunchTime = firstMapTaskLaunchTime;
}
public long getFirstReduceTaskLaunchTime() {
return firstReduceTaskLaunchTime;
}
public void setFirstReduceTaskLaunchTime(long firstReduceTaskLaunchTime) {
this.firstReduceTaskLaunchTime = firstReduceTaskLaunchTime;
}
public long getJobFinishTime() {
return jobFinishTime;
}
public void setJobFinishTime(long jobFinishTime) {
this.jobFinishTime = jobFinishTime;
}
public int getNumFinishedMaps() {
return numFinishedMaps;
}
public void setNumFinishedMaps(int numFinishedMaps) {
this.numFinishedMaps = numFinishedMaps;
}
public int getNumFailedMaps() {
return numFailedMaps;
}
public void setNumFailedMaps(int numFailedMaps) {
this.numFailedMaps = numFailedMaps;
}
// public int getNumSlotsPerMap() {
// return numSlotsPerMap;
// }
//
// public void setNumSlotsPerMap(int numSlotsPerMap) {
// this.numSlotsPerMap = numSlotsPerMap;
// }
public int getNumFinishedReduces() {
return numFinishedReduces;
}
public void setNumFinishedReduces(int numFinishedReduces) {
this.numFinishedReduces = numFinishedReduces;
}
public int getNumFailedReduces() {
return numFailedReduces;
}
public void setNumFailedReduces(int numFailedReduces) {
this.numFailedReduces = numFailedReduces;
}
// public int getNumSlotsPerReduce() {
// return numSlotsPerReduce;
// }
//
// public void setNumSlotsPerReduce(int numSlotsPerReduce) {
// this.numSlotsPerReduce = numSlotsPerReduce;
// }
public String getUser() {
return user;
}
public void setUser(String user) {
this.user = user;
}
public String getQueue() {
return queue;
}
public void setQueue(String queue) {
this.queue = queue;
}
public String getJobStatus() {
return jobStatus;
}
public void setJobStatus(String jobStatus) {
this.jobStatus = jobStatus;
}
public long getMapSlotSeconds() {
return mapSlotSeconds;
}
public void setMapSlotSeconds(long mapSlotSeconds) {
this.mapSlotSeconds = mapSlotSeconds;
}
public long getReduceSlotSeconds() {
return reduceSlotSeconds;
}
public void setReduceSlotSeconds(long reduceSlotSeconds) {
this.reduceSlotSeconds = reduceSlotSeconds;
}
// public int getClusterSlotCapacity() {
// return clusterSlotCapacity;
// }
//
// public void setClusterSlotCapacity(int clusterSlotCapacity) {
// this.clusterSlotCapacity = clusterSlotCapacity;
// }
public String getJobSummaryString() {
SummaryBuilder summary = new SummaryBuilder()
.add("jobId", jobId)
.add("submitTime", jobSubmitTime)
.add("launchTime", jobLaunchTime)
.add("firstMapTaskLaunchTime", firstMapTaskLaunchTime)
.add("firstReduceTaskLaunchTime", firstReduceTaskLaunchTime)
.add("finishTime", jobFinishTime)
.add("numMaps", numFinishedMaps + numFailedMaps)
.add("numReduces", numFinishedReduces + numFailedReduces)
.add("user", user)
.add("queue", queue)
.add("status", jobStatus)
.add("mapSlotSeconds", mapSlotSeconds)
.add("reduceSlotSeconds", reduceSlotSeconds);
return summary.toString();
}
static final char EQUALS = '=';
static final char[] charsToEscape = { StringUtils.COMMA, EQUALS,
StringUtils.ESCAPE_CHAR };
static class SummaryBuilder {
final StringBuilder buffer = new StringBuilder();
// A little optimization for a very common case
SummaryBuilder add(String key, long value) {
return _add(key, Long.toString(value));
}
<T> SummaryBuilder add(String key, T value) {
return _add(key, StringUtils.escapeString(String.valueOf(value),
StringUtils.ESCAPE_CHAR, charsToEscape));
}
SummaryBuilder add(SummaryBuilder summary) {
if (buffer.length() > 0)
buffer.append(StringUtils.COMMA);
buffer.append(summary.buffer);
return this;
}
SummaryBuilder _add(String key, String value) {
if (buffer.length() > 0)
buffer.append(StringUtils.COMMA);
buffer.append(key).append(EQUALS).append(value);
return this;
}
@Override
public String toString() {
return buffer.toString();
}
}
}

View File

@ -0,0 +1,74 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app;
import org.apache.hadoop.mapreduce.v2.MRConstants;
public interface AMConstants {
public static final String CONTAINERLAUNCHER_THREADPOOL_SIZE =
"yarn.mapreduce.containerlauncher.threadpool-size";
public static final String AM_RM_SCHEDULE_INTERVAL =
"yarn.appMaster.scheduler.interval";
public static final int DEFAULT_AM_RM_SCHEDULE_INTERVAL = 2000;
public static final String AM_TASK_LISTENER_THREADS =
MRConstants.YARN_MR_PREFIX + "task.listener.threads";
public static final int DEFAULT_AM_TASK_LISTENER_THREADS = 10;
public static final String AM_JOB_CLIENT_THREADS =
MRConstants.YARN_MR_PREFIX + "job.client.threads";
public static final int DEFAULT_AM_JOB_CLIENT_THREADS = 1;
public static final String SPECULATOR_CLASS =
MRConstants.YARN_MR_PREFIX + "speculator.class";
public static final String TASK_RUNTIME_ESTIMATOR_CLASS =
MRConstants.YARN_MR_PREFIX + "task.runtime.estimator.class";
public static final String TASK_ATTEMPT_PROGRESS_RUNTIME_LINEARIZER_CLASS =
MRConstants.YARN_MR_PREFIX + "task.runtime.linearizer.class";
public static final String EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS =
MRConstants.YARN_MR_PREFIX
+ "task.runtime.estimator.exponential.smooth.lambda";
public static final String EXPONENTIAL_SMOOTHING_SMOOTH_RATE =
MRConstants.YARN_MR_PREFIX
+ "task.runtime.estimator.exponential.smooth.smoothsrate";
public static final String RECOVERY_ENABLE = MRConstants.YARN_MR_PREFIX
+ "recovery.enable";
public static final float DEFAULT_REDUCE_RAMP_UP_LIMIT = 0.5f;
public static final String REDUCE_RAMPUP_UP_LIMIT = MRConstants.YARN_MR_PREFIX
+ "reduce.rampup.limit";
public static final float DEFAULT_REDUCE_PREEMPTION_LIMIT = 0.5f;
public static final String REDUCE_PREEMPTION_LIMIT = MRConstants.YARN_MR_PREFIX
+ "reduce.preemption.limit";
public static final String NODE_BLACKLISTING_ENABLE = MRConstants.YARN_MR_PREFIX
+ "node.blacklisting.enable";
}

View File

@ -0,0 +1,55 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app;
import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.event.EventHandler;
/**
* Context interface for sharing information across components in YARN App.
*/
@InterfaceAudience.Private
public interface AppContext {
ApplicationId getApplicationID();
ApplicationAttemptId getApplicationAttemptId();
String getApplicationName();
long getStartTime();
CharSequence getUser();
Job getJob(JobId jobID);
Map<JobId, Job> getAllJobs();
EventHandler getEventHandler();
Clock getClock();
}

View File

@ -0,0 +1,576 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app;
import java.io.File;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
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.Path;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.LocalContainerLauncher;
import org.apache.hadoop.mapred.TaskAttemptListenerImpl;
import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler;
import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
import org.apache.hadoop.mapreduce.v2.MRConstants;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
import org.apache.hadoop.mapreduce.v2.app.client.MRClientService;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobFinishEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl;
import org.apache.hadoop.mapreduce.v2.app.local.LocalContainerAllocator;
import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
import org.apache.hadoop.mapreduce.v2.app.recover.Recovery;
import org.apache.hadoop.mapreduce.v2.app.recover.RecoveryService;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator;
import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleaner;
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanerImpl;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
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.yarn.Clock;
import org.apache.hadoop.yarn.SystemClock;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.service.CompositeService;
import org.apache.hadoop.yarn.service.Service;
/**
* The Map-Reduce Application Master.
* The state machine is encapsulated in the implementation of Job interface.
* All state changes happens via Job interface. Each event
* results in a Finite State Transition in Job.
*
* MR AppMaster is the composition of loosely coupled services. The services
* interact with each other via events. The components resembles the
* Actors model. The component acts on received event and send out the
* events to other components.
* This keeps it highly concurrent with no or minimal synchronization needs.
*
* The events are dispatched by a central Dispatch mechanism. All components
* register to the Dispatcher.
*
* The information is shared across different components using AppContext.
*/
public class MRAppMaster extends CompositeService {
private static final Log LOG = LogFactory.getLog(MRAppMaster.class);
private Clock clock;
private final long startTime = System.currentTimeMillis();
private String appName;
private final int startCount;
private final ApplicationId appID;
private final ApplicationAttemptId appAttemptID;
protected final MRAppMetrics metrics;
private Set<TaskId> completedTasksFromPreviousRun;
private AppContext context;
private Dispatcher dispatcher;
private ClientService clientService;
private ContainerAllocator containerAllocator;
private ContainerLauncher containerLauncher;
private TaskCleaner taskCleaner;
private Speculator speculator;
private TaskAttemptListener taskAttemptListener;
private JobTokenSecretManager jobTokenSecretManager =
new JobTokenSecretManager();
private Job job;
public MRAppMaster(ApplicationId applicationId, int startCount) {
this(applicationId, new SystemClock(), startCount);
}
public MRAppMaster(ApplicationId applicationId, Clock clock, int startCount) {
super(MRAppMaster.class.getName());
this.clock = clock;
this.appID = applicationId;
this.appAttemptID = RecordFactoryProvider.getRecordFactory(null)
.newRecordInstance(ApplicationAttemptId.class);
this.appAttemptID.setApplicationId(appID);
this.appAttemptID.setAttemptId(startCount);
this.startCount = startCount;
this.metrics = MRAppMetrics.create();
LOG.info("Created MRAppMaster for application " + applicationId);
}
@Override
public void init(final Configuration conf) {
context = new RunningAppContext();
// Job name is the same as the app name util we support DAG of jobs
// for an app later
appName = conf.get(MRJobConfig.JOB_NAME, "<missing app name>");
if (conf.getBoolean(AMConstants.RECOVERY_ENABLE, false)
&& startCount > 1) {
LOG.info("Recovery is enabled. Will try to recover from previous life.");
Recovery recoveryServ = new RecoveryService(appID, clock, startCount);
addIfService(recoveryServ);
dispatcher = recoveryServ.getDispatcher();
clock = recoveryServ.getClock();
completedTasksFromPreviousRun = recoveryServ.getCompletedTasks();
} else {
dispatcher = new AsyncDispatcher();
addIfService(dispatcher);
}
//service to handle requests to TaskUmbilicalProtocol
taskAttemptListener = createTaskAttemptListener(context);
addIfService(taskAttemptListener);
//service to do the task cleanup
taskCleaner = createTaskCleaner(context);
addIfService(taskCleaner);
//service to handle requests from JobClient
clientService = createClientService(context);
addIfService(clientService);
//service to log job history events
EventHandler<JobHistoryEvent> historyService =
createJobHistoryHandler(context);
addIfService(historyService);
JobEventDispatcher synchronousJobEventDispatcher = new JobEventDispatcher();
//register the event dispatchers
dispatcher.register(JobEventType.class, synchronousJobEventDispatcher);
dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
dispatcher.register(TaskAttemptEventType.class,
new TaskAttemptEventDispatcher());
dispatcher.register(TaskCleaner.EventType.class, taskCleaner);
dispatcher.register(org.apache.hadoop.mapreduce.jobhistory.EventType.class,
historyService);
if (conf.getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
|| conf.getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
//optional service to speculate on task attempts' progress
speculator = createSpeculator(conf, context);
addIfService(speculator);
}
dispatcher.register(Speculator.EventType.class,
new SpeculatorEventDispatcher());
Credentials fsTokens = new Credentials();
if (UserGroupInformation.isSecurityEnabled()) {
// Read the file-system tokens from the localized tokens-file.
try {
Path jobSubmitDir =
FileContext.getLocalFSFileContext().makeQualified(
new Path(new File(MRConstants.JOB_SUBMIT_DIR)
.getAbsolutePath()));
Path jobTokenFile =
new Path(jobSubmitDir, MRConstants.APPLICATION_TOKENS_FILE);
fsTokens.addAll(Credentials.readTokenStorageFile(jobTokenFile, conf));
LOG.info("jobSubmitDir=" + jobSubmitDir + " jobTokenFile="
+ jobTokenFile);
UserGroupInformation currentUser =
UserGroupInformation.getCurrentUser();
for (Token<? extends TokenIdentifier> tk : fsTokens.getAllTokens()) {
LOG.info(" --- DEBUG: Token of kind " + tk.getKind()
+ "in current ugi in the AppMaster for service "
+ tk.getService());
currentUser.addToken(tk); // For use by AppMaster itself.
}
} catch (IOException e) {
throw new YarnException(e);
}
}
super.init(conf);
//---- start of what used to be startJobs() code:
Configuration config = getConfig();
job = createJob(config, fsTokens);
/** create a job event for job intialization */
JobEvent initJobEvent = new JobEvent(job.getID(), JobEventType.JOB_INIT);
/** send init to the job (this does NOT trigger job execution) */
synchronousJobEventDispatcher.handle(initJobEvent);
// send init to speculator. This won't yest start as dispatcher isn't
// started yet.
dispatcher.getEventHandler().handle(
new SpeculatorEvent(job.getID(), clock.getTime()));
// JobImpl's InitTransition is done (call above is synchronous), so the
// "uber-decision" (MR-1220) has been made. Query job and switch to
// ubermode if appropriate (by registering different container-allocator
// and container-launcher services/event-handlers).
if (job.isUber()) {
LOG.info("MRAppMaster uberizing job " + job.getID()
+ " in local container (\"uber-AM\").");
} else {
LOG.info("MRAppMaster launching normal, non-uberized, multi-container "
+ "job " + job.getID() + ".");
}
// service to allocate containers from RM (if non-uber) or to fake it (uber)
containerAllocator =
createContainerAllocator(clientService, context, job.isUber());
addIfService(containerAllocator);
dispatcher.register(ContainerAllocator.EventType.class, containerAllocator);
if (containerAllocator instanceof Service) {
((Service) containerAllocator).init(config);
}
// corresponding service to launch allocated containers via NodeManager
containerLauncher = createContainerLauncher(context, job.isUber());
addIfService(containerLauncher);
dispatcher.register(ContainerLauncher.EventType.class, containerLauncher);
if (containerLauncher instanceof Service) {
((Service) containerLauncher).init(config);
}
} // end of init()
/** Create and initialize (but don't start) a single job.
* @param fsTokens */
protected Job createJob(Configuration conf, Credentials fsTokens) {
// create single job
Job newJob = new JobImpl(appID, conf, dispatcher.getEventHandler(),
taskAttemptListener, jobTokenSecretManager, fsTokens, clock, startCount,
completedTasksFromPreviousRun, metrics);
((RunningAppContext) context).jobs.put(newJob.getID(), newJob);
dispatcher.register(JobFinishEvent.Type.class,
new EventHandler<JobFinishEvent>() {
@Override
public void handle(JobFinishEvent event) {
// job has finished
// this is the only job, so shut down the Appmaster
// note in a workflow scenario, this may lead to creation of a new
// job (FIXME?)
// TODO:currently just wait for some time so clients can know the
// final states. Will be removed once RM come on.
try {
Thread.sleep(5000);
} catch (InterruptedException e) {
e.printStackTrace();
}
LOG.info("Calling stop for all the services");
try {
stop();
} catch (Throwable t) {
LOG.warn("Graceful stop failed ", t);
}
//TODO: this is required because rpc server does not shut down
// in spite of calling server.stop().
//Bring the process down by force.
//Not needed after HADOOP-7140
LOG.info("Exiting MR AppMaster..GoodBye!");
System.exit(0);
}
});
return newJob;
} // end createJob()
protected void addIfService(Object object) {
if (object instanceof Service) {
addService((Service) object);
}
}
protected EventHandler<JobHistoryEvent> createJobHistoryHandler(
AppContext context) {
JobHistoryEventHandler eventHandler = new JobHistoryEventHandler(context,
getStartCount());
return eventHandler;
}
protected Speculator createSpeculator(Configuration conf, AppContext context) {
Class<? extends Speculator> speculatorClass;
try {
speculatorClass
// "yarn.mapreduce.job.speculator.class"
= conf.getClass(AMConstants.SPECULATOR_CLASS,
DefaultSpeculator.class,
Speculator.class);
Constructor<? extends Speculator> speculatorConstructor
= speculatorClass.getConstructor
(Configuration.class, AppContext.class);
Speculator result = speculatorConstructor.newInstance(conf, context);
return result;
} catch (InstantiationException ex) {
LOG.error("Can't make a speculator -- check "
+ AMConstants.SPECULATOR_CLASS + " " + ex);
throw new YarnException(ex);
} catch (IllegalAccessException ex) {
LOG.error("Can't make a speculator -- check "
+ AMConstants.SPECULATOR_CLASS + " " + ex);
throw new YarnException(ex);
} catch (InvocationTargetException ex) {
LOG.error("Can't make a speculator -- check "
+ AMConstants.SPECULATOR_CLASS + " " + ex);
throw new YarnException(ex);
} catch (NoSuchMethodException ex) {
LOG.error("Can't make a speculator -- check "
+ AMConstants.SPECULATOR_CLASS + " " + ex);
throw new YarnException(ex);
}
}
protected TaskAttemptListener createTaskAttemptListener(AppContext context) {
TaskAttemptListener lis =
new TaskAttemptListenerImpl(context, jobTokenSecretManager);
return lis;
}
protected TaskCleaner createTaskCleaner(AppContext context) {
return new TaskCleanerImpl(context);
}
protected ContainerAllocator createContainerAllocator(
ClientService clientService, AppContext context, boolean isLocal) {
//return new StaticContainerAllocator(context);
return isLocal
? new LocalContainerAllocator(clientService, context)
: new RMContainerAllocator(clientService, context);
}
protected ContainerLauncher createContainerLauncher(AppContext context,
boolean isLocal) {
return isLocal
? new LocalContainerLauncher(context,
(TaskUmbilicalProtocol) taskAttemptListener)
: new ContainerLauncherImpl(context);
}
//TODO:should have an interface for MRClientService
protected ClientService createClientService(AppContext context) {
return new MRClientService(context);
}
public ApplicationId getAppID() {
return appID;
}
public int getStartCount() {
return startCount;
}
public AppContext getContext() {
return context;
}
public Dispatcher getDispatcher() {
return dispatcher;
}
public Set<TaskId> getCompletedTaskFromPreviousRun() {
return completedTasksFromPreviousRun;
}
public ContainerAllocator getContainerAllocator() {
return containerAllocator;
}
public ContainerLauncher getContainerLauncher() {
return containerLauncher;
}
public TaskAttemptListener getTaskAttemptListener() {
return taskAttemptListener;
}
class RunningAppContext implements AppContext {
private Map<JobId, Job> jobs = new ConcurrentHashMap<JobId, Job>();
@Override
public ApplicationAttemptId getApplicationAttemptId() {
return appAttemptID;
}
@Override
public ApplicationId getApplicationID() {
return appID;
}
@Override
public String getApplicationName() {
return appName;
}
@Override
public long getStartTime() {
return startTime;
}
@Override
public Job getJob(JobId jobID) {
return jobs.get(jobID);
}
@Override
public Map<JobId, Job> getAllJobs() {
return jobs;
}
@Override
public EventHandler getEventHandler() {
return dispatcher.getEventHandler();
}
@Override
public CharSequence getUser() {
return getConfig().get(MRJobConfig.USER_NAME);
}
@Override
public Clock getClock() {
return clock;
}
}
@Override
public void start() {
// metrics system init is really init & start.
// It's more test friendly to put it here.
DefaultMetricsSystem.initialize("MRAppMaster");
startJobs();
//start all the components
super.start();
}
/**
* This can be overridden to instantiate multiple jobs and create a
* workflow.
*
* TODO: Rework the design to actually support this. Currently much of the
* job stuff has been moved to init() above to support uberization (MR-1220).
* In a typical workflow, one presumably would want to uberize only a subset
* of the jobs (the "small" ones), which is awkward with the current design.
*/
protected void startJobs() {
/** create a job-start event to get this ball rolling */
JobEvent startJobEvent = new JobEvent(job.getID(), JobEventType.JOB_START);
/** send the job-start event. this triggers the job execution. */
dispatcher.getEventHandler().handle(startJobEvent);
}
private class JobEventDispatcher implements EventHandler<JobEvent> {
@Override
public void handle(JobEvent event) {
((EventHandler<JobEvent>)context.getJob(event.getJobId())).handle(event);
}
}
private class TaskEventDispatcher implements EventHandler<TaskEvent> {
@Override
public void handle(TaskEvent event) {
Task task = context.getJob(event.getTaskID().getJobId()).getTask(
event.getTaskID());
((EventHandler<TaskEvent>)task).handle(event);
}
}
private class TaskAttemptEventDispatcher
implements EventHandler<TaskAttemptEvent> {
@Override
public void handle(TaskAttemptEvent event) {
Job job = context.getJob(event.getTaskAttemptID().getTaskId().getJobId());
Task task = job.getTask(event.getTaskAttemptID().getTaskId());
TaskAttempt attempt = task.getAttempt(event.getTaskAttemptID());
((EventHandler<TaskAttemptEvent>) attempt).handle(event);
}
}
private class SpeculatorEventDispatcher implements
EventHandler<SpeculatorEvent> {
@Override
public void handle(SpeculatorEvent event) {
if (getConfig().getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
|| getConfig().getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
// Speculator IS enabled, direct the event to there.
speculator.handle(event);
}
}
}
public static void main(String[] args) {
try {
//Configuration.addDefaultResource("job.xml");
ApplicationId applicationId = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class);
applicationId.setClusterTimestamp(Long.valueOf(args[0]));
applicationId.setId(Integer.valueOf(args[1]));
int failCount = Integer.valueOf(args[2]);
MRAppMaster appMaster = new MRAppMaster(applicationId, failCount);
YarnConfiguration conf = new YarnConfiguration(new JobConf());
conf.addResource(new Path(MRConstants.JOB_CONF_FILE));
conf.set(MRJobConfig.USER_NAME,
System.getProperty("user.name"));
UserGroupInformation.setConfiguration(conf);
appMaster.init(conf);
appMaster.start();
} catch (Throwable t) {
LOG.error("Caught throwable. Exiting:", t);
System.exit(1);
}
}
}

View File

@ -0,0 +1,58 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app;
import java.lang.annotation.Annotation;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.KerberosInfo;
import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.token.TokenInfo;
import org.apache.hadoop.security.token.TokenSelector;
import org.apache.hadoop.yarn.proto.MRClientProtocol;
import org.apache.hadoop.yarn.security.ApplicationTokenSelector;
public class MRClientSecurityInfo extends SecurityInfo {
@Override
public KerberosInfo getKerberosInfo(Class<?> protocol, Configuration conf) {
return null;
}
@Override
public TokenInfo getTokenInfo(Class<?> protocol, Configuration conf) {
if (!protocol.equals(MRClientProtocol.MRClientProtocolService.BlockingInterface.class)) {
return null;
}
return new TokenInfo() {
@Override
public Class<? extends Annotation> annotationType() {
return null;
}
@Override
public Class<? extends TokenSelector<? extends TokenIdentifier>>
value() {
return ApplicationTokenSelector.class;
}
};
}
}

View File

@ -0,0 +1,35 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app;
import java.net.InetSocketAddress;
import org.apache.hadoop.mapred.Task;
import org.apache.hadoop.mapred.WrappedJvmID;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
public interface TaskAttemptListener {
InetSocketAddress getAddress();
void register(TaskAttemptId attemptID, Task task, WrappedJvmID jvmID);
void unregister(TaskAttemptId attemptID, WrappedJvmID jvmID);
}

View File

@ -0,0 +1,137 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.service.AbstractService;
/**
* This class keeps track of tasks that have already been launched. It
* determines if a task is alive and running or marks a task as dead if it does
* not hear from it for a long time.
*
*/
public class TaskHeartbeatHandler extends AbstractService {
private static final Log LOG = LogFactory.getLog(TaskHeartbeatHandler.class);
//thread which runs periodically to see the last time since a heartbeat is
//received from a task.
private Thread lostTaskCheckerThread;
private volatile boolean stopped;
private int taskTimeOut = 5*60*1000;//5 mins
private final EventHandler eventHandler;
private final Clock clock;
private Map<TaskAttemptId, Long> runningAttempts
= new HashMap<TaskAttemptId, Long>();
public TaskHeartbeatHandler(EventHandler eventHandler, Clock clock) {
super("TaskHeartbeatHandler");
this.eventHandler = eventHandler;
this.clock = clock;
}
@Override
public void init(Configuration conf) {
super.init(conf);
taskTimeOut = conf.getInt("mapreduce.task.timeout", 5*60*1000);
}
@Override
public void start() {
lostTaskCheckerThread = new Thread(new PingChecker());
lostTaskCheckerThread.start();
super.start();
}
@Override
public void stop() {
stopped = true;
lostTaskCheckerThread.interrupt();
super.stop();
}
public synchronized void receivedPing(TaskAttemptId attemptID) {
//only put for the registered attempts
if (runningAttempts.containsKey(attemptID)) {
runningAttempts.put(attemptID, clock.getTime());
}
}
public synchronized void register(TaskAttemptId attemptID) {
runningAttempts.put(attemptID, clock.getTime());
}
public synchronized void unregister(TaskAttemptId attemptID) {
runningAttempts.remove(attemptID);
}
private class PingChecker implements Runnable {
@Override
public void run() {
while (!stopped && !Thread.currentThread().isInterrupted()) {
synchronized (TaskHeartbeatHandler.this) {
Iterator<Map.Entry<TaskAttemptId, Long>> iterator =
runningAttempts.entrySet().iterator();
//avoid calculating current time everytime in loop
long currentTime = clock.getTime();
while (iterator.hasNext()) {
Map.Entry<TaskAttemptId, Long> entry = iterator.next();
if (currentTime > entry.getValue() + taskTimeOut) {
//task is lost, remove from the list and raise lost event
iterator.remove();
eventHandler.handle(
new TaskAttemptDiagnosticsUpdateEvent(entry.getKey(),
"AttemptID:" + entry.getKey().toString() +
" Timed out after " + taskTimeOut/1000 + " secs"));
eventHandler.handle(new TaskAttemptEvent(entry
.getKey(), TaskAttemptEventType.TA_TIMED_OUT));
}
}
}
try {
Thread.sleep(taskTimeOut);
} catch (InterruptedException e) {
LOG.info("TaskHeartbeatHandler thread interrupted");
break;
}
}
}
}
}

View File

@ -0,0 +1,28 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.client;
import java.net.InetSocketAddress;
public interface ClientService {
InetSocketAddress getBindAddress();
int getHttpPort();
}

View File

@ -0,0 +1,392 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.client;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.security.AccessControlException;
import java.util.Arrays;
import java.util.Collection;
import org.apache.avro.ipc.Server;
import org.apache.commons.codec.binary.Base64;
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.MRClientProtocol;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptResponse;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetCountersRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetCountersResponse;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDiagnosticsRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDiagnosticsResponse;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportResponse;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptCompletionEventsRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptCompletionEventsResponse;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptReportRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptReportResponse;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportResponse;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportsRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportsResponse;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillJobRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillJobResponse;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskAttemptRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskAttemptResponse;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskResponse;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobDiagnosticsUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
import org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.RPCUtil;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier;
import org.apache.hadoop.yarn.security.SchedulerSecurityInfo;
import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.webapp.WebApp;
import org.apache.hadoop.yarn.webapp.WebApps;
/**
* This module is responsible for talking to the
* jobclient (user facing).
*
*/
public class MRClientService extends AbstractService
implements ClientService {
static final Log LOG = LogFactory.getLog(MRClientService.class);
private MRClientProtocol protocolHandler;
private Server server;
private WebApp webApp;
private InetSocketAddress bindAddress;
private AppContext appContext;
public MRClientService(AppContext appContext) {
super("MRClientService");
this.appContext = appContext;
this.protocolHandler = new MRClientProtocolHandler();
}
public void start() {
Configuration conf = new Configuration(getConfig()); // Just for not messing up sec-info class config
YarnRPC rpc = YarnRPC.create(conf);
InetSocketAddress address = NetUtils.createSocketAddr("0.0.0.0:0");
InetAddress hostNameResolved = null;
try {
hostNameResolved = InetAddress.getLocalHost();
} catch (UnknownHostException e) {
throw new YarnException(e);
}
ClientToAMSecretManager secretManager = null;
if (UserGroupInformation.isSecurityEnabled()) {
secretManager = new ClientToAMSecretManager();
String secretKeyStr =
System
.getenv(ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME);
byte[] bytes = Base64.decodeBase64(secretKeyStr);
ApplicationTokenIdentifier identifier =
new ApplicationTokenIdentifier(this.appContext.getApplicationID());
secretManager.setMasterKey(identifier, bytes);
conf.setClass(
YarnConfiguration.YARN_SECURITY_INFO,
SchedulerSecurityInfo.class, SecurityInfo.class); // Same for now.
}
server =
rpc.getServer(MRClientProtocol.class, protocolHandler, address,
conf, secretManager,
conf.getInt(AMConstants.AM_JOB_CLIENT_THREADS,
AMConstants.DEFAULT_AM_JOB_CLIENT_THREADS));
server.start();
this.bindAddress =
NetUtils.createSocketAddr(hostNameResolved.getHostAddress()
+ ":" + server.getPort());
LOG.info("Instantiated MRClientService at " + this.bindAddress);
try {
webApp = WebApps.$for("yarn", AppContext.class, appContext).with(conf).
start(new AMWebApp());
} catch (Exception e) {
LOG.error("Webapps failed to start. Ignoring for now:", e);
}
super.start();
}
public void stop() {
server.close();
if (webApp != null) {
webApp.stop();
}
super.stop();
}
@Override
public InetSocketAddress getBindAddress() {
return bindAddress;
}
@Override
public int getHttpPort() {
return webApp.port();
}
class MRClientProtocolHandler implements MRClientProtocol {
private RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
private Job verifyAndGetJob(JobId jobID,
boolean modifyAccess) throws YarnRemoteException {
Job job = appContext.getJob(jobID);
if (job == null) {
throw RPCUtil.getRemoteException("Unknown job " + jobID);
}
//TODO fix job acls.
//JobACL operation = JobACL.VIEW_JOB;
//if (modifyAccess) {
// operation = JobACL.MODIFY_JOB;
//}
//TO disable check access ofr now.
//checkAccess(job, operation);
return job;
}
private Task verifyAndGetTask(TaskId taskID,
boolean modifyAccess) throws YarnRemoteException {
Task task = verifyAndGetJob(taskID.getJobId(),
modifyAccess).getTask(taskID);
if (task == null) {
throw RPCUtil.getRemoteException("Unknown Task " + taskID);
}
return task;
}
private TaskAttempt verifyAndGetAttempt(TaskAttemptId attemptID,
boolean modifyAccess) throws YarnRemoteException {
TaskAttempt attempt = verifyAndGetTask(attemptID.getTaskId(),
modifyAccess).getAttempt(attemptID);
if (attempt == null) {
throw RPCUtil.getRemoteException("Unknown TaskAttempt " + attemptID);
}
return attempt;
}
private void checkAccess(Job job, JobACL jobOperation)
throws YarnRemoteException {
if (!UserGroupInformation.isSecurityEnabled()) {
return;
}
UserGroupInformation callerUGI;
try {
callerUGI = UserGroupInformation.getCurrentUser();
} catch (IOException e) {
throw RPCUtil.getRemoteException(e);
}
if(!job.checkAccess(callerUGI, jobOperation)) {
throw RPCUtil.getRemoteException(new AccessControlException("User "
+ callerUGI.getShortUserName() + " cannot perform operation "
+ jobOperation.name() + " on " + job.getID()));
}
}
@Override
public GetCountersResponse getCounters(GetCountersRequest request)
throws YarnRemoteException {
JobId jobId = request.getJobId();
Job job = verifyAndGetJob(jobId, false);
GetCountersResponse response =
recordFactory.newRecordInstance(GetCountersResponse.class);
response.setCounters(job.getCounters());
return response;
}
@Override
public GetJobReportResponse getJobReport(GetJobReportRequest request)
throws YarnRemoteException {
JobId jobId = request.getJobId();
Job job = verifyAndGetJob(jobId, false);
GetJobReportResponse response =
recordFactory.newRecordInstance(GetJobReportResponse.class);
response.setJobReport(job.getReport());
return response;
}
@Override
public GetTaskAttemptReportResponse getTaskAttemptReport(
GetTaskAttemptReportRequest request) throws YarnRemoteException {
TaskAttemptId taskAttemptId = request.getTaskAttemptId();
GetTaskAttemptReportResponse response =
recordFactory.newRecordInstance(GetTaskAttemptReportResponse.class);
response.setTaskAttemptReport(
verifyAndGetAttempt(taskAttemptId, false).getReport());
return response;
}
@Override
public GetTaskReportResponse getTaskReport(GetTaskReportRequest request)
throws YarnRemoteException {
TaskId taskId = request.getTaskId();
GetTaskReportResponse response =
recordFactory.newRecordInstance(GetTaskReportResponse.class);
response.setTaskReport(verifyAndGetTask(taskId, false).getReport());
return response;
}
@Override
public GetTaskAttemptCompletionEventsResponse getTaskAttemptCompletionEvents(
GetTaskAttemptCompletionEventsRequest request)
throws YarnRemoteException {
JobId jobId = request.getJobId();
int fromEventId = request.getFromEventId();
int maxEvents = request.getMaxEvents();
Job job = verifyAndGetJob(jobId, false);
GetTaskAttemptCompletionEventsResponse response =
recordFactory.newRecordInstance(GetTaskAttemptCompletionEventsResponse.class);
response.addAllCompletionEvents(Arrays.asList(
job.getTaskAttemptCompletionEvents(fromEventId, maxEvents)));
return response;
}
@Override
public KillJobResponse killJob(KillJobRequest request)
throws YarnRemoteException {
JobId jobId = request.getJobId();
String message = "Kill Job received from client " + jobId;
LOG.info(message);
verifyAndGetJob(jobId, true);
appContext.getEventHandler().handle(
new JobDiagnosticsUpdateEvent(jobId, message));
appContext.getEventHandler().handle(
new JobEvent(jobId, JobEventType.JOB_KILL));
KillJobResponse response =
recordFactory.newRecordInstance(KillJobResponse.class);
return response;
}
@Override
public KillTaskResponse killTask(KillTaskRequest request)
throws YarnRemoteException {
TaskId taskId = request.getTaskId();
String message = "Kill task received from client " + taskId;
LOG.info(message);
verifyAndGetTask(taskId, true);
appContext.getEventHandler().handle(
new TaskEvent(taskId, TaskEventType.T_KILL));
KillTaskResponse response =
recordFactory.newRecordInstance(KillTaskResponse.class);
return response;
}
@Override
public KillTaskAttemptResponse killTaskAttempt(
KillTaskAttemptRequest request) throws YarnRemoteException {
TaskAttemptId taskAttemptId = request.getTaskAttemptId();
String message = "Kill task attempt received from client " + taskAttemptId;
LOG.info(message);
verifyAndGetAttempt(taskAttemptId, true);
appContext.getEventHandler().handle(
new TaskAttemptDiagnosticsUpdateEvent(taskAttemptId, message));
appContext.getEventHandler().handle(
new TaskAttemptEvent(taskAttemptId,
TaskAttemptEventType.TA_KILL));
KillTaskAttemptResponse response =
recordFactory.newRecordInstance(KillTaskAttemptResponse.class);
return response;
}
@Override
public GetDiagnosticsResponse getDiagnostics(
GetDiagnosticsRequest request) throws YarnRemoteException {
TaskAttemptId taskAttemptId = request.getTaskAttemptId();
GetDiagnosticsResponse response =
recordFactory.newRecordInstance(GetDiagnosticsResponse.class);
response.addAllDiagnostics(
verifyAndGetAttempt(taskAttemptId, false).getDiagnostics());
return response;
}
@Override
public FailTaskAttemptResponse failTaskAttempt(
FailTaskAttemptRequest request) throws YarnRemoteException {
TaskAttemptId taskAttemptId = request.getTaskAttemptId();
String message = "Fail task attempt received from client " + taskAttemptId;
LOG.info(message);
verifyAndGetAttempt(taskAttemptId, true);
appContext.getEventHandler().handle(
new TaskAttemptDiagnosticsUpdateEvent(taskAttemptId, message));
appContext.getEventHandler().handle(
new TaskAttemptEvent(taskAttemptId,
TaskAttemptEventType.TA_FAILMSG));
FailTaskAttemptResponse response = recordFactory.
newRecordInstance(FailTaskAttemptResponse.class);
return response;
}
@Override
public GetTaskReportsResponse getTaskReports(
GetTaskReportsRequest request) throws YarnRemoteException {
JobId jobId = request.getJobId();
TaskType taskType = request.getTaskType();
GetTaskReportsResponse response =
recordFactory.newRecordInstance(GetTaskReportsResponse.class);
Job job = verifyAndGetJob(jobId, false);
LOG.info("Getting task report for " + taskType + " " + jobId);
Collection<Task> tasks = job.getTasks(taskType).values();
LOG.info("Getting task report size " + tasks.size());
for (Task task : tasks) {
response.addTaskReport(task.getReport());
}
return response;
}
}
}

View File

@ -0,0 +1,59 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.mapreduce.JobACL;
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.security.UserGroupInformation;
/**
* Main interface to interact with the job. Provides only getters.
*/
public interface Job {
JobId getID();
String getName();
JobState getState();
JobReport getReport();
Counters getCounters();
Map<TaskId,Task> getTasks();
Map<TaskId,Task> getTasks(TaskType taskType);
Task getTask(TaskId taskID);
List<String> getDiagnostics();
int getTotalMaps();
int getTotalReduces();
int getCompletedMaps();
int getCompletedReduces();
boolean isUber();
TaskAttemptCompletionEvent[]
getTaskAttemptCompletionEvents(int fromEventId, int maxEvents);
boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation);
}

View File

@ -0,0 +1,58 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job;
import java.util.Map;
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
/**
* Read only view of Task.
*/
public interface Task {
TaskId getID();
TaskReport getReport();
TaskState getState();
Counters getCounters();
float getProgress();
TaskType getType();
Map<TaskAttemptId, TaskAttempt> getAttempts();
TaskAttempt getAttempt(TaskAttemptId attemptID);
/** Has Task reached the final state or not.
*/
boolean isFinished();
/**
* Can the output of the taskAttempt be committed. Note that once the task
* gives a go for a commit, further canCommit requests from any other attempts
* should return false.
*
* @param taskAttemptID
* @return whether the attempt's output can be committed or not.
*/
boolean canCommit(TaskAttemptId taskAttemptID);
}

View File

@ -0,0 +1,66 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job;
import java.util.List;
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptReport;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
import org.apache.hadoop.yarn.api.records.ContainerId;
/**
* Read only view of TaskAttempt.
*/
public interface TaskAttempt {
TaskAttemptId getID();
TaskAttemptReport getReport();
List<String> getDiagnostics();
Counters getCounters();
float getProgress();
TaskAttemptState getState();
/** Has attempt reached the final state or not.
*/
boolean isFinished();
/**If container Assigned then return container ID, otherwise null.
*/
ContainerId getAssignedContainerID();
/**If container Assigned then return container mgr address, otherwise null.
*/
String getAssignedContainerMgrAddress();
/**If container Assigned then return the node's http address, otherwise null.
*/
String getNodeHttpAddress();
/** Returns time at which container is launched. If container is not launched
* yet, returns 0.
*/
long getLaunchTime();
/** Returns attempt's finish time. If attempt is not finished
* yet, returns 0.
*/
long getFinishTime();
}

View File

@ -0,0 +1,42 @@
package org.apache.hadoop.mapreduce.v2.app.job.event;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
public class JobCounterUpdateEvent extends JobEvent {
List<CounterIncrementalUpdate> counterUpdates = null;
public JobCounterUpdateEvent(JobId jobId) {
super(jobId, JobEventType.JOB_COUNTER_UPDATE);
counterUpdates = new ArrayList<JobCounterUpdateEvent.CounterIncrementalUpdate>();
}
public void addCounterUpdate(Enum<?> key, long incrValue) {
counterUpdates.add(new CounterIncrementalUpdate(key, incrValue));
}
public List<CounterIncrementalUpdate> getCounterUpdates() {
return counterUpdates;
}
public static class CounterIncrementalUpdate {
Enum<?> key;
long incrValue;
public CounterIncrementalUpdate(Enum<?> key, long incrValue) {
this.key = key;
this.incrValue = incrValue;
}
public Enum<?> getCounterKey() {
return key;
}
public long getIncrementValue() {
return incrValue;
}
}
}

View File

@ -0,0 +1,36 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
public class JobDiagnosticsUpdateEvent extends JobEvent {
private String diagnosticUpdate;
public JobDiagnosticsUpdateEvent(JobId jobID, String diagnostic) {
super(jobID, JobEventType.JOB_DIAGNOSTIC_UPDATE);
this.diagnosticUpdate = diagnostic;
}
public String getDiagnosticUpdate() {
return this.diagnosticUpdate;
}
}

View File

@ -0,0 +1,41 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.yarn.event.AbstractEvent;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
/**
* This class encapsulates job related events.
*
*/
public class JobEvent extends AbstractEvent<JobEventType> {
private JobId jobID;
public JobEvent(JobId jobID, JobEventType type) {
super(type);
this.jobID = jobID;
}
public JobId getJobId() {
return jobID;
}
}

View File

@ -0,0 +1,48 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
/**
* Event types handled by Job.
*/
public enum JobEventType {
//Producer:Client
JOB_KILL,
//Producer:MRAppMaster
JOB_INIT,
JOB_START,
//Producer:Task
JOB_TASK_COMPLETED,
JOB_MAP_TASK_RESCHEDULED,
JOB_TASK_ATTEMPT_COMPLETED,
//Producer:Job
JOB_COMPLETED,
//Producer:Any component
JOB_DIAGNOSTIC_UPDATE,
INTERNAL_ERROR,
JOB_COUNTER_UPDATE,
//Producer:TaskAttemptListener
JOB_TASK_ATTEMPT_FETCH_FAILURE
}

View File

@ -0,0 +1,42 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.yarn.event.AbstractEvent;
public class JobFinishEvent
extends AbstractEvent<JobFinishEvent.Type> {
public enum Type {
STATE_CHANGED
}
private JobId jobID;
public JobFinishEvent(JobId jobID) {
super(Type.STATE_CHANGED);
this.jobID = jobID;
}
public JobId getJobId() {
return jobID;
}
}

View File

@ -0,0 +1,38 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
public class JobMapTaskRescheduledEvent extends JobEvent {
private TaskId taskID;
public JobMapTaskRescheduledEvent(TaskId taskID) {
super(taskID.getJobId(), JobEventType.JOB_MAP_TASK_RESCHEDULED);
this.taskID = taskID;
}
public TaskId getTaskID() {
return taskID;
}
}

View File

@ -0,0 +1,37 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent;
public class JobTaskAttemptCompletedEvent extends JobEvent {
private TaskAttemptCompletionEvent completionEvent;
public JobTaskAttemptCompletedEvent(TaskAttemptCompletionEvent completionEvent) {
super(completionEvent.getAttemptId().getTaskId().getJobId(),
JobEventType.JOB_TASK_ATTEMPT_COMPLETED);
this.completionEvent = completionEvent;
}
public TaskAttemptCompletionEvent getCompletionEvent() {
return completionEvent;
}
}

View File

@ -0,0 +1,48 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import java.util.List;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
public class JobTaskAttemptFetchFailureEvent extends JobEvent {
private final TaskAttemptId reduce;
private final List<TaskAttemptId> maps;
public JobTaskAttemptFetchFailureEvent(TaskAttemptId reduce,
List<TaskAttemptId> maps) {
super(reduce.getTaskId().getJobId(),
JobEventType.JOB_TASK_ATTEMPT_FETCH_FAILURE);
this.reduce = reduce;
this.maps = maps;
}
public List<TaskAttemptId> getMaps() {
return maps;
}
public TaskAttemptId getReduce() {
return reduce;
}
}

View File

@ -0,0 +1,43 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
public class JobTaskEvent extends JobEvent {
private TaskId taskID;
private TaskState taskState;
public JobTaskEvent(TaskId taskID, TaskState taskState) {
super(taskID.getJobId(), JobEventType.JOB_TASK_COMPLETED);
this.taskID = taskID;
this.taskState = taskState;
}
public TaskId getTaskID() {
return taskID;
}
public TaskState getState() {
return taskState;
}
}

View File

@ -0,0 +1,39 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.yarn.api.records.Container;
public class TaskAttemptContainerAssignedEvent extends TaskAttemptEvent {
private final Container container;
public TaskAttemptContainerAssignedEvent(TaskAttemptId id,
Container container) {
super(id, TaskAttemptEventType.TA_ASSIGNED);
this.container = container;
}
public Container getContainer() {
return this.container;
}
}

View File

@ -0,0 +1,37 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
public class TaskAttemptDiagnosticsUpdateEvent extends TaskAttemptEvent {
private String diagnosticInfo;
public TaskAttemptDiagnosticsUpdateEvent(TaskAttemptId attemptID,
String diagnosticInfo) {
super(attemptID, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE);
this.diagnosticInfo = diagnosticInfo;
}
public String getDiagnosticInfo() {
return diagnosticInfo;
}
}

View File

@ -0,0 +1,41 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.yarn.event.AbstractEvent;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
/**
* This class encapsulates task attempt related events.
*
*/
public class TaskAttemptEvent extends AbstractEvent<TaskAttemptEventType> {
private TaskAttemptId attemptID;
public TaskAttemptEvent(TaskAttemptId id, TaskAttemptEventType type) {
super(type);
this.attemptID = id;
}
public TaskAttemptId getTaskAttemptID() {
return attemptID;
}
}

View File

@ -0,0 +1,55 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
/**
* Event types handled by TaskAttempt.
*/
public enum TaskAttemptEventType {
//Producer:Task
TA_SCHEDULE,
TA_RESCHEDULE,
//Producer:Client, Task
TA_KILL,
//Producer:ContainerAllocator
TA_ASSIGNED,
TA_CONTAINER_COMPLETED,
//Producer:ContainerLauncher
TA_CONTAINER_LAUNCHED,
TA_CONTAINER_LAUNCH_FAILED,
TA_CONTAINER_CLEANED,
//Producer:TaskAttemptListener
TA_DIAGNOSTICS_UPDATE,
TA_COMMIT_PENDING,
TA_DONE,
TA_FAILMSG,
TA_UPDATE,
TA_TIMED_OUT,
//Producer:TaskCleaner
TA_CLEANUP_DONE,
//Producer:Job
TA_TOO_MANY_FETCH_FAILURE,
}

View File

@ -0,0 +1,61 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import java.util.List;
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.api.records.Phase;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
public class TaskAttemptStatusUpdateEvent extends TaskAttemptEvent {
private TaskAttemptStatus reportedTaskAttemptStatus;
public TaskAttemptStatusUpdateEvent(TaskAttemptId id,
TaskAttemptStatus taskAttemptStatus) {
super(id, TaskAttemptEventType.TA_UPDATE);
this.reportedTaskAttemptStatus = taskAttemptStatus;
}
public TaskAttemptStatus getReportedTaskAttemptStatus() {
return reportedTaskAttemptStatus;
}
/**
* The internal TaskAttemptStatus object corresponding to remote Task status.
*
*/
public static class TaskAttemptStatus {
public TaskAttemptId id;
public float progress;
public Counters counters;
public String diagnosticInfo;
public String stateString;
public Phase phase;
public long outputSize;
public List<TaskAttemptId> fetchFailedMaps;
public long mapFinishTime;
public long shuffleFinishTime;
public long sortFinishTime;
public TaskAttemptState taskState;
}
}

View File

@ -0,0 +1,40 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.yarn.event.AbstractEvent;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
/**
* this class encapsulates task related events.
*
*/
public class TaskEvent extends AbstractEvent<TaskEventType> {
private TaskId taskID;
public TaskEvent(TaskId taskID, TaskEventType type) {
super(type);
this.taskID = taskID;
}
public TaskId getTaskID() {
return taskID;
}
}

View File

@ -0,0 +1,41 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
/**
* Event types handled by Task.
*/
public enum TaskEventType {
//Producer:Client, Job
T_KILL,
//Producer:Job
T_SCHEDULE,
//Producer:Speculator
T_ADD_SPEC_ATTEMPT,
//Producer:TaskAttempt
T_ATTEMPT_LAUNCHED,
T_ATTEMPT_COMMIT_PENDING,
T_ATTEMPT_FAILED,
T_ATTEMPT_SUCCEEDED,
T_ATTEMPT_KILLED
}

View File

@ -0,0 +1,37 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.event;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
public class TaskTAttemptEvent extends TaskEvent {
private TaskAttemptId attemptID;
public TaskTAttemptEvent(TaskAttemptId id, TaskEventType type) {
super(id.getTaskId(), type);
this.attemptID = id;
}
public TaskAttemptId getTaskAttemptID() {
return attemptID;
}
}

View File

@ -0,0 +1,97 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.impl;
import java.util.Collection;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.MapTaskAttemptImpl;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.event.EventHandler;
public class MapTaskImpl extends TaskImpl {
private final TaskSplitMetaInfo taskSplitMetaInfo;
public MapTaskImpl(JobId jobId, int partition, EventHandler eventHandler,
Path remoteJobConfFile, Configuration conf,
TaskSplitMetaInfo taskSplitMetaInfo,
TaskAttemptListener taskAttemptListener, OutputCommitter committer,
Token<JobTokenIdentifier> jobToken,
Collection<Token<? extends TokenIdentifier>> fsTokens, Clock clock,
Set<TaskId> completedTasksFromPreviousRun, int startCount,
MRAppMetrics metrics) {
super(jobId, TaskType.MAP, partition, eventHandler, remoteJobConfFile,
conf, taskAttemptListener, committer, jobToken, fsTokens, clock,
completedTasksFromPreviousRun, startCount, metrics);
this.taskSplitMetaInfo = taskSplitMetaInfo;
}
@Override
protected int getMaxAttempts() {
return conf.getInt(MRJobConfig.MAP_MAX_ATTEMPTS, 4);
}
@Override
protected TaskAttemptImpl createAttempt() {
return new MapTaskAttemptImpl(getID(), nextAttemptNumber,
eventHandler, jobFile,
partition, taskSplitMetaInfo, conf, taskAttemptListener,
committer, jobToken, fsTokens, clock);
}
@Override
public TaskType getType() {
return TaskType.MAP;
}
protected TaskSplitMetaInfo getTaskSplitMetaInfo() {
return this.taskSplitMetaInfo;
}
/**
* @return a String formatted as a comma-separated list of splits.
*/
@Override
protected String getSplitsAsString() {
String[] splits = getTaskSplitMetaInfo().getLocations();
if (splits == null || splits.length == 0)
return "";
StringBuilder sb = new StringBuilder();
for (int i = 0; i < splits.length; i++) {
if (i != 0) sb.append(",");
sb.append(splits[i]);
}
return sb.toString();
}
}

View File

@ -0,0 +1,75 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.impl;
import java.util.Collection;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.ReduceTaskAttemptImpl;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.event.EventHandler;
public class ReduceTaskImpl extends TaskImpl {
private final int numMapTasks;
public ReduceTaskImpl(JobId jobId, int partition,
EventHandler eventHandler, Path jobFile, Configuration conf,
int numMapTasks, TaskAttemptListener taskAttemptListener,
OutputCommitter committer, Token<JobTokenIdentifier> jobToken,
Collection<Token<? extends TokenIdentifier>> fsTokens, Clock clock,
Set<TaskId> completedTasksFromPreviousRun, int startCount,
MRAppMetrics metrics) {
super(jobId, TaskType.REDUCE, partition, eventHandler, jobFile, conf,
taskAttemptListener, committer, jobToken, fsTokens, clock,
completedTasksFromPreviousRun, startCount, metrics);
this.numMapTasks = numMapTasks;
}
@Override
protected int getMaxAttempts() {
return conf.getInt(MRJobConfig.REDUCE_MAX_ATTEMPTS, 4);
}
@Override
protected TaskAttemptImpl createAttempt() {
return new ReduceTaskAttemptImpl(getID(), nextAttemptNumber,
eventHandler, jobFile,
partition, numMapTasks, conf, taskAttemptListener,
committer, jobToken, fsTokens, clock);
}
@Override
public TaskType getType() {
return TaskType.REDUCE;
}
}

View File

@ -0,0 +1,887 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.job.impl;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
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.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
import org.apache.hadoop.mapreduce.jobhistory.TaskFailedEvent;
import org.apache.hadoop.mapreduce.jobhistory.TaskFinishedEvent;
import org.apache.hadoop.mapreduce.jobhistory.TaskStartedEvent;
import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
import org.apache.hadoop.mapreduce.v2.api.records.Counters;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEventStatus;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerFailedEvent;
import org.apache.hadoop.mapreduce.v2.app.TaskAttemptListener;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobDiagnosticsUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobMapTaskRescheduledEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskAttemptCompletedEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobTaskEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptEvent;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
import org.apache.hadoop.yarn.state.MultipleArcTransition;
import org.apache.hadoop.yarn.state.SingleArcTransition;
import org.apache.hadoop.yarn.state.StateMachine;
import org.apache.hadoop.yarn.state.StateMachineFactory;
/**
* Implementation of Task interface.
*/
public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
private static final Log LOG = LogFactory.getLog(TaskImpl.class);
protected final Configuration conf;
protected final Path jobFile;
protected final OutputCommitter committer;
protected final int partition;
protected final TaskAttemptListener taskAttemptListener;
protected final EventHandler eventHandler;
private final TaskId taskId;
private Map<TaskAttemptId, TaskAttempt> attempts;
private final int maxAttempts;
protected final Clock clock;
private final Lock readLock;
private final Lock writeLock;
private final MRAppMetrics metrics;
private long scheduledTime;
private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
protected Collection<Token<? extends TokenIdentifier>> fsTokens;
protected Token<JobTokenIdentifier> jobToken;
// counts the number of attempts that are either running or in a state where
// they will come to be running when they get a Container
private int numberUncompletedAttempts = 0;
private boolean historyTaskStartGenerated = false;
private static final SingleArcTransition<TaskImpl, TaskEvent>
ATTEMPT_KILLED_TRANSITION = new AttemptKilledTransition();
private static final SingleArcTransition<TaskImpl, TaskEvent>
KILL_TRANSITION = new KillTransition();
private static final StateMachineFactory
<TaskImpl, TaskState, TaskEventType, TaskEvent>
stateMachineFactory
= new StateMachineFactory<TaskImpl, TaskState, TaskEventType, TaskEvent>
(TaskState.NEW)
// define the state machine of Task
// Transitions from NEW state
.addTransition(TaskState.NEW, TaskState.SCHEDULED,
TaskEventType.T_SCHEDULE, new InitialScheduleTransition())
.addTransition(TaskState.NEW, TaskState.KILLED,
TaskEventType.T_KILL, new KillNewTransition())
// Transitions from SCHEDULED state
//when the first attempt is launched, the task state is set to RUNNING
.addTransition(TaskState.SCHEDULED, TaskState.RUNNING,
TaskEventType.T_ATTEMPT_LAUNCHED, new LaunchTransition())
.addTransition(TaskState.SCHEDULED, TaskState.KILL_WAIT,
TaskEventType.T_KILL, KILL_TRANSITION)
.addTransition(TaskState.SCHEDULED, TaskState.SCHEDULED,
TaskEventType.T_ATTEMPT_KILLED, ATTEMPT_KILLED_TRANSITION)
.addTransition(TaskState.SCHEDULED,
EnumSet.of(TaskState.SCHEDULED, TaskState.FAILED),
TaskEventType.T_ATTEMPT_FAILED,
new AttemptFailedTransition())
// Transitions from RUNNING state
.addTransition(TaskState.RUNNING, TaskState.RUNNING,
TaskEventType.T_ATTEMPT_LAUNCHED) //more attempts may start later
.addTransition(TaskState.RUNNING, TaskState.RUNNING,
TaskEventType.T_ATTEMPT_COMMIT_PENDING,
new AttemptCommitPendingTransition())
.addTransition(TaskState.RUNNING, TaskState.RUNNING,
TaskEventType.T_ADD_SPEC_ATTEMPT, new RedundantScheduleTransition())
.addTransition(TaskState.RUNNING, TaskState.SUCCEEDED,
TaskEventType.T_ATTEMPT_SUCCEEDED,
new AttemptSucceededTransition())
.addTransition(TaskState.RUNNING, TaskState.RUNNING,
TaskEventType.T_ATTEMPT_KILLED,
ATTEMPT_KILLED_TRANSITION)
.addTransition(TaskState.RUNNING,
EnumSet.of(TaskState.RUNNING, TaskState.FAILED),
TaskEventType.T_ATTEMPT_FAILED,
new AttemptFailedTransition())
.addTransition(TaskState.RUNNING, TaskState.KILL_WAIT,
TaskEventType.T_KILL, KILL_TRANSITION)
// Transitions from KILL_WAIT state
.addTransition(TaskState.KILL_WAIT,
EnumSet.of(TaskState.KILL_WAIT, TaskState.KILLED),
TaskEventType.T_ATTEMPT_KILLED,
new KillWaitAttemptKilledTransition())
// Ignore-able transitions.
.addTransition(
TaskState.KILL_WAIT,
TaskState.KILL_WAIT,
EnumSet.of(TaskEventType.T_KILL,
TaskEventType.T_ATTEMPT_LAUNCHED,
TaskEventType.T_ATTEMPT_COMMIT_PENDING,
TaskEventType.T_ATTEMPT_FAILED,
TaskEventType.T_ATTEMPT_SUCCEEDED,
TaskEventType.T_ADD_SPEC_ATTEMPT))
// Transitions from SUCCEEDED state
.addTransition(TaskState.SUCCEEDED, //only possible for map tasks
EnumSet.of(TaskState.SCHEDULED, TaskState.FAILED),
TaskEventType.T_ATTEMPT_FAILED, new MapRetroactiveFailureTransition())
// Ignore-able transitions.
.addTransition(
TaskState.SUCCEEDED, TaskState.SUCCEEDED,
EnumSet.of(TaskEventType.T_KILL,
TaskEventType.T_ADD_SPEC_ATTEMPT,
TaskEventType.T_ATTEMPT_LAUNCHED,
TaskEventType.T_ATTEMPT_KILLED))
// Transitions from FAILED state
.addTransition(TaskState.FAILED, TaskState.FAILED,
EnumSet.of(TaskEventType.T_KILL,
TaskEventType.T_ADD_SPEC_ATTEMPT))
// Transitions from KILLED state
.addTransition(TaskState.KILLED, TaskState.KILLED,
EnumSet.of(TaskEventType.T_KILL,
TaskEventType.T_ADD_SPEC_ATTEMPT))
// create the topology tables
.installTopology();
private final StateMachine<TaskState, TaskEventType, TaskEvent>
stateMachine;
protected int nextAttemptNumber;
//should be set to one which comes first
//saying COMMIT_PENDING
private TaskAttemptId commitAttempt;
private TaskAttemptId successfulAttempt;
private int failedAttempts;
private int finishedAttempts;//finish are total of success, failed and killed
@Override
public TaskState getState() {
return stateMachine.getCurrentState();
}
public TaskImpl(JobId jobId, TaskType taskType, int partition,
EventHandler eventHandler, Path remoteJobConfFile, Configuration conf,
TaskAttemptListener taskAttemptListener, OutputCommitter committer,
Token<JobTokenIdentifier> jobToken,
Collection<Token<? extends TokenIdentifier>> fsTokens, Clock clock,
Set<TaskId> completedTasksFromPreviousRun, int startCount,
MRAppMetrics metrics) {
this.conf = conf;
this.clock = clock;
this.jobFile = remoteJobConfFile;
ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
readLock = readWriteLock.readLock();
writeLock = readWriteLock.writeLock();
this.attempts = Collections.emptyMap();
// This overridable method call is okay in a constructor because we
// have a convention that none of the overrides depends on any
// fields that need initialization.
maxAttempts = getMaxAttempts();
taskId = recordFactory.newRecordInstance(TaskId.class);
taskId.setJobId(jobId);
taskId.setId(partition);
taskId.setTaskType(taskType);
this.partition = partition;
this.taskAttemptListener = taskAttemptListener;
this.eventHandler = eventHandler;
this.committer = committer;
this.fsTokens = fsTokens;
this.jobToken = jobToken;
this.metrics = metrics;
if (completedTasksFromPreviousRun != null
&& completedTasksFromPreviousRun.contains(taskId)) {
LOG.info("Task is from previous run " + taskId);
startCount = startCount - 1;
}
//attempt ids are generated based on MR app startCount so that attempts
//from previous lives don't overstep the current one.
//this assumes that a task won't have more than 1000 attempts in its single
//life
nextAttemptNumber = (startCount - 1) * 1000;
// This "this leak" is okay because the retained pointer is in an
// instance variable.
stateMachine = stateMachineFactory.make(this);
}
@Override
public Map<TaskAttemptId, TaskAttempt> getAttempts() {
readLock.lock();
try {
if (attempts.size() <= 1) {
return attempts;
}
Map<TaskAttemptId, TaskAttempt> result
= new LinkedHashMap<TaskAttemptId, TaskAttempt>();
result.putAll(attempts);
return result;
} finally {
readLock.unlock();
}
}
@Override
public TaskAttempt getAttempt(TaskAttemptId attemptID) {
readLock.lock();
try {
return attempts.get(attemptID);
} finally {
readLock.unlock();
}
}
@Override
public TaskId getID() {
return taskId;
}
@Override
public boolean isFinished() {
readLock.lock();
try {
// TODO: Use stateMachine level method?
return (getState() == TaskState.SUCCEEDED ||
getState() == TaskState.FAILED ||
getState() == TaskState.KILLED);
} finally {
readLock.unlock();
}
}
@Override
public TaskReport getReport() {
TaskReport report = recordFactory.newRecordInstance(TaskReport.class);
readLock.lock();
try {
report.setTaskId(taskId);
report.setStartTime(getLaunchTime());
report.setFinishTime(getFinishTime());
report.setTaskState(getState());
report.setProgress(getProgress());
report.setCounters(getCounters());
for (TaskAttempt attempt : attempts.values()) {
if (TaskAttemptState.RUNNING.equals(attempt.getState())) {
report.addRunningAttempt(attempt.getID());
}
}
report.setSuccessfulAttempt(successfulAttempt);
for (TaskAttempt att : attempts.values()) {
String prefix = "AttemptID:" + att.getID() + " Info:";
for (CharSequence cs : att.getDiagnostics()) {
report.addDiagnostics(prefix + cs);
}
}
return report;
} finally {
readLock.unlock();
}
}
@Override
public Counters getCounters() {
Counters counters = null;
readLock.lock();
try {
TaskAttempt bestAttempt = selectBestAttempt();
if (bestAttempt != null) {
counters = bestAttempt.getCounters();
} else {
counters = recordFactory.newRecordInstance(Counters.class);
// counters.groups = new HashMap<CharSequence, CounterGroup>();
}
return counters;
} finally {
readLock.unlock();
}
}
@Override
public float getProgress() {
readLock.lock();
try {
TaskAttempt bestAttempt = selectBestAttempt();
if (bestAttempt == null) {
return 0;
}
return bestAttempt.getProgress();
} finally {
readLock.unlock();
}
}
//this is always called in read/write lock
private long getLaunchTime() {
long launchTime = 0;
for (TaskAttempt at : attempts.values()) {
//select the least launch time of all attempts
if (launchTime == 0 || launchTime > at.getLaunchTime()) {
launchTime = at.getLaunchTime();
}
}
if (launchTime == 0) {
return this.scheduledTime;
}
return launchTime;
}
//this is always called in read/write lock
//TODO Verify behaviour is Task is killed (no finished attempt)
private long getFinishTime() {
if (!isFinished()) {
return 0;
}
long finishTime = 0;
for (TaskAttempt at : attempts.values()) {
//select the max finish time of all attempts
if (finishTime < at.getFinishTime()) {
finishTime = at.getFinishTime();
}
}
return finishTime;
}
private long getFinishTime(TaskAttemptId taId) {
if (taId == null) {
return clock.getTime();
}
long finishTime = 0;
for (TaskAttempt at : attempts.values()) {
//select the max finish time of all attempts
if (at.getID().equals(taId)) {
return at.getFinishTime();
}
}
return finishTime;
}
private TaskState finished(TaskState finalState) {
if (getState() == TaskState.RUNNING) {
metrics.endRunningTask(this);
}
return finalState;
}
//select the nextAttemptNumber with best progress
// always called inside the Read Lock
private TaskAttempt selectBestAttempt() {
float progress = 0f;
TaskAttempt result = null;
for (TaskAttempt at : attempts.values()) {
if (result == null) {
result = at; //The first time around
}
//TODO: consider the nextAttemptNumber only if it is not failed/killed ?
// calculate the best progress
if (at.getProgress() > progress) {
result = at;
progress = at.getProgress();
}
}
return result;
}
@Override
public boolean canCommit(TaskAttemptId taskAttemptID) {
readLock.lock();
boolean canCommit = false;
try {
if (commitAttempt != null) {
canCommit = taskAttemptID.equals(commitAttempt);
LOG.info("Result of canCommit for " + taskAttemptID + ":" + canCommit);
}
} finally {
readLock.unlock();
}
return canCommit;
}
protected abstract TaskAttemptImpl createAttempt();
// No override of this method may require that the subclass be initialized.
protected abstract int getMaxAttempts();
protected TaskAttempt getSuccessfulAttempt() {
readLock.lock();
try {
if (null == successfulAttempt) {
return null;
}
return attempts.get(successfulAttempt);
} finally {
readLock.unlock();
}
}
// This is always called in the Write Lock
private void addAndScheduleAttempt() {
TaskAttempt attempt = createAttempt();
LOG.info("Created attempt " + attempt.getID());
switch (attempts.size()) {
case 0:
attempts = Collections.singletonMap(attempt.getID(), attempt);
break;
case 1:
Map newAttempts
= new LinkedHashMap<TaskAttemptId, TaskAttempt>(maxAttempts);
newAttempts.putAll(attempts);
attempts = newAttempts;
attempts.put(attempt.getID(), attempt);
break;
default:
attempts.put(attempt.getID(), attempt);
break;
}
++nextAttemptNumber;
++numberUncompletedAttempts;
//schedule the nextAttemptNumber
if (failedAttempts > 0) {
eventHandler.handle(new TaskAttemptEvent(attempt.getID(),
TaskAttemptEventType.TA_RESCHEDULE));
} else {
eventHandler.handle(new TaskAttemptEvent(attempt.getID(),
TaskAttemptEventType.TA_SCHEDULE));
}
}
@Override
public void handle(TaskEvent event) {
LOG.info("Processing " + event.getTaskID() + " of type " + event.getType());
try {
writeLock.lock();
TaskState oldState = getState();
try {
stateMachine.doTransition(event.getType(), event);
} catch (InvalidStateTransitonException e) {
LOG.error("Can't handle this event at current state", e);
internalError(event.getType());
}
if (oldState != getState()) {
LOG.info(taskId + " Task Transitioned from " + oldState + " to "
+ getState());
}
} finally {
writeLock.unlock();
}
}
private void internalError(TaskEventType type) {
eventHandler.handle(new JobDiagnosticsUpdateEvent(
this.taskId.getJobId(), "Invalid event " + type +
" on Task " + this.taskId));
eventHandler.handle(new JobEvent(this.taskId.getJobId(),
JobEventType.INTERNAL_ERROR));
}
// always called inside a transition, in turn inside the Write Lock
private void handleTaskAttemptCompletion(TaskAttemptId attemptId,
TaskAttemptCompletionEventStatus status) {
finishedAttempts++;
TaskAttempt attempt = attempts.get(attemptId);
//raise the completion event only if the container is assigned
// to nextAttemptNumber
if (attempt.getNodeHttpAddress() != null) {
TaskAttemptCompletionEvent tce = recordFactory.newRecordInstance(TaskAttemptCompletionEvent.class);
tce.setEventId(-1);
//TODO: XXXXXX hardcoded port
tce.setMapOutputServerAddress("http://" + attempt.getNodeHttpAddress().split(":")[0] + ":8080");
tce.setStatus(status);
tce.setAttemptId(attempt.getID());
int runTime = 0;
if (attempt.getFinishTime() != 0 && attempt.getLaunchTime() !=0)
runTime = (int)(attempt.getFinishTime() - attempt.getLaunchTime());
tce.setAttemptRunTime(runTime);
//raise the event to job so that it adds the completion event to its
//data structures
eventHandler.handle(new JobTaskAttemptCompletedEvent(tce));
}
}
private static TaskFinishedEvent createTaskFinishedEvent(TaskImpl task, TaskState taskState) {
TaskFinishedEvent tfe =
new TaskFinishedEvent(TypeConverter.fromYarn(task.taskId),
task.getFinishTime(task.successfulAttempt),
TypeConverter.fromYarn(task.taskId.getTaskType()),
taskState.toString(),
TypeConverter.fromYarn(task.getCounters()));
return tfe;
}
private static TaskFailedEvent createTaskFailedEvent(TaskImpl task, List<String> diag, TaskState taskState, TaskAttemptId taId) {
StringBuilder errorSb = new StringBuilder();
if (diag != null) {
for (String d : diag) {
errorSb.append(", ").append(d);
}
}
TaskFailedEvent taskFailedEvent = new TaskFailedEvent(
TypeConverter.fromYarn(task.taskId),
// Hack since getFinishTime needs isFinished to be true and that doesn't happen till after the transition.
task.getFinishTime(taId),
TypeConverter.fromYarn(task.getType()),
errorSb.toString(),
taskState.toString(),
taId == null ? null : TypeConverter.fromYarn(taId));
return taskFailedEvent;
}
/**
* @return a String representation of the splits.
*
* Subclasses can override this method to provide their own representations
* of splits (if any).
*
*/
protected String getSplitsAsString(){
return "";
}
private static class InitialScheduleTransition
implements SingleArcTransition<TaskImpl, TaskEvent> {
@Override
public void transition(TaskImpl task, TaskEvent event) {
task.addAndScheduleAttempt();
task.scheduledTime = task.clock.getTime();
TaskStartedEvent tse = new TaskStartedEvent(
TypeConverter.fromYarn(task.taskId), task.getLaunchTime(),
TypeConverter.fromYarn(task.taskId.getTaskType()),
task.getSplitsAsString());
task.eventHandler
.handle(new JobHistoryEvent(task.taskId.getJobId(), tse));
task.historyTaskStartGenerated = true;
}
}
// Used when creating a new attempt while one is already running.
// Currently we do this for speculation. In the future we may do this
// for tasks that failed in a way that might indicate application code
// problems, so we can take later failures in parallel and flush the
// job quickly when this happens.
private static class RedundantScheduleTransition
implements SingleArcTransition<TaskImpl, TaskEvent> {
@Override
public void transition(TaskImpl task, TaskEvent event) {
LOG.info("Scheduling a redundant attempt for task " + task.taskId);
task.addAndScheduleAttempt();
}
}
private static class AttemptCommitPendingTransition
implements SingleArcTransition<TaskImpl, TaskEvent> {
@Override
public void transition(TaskImpl task, TaskEvent event) {
TaskTAttemptEvent ev = (TaskTAttemptEvent) event;
// The nextAttemptNumber is commit pending, decide on set the commitAttempt
TaskAttemptId attemptID = ev.getTaskAttemptID();
if (task.commitAttempt == null) {
// TODO: validate attemptID
task.commitAttempt = attemptID;
LOG.info(attemptID + " given a go for committing the task output.");
} else {
// Don't think this can be a pluggable decision, so simply raise an
// event for the TaskAttempt to delete its output.
LOG.info(task.commitAttempt
+ " already given a go for committing the task output, so killing "
+ attemptID);
task.eventHandler.handle(new TaskAttemptEvent(
attemptID, TaskAttemptEventType.TA_KILL));
}
}
}
private static class AttemptSucceededTransition
implements SingleArcTransition<TaskImpl, TaskEvent> {
@Override
public void transition(TaskImpl task, TaskEvent event) {
task.handleTaskAttemptCompletion(
((TaskTAttemptEvent) event).getTaskAttemptID(),
TaskAttemptCompletionEventStatus.SUCCEEDED);
--task.numberUncompletedAttempts;
task.successfulAttempt = ((TaskTAttemptEvent) event).getTaskAttemptID();
task.eventHandler.handle(new JobTaskEvent(
task.taskId, TaskState.SUCCEEDED));
LOG.info("Task succeeded with attempt " + task.successfulAttempt);
// issue kill to all other attempts
if (task.historyTaskStartGenerated) {
TaskFinishedEvent tfe = createTaskFinishedEvent(task,
TaskState.SUCCEEDED);
task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
tfe));
}
for (TaskAttempt attempt : task.attempts.values()) {
if (attempt.getID() != task.successfulAttempt &&
// This is okay because it can only talk us out of sending a
// TA_KILL message to an attempt that doesn't need one for
// other reasons.
!attempt.isFinished()) {
LOG.info("Issuing kill to other attempt " + attempt.getID());
task.eventHandler.handle(
new TaskAttemptEvent(attempt.getID(),
TaskAttemptEventType.TA_KILL));
}
}
task.finished(TaskState.SUCCEEDED);
}
}
private static class AttemptKilledTransition implements
SingleArcTransition<TaskImpl, TaskEvent> {
@Override
public void transition(TaskImpl task, TaskEvent event) {
task.handleTaskAttemptCompletion(
((TaskTAttemptEvent) event).getTaskAttemptID(),
TaskAttemptCompletionEventStatus.KILLED);
--task.numberUncompletedAttempts;
if (task.successfulAttempt == null) {
task.addAndScheduleAttempt();
}
}
}
private static class KillWaitAttemptKilledTransition implements
MultipleArcTransition<TaskImpl, TaskEvent, TaskState> {
protected TaskState finalState = TaskState.KILLED;
@Override
public TaskState transition(TaskImpl task, TaskEvent event) {
task.handleTaskAttemptCompletion(
((TaskTAttemptEvent) event).getTaskAttemptID(),
TaskAttemptCompletionEventStatus.KILLED);
// check whether all attempts are finished
if (task.finishedAttempts == task.attempts.size()) {
if (task.historyTaskStartGenerated) {
TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, null,
finalState, null); // TODO JH verify failedAttempt null
task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
taskFailedEvent));
} else {
LOG.debug("Not generating HistoryFinish event since start event not" +
" generated for task: " + task.getID());
}
task.eventHandler.handle(
new JobTaskEvent(task.taskId, finalState));
return finalState;
}
return task.getState();
}
}
private static class AttemptFailedTransition implements
MultipleArcTransition<TaskImpl, TaskEvent, TaskState> {
@Override
public TaskState transition(TaskImpl task, TaskEvent event) {
task.failedAttempts++;
TaskTAttemptEvent castEvent = (TaskTAttemptEvent) event;
TaskAttempt attempt = task.attempts.get(castEvent.getTaskAttemptID());
if (attempt.getAssignedContainerMgrAddress() != null) {
//container was assigned
task.eventHandler.handle(new ContainerFailedEvent(attempt.getID(),
attempt.getAssignedContainerMgrAddress()));
}
if (task.failedAttempts < task.maxAttempts) {
task.handleTaskAttemptCompletion(
((TaskTAttemptEvent) event).getTaskAttemptID(),
TaskAttemptCompletionEventStatus.FAILED);
// we don't need a new event if we already have a spare
if (--task.numberUncompletedAttempts == 0
&& task.successfulAttempt == null) {
task.addAndScheduleAttempt();
}
} else {
task.handleTaskAttemptCompletion(
((TaskTAttemptEvent) event).getTaskAttemptID(),
TaskAttemptCompletionEventStatus.TIPFAILED);
TaskTAttemptEvent ev = (TaskTAttemptEvent) event;
TaskAttemptId taId = ev.getTaskAttemptID();
if (task.historyTaskStartGenerated) {
TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, attempt.getDiagnostics(),
TaskState.FAILED, taId);
task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
taskFailedEvent));
} else {
LOG.debug("Not generating HistoryFinish event since start event not" +
" generated for task: " + task.getID());
}
task.eventHandler.handle(
new JobTaskEvent(task.taskId, TaskState.FAILED));
return task.finished(TaskState.FAILED);
}
return getDefaultState(task);
}
protected TaskState getDefaultState(Task task) {
return task.getState();
}
protected void unSucceed(TaskImpl task) {
++task.numberUncompletedAttempts;
task.successfulAttempt = null;
}
}
private static class MapRetroactiveFailureTransition
extends AttemptFailedTransition {
@Override
public TaskState transition(TaskImpl task, TaskEvent event) {
//verify that this occurs only for map task
//TODO: consider moving it to MapTaskImpl
if (!TaskType.MAP.equals(task.getType())) {
LOG.error("Unexpected event for REDUCE task " + event.getType());
task.internalError(event.getType());
}
// tell the job about the rescheduling
task.eventHandler.handle(
new JobMapTaskRescheduledEvent(task.taskId));
// super.transition is mostly coded for the case where an
// UNcompleted task failed. When a COMPLETED task retroactively
// fails, we have to let AttemptFailedTransition.transition
// believe that there's no redundancy.
unSucceed(task);
return super.transition(task, event);
}
@Override
protected TaskState getDefaultState(Task task) {
return TaskState.SCHEDULED;
}
}
private static class KillNewTransition
implements SingleArcTransition<TaskImpl, TaskEvent> {
@Override
public void transition(TaskImpl task, TaskEvent event) {
if (task.historyTaskStartGenerated) {
TaskFailedEvent taskFailedEvent = createTaskFailedEvent(task, null,
TaskState.KILLED, null); // TODO Verify failedAttemptId is null
task.eventHandler.handle(new JobHistoryEvent(task.taskId.getJobId(),
taskFailedEvent));
}else {
LOG.debug("Not generating HistoryFinish event since start event not" +
" generated for task: " + task.getID());
}
task.eventHandler.handle(
new JobTaskEvent(task.taskId, TaskState.KILLED));
task.metrics.endWaitingTask(task);
}
}
private void killUnfinishedAttempt(TaskAttempt attempt, String logMsg) {
if (attempt != null && !attempt.isFinished()) {
eventHandler.handle(
new TaskAttemptEvent(attempt.getID(),
TaskAttemptEventType.TA_KILL));
}
}
private static class KillTransition
implements SingleArcTransition<TaskImpl, TaskEvent> {
@Override
public void transition(TaskImpl task, TaskEvent event) {
// issue kill to all non finished attempts
for (TaskAttempt attempt : task.attempts.values()) {
task.killUnfinishedAttempt
(attempt, "Task KILL is received. Killing attempt!");
}
task.numberUncompletedAttempts = 0;
}
}
static class LaunchTransition
implements SingleArcTransition<TaskImpl, TaskEvent> {
@Override
public void transition(TaskImpl task, TaskEvent event) {
task.metrics.launchedTask(task);
task.metrics.runningTask(task);
}
}
}

View File

@ -0,0 +1,31 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.launcher;
import org.apache.hadoop.yarn.event.EventHandler;
public interface ContainerLauncher
extends EventHandler<ContainerLauncherEvent> {
enum EventType {
CONTAINER_REMOTE_LAUNCH,
CONTAINER_REMOTE_CLEANUP
}
}

View File

@ -0,0 +1,114 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.launcher;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerToken;
import org.apache.hadoop.yarn.event.AbstractEvent;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
public class ContainerLauncherEvent
extends AbstractEvent<ContainerLauncher.EventType> {
private TaskAttemptId taskAttemptID;
private ContainerId containerID;
private String containerMgrAddress;
private ContainerToken containerToken;
public ContainerLauncherEvent(TaskAttemptId taskAttemptID,
ContainerId containerID,
String containerMgrAddress,
ContainerToken containerToken,
ContainerLauncher.EventType type) {
super(type);
this.taskAttemptID = taskAttemptID;
this.containerID = containerID;
this.containerMgrAddress = containerMgrAddress;
this.containerToken = containerToken;
}
public TaskAttemptId getTaskAttemptID() {
return this.taskAttemptID;
}
public ContainerId getContainerID() {
return containerID;
}
public String getContainerMgrAddress() {
return containerMgrAddress;
}
public ContainerToken getContainerToken() {
return containerToken;
}
@Override
public String toString() {
return super.toString() + " for taskAttempt " + taskAttemptID;
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result
+ ((containerID == null) ? 0 : containerID.hashCode());
result = prime * result
+ ((containerMgrAddress == null) ? 0 : containerMgrAddress.hashCode());
result = prime * result
+ ((containerToken == null) ? 0 : containerToken.hashCode());
result = prime * result
+ ((taskAttemptID == null) ? 0 : taskAttemptID.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj)
return true;
if (obj == null)
return false;
if (getClass() != obj.getClass())
return false;
ContainerLauncherEvent other = (ContainerLauncherEvent) obj;
if (containerID == null) {
if (other.containerID != null)
return false;
} else if (!containerID.equals(other.containerID))
return false;
if (containerMgrAddress == null) {
if (other.containerMgrAddress != null)
return false;
} else if (!containerMgrAddress.equals(other.containerMgrAddress))
return false;
if (containerToken == null) {
if (other.containerToken != null)
return false;
} else if (!containerToken.equals(other.containerToken))
return false;
if (taskAttemptID == null) {
if (other.taskAttemptID != null)
return false;
} else if (!taskAttemptID.equals(other.taskAttemptID))
return false;
return true;
}
}

View File

@ -0,0 +1,279 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.launcher;
import java.io.IOException;
import java.security.PrivilegedAction;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
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.io.Text;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.ContainerManager;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerToken;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.service.AbstractService;
/**
* This class is responsible for launching of containers.
*/
public class ContainerLauncherImpl extends AbstractService implements
ContainerLauncher {
private static final Log LOG = LogFactory.getLog(ContainerLauncherImpl.class);
private AppContext context;
private ThreadPoolExecutor launcherPool;
private Thread eventHandlingThread;
private BlockingQueue<ContainerLauncherEvent> eventQueue =
new LinkedBlockingQueue<ContainerLauncherEvent>();
private RecordFactory recordFactory;
//have a cache/map of UGIs so as to avoid creating too many RPC
//client connection objects to the same NodeManager
private Map<String, UserGroupInformation> ugiMap =
new HashMap<String, UserGroupInformation>();
public ContainerLauncherImpl(AppContext context) {
super(ContainerLauncherImpl.class.getName());
this.context = context;
}
@Override
public synchronized void init(Configuration conf) {
// Clone configuration for this component so that the SecurityInfo setting
// doesn't affect the original configuration
Configuration myLocalConfig = new Configuration(conf);
myLocalConfig.setClass(
YarnConfiguration.YARN_SECURITY_INFO,
ContainerManagerSecurityInfo.class, SecurityInfo.class);
this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
super.init(myLocalConfig);
}
public void start() {
launcherPool =
new ThreadPoolExecutor(getConfig().getInt(
AMConstants.CONTAINERLAUNCHER_THREADPOOL_SIZE, 10),
Integer.MAX_VALUE, 1, TimeUnit.HOURS,
new LinkedBlockingQueue<Runnable>());
launcherPool.prestartAllCoreThreads(); // Wait for work.
eventHandlingThread = new Thread(new Runnable() {
@Override
public void run() {
ContainerLauncherEvent event = null;
while (!Thread.currentThread().isInterrupted()) {
try {
event = eventQueue.take();
} catch (InterruptedException e) {
LOG.error("Returning, interrupted : " + e);
return;
}
// the events from the queue are handled in parallel
// using a thread pool
launcherPool.execute(new EventProcessor(event));
// TODO: Group launching of multiple containers to a single
// NodeManager into a single connection
}
}
});
eventHandlingThread.start();
super.start();
}
public void stop() {
eventHandlingThread.interrupt();
launcherPool.shutdown();
super.stop();
}
protected ContainerManager getCMProxy(ContainerId containerID,
final String containerManagerBindAddr, ContainerToken containerToken)
throws IOException {
UserGroupInformation user = UserGroupInformation.getCurrentUser();
// TODO: Synchronization problems!!
if (UserGroupInformation.isSecurityEnabled()) {
if(!ugiMap.containsKey(containerManagerBindAddr)) {
Token<ContainerTokenIdentifier> token =
new Token<ContainerTokenIdentifier>(
containerToken.getIdentifier().array(),
containerToken.getPassword().array(), new Text(
containerToken.getKind()), new Text(
containerToken.getService()));
//the user in createRemoteUser in this context is not important
user = UserGroupInformation.createRemoteUser(containerManagerBindAddr);
user.addToken(token);
ugiMap.put(containerManagerBindAddr, user);
} else {
user = ugiMap.get(containerManagerBindAddr);
}
}
ContainerManager proxy =
user.doAs(new PrivilegedAction<ContainerManager>() {
@Override
public ContainerManager run() {
YarnRPC rpc = YarnRPC.create(getConfig());
return (ContainerManager) rpc.getProxy(ContainerManager.class,
NetUtils.createSocketAddr(containerManagerBindAddr),
getConfig());
}
});
return proxy;
}
/**
* Setup and start the container on remote nodemanager.
*/
private class EventProcessor implements Runnable {
private ContainerLauncherEvent event;
EventProcessor(ContainerLauncherEvent event) {
this.event = event;
}
@Override
public void run() {
LOG.info("Processing the event " + event.toString());
// Load ContainerManager tokens before creating a connection.
// TODO: Do it only once per NodeManager.
final String containerManagerBindAddr = event.getContainerMgrAddress();
ContainerId containerID = event.getContainerID();
ContainerToken containerToken = event.getContainerToken();
switch(event.getType()) {
case CONTAINER_REMOTE_LAUNCH:
ContainerRemoteLaunchEvent launchEv = (ContainerRemoteLaunchEvent) event;
TaskAttemptId taskAttemptID = launchEv.getTaskAttemptID();
try {
ContainerManager proxy =
getCMProxy(containerID, containerManagerBindAddr, containerToken);
// Construct the actual Container
ContainerLaunchContext containerLaunchContext =
launchEv.getContainer();
// Now launch the actual container
StartContainerRequest startRequest = recordFactory
.newRecordInstance(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext);
proxy.startContainer(startRequest);
LOG.info("Returning from container-launch for " + taskAttemptID);
// after launching, send launched event to task attempt to move
// it from ASSIGNED to RUNNING state
context.getEventHandler().handle(
new TaskAttemptEvent(taskAttemptID,
TaskAttemptEventType.TA_CONTAINER_LAUNCHED));
} catch (Throwable t) {
String message = "Container launch failed for " + containerID
+ " : " + StringUtils.stringifyException(t);
LOG.error(message);
context.getEventHandler().handle(
new TaskAttemptDiagnosticsUpdateEvent(taskAttemptID, message));
context.getEventHandler().handle(
new TaskAttemptEvent(taskAttemptID,
TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED));
}
break;
case CONTAINER_REMOTE_CLEANUP:
// We will have to remove the launch (meant "cleanup"? FIXME) event if it is still in eventQueue
// and not yet processed
if (eventQueue.contains(event)) {
eventQueue.remove(event); // TODO: Any synchro needed?
//deallocate the container
context.getEventHandler().handle(
new ContainerAllocatorEvent(event.getTaskAttemptID(),
ContainerAllocator.EventType.CONTAINER_DEALLOCATE));
} else {
try {
ContainerManager proxy =
getCMProxy(containerID, containerManagerBindAddr, containerToken);
// TODO:check whether container is launched
// kill the remote container if already launched
StopContainerRequest stopRequest = recordFactory
.newRecordInstance(StopContainerRequest.class);
stopRequest.setContainerId(event.getContainerID());
proxy.stopContainer(stopRequest);
} catch (Throwable t) {
//ignore the cleanup failure
LOG.warn("cleanup failed for container " + event.getContainerID() ,
t);
}
// after killing, send killed event to taskattempt
context.getEventHandler().handle(
new TaskAttemptEvent(event.getTaskAttemptID(),
TaskAttemptEventType.TA_CONTAINER_CLEANED));
}
break;
}
}
}
@Override
public void handle(ContainerLauncherEvent event) {
try {
eventQueue.put(event);
} catch (InterruptedException e) {
throw new YarnException(e);
}
}
}

View File

@ -0,0 +1,40 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.launcher;
import org.apache.hadoop.mapred.Task;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerToken;
public abstract class ContainerRemoteLaunchEvent extends ContainerLauncherEvent {
public ContainerRemoteLaunchEvent(TaskAttemptId taskAttemptID,
ContainerId containerID, String containerMgrAddress,
ContainerToken containerToken) {
super(taskAttemptID, containerID, containerMgrAddress,
containerToken,
ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH);
}
public abstract ContainerLaunchContext getContainer();
public abstract Task getRemoteTask();
}

View File

@ -0,0 +1,100 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.local;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerRequestEvent;
import org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.util.Records;
/**
* Allocates containers locally. Doesn't allocate a real container;
* instead sends an allocated event for all requests.
*/
public class LocalContainerAllocator extends RMCommunicator
implements ContainerAllocator {
private static final Log LOG =
LogFactory.getLog(LocalContainerAllocator.class);
private final EventHandler eventHandler;
private final ApplicationId appID;
private AtomicInteger containerCount = new AtomicInteger();
private final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
public LocalContainerAllocator(ClientService clientService,
AppContext context) {
super(clientService, context);
this.eventHandler = context.getEventHandler();
this.appID = context.getApplicationID();
}
@Override
public void handle(ContainerAllocatorEvent event) {
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
LOG.info("Processing the event " + event.toString());
ContainerId cID = recordFactory.newRecordInstance(ContainerId.class);
cID.setAppId(appID);
// use negative ids to denote that these are local. Need a better way ??
cID.setId((-1) * containerCount.getAndIncrement());
Container container = recordFactory.newRecordInstance(Container.class);
container.setId(cID);
NodeId nodeId = Records.newRecord(NodeId.class);
nodeId.setHost("localhost");
nodeId.setPort(1234);
container.setNodeId(nodeId);
container.setContainerToken(null);
container.setNodeHttpAddress("localhost:9999");
// send the container-assigned event to task attempt
if (event.getAttemptID().getTaskId().getTaskType() == TaskType.MAP) {
JobCounterUpdateEvent jce =
new JobCounterUpdateEvent(event.getAttemptID().getTaskId()
.getJobId());
// TODO Setting OTHER_LOCAL_MAP for now.
jce.addCounterUpdate(JobCounter.OTHER_LOCAL_MAPS, 1);
eventHandler.handle(jce);
}
eventHandler.handle(new TaskAttemptContainerAssignedEvent(
event.getAttemptID(), container));
}
}
}

View File

@ -0,0 +1,183 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.metrics;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.metrics2.MetricsSystem;
import org.apache.hadoop.metrics2.annotation.Metric;
import org.apache.hadoop.metrics2.annotation.Metrics;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.lib.MutableCounterInt;
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
import org.apache.hadoop.metrics2.source.JvmMetrics;
@Metrics(about="MR App Metrics", context="mapred")
public class MRAppMetrics {
@Metric MutableCounterInt jobsSubmitted;
@Metric MutableCounterInt jobsCompleted;
@Metric MutableCounterInt jobsFailed;
@Metric MutableCounterInt jobsKilled;
@Metric MutableGaugeInt jobsPreparing;
@Metric MutableGaugeInt jobsRunning;
@Metric MutableCounterInt mapsLaunched;
@Metric MutableCounterInt mapsCompleted;
@Metric MutableCounterInt mapsFailed;
@Metric MutableCounterInt mapsKilled;
@Metric MutableGaugeInt mapsRunning;
@Metric MutableGaugeInt mapsWaiting;
@Metric MutableCounterInt reducesLaunched;
@Metric MutableCounterInt reducesCompleted;
@Metric MutableCounterInt reducesFailed;
@Metric MutableCounterInt reducesKilled;
@Metric MutableGaugeInt reducesRunning;
@Metric MutableGaugeInt reducesWaiting;
public static MRAppMetrics create() {
return create(DefaultMetricsSystem.instance());
}
public static MRAppMetrics create(MetricsSystem ms) {
JvmMetrics.create("MRAppMaster", null, ms);
return ms.register(new MRAppMetrics());
}
// potential instrumentation interface methods
public void submittedJob(Job job) {
jobsSubmitted.incr();
}
public void completedJob(Job job) {
jobsCompleted.incr();
}
public void failedJob(Job job) {
jobsFailed.incr();
}
public void killedJob(Job job) {
jobsKilled.incr();
}
public void preparingJob(Job job) {
jobsPreparing.incr();
}
public void endPreparingJob(Job job) {
jobsPreparing.decr();
}
public void runningJob(Job job) {
jobsRunning.incr();
}
public void endRunningJob(Job job) {
jobsRunning.decr();
}
public void launchedTask(Task task) {
switch (task.getType()) {
case MAP:
mapsLaunched.incr();
break;
case REDUCE:
reducesLaunched.incr();
break;
}
endWaitingTask(task);
}
public void completedTask(Task task) {
switch (task.getType()) {
case MAP:
mapsCompleted.incr();
break;
case REDUCE:
reducesCompleted.incr();
break;
}
}
public void failedTask(Task task) {
switch (task.getType()) {
case MAP:
mapsFailed.incr();
break;
case REDUCE:
reducesFailed.incr();
break;
}
}
public void killedTask(Task task) {
switch (task.getType()) {
case MAP:
mapsKilled.incr();
break;
case REDUCE:
reducesKilled.incr();
break;
}
}
public void runningTask(Task task) {
switch (task.getType()) {
case MAP:
mapsRunning.incr();
break;
case REDUCE:
reducesRunning.incr();
break;
}
}
public void endRunningTask(Task task) {
switch (task.getType()) {
case MAP:
mapsRunning.decr();
break;
case REDUCE:
reducesRunning.decr();
break;
}
}
public void waitingTask(Task task) {
switch (task.getType()) {
case MAP:
mapsWaiting.incr();
break;
case REDUCE:
reducesWaiting.incr();
}
}
public void endWaitingTask(Task task) {
switch (task.getType()) {
case MAP:
mapsWaiting.decr();
break;
case REDUCE:
reducesWaiting.decr();
break;
}
}
}

View File

@ -0,0 +1,43 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.recover;
import org.apache.hadoop.yarn.Clock;
class ControlledClock implements Clock {
private long time = -1;
private final Clock actualClock;
ControlledClock(Clock actualClock) {
this.actualClock = actualClock;
}
synchronized void setTime(long time) {
this.time = time;
}
synchronized void reset() {
time = -1;
}
@Override
public synchronized long getTime() {
if (time != -1) {
return time;
}
return actualClock.getTime();
}
}

View File

@ -0,0 +1,34 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.recover;
import java.util.Set;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.event.Dispatcher;
public interface Recovery {
Dispatcher getDispatcher();
Clock getClock();
Set<TaskId> getCompletedTasks();
}

View File

@ -0,0 +1,368 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.recover;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
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.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.JobInfo;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskAttemptInfo;
import org.apache.hadoop.mapreduce.jobhistory.JobHistoryParser.TaskInfo;
import org.apache.hadoop.mapreduce.v2.api.records.Phase;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskTAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleaner;
import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanupEvent;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.service.CompositeService;
import org.apache.hadoop.yarn.service.Service;
/*
* Recovers the completed tasks from the previous life of Application Master.
* The completed tasks are deciphered from the history file of the previous life.
* Recovery service intercepts and replay the events for completed tasks.
* While recovery is in progress, the scheduling of new tasks are delayed by
* buffering the task schedule events.
* The recovery service controls the clock while recovery is in progress.
*/
//TODO:
//task cleanup for all non completed tasks
//change job output committer to have
// - atomic job output promotion
// - recover output of completed tasks
public class RecoveryService extends CompositeService implements Recovery {
private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
private static final Log LOG = LogFactory.getLog(RecoveryService.class);
private final ApplicationId appID;
private final Dispatcher dispatcher;
private final ControlledClock clock;
private final int startCount;
private JobInfo jobInfo = null;
private final Map<TaskId, TaskInfo> completedTasks =
new HashMap<TaskId, TaskInfo>();
private final List<TaskEvent> pendingTaskScheduleEvents =
new ArrayList<TaskEvent>();
private volatile boolean recoveryMode = false;
public RecoveryService(ApplicationId appID, Clock clock, int startCount) {
super("RecoveringDispatcher");
this.appID = appID;
this.startCount = startCount;
this.dispatcher = new RecoveryDispatcher();
this.clock = new ControlledClock(clock);
addService((Service) dispatcher);
}
@Override
public void init(Configuration conf) {
super.init(conf);
// parse the history file
try {
parse();
if (completedTasks.size() > 0) {
recoveryMode = true;
LOG.info("SETTING THE RECOVERY MODE TO TRUE. NO OF COMPLETED TASKS " +
"TO RECOVER " + completedTasks.size());
LOG.info("Job launch time " + jobInfo.getLaunchTime());
clock.setTime(jobInfo.getLaunchTime());
}
} catch (IOException e) {
LOG.warn(e);
LOG.warn("Could not parse the old history file. Aborting recovery. "
+ "Starting afresh.");
}
}
@Override
public Dispatcher getDispatcher() {
return dispatcher;
}
@Override
public Clock getClock() {
return clock;
}
@Override
public Set<TaskId> getCompletedTasks() {
return completedTasks.keySet();
}
private void parse() throws IOException {
// TODO: parse history file based on startCount
String jobName = TypeConverter.fromYarn(appID).toString();
String jobhistoryDir = JobHistoryUtils.getConfiguredHistoryStagingDirPrefix(getConfig());
FSDataInputStream in = null;
Path historyFile = null;
Path histDirPath = FileContext.getFileContext(getConfig()).makeQualified(
new Path(jobhistoryDir));
FileContext fc = FileContext.getFileContext(histDirPath.toUri(),
getConfig());
historyFile = fc.makeQualified(JobHistoryUtils.getStagingJobHistoryFile(
histDirPath, jobName, startCount - 1)); //read the previous history file
in = fc.open(historyFile);
JobHistoryParser parser = new JobHistoryParser(in);
jobInfo = parser.parse();
Map<org.apache.hadoop.mapreduce.TaskID, TaskInfo> taskInfos = jobInfo
.getAllTasks();
for (TaskInfo taskInfo : taskInfos.values()) {
if (TaskState.SUCCEEDED.toString().equals(taskInfo.getTaskStatus())) {
completedTasks
.put(TypeConverter.toYarn(taskInfo.getTaskId()), taskInfo);
LOG.info("Read from history task "
+ TypeConverter.toYarn(taskInfo.getTaskId()));
}
}
LOG.info("Read completed tasks from history "
+ completedTasks.size());
}
class RecoveryDispatcher extends AsyncDispatcher {
private final EventHandler actualHandler;
private final EventHandler handler;
RecoveryDispatcher() {
actualHandler = super.getEventHandler();
handler = new InterceptingEventHandler(actualHandler);
}
@Override
public void dispatch(Event event) {
if (recoveryMode) {
if (event.getType() == TaskAttemptEventType.TA_CONTAINER_LAUNCHED) {
TaskAttemptInfo attInfo = getTaskAttemptInfo(((TaskAttemptEvent) event)
.getTaskAttemptID());
LOG.info("Attempt start time " + attInfo.getStartTime());
clock.setTime(attInfo.getStartTime());
} else if (event.getType() == TaskAttemptEventType.TA_DONE
|| event.getType() == TaskAttemptEventType.TA_FAILMSG
|| event.getType() == TaskAttemptEventType.TA_KILL) {
TaskAttemptInfo attInfo = getTaskAttemptInfo(((TaskAttemptEvent) event)
.getTaskAttemptID());
LOG.info("Attempt finish time " + attInfo.getFinishTime());
clock.setTime(attInfo.getFinishTime());
}
else if (event.getType() == TaskEventType.T_ATTEMPT_FAILED
|| event.getType() == TaskEventType.T_ATTEMPT_KILLED
|| event.getType() == TaskEventType.T_ATTEMPT_SUCCEEDED) {
TaskTAttemptEvent tEvent = (TaskTAttemptEvent) event;
LOG.info("Recovered Task attempt " + tEvent.getTaskAttemptID());
TaskInfo taskInfo = completedTasks.get(tEvent.getTaskAttemptID()
.getTaskId());
taskInfo.getAllTaskAttempts().remove(
TypeConverter.fromYarn(tEvent.getTaskAttemptID()));
// remove the task info from completed tasks if all attempts are
// recovered
if (taskInfo.getAllTaskAttempts().size() == 0) {
completedTasks.remove(tEvent.getTaskAttemptID().getTaskId());
// checkForRecoveryComplete
LOG.info("CompletedTasks() " + completedTasks.size());
if (completedTasks.size() == 0) {
recoveryMode = false;
clock.reset();
LOG.info("Setting the recovery mode to false. " +
"Recovery is complete!");
// send all pending tasks schedule events
for (TaskEvent tEv : pendingTaskScheduleEvents) {
actualHandler.handle(tEv);
}
}
}
}
}
super.dispatch(event);
}
@Override
public EventHandler getEventHandler() {
return handler;
}
}
private TaskAttemptInfo getTaskAttemptInfo(TaskAttemptId id) {
TaskInfo taskInfo = completedTasks.get(id.getTaskId());
return taskInfo.getAllTaskAttempts().get(TypeConverter.fromYarn(id));
}
private class InterceptingEventHandler implements EventHandler {
EventHandler actualHandler;
InterceptingEventHandler(EventHandler actualHandler) {
this.actualHandler = actualHandler;
}
@Override
public void handle(Event event) {
if (!recoveryMode) {
// delegate to the dispatcher one
actualHandler.handle(event);
return;
}
else if (event.getType() == TaskEventType.T_SCHEDULE) {
TaskEvent taskEvent = (TaskEvent) event;
// delay the scheduling of new tasks till previous ones are recovered
if (completedTasks.get(taskEvent.getTaskID()) == null) {
LOG.debug("Adding to pending task events "
+ taskEvent.getTaskID());
pendingTaskScheduleEvents.add(taskEvent);
return;
}
}
else if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
TaskAttemptId aId = ((ContainerAllocatorEvent) event).getAttemptID();
TaskAttemptInfo attInfo = getTaskAttemptInfo(aId);
LOG.debug("CONTAINER_REQ " + aId);
sendAssignedEvent(aId, attInfo);
return;
}
else if (event.getType() == TaskCleaner.EventType.TASK_CLEAN) {
TaskAttemptId aId = ((TaskCleanupEvent) event).getAttemptID();
LOG.debug("TASK_CLEAN");
actualHandler.handle(new TaskAttemptEvent(aId,
TaskAttemptEventType.TA_CLEANUP_DONE));
return;
}
else if (event.getType() == ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH) {
TaskAttemptId aId = ((ContainerRemoteLaunchEvent) event)
.getTaskAttemptID();
TaskAttemptInfo attInfo = getTaskAttemptInfo(aId);
actualHandler.handle(new TaskAttemptEvent(aId,
TaskAttemptEventType.TA_CONTAINER_LAUNCHED));
// send the status update event
sendStatusUpdateEvent(aId, attInfo);
TaskAttemptState state = TaskAttemptState.valueOf(attInfo.getTaskStatus());
switch (state) {
case SUCCEEDED:
// send the done event
LOG.info("Sending done event to " + aId);
actualHandler.handle(new TaskAttemptEvent(aId,
TaskAttemptEventType.TA_DONE));
break;
case KILLED:
LOG.info("Sending kill event to " + aId);
actualHandler.handle(new TaskAttemptEvent(aId,
TaskAttemptEventType.TA_KILL));
break;
default:
LOG.info("Sending fail event to " + aId);
actualHandler.handle(new TaskAttemptEvent(aId,
TaskAttemptEventType.TA_FAILMSG));
break;
}
return;
}
// delegate to the actual handler
actualHandler.handle(event);
}
private void sendStatusUpdateEvent(TaskAttemptId yarnAttemptID,
TaskAttemptInfo attemptInfo) {
LOG.info("Sending status update event to " + yarnAttemptID);
TaskAttemptStatus taskAttemptStatus = new TaskAttemptStatus();
taskAttemptStatus.id = yarnAttemptID;
taskAttemptStatus.progress = 1.0f;
taskAttemptStatus.diagnosticInfo = "";
taskAttemptStatus.stateString = attemptInfo.getTaskStatus();
// taskAttemptStatus.outputSize = attemptInfo.getOutputSize();
taskAttemptStatus.phase = Phase.CLEANUP;
org.apache.hadoop.mapreduce.Counters cntrs = attemptInfo.getCounters();
if (cntrs == null) {
taskAttemptStatus.counters = null;
} else {
taskAttemptStatus.counters = TypeConverter.toYarn(attemptInfo
.getCounters());
}
actualHandler.handle(new TaskAttemptStatusUpdateEvent(
taskAttemptStatus.id, taskAttemptStatus));
}
private void sendAssignedEvent(TaskAttemptId yarnAttemptID,
TaskAttemptInfo attemptInfo) {
LOG.info("Sending assigned event to " + yarnAttemptID);
ContainerId cId = recordFactory
.newRecordInstance(ContainerId.class);
Container container = recordFactory
.newRecordInstance(Container.class);
container.setId(cId);
container.setNodeId(recordFactory
.newRecordInstance(NodeId.class));
container.setContainerToken(null);
container.setNodeHttpAddress(attemptInfo.getHostname() + ":" +
attemptInfo.getHttpPort());
actualHandler.handle(new TaskAttemptContainerAssignedEvent(yarnAttemptID,
container));
}
}
}

View File

@ -0,0 +1,32 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.rm;
import org.apache.hadoop.yarn.event.EventHandler;
public interface ContainerAllocator extends EventHandler<ContainerAllocatorEvent>{
enum EventType {
CONTAINER_REQ,
CONTAINER_DEALLOCATE,
CONTAINER_FAILED
}
}

View File

@ -0,0 +1,38 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.rm;
import org.apache.hadoop.yarn.event.AbstractEvent;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
public class ContainerAllocatorEvent extends
AbstractEvent<ContainerAllocator.EventType> {
private TaskAttemptId attemptID;
public ContainerAllocatorEvent(TaskAttemptId attemptID,
ContainerAllocator.EventType type) {
super(type);
this.attemptID = attemptID;
}
public TaskAttemptId getAttemptID() {
return attemptID;
}
}

View File

@ -0,0 +1,18 @@
package org.apache.hadoop.mapreduce.v2.app.rm;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
public class ContainerFailedEvent extends ContainerAllocatorEvent {
private final String contMgrAddress;
public ContainerFailedEvent(TaskAttemptId attemptID, String contMgrAddr) {
super(attemptID, ContainerAllocator.EventType.CONTAINER_FAILED);
this.contMgrAddress = contMgrAddr;
}
public String getContMgrAddress() {
return contMgrAddress;
}
}

View File

@ -0,0 +1,68 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.rm;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.yarn.api.records.Resource;
public class ContainerRequestEvent extends ContainerAllocatorEvent {
private final Resource capability;
private final String[] hosts;
private final String[] racks;
private boolean earlierAttemptFailed = false;
public ContainerRequestEvent(TaskAttemptId attemptID,
Resource capability,
String[] hosts, String[] racks) {
super(attemptID, ContainerAllocator.EventType.CONTAINER_REQ);
this.capability = capability;
this.hosts = hosts;
this.racks = racks;
}
ContainerRequestEvent(TaskAttemptId attemptID, Resource capability) {
this(attemptID, capability, new String[0], new String[0]);
this.earlierAttemptFailed = true;
}
public static ContainerRequestEvent createContainerRequestEventForFailedContainer(
TaskAttemptId attemptID,
Resource capability) {
//ContainerRequest for failed events does not consider rack / node locality?
return new ContainerRequestEvent(attemptID, capability);
}
public Resource getCapability() {
return capability;
}
public String[] getHosts() {
return hosts;
}
public String[] getRacks() {
return racks;
}
public boolean getEarlierAttemptFailed() {
return earlierAttemptFailed;
}
}

View File

@ -0,0 +1,280 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.rm;
import java.io.IOException;
import java.security.PrivilegedAction;
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.fs.CommonConfigurationKeys;
import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityInfo;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.AMRMProtocol;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.AMResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.SchedulerSecurityInfo;
import org.apache.hadoop.yarn.service.AbstractService;
/**
* Registers/unregisters to RM and sends heartbeats to RM.
*/
public class RMCommunicator extends AbstractService {
private static final Log LOG = LogFactory.getLog(RMContainerAllocator.class);
private int rmPollInterval;//millis
protected ApplicationId applicationId;
protected ApplicationAttemptId applicationAttemptId;
private volatile boolean stopped;
protected Thread allocatorThread;
protected EventHandler eventHandler;
protected AMRMProtocol scheduler;
private final ClientService clientService;
private int lastResponseID;
private Resource minContainerCapability;
private Resource maxContainerCapability;
private final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
private final AppContext context;
private Job job;
public RMCommunicator(ClientService clientService, AppContext context) {
super("RMCommunicator");
this.clientService = clientService;
this.context = context;
this.eventHandler = context.getEventHandler();
this.applicationId = context.getApplicationID();
this.applicationAttemptId = context.getApplicationAttemptId();
}
@Override
public void init(Configuration conf) {
super.init(conf);
rmPollInterval =
conf.getInt(AMConstants.AM_RM_SCHEDULE_INTERVAL,
AMConstants.DEFAULT_AM_RM_SCHEDULE_INTERVAL);
}
@Override
public void start() {
scheduler= createSchedulerProxy();
//LOG.info("Scheduler is " + scheduler);
register();
startAllocatorThread();
JobID id = TypeConverter.fromYarn(context.getApplicationID());
JobId jobId = TypeConverter.toYarn(id);
job = context.getJob(jobId);
super.start();
}
protected AppContext getContext() {
return context;
}
protected Job getJob() {
return job;
}
protected void register() {
//Register
String host =
clientService.getBindAddress().getAddress().getHostAddress();
try {
RegisterApplicationMasterRequest request =
recordFactory.newRecordInstance(RegisterApplicationMasterRequest.class);
request.setApplicationAttemptId(applicationAttemptId);
request.setHost(host);
request.setRpcPort(clientService.getBindAddress().getPort());
request.setTrackingUrl(host + ":" + clientService.getHttpPort());
RegisterApplicationMasterResponse response =
scheduler.registerApplicationMaster(request);
minContainerCapability = response.getMinimumResourceCapability();
maxContainerCapability = response.getMaximumResourceCapability();
LOG.info("minContainerCapability: " + minContainerCapability.getMemory());
LOG.info("maxContainerCapability: " + maxContainerCapability.getMemory());
} catch (Exception are) {
LOG.info("Exception while registering", are);
throw new YarnException(are);
}
}
protected void unregister() {
try {
String finalState = "RUNNING";
if (job.getState() == JobState.SUCCEEDED) {
finalState = "SUCCEEDED";
} else if (job.getState() == JobState.KILLED) {
finalState = "KILLED";
} else if (job.getState() == JobState.FAILED
|| job.getState() == JobState.ERROR) {
finalState = "FAILED";
}
StringBuffer sb = new StringBuffer();
for (String s : job.getDiagnostics()) {
sb.append(s).append("\n");
}
LOG.info("Setting job diagnostics to " + sb.toString());
String historyUrl = JobHistoryUtils.getHistoryUrl(getConfig(),
context.getApplicationID());
LOG.info("History url is " + historyUrl);
FinishApplicationMasterRequest request =
recordFactory.newRecordInstance(FinishApplicationMasterRequest.class);
request.setAppAttemptId(this.applicationAttemptId);
request.setFinalState(finalState.toString());
request.setDiagnostics(sb.toString());
request.setTrackingUrl(historyUrl);
scheduler.finishApplicationMaster(request);
} catch(Exception are) {
LOG.info("Exception while unregistering ", are);
}
}
protected Resource getMinContainerCapability() {
return minContainerCapability;
}
protected Resource getMaxContainerCapability() {
return maxContainerCapability;
}
@Override
public void stop() {
stopped = true;
allocatorThread.interrupt();
try {
allocatorThread.join();
} catch (InterruptedException ie) {
LOG.info("InterruptedException while stopping", ie);
}
unregister();
super.stop();
}
protected void startAllocatorThread() {
allocatorThread = new Thread(new Runnable() {
@Override
public void run() {
while (!stopped && !Thread.currentThread().isInterrupted()) {
try {
Thread.sleep(rmPollInterval);
try {
heartbeat();
} catch (Exception e) {
LOG.error("ERROR IN CONTACTING RM. ", e);
// TODO: for other exceptions
}
} catch (InterruptedException e) {
LOG.info("Allocated thread interrupted. Returning.");
return;
}
}
}
});
allocatorThread.start();
}
protected AMRMProtocol createSchedulerProxy() {
final YarnRPC rpc = YarnRPC.create(getConfig());
final Configuration conf = new Configuration(getConfig());
final String serviceAddr = conf.get(
YarnConfiguration.SCHEDULER_ADDRESS,
YarnConfiguration.DEFAULT_SCHEDULER_BIND_ADDRESS);
UserGroupInformation currentUser;
try {
currentUser = UserGroupInformation.getCurrentUser();
} catch (IOException e) {
throw new YarnException(e);
}
if (UserGroupInformation.isSecurityEnabled()) {
conf.setClass(YarnConfiguration.YARN_SECURITY_INFO,
SchedulerSecurityInfo.class, SecurityInfo.class);
String tokenURLEncodedStr = System.getenv().get(
ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME);
LOG.debug("AppMasterToken is " + tokenURLEncodedStr);
Token<? extends TokenIdentifier> token = new Token<TokenIdentifier>();
try {
token.decodeFromUrlString(tokenURLEncodedStr);
} catch (IOException e) {
throw new YarnException(e);
}
currentUser.addToken(token);
}
return currentUser.doAs(new PrivilegedAction<AMRMProtocol>() {
@Override
public AMRMProtocol run() {
return (AMRMProtocol) rpc.getProxy(AMRMProtocol.class,
NetUtils.createSocketAddr(serviceAddr), conf);
}
});
}
protected synchronized void heartbeat() throws Exception {
AllocateRequest allocateRequest =
recordFactory.newRecordInstance(AllocateRequest.class);
allocateRequest.setApplicationAttemptId(applicationAttemptId);
allocateRequest.setResponseId(lastResponseID);
allocateRequest.addAllAsks(new ArrayList<ResourceRequest>());
allocateRequest.addAllReleases(new ArrayList<ContainerId>());
AllocateResponse allocateResponse = scheduler.allocate(allocateRequest);
AMResponse response = allocateResponse.getAMResponse();
if (response.getReboot()) {
LOG.info("Event from RM: shutting down Application Master");
}
}
}

View File

@ -0,0 +1,784 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.rm;
import java.net.InetAddress;
import java.net.UnknownHostException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
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.mapreduce.JobCounter;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobCounterUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobDiagnosticsUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.JobEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerAssignedEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptDiagnosticsUpdateEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.yarn.api.records.AMResponse;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.util.RackResolver;
/**
* Allocates the container from the ResourceManager scheduler.
*/
public class RMContainerAllocator extends RMContainerRequestor
implements ContainerAllocator {
private static final Log LOG = LogFactory.getLog(RMContainerAllocator.class);
public static final
float DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART = 0.05f;
private static final Priority PRIORITY_FAST_FAIL_MAP;
private static final Priority PRIORITY_REDUCE;
private static final Priority PRIORITY_MAP;
static {
PRIORITY_FAST_FAIL_MAP = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(Priority.class);
PRIORITY_FAST_FAIL_MAP.setPriority(5);
PRIORITY_REDUCE = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(Priority.class);
PRIORITY_REDUCE.setPriority(10);
PRIORITY_MAP = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(Priority.class);
PRIORITY_MAP.setPriority(20);
}
/*
Vocabulory Used:
pending -> requests which are NOT yet sent to RM
scheduled -> requests which are sent to RM but not yet assigned
assigned -> requests which are assigned to a container
completed -> request corresponding to which container has completed
Lifecycle of map
scheduled->assigned->completed
Lifecycle of reduce
pending->scheduled->assigned->completed
Maps are scheduled as soon as their requests are received. Reduces are
added to the pending and are ramped up (added to scheduled) based
on completed maps and current availability in the cluster.
*/
//reduces which are not yet scheduled
private final LinkedList<ContainerRequest> pendingReduces =
new LinkedList<ContainerRequest>();
//holds information about the assigned containers to task attempts
private final AssignedRequests assignedRequests = new AssignedRequests();
//holds scheduled requests to be fulfilled by RM
private final ScheduledRequests scheduledRequests = new ScheduledRequests();
private int containersAllocated = 0;
private int containersReleased = 0;
private int hostLocalAssigned = 0;
private int rackLocalAssigned = 0;
private boolean recalculateReduceSchedule = false;
private int mapResourceReqt;//memory
private int reduceResourceReqt;//memory
private int completedMaps = 0;
private int completedReduces = 0;
private boolean reduceStarted = false;
private float maxReduceRampupLimit = 0;
private float maxReducePreemptionLimit = 0;
private float reduceSlowStart = 0;
public RMContainerAllocator(ClientService clientService, AppContext context) {
super(clientService, context);
}
@Override
public void init(Configuration conf) {
super.init(conf);
reduceSlowStart = conf.getFloat(
MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART,
DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART);
maxReduceRampupLimit = conf.getFloat(
AMConstants.REDUCE_RAMPUP_UP_LIMIT,
AMConstants.DEFAULT_REDUCE_RAMP_UP_LIMIT);
maxReducePreemptionLimit = conf.getFloat(
AMConstants.REDUCE_PREEMPTION_LIMIT,
AMConstants.DEFAULT_REDUCE_PREEMPTION_LIMIT);
RackResolver.init(conf);
}
@Override
protected synchronized void heartbeat() throws Exception {
LOG.info("Before Scheduling: " + getStat());
List<Container> allocatedContainers = getResources();
LOG.info("After Scheduling: " + getStat());
if (allocatedContainers.size() > 0) {
LOG.info("Before Assign: " + getStat());
scheduledRequests.assign(allocatedContainers);
LOG.info("After Assign: " + getStat());
}
if (recalculateReduceSchedule) {
preemptReducesIfNeeded();
scheduleReduces();
recalculateReduceSchedule = false;
}
}
@Override
public void stop() {
super.stop();
LOG.info("Final Stats: " + getStat());
}
@Override
public synchronized void handle(ContainerAllocatorEvent event) {
LOG.info("Processing the event " + event.toString());
recalculateReduceSchedule = true;
if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {
ContainerRequestEvent reqEvent = (ContainerRequestEvent) event;
if (reqEvent.getAttemptID().getTaskId().getTaskType().equals(TaskType.MAP)) {
if (mapResourceReqt == 0) {
mapResourceReqt = reqEvent.getCapability().getMemory();
int minSlotMemSize = getMinContainerCapability().getMemory();
mapResourceReqt = (int) Math.ceil((float) mapResourceReqt/minSlotMemSize) * minSlotMemSize;
LOG.info("mapResourceReqt:"+mapResourceReqt);
if (mapResourceReqt > getMaxContainerCapability().getMemory()) {
String diagMsg = "MAP capability required is more than the supported " +
"max container capability in the cluster. Killing the Job. mapResourceReqt: " +
mapResourceReqt + " maxContainerCapability:" + getMaxContainerCapability().getMemory();
LOG.info(diagMsg);
eventHandler.handle(new JobDiagnosticsUpdateEvent(
getJob().getID(), diagMsg));
eventHandler.handle(new JobEvent(getJob().getID(), JobEventType.JOB_KILL));
}
}
//set the rounded off memory
reqEvent.getCapability().setMemory(mapResourceReqt);
scheduledRequests.addMap(reqEvent);//maps are immediately scheduled
} else {
if (reduceResourceReqt == 0) {
reduceResourceReqt = reqEvent.getCapability().getMemory();
int minSlotMemSize = getMinContainerCapability().getMemory();
//round off on slotsize
reduceResourceReqt = (int) Math.ceil((float) reduceResourceReqt/minSlotMemSize) * minSlotMemSize;
LOG.info("reduceResourceReqt:"+reduceResourceReqt);
if (reduceResourceReqt > getMaxContainerCapability().getMemory()) {
String diagMsg = "REDUCE capability required is more than the supported " +
"max container capability in the cluster. Killing the Job. reduceResourceReqt: " +
reduceResourceReqt + " maxContainerCapability:" + getMaxContainerCapability().getMemory();
LOG.info(diagMsg);
eventHandler.handle(new JobDiagnosticsUpdateEvent(
getJob().getID(), diagMsg));
eventHandler.handle(new JobEvent(getJob().getID(), JobEventType.JOB_KILL));
}
}
//set the rounded off memory
reqEvent.getCapability().setMemory(reduceResourceReqt);
if (reqEvent.getEarlierAttemptFailed()) {
//add to the front of queue for fail fast
pendingReduces.addFirst(new ContainerRequest(reqEvent, PRIORITY_REDUCE));
} else {
pendingReduces.add(new ContainerRequest(reqEvent, PRIORITY_REDUCE));//reduces are added to pending and are slowly ramped up
}
}
} else if (
event.getType() == ContainerAllocator.EventType.CONTAINER_DEALLOCATE) {
TaskAttemptId aId = event.getAttemptID();
boolean removed = scheduledRequests.remove(aId);
if (!removed) {
ContainerId containerId = assignedRequests.get(aId);
if (containerId != null) {
removed = true;
assignedRequests.remove(aId);
containersReleased++;
release(containerId);
}
}
if (!removed) {
LOG.error("Could not deallocate container for task attemptId " +
aId);
}
} else if (
event.getType() == ContainerAllocator.EventType.CONTAINER_FAILED) {
ContainerFailedEvent fEv = (ContainerFailedEvent) event;
String host = getHost(fEv.getContMgrAddress());
containerFailedOnHost(host);
}
}
private static String getHost(String contMgrAddress) {
String host = contMgrAddress;
String[] hostport = host.split(":");
if (hostport.length == 2) {
host = hostport[0];
}
return host;
}
private void preemptReducesIfNeeded() {
if (reduceResourceReqt == 0) {
return; //no reduces
}
//check if reduces have taken over the whole cluster and there are
//unassigned maps
if (scheduledRequests.maps.size() > 0) {
int memLimit = getMemLimit();
int availableMemForMap = memLimit - ((assignedRequests.reduces.size() -
assignedRequests.preemptionWaitingReduces.size()) * reduceResourceReqt);
//availableMemForMap must be sufficient to run atleast 1 map
if (availableMemForMap < mapResourceReqt) {
//to make sure new containers are given to maps and not reduces
//ramp down all scheduled reduces if any
//(since reduces are scheduled at higher priority than maps)
LOG.info("Ramping down all scheduled reduces:" + scheduledRequests.reduces.size());
for (ContainerRequest req : scheduledRequests.reduces.values()) {
pendingReduces.add(req);
}
scheduledRequests.reduces.clear();
//preempt for making space for atleast one map
int premeptionLimit = Math.max(mapResourceReqt,
(int) (maxReducePreemptionLimit * memLimit));
int preemptMem = Math.min(scheduledRequests.maps.size() * mapResourceReqt,
premeptionLimit);
int toPreempt = (int) Math.ceil((float) preemptMem/reduceResourceReqt);
toPreempt = Math.min(toPreempt, assignedRequests.reduces.size());
LOG.info("Going to preempt " + toPreempt);
assignedRequests.preemptReduce(toPreempt);
}
}
}
private void scheduleReduces() {
if (pendingReduces.size() == 0) {
return;
}
LOG.info("Recalculating schedule...");
//if all maps are assigned, then ramp up all reduces irrespective of the
//headroom
if (scheduledRequests.maps.size() == 0 && pendingReduces.size() > 0) {
LOG.info("All maps assigned. Ramping up all remaining reduces:" + pendingReduces.size());
for (ContainerRequest req : pendingReduces) {
scheduledRequests.addReduce(req);
}
pendingReduces.clear();
return;
}
int totalMaps = assignedRequests.maps.size() + completedMaps + scheduledRequests.maps.size();
//check for slow start
if (!reduceStarted) {//not set yet
int completedMapsForReduceSlowstart = (int)Math.ceil(reduceSlowStart *
totalMaps);
if(completedMaps < completedMapsForReduceSlowstart) {
LOG.info("Reduce slow start threshold not met. " +
"completedMapsForReduceSlowstart " + completedMapsForReduceSlowstart);
return;
} else {
LOG.info("Reduce slow start threshold reached. Scheduling reduces.");
reduceStarted = true;
}
}
float completedMapPercent = 0f;
if (totalMaps != 0) {//support for 0 maps
completedMapPercent = (float)completedMaps/totalMaps;
} else {
completedMapPercent = 1;
}
int netScheduledMapMem = scheduledRequests.maps.size() * mapResourceReqt
+ assignedRequests.maps.size() * mapResourceReqt;
int netScheduledReduceMem = scheduledRequests.reduces.size()
* reduceResourceReqt + assignedRequests.reduces.size()
* reduceResourceReqt;
int finalMapMemLimit = 0;
int finalReduceMemLimit = 0;
// ramp up the reduces based on completed map percentage
int totalMemLimit = getMemLimit();
int idealReduceMemLimit = Math.min((int)(completedMapPercent * totalMemLimit),
(int) (maxReduceRampupLimit * totalMemLimit));
int idealMapMemLimit = totalMemLimit - idealReduceMemLimit;
// check if there aren't enough maps scheduled, give the free map capacity
// to reduce
if (idealMapMemLimit > netScheduledMapMem) {
int unusedMapMemLimit = idealMapMemLimit - netScheduledMapMem;
finalReduceMemLimit = idealReduceMemLimit + unusedMapMemLimit;
finalMapMemLimit = totalMemLimit - finalReduceMemLimit;
} else {
finalMapMemLimit = idealMapMemLimit;
finalReduceMemLimit = idealReduceMemLimit;
}
LOG.info("completedMapPercent " + completedMapPercent +
" totalMemLimit:" + totalMemLimit +
" finalMapMemLimit:" + finalMapMemLimit +
" finalReduceMemLimit:" + finalReduceMemLimit +
" netScheduledMapMem:" + netScheduledMapMem +
" netScheduledReduceMem:" + netScheduledReduceMem);
int rampUp = (finalReduceMemLimit - netScheduledReduceMem)
/ reduceResourceReqt;
if (rampUp > 0) {
rampUp = Math.min(rampUp, pendingReduces.size());
LOG.info("Ramping up " + rampUp);
//more reduce to be scheduled
for (int i = 0; i < rampUp; i++) {
ContainerRequest request = pendingReduces.removeFirst();
scheduledRequests.addReduce(request);
}
} else if (rampUp < 0){
int rampDown = -1 * rampUp;
rampDown = Math.min(rampDown, scheduledRequests.reduces.size());
LOG.info("Ramping down " + rampDown);
//remove from the scheduled and move back to pending
for (int i = 0; i < rampDown; i++) {
ContainerRequest request = scheduledRequests.removeReduce();
pendingReduces.add(request);
}
}
}
/**
* Synchronized to avoid findbugs warnings
*/
private synchronized String getStat() {
return "PendingReduces:" + pendingReduces.size() +
" ScheduledMaps:" + scheduledRequests.maps.size() +
" ScheduledReduces:" + scheduledRequests.reduces.size() +
" AssignedMaps:" + assignedRequests.maps.size() +
" AssignedReduces:" + assignedRequests.reduces.size() +
" completedMaps:" + completedMaps +
" completedReduces:" + completedReduces +
" containersAllocated:" + containersAllocated +
" containersReleased:" + containersReleased +
" hostLocalAssigned:" + hostLocalAssigned +
" rackLocalAssigned:" + rackLocalAssigned +
" availableResources(headroom):" + getAvailableResources();
}
private List<Container> getResources() throws Exception {
int headRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;//first time it would be null
AMResponse response = makeRemoteRequest();
int newHeadRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;
List<Container> newContainers = response.getNewContainerList();
List<Container> finishedContainers = response.getFinishedContainerList();
if (newContainers.size() + finishedContainers.size() > 0 || headRoom != newHeadRoom) {
//something changed
recalculateReduceSchedule = true;
}
List<Container> allocatedContainers = new ArrayList<Container>();
for (Container cont : newContainers) {
allocatedContainers.add(cont);
LOG.debug("Received new Container :" + cont);
}
for (Container cont : finishedContainers) {
LOG.info("Received completed container " + cont);
TaskAttemptId attemptID = assignedRequests.get(cont.getId());
if (attemptID == null) {
LOG.error("Container complete event for unknown container id "
+ cont.getId());
} else {
assignedRequests.remove(attemptID);
if (attemptID.getTaskId().getTaskType().equals(TaskType.MAP)) {
completedMaps++;
} else {
completedReduces++;
}
// send the container completed event to Task attempt
eventHandler.handle(new TaskAttemptEvent(attemptID,
TaskAttemptEventType.TA_CONTAINER_COMPLETED));
// Send the diagnostics
String diagnostics = cont.getContainerStatus().getDiagnostics();
eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptID,
diagnostics));
}
}
return newContainers;
}
private int getMemLimit() {
int headRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;
return headRoom + assignedRequests.maps.size() * mapResourceReqt +
assignedRequests.reduces.size() * reduceResourceReqt;
}
private class ScheduledRequests {
private final LinkedList<TaskAttemptId> earlierFailedMaps =
new LinkedList<TaskAttemptId>();
/** Maps from a host to a list of Map tasks with data on the host */
private final Map<String, LinkedList<TaskAttemptId>> mapsHostMapping =
new HashMap<String, LinkedList<TaskAttemptId>>();
private final Map<String, LinkedList<TaskAttemptId>> mapsRackMapping =
new HashMap<String, LinkedList<TaskAttemptId>>();
private final Map<TaskAttemptId, ContainerRequest> maps =
new LinkedHashMap<TaskAttemptId, ContainerRequest>();
private final LinkedHashMap<TaskAttemptId, ContainerRequest> reduces =
new LinkedHashMap<TaskAttemptId, ContainerRequest>();
boolean remove(TaskAttemptId tId) {
ContainerRequest req = null;
if (tId.getTaskId().getTaskType().equals(TaskType.MAP)) {
req = maps.remove(tId);
} else {
req = reduces.remove(tId);
}
if (req == null) {
return false;
} else {
decContainerReq(req);
return true;
}
}
ContainerRequest removeReduce() {
Iterator<Entry<TaskAttemptId, ContainerRequest>> it = reduces.entrySet().iterator();
if (it.hasNext()) {
Entry<TaskAttemptId, ContainerRequest> entry = it.next();
it.remove();
decContainerReq(entry.getValue());
return entry.getValue();
}
return null;
}
void addMap(ContainerRequestEvent event) {
ContainerRequest request = null;
if (event.getEarlierAttemptFailed()) {
earlierFailedMaps.add(event.getAttemptID());
request = new ContainerRequest(event, PRIORITY_FAST_FAIL_MAP);
} else {
for (String host : event.getHosts()) {
//host comes from data splitLocations which are hostnames. Containers
// use IP addresses.
//TODO Temporary fix for locality. Use resolvers from h-common.
// Cache to make this more efficient ?
InetAddress addr = null;
try {
addr = InetAddress.getByName(host);
} catch (UnknownHostException e) {
LOG.warn("Unable to resolve host to IP for host [: " + host + "]");
}
if (addr != null) //Fallback to host if resolve fails.
host = addr.getHostAddress();
LinkedList<TaskAttemptId> list = mapsHostMapping.get(host);
if (list == null) {
list = new LinkedList<TaskAttemptId>();
mapsHostMapping.put(host, list);
}
list.add(event.getAttemptID());
LOG.info("Added attempt req to host " + host);
}
for (String rack: event.getRacks()) {
LinkedList<TaskAttemptId> list = mapsRackMapping.get(rack);
if (list == null) {
list = new LinkedList<TaskAttemptId>();
mapsRackMapping.put(rack, list);
}
list.add(event.getAttemptID());
LOG.info("Added attempt req to rack " + rack);
}
request = new ContainerRequest(event, PRIORITY_MAP);
}
maps.put(event.getAttemptID(), request);
addContainerReq(request);
}
void addReduce(ContainerRequest req) {
reduces.put(req.attemptID, req);
addContainerReq(req);
}
private void assign(List<Container> allocatedContainers) {
Iterator<Container> it = allocatedContainers.iterator();
LOG.info("Got allocated containers " + allocatedContainers.size());
containersAllocated += allocatedContainers.size();
while (it.hasNext()) {
Container allocated = it.next();
LOG.info("Assigning container " + allocated);
ContainerRequest assigned = assign(allocated);
if (assigned != null) {
// Update resource requests
decContainerReq(assigned);
// send the container-assigned event to task attempt
eventHandler.handle(new TaskAttemptContainerAssignedEvent(
assigned.attemptID, allocated));
assignedRequests.add(allocated.getId(), assigned.attemptID);
LOG.info("Assigned container (" + allocated + ") " +
" to task " + assigned.attemptID +
" on node " + allocated.getNodeId().toString());
} else {
//not assigned to any request, release the container
LOG.info("Releasing unassigned and invalid container " + allocated
+ ". RM has gone crazy, someone go look!"
+ " Hey RM, if you are so rich, go donate to non-profits!");
containersReleased++;
release(allocated.getId());
}
}
}
private ContainerRequest assign(Container allocated) {
ContainerRequest assigned = null;
if (mapResourceReqt != reduceResourceReqt) {
//assign based on size
LOG.info("Assigning based on container size");
if (allocated.getResource().getMemory() == mapResourceReqt) {
assigned = assignToFailedMap(allocated);
if (assigned == null) {
assigned = assignToMap(allocated);
}
} else if (allocated.getResource().getMemory() == reduceResourceReqt) {
assigned = assignToReduce(allocated);
}
return assigned;
}
//container can be given to either map or reduce
//assign based on priority
//try to assign to earlierFailedMaps if present
assigned = assignToFailedMap(allocated);
//Assign to reduces before assigning to maps ?
if (assigned == null) {
assigned = assignToReduce(allocated);
}
//try to assign to maps if present
if (assigned == null) {
assigned = assignToMap(allocated);
}
return assigned;
}
private ContainerRequest assignToFailedMap(Container allocated) {
//try to assign to earlierFailedMaps if present
ContainerRequest assigned = null;
while (assigned == null && earlierFailedMaps.size() > 0 &&
allocated.getResource().getMemory() >= mapResourceReqt) {
TaskAttemptId tId = earlierFailedMaps.removeFirst();
if (maps.containsKey(tId)) {
assigned = maps.remove(tId);
JobCounterUpdateEvent jce =
new JobCounterUpdateEvent(assigned.attemptID.getTaskId().getJobId());
jce.addCounterUpdate(JobCounter.OTHER_LOCAL_MAPS, 1);
eventHandler.handle(jce);
LOG.info("Assigned from earlierFailedMaps");
break;
}
}
return assigned;
}
private ContainerRequest assignToReduce(Container allocated) {
ContainerRequest assigned = null;
//try to assign to reduces if present
if (assigned == null && reduces.size() > 0
&& allocated.getResource().getMemory() >= reduceResourceReqt) {
TaskAttemptId tId = reduces.keySet().iterator().next();
assigned = reduces.remove(tId);
LOG.info("Assigned to reduce");
}
return assigned;
}
private ContainerRequest assignToMap(Container allocated) {
//try to assign to maps if present
//first by host, then by rack, followed by *
ContainerRequest assigned = null;
while (assigned == null && maps.size() > 0
&& allocated.getResource().getMemory() >= mapResourceReqt) {
String host = getHost(allocated.getNodeId().toString());
LinkedList<TaskAttemptId> list = mapsHostMapping.get(host);
while (list != null && list.size() > 0) {
LOG.info("Host matched to the request list " + host);
TaskAttemptId tId = list.removeFirst();
if (maps.containsKey(tId)) {
assigned = maps.remove(tId);
JobCounterUpdateEvent jce =
new JobCounterUpdateEvent(assigned.attemptID.getTaskId().getJobId());
jce.addCounterUpdate(JobCounter.DATA_LOCAL_MAPS, 1);
eventHandler.handle(jce);
hostLocalAssigned++;
LOG.info("Assigned based on host match " + host);
break;
}
}
if (assigned == null) {
String rack = RackResolver.resolve(host).getNetworkLocation();
list = mapsRackMapping.get(rack);
while (list != null && list.size() > 0) {
TaskAttemptId tId = list.removeFirst();
if (maps.containsKey(tId)) {
assigned = maps.remove(tId);
JobCounterUpdateEvent jce =
new JobCounterUpdateEvent(assigned.attemptID.getTaskId().getJobId());
jce.addCounterUpdate(JobCounter.RACK_LOCAL_MAPS, 1);
eventHandler.handle(jce);
rackLocalAssigned++;
LOG.info("Assigned based on rack match " + rack);
break;
}
}
if (assigned == null && maps.size() > 0) {
TaskAttemptId tId = maps.keySet().iterator().next();
assigned = maps.remove(tId);
JobCounterUpdateEvent jce =
new JobCounterUpdateEvent(assigned.attemptID.getTaskId().getJobId());
jce.addCounterUpdate(JobCounter.OTHER_LOCAL_MAPS, 1);
eventHandler.handle(jce);
LOG.info("Assigned based on * match");
break;
}
}
}
return assigned;
}
}
private class AssignedRequests {
private final Map<ContainerId, TaskAttemptId> containerToAttemptMap =
new HashMap<ContainerId, TaskAttemptId>();
private final LinkedHashMap<TaskAttemptId, ContainerId> maps =
new LinkedHashMap<TaskAttemptId, ContainerId>();
private final LinkedHashMap<TaskAttemptId, ContainerId> reduces =
new LinkedHashMap<TaskAttemptId, ContainerId>();
private final Set<TaskAttemptId> preemptionWaitingReduces =
new HashSet<TaskAttemptId>();
void add(ContainerId containerId, TaskAttemptId tId) {
LOG.info("Assigned container " + containerId.toString()
+ " to " + tId);
containerToAttemptMap.put(containerId, tId);
if (tId.getTaskId().getTaskType().equals(TaskType.MAP)) {
maps.put(tId, containerId);
} else {
reduces.put(tId, containerId);
}
}
void preemptReduce(int toPreempt) {
List<TaskAttemptId> reduceList = new ArrayList(reduces.keySet());
//sort reduces on progress
Collections.sort(reduceList,
new Comparator<TaskAttemptId>() {
@Override
public int compare(TaskAttemptId o1, TaskAttemptId o2) {
float p = getJob().getTask(o1.getTaskId()).getAttempt(o1).getProgress() -
getJob().getTask(o2.getTaskId()).getAttempt(o2).getProgress();
return p >= 0 ? 1 : -1;
}
});
for (int i = 0; i < toPreempt && reduceList.size() > 0; i++) {
TaskAttemptId id = reduceList.remove(0);//remove the one on top
LOG.info("Preempting " + id);
preemptionWaitingReduces.add(id);
eventHandler.handle(new TaskAttemptEvent(id, TaskAttemptEventType.TA_KILL));
}
}
boolean remove(TaskAttemptId tId) {
ContainerId containerId = null;
if (tId.getTaskId().getTaskType().equals(TaskType.MAP)) {
containerId = maps.remove(tId);
} else {
containerId = reduces.remove(tId);
if (containerId != null) {
boolean preempted = preemptionWaitingReduces.remove(tId);
if (preempted) {
LOG.info("Reduce preemption successful " + tId);
}
}
}
if (containerId != null) {
containerToAttemptMap.remove(containerId);
return true;
}
return false;
}
TaskAttemptId get(ContainerId cId) {
return containerToAttemptMap.get(cId);
}
ContainerId get(TaskAttemptId tId) {
if (tId.getTaskId().getTaskType().equals(TaskType.MAP)) {
return maps.get(tId);
} else {
return reduces.get(tId);
}
}
}
}

View File

@ -0,0 +1,274 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.rm;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.TreeSet;
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.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.records.AMResponse;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
/**
* Keeps the data structures to send container requests to RM.
*/
public abstract class RMContainerRequestor extends RMCommunicator {
private static final Log LOG = LogFactory.getLog(RMContainerRequestor.class);
static final String ANY = "*";
private int lastResponseID;
private Resource availableResources;
private final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
//Key -> Priority
//Value -> Map
//Key->ResourceName (e.g., hostname, rackname, *)
//Value->Map
//Key->Resource Capability
//Value->ResourceReqeust
private final Map<Priority, Map<String, Map<Resource, ResourceRequest>>>
remoteRequestsTable =
new TreeMap<Priority, Map<String, Map<Resource, ResourceRequest>>>();
private final Set<ResourceRequest> ask = new TreeSet<ResourceRequest>();
private final Set<ContainerId> release = new TreeSet<ContainerId>();
private boolean nodeBlacklistingEnabled;
private int maxTaskFailuresPerNode;
private final Map<String, Integer> nodeFailures = new HashMap<String, Integer>();
private final Set<String> blacklistedNodes = new HashSet<String>();
public RMContainerRequestor(ClientService clientService, AppContext context) {
super(clientService, context);
}
static class ContainerRequest {
final TaskAttemptId attemptID;
final Resource capability;
final String[] hosts;
final String[] racks;
//final boolean earlierAttemptFailed;
final Priority priority;
public ContainerRequest(ContainerRequestEvent event, Priority priority) {
this.attemptID = event.getAttemptID();
this.capability = event.getCapability();
this.hosts = event.getHosts();
this.racks = event.getRacks();
//this.earlierAttemptFailed = event.getEarlierAttemptFailed();
this.priority = priority;
}
}
@Override
public void init(Configuration conf) {
super.init(conf);
nodeBlacklistingEnabled =
conf.getBoolean(AMConstants.NODE_BLACKLISTING_ENABLE, true);
LOG.info("nodeBlacklistingEnabled:" + nodeBlacklistingEnabled);
maxTaskFailuresPerNode =
conf.getInt(MRJobConfig.MAX_TASK_FAILURES_PER_TRACKER, 3);
LOG.info("maxTaskFailuresPerNode is " + maxTaskFailuresPerNode);
}
protected abstract void heartbeat() throws Exception;
protected AMResponse makeRemoteRequest() throws YarnRemoteException {
AllocateRequest allocateRequest = recordFactory
.newRecordInstance(AllocateRequest.class);
allocateRequest.setApplicationAttemptId(applicationAttemptId);
allocateRequest.setResponseId(lastResponseID);
allocateRequest.addAllAsks(new ArrayList<ResourceRequest>(ask));
allocateRequest.addAllReleases(new ArrayList<ContainerId>(release));
AllocateResponse allocateResponse = scheduler.allocate(allocateRequest);
AMResponse response = allocateResponse.getAMResponse();
lastResponseID = response.getResponseId();
availableResources = response.getAvailableResources();
LOG.info("getResources() for " + applicationId + ":" + " ask="
+ ask.size() + " release= " + release.size() + " newContainers="
+ response.getNewContainerCount() + " finishedContainers="
+ response.getFinishedContainerCount()
+ " resourcelimit=" + availableResources);
ask.clear();
release.clear();
return response;
}
protected void containerFailedOnHost(String hostName) {
if (!nodeBlacklistingEnabled) {
return;
}
if (blacklistedNodes.contains(hostName)) {
LOG.info("Host " + hostName + " is already blacklisted.");
return; //already blacklisted
}
Integer failures = nodeFailures.remove(hostName);
failures = failures == null ? 0 : failures;
failures++;
LOG.info(failures + " failures on node " + hostName);
if (failures >= maxTaskFailuresPerNode) {
blacklistedNodes.add(hostName);
LOG.info("Blacklisted host " + hostName);
//remove all the requests corresponding to this hostname
for (Map<String, Map<Resource, ResourceRequest>> remoteRequests
: remoteRequestsTable.values()){
//remove from host
Map<Resource, ResourceRequest> reqMap = remoteRequests.remove(hostName);
if (reqMap != null) {
for (ResourceRequest req : reqMap.values()) {
ask.remove(req);
}
}
//TODO: remove from rack
}
} else {
nodeFailures.put(hostName, failures);
}
}
protected Resource getAvailableResources() {
return availableResources;
}
protected void addContainerReq(ContainerRequest req) {
// Create resource requests
for (String host : req.hosts) {
// Data-local
addResourceRequest(req.priority, host, req.capability);
}
// Nothing Rack-local for now
for (String rack : req.racks) {
addResourceRequest(req.priority, rack, req.capability);
}
// Off-switch
addResourceRequest(req.priority, ANY, req.capability);
}
protected void decContainerReq(ContainerRequest req) {
// Update resource requests
for (String hostName : req.hosts) {
decResourceRequest(req.priority, hostName, req.capability);
}
for (String rack : req.racks) {
decResourceRequest(req.priority, rack, req.capability);
}
decResourceRequest(req.priority, ANY, req.capability);
}
private void addResourceRequest(Priority priority, String resourceName,
Resource capability) {
Map<String, Map<Resource, ResourceRequest>> remoteRequests =
this.remoteRequestsTable.get(priority);
if (remoteRequests == null) {
remoteRequests = new HashMap<String, Map<Resource, ResourceRequest>>();
this.remoteRequestsTable.put(priority, remoteRequests);
LOG.info("Added priority=" + priority);
}
Map<Resource, ResourceRequest> reqMap = remoteRequests.get(resourceName);
if (reqMap == null) {
reqMap = new HashMap<Resource, ResourceRequest>();
remoteRequests.put(resourceName, reqMap);
}
ResourceRequest remoteRequest = reqMap.get(capability);
if (remoteRequest == null) {
remoteRequest = recordFactory.newRecordInstance(ResourceRequest.class);
remoteRequest.setPriority(priority);
remoteRequest.setHostName(resourceName);
remoteRequest.setCapability(capability);
remoteRequest.setNumContainers(0);
reqMap.put(capability, remoteRequest);
}
remoteRequest.setNumContainers(remoteRequest.getNumContainers() + 1);
// Note this down for next interaction with ResourceManager
ask.add(remoteRequest);
LOG.info("addResourceRequest:" + " applicationId=" + applicationId.getId()
+ " priority=" + priority.getPriority() + " resourceName=" + resourceName
+ " numContainers=" + remoteRequest.getNumContainers() + " #asks="
+ ask.size());
}
private void decResourceRequest(Priority priority, String resourceName,
Resource capability) {
Map<String, Map<Resource, ResourceRequest>> remoteRequests =
this.remoteRequestsTable.get(priority);
Map<Resource, ResourceRequest> reqMap = remoteRequests.get(resourceName);
ResourceRequest remoteRequest = reqMap.get(capability);
LOG.info("BEFORE decResourceRequest:" + " applicationId=" + applicationId.getId()
+ " priority=" + priority.getPriority() + " resourceName=" + resourceName
+ " numContainers=" + remoteRequest.getNumContainers() + " #asks="
+ ask.size());
remoteRequest.setNumContainers(remoteRequest.getNumContainers() -1);
if (remoteRequest.getNumContainers() == 0) {
reqMap.remove(capability);
if (reqMap.size() == 0) {
remoteRequests.remove(resourceName);
}
if (remoteRequests.size() == 0) {
remoteRequestsTable.remove(priority);
}
//remove from ask if it may have
ask.remove(remoteRequest);
} else {
ask.add(remoteRequest);//this will override the request if ask doesn't
//already have it.
}
LOG.info("AFTER decResourceRequest:" + " applicationId="
+ applicationId.getId() + " priority=" + priority.getPriority()
+ " resourceName=" + resourceName + " numContainers="
+ remoteRequest.getNumContainers() + " #asks=" + ask.size());
}
protected void release(ContainerId containerId) {
release.add(containerId);
}
}

View File

@ -0,0 +1,78 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.speculate;
public class DataStatistics {
private int count = 0;
private double sum = 0;
private double sumSquares = 0;
public DataStatistics() {
}
public DataStatistics(double initNum) {
this.count = 1;
this.sum = initNum;
this.sumSquares = initNum * initNum;
}
public synchronized void add(double newNum) {
this.count++;
this.sum += newNum;
this.sumSquares += newNum * newNum;
}
public synchronized void updateStatistics(double old, double update) {
this.sum += update - old;
this.sumSquares += (update * update) - (old * old);
}
public synchronized double mean() {
return count == 0 ? 0.0 : sum/count;
}
public synchronized double var() {
// E(X^2) - E(X)^2
if (count <= 1) {
return 0.0;
}
double mean = mean();
return Math.max((sumSquares/count) - mean * mean, 0.0d);
}
public synchronized double std() {
return Math.sqrt(this.var());
}
public synchronized double outlier(float sigma) {
if (count != 0.0) {
return mean() + std() * sigma;
}
return 0.0;
}
public synchronized double count() {
return count;
}
public String toString() {
return "DataStatistics: count is " + count + ", sum is " + sum +
", sumSquares is " + sumSquares + " mean is " + mean() + " std() is " + std();
}
}

View File

@ -0,0 +1,512 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.speculate;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
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.v2.MRConstants;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
import org.apache.hadoop.yarn.Clock;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.service.AbstractService;
public class DefaultSpeculator extends AbstractService implements
Speculator {
private static final long ON_SCHEDULE = Long.MIN_VALUE;
private static final long ALREADY_SPECULATING = Long.MIN_VALUE + 1;
private static final long TOO_NEW = Long.MIN_VALUE + 2;
private static final long PROGRESS_IS_GOOD = Long.MIN_VALUE + 3;
private static final long NOT_RUNNING = Long.MIN_VALUE + 4;
private static final long TOO_LATE_TO_SPECULATE = Long.MIN_VALUE + 5;
private static final long SOONEST_RETRY_AFTER_NO_SPECULATE = 1000L * 1L;
private static final long SOONEST_RETRY_AFTER_SPECULATE = 1000L * 15L;
private static final double PROPORTION_RUNNING_TASKS_SPECULATABLE = 0.1;
private static final double PROPORTION_TOTAL_TASKS_SPECULATABLE = 0.01;
private static final int MINIMUM_ALLOWED_SPECULATIVE_TASKS = 10;
private static final Log LOG = LogFactory.getLog(DefaultSpeculator.class);
private final ConcurrentMap<TaskId, Boolean> runningTasks
= new ConcurrentHashMap<TaskId, Boolean>();
private final Map<Task, AtomicBoolean> pendingSpeculations
= new ConcurrentHashMap<Task, AtomicBoolean>();
// These are the current needs, not the initial needs. For each job, these
// record the number of attempts that exist and that are actively
// waiting for a container [as opposed to running or finished]
private final ConcurrentMap<JobId, AtomicInteger> mapContainerNeeds
= new ConcurrentHashMap<JobId, AtomicInteger>();
private final ConcurrentMap<JobId, AtomicInteger> reduceContainerNeeds
= new ConcurrentHashMap<JobId, AtomicInteger>();
private final Set<TaskId> mayHaveSpeculated = new HashSet();
private final Configuration conf;
private AppContext context;
private Thread speculationBackgroundThread = null;
private BlockingQueue<SpeculatorEvent> eventQueue
= new LinkedBlockingQueue<SpeculatorEvent>();
private TaskRuntimeEstimator estimator;
private BlockingQueue<Object> scanControl = new LinkedBlockingQueue<Object>();
private final Clock clock;
private final EventHandler<TaskEvent> eventHandler;
public DefaultSpeculator(Configuration conf, AppContext context) {
this(conf, context, context.getClock());
}
public DefaultSpeculator(Configuration conf, AppContext context, Clock clock) {
this(conf, context, getEstimator(conf, context), clock);
}
static private TaskRuntimeEstimator getEstimator
(Configuration conf, AppContext context) {
TaskRuntimeEstimator estimator;
try {
// "yarn.mapreduce.job.task.runtime.estimator.class"
Class<? extends TaskRuntimeEstimator> estimatorClass
= conf.getClass(AMConstants.TASK_RUNTIME_ESTIMATOR_CLASS,
LegacyTaskRuntimeEstimator.class,
TaskRuntimeEstimator.class);
Constructor<? extends TaskRuntimeEstimator> estimatorConstructor
= estimatorClass.getConstructor();
estimator = estimatorConstructor.newInstance();
estimator.contextualize(conf, context);
} catch (InstantiationException ex) {
LOG.error("Can't make a speculation runtime extimator" + ex);
throw new YarnException(ex);
} catch (IllegalAccessException ex) {
LOG.error("Can't make a speculation runtime extimator" + ex);
throw new YarnException(ex);
} catch (InvocationTargetException ex) {
LOG.error("Can't make a speculation runtime extimator" + ex);
throw new YarnException(ex);
} catch (NoSuchMethodException ex) {
LOG.error("Can't make a speculation runtime extimator" + ex);
throw new YarnException(ex);
}
return estimator;
}
// This constructor is designed to be called by other constructors.
// However, it's public because we do use it in the test cases.
// Normally we figure out our own estimator.
public DefaultSpeculator
(Configuration conf, AppContext context,
TaskRuntimeEstimator estimator, Clock clock) {
super(DefaultSpeculator.class.getName());
this.conf = conf;
this.context = context;
this.estimator = estimator;
this.clock = clock;
this.eventHandler = context.getEventHandler();
}
/* ************************************************************* */
// This is the task-mongering that creates the two new threads -- one for
// processing events from the event queue and one for periodically
// looking for speculation opportunities
@Override
public void start() {
Runnable speculationBackgroundCore
= new Runnable() {
@Override
public void run() {
while (!Thread.currentThread().isInterrupted()) {
long backgroundRunStartTime = clock.getTime();
try {
int speculations = computeSpeculations();
long mininumRecomp
= speculations > 0 ? SOONEST_RETRY_AFTER_SPECULATE
: SOONEST_RETRY_AFTER_NO_SPECULATE;
long wait = Math.max(mininumRecomp,
clock.getTime() - backgroundRunStartTime);
if (speculations > 0) {
LOG.info("We launched " + speculations
+ " speculations. Sleeping " + wait + " milliseconds.");
}
Object pollResult
= scanControl.poll(wait, TimeUnit.MILLISECONDS);
} catch (InterruptedException e) {
LOG.error("Background thread returning, interrupted : " + e);
e.printStackTrace(System.out);
return;
}
}
}
};
speculationBackgroundThread = new Thread
(speculationBackgroundCore, "DefaultSpeculator background processing");
speculationBackgroundThread.start();
super.start();
}
@Override
public void stop() {
// this could be called before background thread is established
if (speculationBackgroundThread != null) {
speculationBackgroundThread.interrupt();
}
super.stop();
}
@Override
public void handleAttempt(TaskAttemptStatus status) {
long timestamp = clock.getTime();
statusUpdate(status, timestamp);
}
// This section is not part of the Speculator interface; it's used only for
// testing
public boolean eventQueueEmpty() {
return eventQueue.isEmpty();
}
// This interface is intended to be used only for test cases.
public void scanForSpeculations() {
LOG.info("We got asked to run a debug speculation scan.");
// debug
System.out.println("We got asked to run a debug speculation scan.");
System.out.println("There are " + scanControl.size()
+ " events stacked already.");
scanControl.add(new Object());
Thread.yield();
}
/* ************************************************************* */
// This section contains the code that gets run for a SpeculatorEvent
private AtomicInteger containerNeed(TaskId taskID) {
JobId jobID = taskID.getJobId();
TaskType taskType = taskID.getTaskType();
ConcurrentMap<JobId, AtomicInteger> relevantMap
= taskType == TaskType.MAP ? mapContainerNeeds : reduceContainerNeeds;
AtomicInteger result = relevantMap.get(jobID);
if (result == null) {
relevantMap.putIfAbsent(jobID, new AtomicInteger(0));
result = relevantMap.get(jobID);
}
return result;
}
private synchronized void processSpeculatorEvent(SpeculatorEvent event) {
switch (event.getType()) {
case ATTEMPT_STATUS_UPDATE:
statusUpdate(event.getReportedStatus(), event.getTimestamp());
break;
case TASK_CONTAINER_NEED_UPDATE:
{
AtomicInteger need = containerNeed(event.getTaskID());
need.addAndGet(event.containersNeededChange());
break;
}
case ATTEMPT_START:
{
LOG.info("ATTEMPT_START " + event.getTaskID());
estimator.enrollAttempt
(event.getReportedStatus(), event.getTimestamp());
break;
}
case JOB_CREATE:
{
LOG.info("JOB_CREATE " + event.getJobID());
estimator.contextualize(getConfig(), context);
break;
}
}
}
/**
* Absorbs one TaskAttemptStatus
*
* @param reportedStatus the status report that we got from a task attempt
* that we want to fold into the speculation data for this job
* @param timestamp the time this status corresponds to. This matters
* because statuses contain progress.
*/
protected void statusUpdate(TaskAttemptStatus reportedStatus, long timestamp) {
String stateString = reportedStatus.taskState.toString();
TaskAttemptId attemptID = reportedStatus.id;
TaskId taskID = attemptID.getTaskId();
Job job = context.getJob(taskID.getJobId());
if (job == null) {
return;
}
Task task = job.getTask(taskID);
if (task == null) {
return;
}
estimator.updateAttempt(reportedStatus, timestamp);
// If the task is already known to be speculation-bait, don't do anything
if (pendingSpeculations.get(task) != null) {
if (pendingSpeculations.get(task).get()) {
return;
}
}
if (stateString.equals(TaskAttemptState.RUNNING.name())) {
runningTasks.putIfAbsent(taskID, Boolean.TRUE);
} else {
runningTasks.remove(taskID, Boolean.TRUE);
}
}
/* ************************************************************* */
// This is the code section that runs periodically and adds speculations for
// those jobs that need them.
// This can return a few magic values for tasks that shouldn't speculate:
// returns ON_SCHEDULE if thresholdRuntime(taskID) says that we should not
// considering speculating this task
// returns ALREADY_SPECULATING if that is true. This has priority.
// returns TOO_NEW if our companion task hasn't gotten any information
// returns PROGRESS_IS_GOOD if the task is sailing through
// returns NOT_RUNNING if the task is not running
//
// All of these values are negative. Any value that should be allowed to
// speculate is 0 or positive.
private long speculationValue(TaskId taskID, long now) {
Job job = context.getJob(taskID.getJobId());
Task task = job.getTask(taskID);
Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
long acceptableRuntime = Long.MIN_VALUE;
long result = Long.MIN_VALUE;
if (!mayHaveSpeculated.contains(taskID)) {
acceptableRuntime = estimator.thresholdRuntime(taskID);
if (acceptableRuntime == Long.MAX_VALUE) {
return ON_SCHEDULE;
}
}
TaskAttemptId runningTaskAttemptID = null;
int numberRunningAttempts = 0;
for (TaskAttempt taskAttempt : attempts.values()) {
if (taskAttempt.getState() == TaskAttemptState.RUNNING
|| taskAttempt.getState() == TaskAttemptState.ASSIGNED) {
if (++numberRunningAttempts > 1) {
return ALREADY_SPECULATING;
}
runningTaskAttemptID = taskAttempt.getID();
long estimatedRunTime = estimator.estimatedRuntime(runningTaskAttemptID);
long taskAttemptStartTime
= estimator.attemptEnrolledTime(runningTaskAttemptID);
if (taskAttemptStartTime > now) {
// This background process ran before we could process the task
// attempt status change that chronicles the attempt start
return TOO_NEW;
}
long estimatedEndTime = estimatedRunTime + taskAttemptStartTime;
long estimatedReplacementEndTime
= now + estimator.estimatedNewAttemptRuntime(taskID);
if (estimatedEndTime < now) {
return PROGRESS_IS_GOOD;
}
if (estimatedReplacementEndTime >= estimatedEndTime) {
return TOO_LATE_TO_SPECULATE;
}
result = estimatedEndTime - estimatedReplacementEndTime;
}
}
// If we are here, there's at most one task attempt.
if (numberRunningAttempts == 0) {
return NOT_RUNNING;
}
if (acceptableRuntime == Long.MIN_VALUE) {
acceptableRuntime = estimator.thresholdRuntime(taskID);
if (acceptableRuntime == Long.MAX_VALUE) {
return ON_SCHEDULE;
}
}
return result;
}
//Add attempt to a given Task.
protected void addSpeculativeAttempt(TaskId taskID) {
LOG.info
("DefaultSpeculator.addSpeculativeAttempt -- we are speculating " + taskID);
eventHandler.handle(new TaskEvent(taskID, TaskEventType.T_ADD_SPEC_ATTEMPT));
mayHaveSpeculated.add(taskID);
}
@Override
public void handle(SpeculatorEvent event) {
processSpeculatorEvent(event);
}
private int maybeScheduleAMapSpeculation() {
return maybeScheduleASpeculation(TaskType.MAP);
}
private int maybeScheduleAReduceSpeculation() {
return maybeScheduleASpeculation(TaskType.REDUCE);
}
private int maybeScheduleASpeculation(TaskType type) {
int successes = 0;
long now = clock.getTime();
ConcurrentMap<JobId, AtomicInteger> containerNeeds
= type == TaskType.MAP ? mapContainerNeeds : reduceContainerNeeds;
for (ConcurrentMap.Entry<JobId, AtomicInteger> jobEntry : containerNeeds.entrySet()) {
// This race conditon is okay. If we skip a speculation attempt we
// should have tried because the event that lowers the number of
// containers needed to zero hasn't come through, it will next time.
// Also, if we miss the fact that the number of containers needed was
// zero but increased due to a failure it's not too bad to launch one
// container prematurely.
if (jobEntry.getValue().get() > 0) {
continue;
}
int numberSpeculationsAlready = 0;
int numberRunningTasks = 0;
// loop through the tasks of the kind
Job job = context.getJob(jobEntry.getKey());
Map<TaskId, Task> tasks = job.getTasks(type);
int numberAllowedSpeculativeTasks
= (int) Math.max(MINIMUM_ALLOWED_SPECULATIVE_TASKS,
PROPORTION_TOTAL_TASKS_SPECULATABLE * tasks.size());
TaskId bestTaskID = null;
long bestSpeculationValue = -1L;
// this loop is potentially pricey.
// TODO track the tasks that are potentially worth looking at
for (Map.Entry<TaskId, Task> taskEntry : tasks.entrySet()) {
long mySpeculationValue = speculationValue(taskEntry.getKey(), now);
if (mySpeculationValue == ALREADY_SPECULATING) {
++numberSpeculationsAlready;
}
if (mySpeculationValue != NOT_RUNNING) {
++numberRunningTasks;
}
if (mySpeculationValue > bestSpeculationValue) {
bestTaskID = taskEntry.getKey();
bestSpeculationValue = mySpeculationValue;
}
}
numberAllowedSpeculativeTasks
= (int) Math.max(numberAllowedSpeculativeTasks,
PROPORTION_RUNNING_TASKS_SPECULATABLE * numberRunningTasks);
// If we found a speculation target, fire it off
if (bestTaskID != null
&& numberAllowedSpeculativeTasks > numberSpeculationsAlready) {
addSpeculativeAttempt(bestTaskID);
++successes;
}
}
return successes;
}
private int computeSpeculations() {
// We'll try to issue one map and one reduce speculation per job per run
return maybeScheduleAMapSpeculation() + maybeScheduleAReduceSpeculation();
}
}

View File

@ -0,0 +1,195 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.speculate;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.app.AMConstants;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
/**
* This estimator exponentially smooths the rate of progress versus wallclock
* time. Conceivably we could write an estimator that smooths time per
* unit progress, and get different results.
*/
public class ExponentiallySmoothedTaskRuntimeEstimator extends StartEndTimesBase {
private final ConcurrentMap<TaskAttemptId, AtomicReference<EstimateVector>> estimates
= new ConcurrentHashMap<TaskAttemptId, AtomicReference<EstimateVector>>();
private SmoothedValue smoothedValue;
private long lambda;
public enum SmoothedValue {
RATE, TIME_PER_UNIT_PROGRESS
}
ExponentiallySmoothedTaskRuntimeEstimator
(long lambda, SmoothedValue smoothedValue) {
super();
this.smoothedValue = smoothedValue;
this.lambda = lambda;
}
public ExponentiallySmoothedTaskRuntimeEstimator() {
super();
}
// immutable
private class EstimateVector {
final double value;
final float basedOnProgress;
final long atTime;
EstimateVector(double value, float basedOnProgress, long atTime) {
this.value = value;
this.basedOnProgress = basedOnProgress;
this.atTime = atTime;
}
EstimateVector incorporate(float newProgress, long newAtTime) {
if (newAtTime <= atTime || newProgress < basedOnProgress) {
return this;
}
double oldWeighting
= value < 0.0
? 0.0 : Math.exp(((double) (newAtTime - atTime)) / lambda);
double newRead = (newProgress - basedOnProgress) / (newAtTime - atTime);
if (smoothedValue == SmoothedValue.TIME_PER_UNIT_PROGRESS) {
newRead = 1.0 / newRead;
}
return new EstimateVector
(value * oldWeighting + newRead * (1.0 - oldWeighting),
newProgress, newAtTime);
}
}
private void incorporateReading
(TaskAttemptId attemptID, float newProgress, long newTime) {
//TODO: Refactor this method, it seems more complicated than necessary.
AtomicReference<EstimateVector> vectorRef = estimates.get(attemptID);
if (vectorRef == null) {
estimates.putIfAbsent(attemptID, new AtomicReference<EstimateVector>(null));
incorporateReading(attemptID, newProgress, newTime);
return;
}
EstimateVector oldVector = vectorRef.get();
if (oldVector == null) {
if (vectorRef.compareAndSet(null,
new EstimateVector(-1.0, 0.0F, Long.MIN_VALUE))) {
return;
}
incorporateReading(attemptID, newProgress, newTime);
return;
}
while (!vectorRef.compareAndSet
(oldVector, oldVector.incorporate(newProgress, newTime))) {
oldVector = vectorRef.get();
}
}
private EstimateVector getEstimateVector(TaskAttemptId attemptID) {
AtomicReference<EstimateVector> vectorRef = estimates.get(attemptID);
if (vectorRef == null) {
return null;
}
return vectorRef.get();
}
private static final long DEFAULT_EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS
= 1000L * 60;
@Override
public void contextualize(Configuration conf, AppContext context) {
super.contextualize(conf, context);
lambda
= conf.getLong(AMConstants.EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS,
DEFAULT_EXPONENTIAL_SMOOTHING_LAMBDA_MILLISECONDS);
smoothedValue
= conf.getBoolean(AMConstants.EXPONENTIAL_SMOOTHING_SMOOTH_RATE, true)
? SmoothedValue.RATE : SmoothedValue.TIME_PER_UNIT_PROGRESS;
}
@Override
public long estimatedRuntime(TaskAttemptId id) {
Long startTime = startTimes.get(id);
if (startTime == null) {
return -1L;
}
EstimateVector vector = getEstimateVector(id);
if (vector == null) {
return -1L;
}
long sunkTime = vector.atTime - startTime;
double value = vector.value;
float progress = vector.basedOnProgress;
if (value == 0) {
return -1L;
}
double rate = smoothedValue == SmoothedValue.RATE ? value : 1.0 / value;
if (rate == 0.0) {
return -1L;
}
double remainingTime = (1.0 - progress) / rate;
return sunkTime + (long)remainingTime;
}
@Override
public long runtimeEstimateVariance(TaskAttemptId id) {
return -1L;
}
@Override
public void updateAttempt(TaskAttemptStatus status, long timestamp) {
super.updateAttempt(status, timestamp);
TaskAttemptId attemptID = status.id;
float progress = status.progress;
incorporateReading(attemptID, progress, timestamp);
}
}

View File

@ -0,0 +1,150 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.speculate;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
public class LegacyTaskRuntimeEstimator extends StartEndTimesBase {
private final Map<TaskAttempt, AtomicLong> attemptRuntimeEstimates
= new ConcurrentHashMap<TaskAttempt, AtomicLong>();
private final ConcurrentHashMap<TaskAttempt, AtomicLong> attemptRuntimeEstimateVariances
= new ConcurrentHashMap<TaskAttempt, AtomicLong>();
@Override
public void updateAttempt(TaskAttemptStatus status, long timestamp) {
super.updateAttempt(status, timestamp);
TaskAttemptId attemptID = status.id;
TaskId taskID = attemptID.getTaskId();
JobId jobID = taskID.getJobId();
Job job = context.getJob(jobID);
if (job == null) {
return;
}
Task task = job.getTask(taskID);
if (task == null) {
return;
}
TaskAttempt taskAttempt = task.getAttempt(attemptID);
if (taskAttempt == null) {
return;
}
Long boxedStart = startTimes.get(attemptID);
long start = boxedStart == null ? Long.MIN_VALUE : boxedStart;
// We need to do two things.
// 1: If this is a completion, we accumulate statistics in the superclass
// 2: If this is not a completion, we learn more about it.
// This is not a completion, but we're cooking.
//
if (taskAttempt.getState() == TaskAttemptState.RUNNING) {
// See if this task is already in the registry
AtomicLong estimateContainer = attemptRuntimeEstimates.get(taskAttempt);
AtomicLong estimateVarianceContainer
= attemptRuntimeEstimateVariances.get(taskAttempt);
if (estimateContainer == null) {
if (attemptRuntimeEstimates.get(taskAttempt) == null) {
attemptRuntimeEstimates.put(taskAttempt, new AtomicLong());
estimateContainer = attemptRuntimeEstimates.get(taskAttempt);
}
}
if (estimateVarianceContainer == null) {
attemptRuntimeEstimateVariances.putIfAbsent(taskAttempt, new AtomicLong());
estimateVarianceContainer = attemptRuntimeEstimateVariances.get(taskAttempt);
}
long estimate = -1;
long varianceEstimate = -1;
// This code assumes that we'll never consider starting a third
// speculative task attempt if two are already running for this task
if (start > 0 && timestamp > start) {
estimate = (long) ((timestamp - start) / Math.max(0.0001, status.progress));
varianceEstimate = (long) (estimate * status.progress / 10);
}
if (estimateContainer != null) {
estimateContainer.set(estimate);
}
if (estimateVarianceContainer != null) {
estimateVarianceContainer.set(varianceEstimate);
}
}
}
private long storedPerAttemptValue
(Map<TaskAttempt, AtomicLong> data, TaskAttemptId attemptID) {
TaskId taskID = attemptID.getTaskId();
JobId jobID = taskID.getJobId();
Job job = context.getJob(jobID);
Task task = job.getTask(taskID);
if (task == null) {
return -1L;
}
TaskAttempt taskAttempt = task.getAttempt(attemptID);
if (taskAttempt == null) {
return -1L;
}
AtomicLong estimate = data.get(taskAttempt);
return estimate == null ? -1L : estimate.get();
}
@Override
public long estimatedRuntime(TaskAttemptId attemptID) {
return storedPerAttemptValue(attemptRuntimeEstimates, attemptID);
}
@Override
public long runtimeEstimateVariance(TaskAttemptId attemptID) {
return storedPerAttemptValue(attemptRuntimeEstimateVariances, attemptID);
}
}

View File

@ -0,0 +1,72 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.speculate;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
/*
* This class is provided solely as an exemplae of the values that mean
* that nothing needs to be computed. It's not currently used.
*/
public class NullTaskRuntimesEngine implements TaskRuntimeEstimator {
@Override
public void enrollAttempt(TaskAttemptStatus status, long timestamp) {
// no code
}
@Override
public long attemptEnrolledTime(TaskAttemptId attemptID) {
return Long.MAX_VALUE;
}
@Override
public void updateAttempt(TaskAttemptStatus status, long timestamp) {
// no code
}
@Override
public void contextualize(Configuration conf, AppContext context) {
// no code
}
@Override
public long thresholdRuntime(TaskId id) {
return Long.MAX_VALUE;
}
@Override
public long estimatedRuntime(TaskAttemptId id) {
return -1L;
}
@Override
public long estimatedNewAttemptRuntime(TaskId id) {
return -1L;
}
@Override
public long runtimeEstimateVariance(TaskAttemptId id) {
return -1L;
}
}

View File

@ -0,0 +1,45 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.speculate;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
import org.apache.hadoop.yarn.event.EventHandler;
/**
* Speculator component. Task Attempts' status updates are sent to this
* component. Concrete implementation runs the speculative algorithm and
* sends the TaskEventType.T_ADD_ATTEMPT.
*
* An implementation also has to arrange for the jobs to be scanned from
* time to time, to launch the speculations.
*/
public interface Speculator
extends EventHandler<SpeculatorEvent> {
enum EventType {
ATTEMPT_STATUS_UPDATE,
ATTEMPT_START,
TASK_CONTAINER_NEED_UPDATE,
JOB_CREATE
}
// This will be implemented if we go to a model where the events are
// processed within the TaskAttempts' state transitions' code.
public void handleAttempt(TaskAttemptStatus status);
}

View File

@ -0,0 +1,86 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.speculate;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
import org.apache.hadoop.yarn.event.AbstractEvent;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
public class SpeculatorEvent extends AbstractEvent<Speculator.EventType> {
// valid for ATTEMPT_STATUS_UPDATE
private TaskAttemptStatus reportedStatus;
// valid for TASK_CONTAINER_NEED_UPDATE
private TaskId taskID;
private int containersNeededChange;
// valid for CREATE_JOB
private JobId jobID;
public SpeculatorEvent(JobId jobID, long timestamp) {
super(Speculator.EventType.JOB_CREATE, timestamp);
this.jobID = jobID;
}
public SpeculatorEvent(TaskAttemptStatus reportedStatus, long timestamp) {
super(Speculator.EventType.ATTEMPT_STATUS_UPDATE, timestamp);
this.reportedStatus = reportedStatus;
}
public SpeculatorEvent(TaskAttemptId attemptID, boolean flag, long timestamp) {
super(Speculator.EventType.ATTEMPT_START, timestamp);
this.reportedStatus = new TaskAttemptStatus();
this.reportedStatus.id = attemptID;
this.taskID = attemptID.getTaskId();
}
/*
* This c'tor creates a TASK_CONTAINER_NEED_UPDATE event .
* We send a +1 event when a task enters a state where it wants a container,
* and a -1 event when it either gets one or withdraws the request.
* The per job sum of all these events is the number of containers requested
* but not granted. The intent is that we only do speculations when the
* speculation wouldn't compete for containers with tasks which need
* to be run.
*/
public SpeculatorEvent(TaskId taskID, int containersNeededChange) {
super(Speculator.EventType.TASK_CONTAINER_NEED_UPDATE);
this.taskID = taskID;
this.containersNeededChange = containersNeededChange;
}
public TaskAttemptStatus getReportedStatus() {
return reportedStatus;
}
public int containersNeededChange() {
return containersNeededChange;
}
public TaskId getTaskID() {
return taskID;
}
public JobId getJobID() {
return jobID;
}
}

View File

@ -0,0 +1,213 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.speculate;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
abstract class StartEndTimesBase implements TaskRuntimeEstimator {
static final float MINIMUM_COMPLETE_PROPORTION_TO_SPECULATE
= 0.05F;
static final int MINIMUM_COMPLETE_NUMBER_TO_SPECULATE
= 1;
protected Configuration conf = null;
protected AppContext context = null;
protected final Map<TaskAttemptId, Long> startTimes
= new ConcurrentHashMap<TaskAttemptId, Long>();
// XXXX This class design assumes that the contents of AppContext.getAllJobs
// never changes. Is that right?
//
// This assumption comes in in several places, mostly in data structure that
// can grow without limit if a AppContext gets new Job's when the old ones
// run out. Also, these mapper statistics blocks won't cover the Job's
// we don't know about.
protected final Map<Job, DataStatistics> mapperStatistics
= new HashMap<Job, DataStatistics>();
protected final Map<Job, DataStatistics> reducerStatistics
= new HashMap<Job, DataStatistics>();
private final Map<Job, Float> slowTaskRelativeTresholds
= new HashMap<Job, Float>();
protected final Set<Task> doneTasks = new HashSet<Task>();
@Override
public void enrollAttempt(TaskAttemptStatus status, long timestamp) {
startTimes.put(status.id,timestamp);
}
@Override
public long attemptEnrolledTime(TaskAttemptId attemptID) {
Long result = startTimes.get(attemptID);
return result == null ? Long.MAX_VALUE : result;
}
@Override
public void contextualize(Configuration conf, AppContext context) {
this.conf = conf;
this.context = context;
Map<JobId, Job> allJobs = context.getAllJobs();
for (Map.Entry<JobId, Job> entry : allJobs.entrySet()) {
final Job job = entry.getValue();
mapperStatistics.put(job, new DataStatistics());
reducerStatistics.put(job, new DataStatistics());
slowTaskRelativeTresholds.put
(job, conf.getFloat(MRJobConfig.SPECULATIVE_SLOWTASK_THRESHOLD,1.0f));
}
}
protected DataStatistics dataStatisticsForTask(TaskId taskID) {
JobId jobID = taskID.getJobId();
Job job = context.getJob(jobID);
if (job == null) {
return null;
}
Task task = job.getTask(taskID);
if (task == null) {
return null;
}
return task.getType() == TaskType.MAP
? mapperStatistics.get(job)
: task.getType() == TaskType.REDUCE
? reducerStatistics.get(job)
: null;
}
@Override
public long thresholdRuntime(TaskId taskID) {
JobId jobID = taskID.getJobId();
Job job = context.getJob(jobID);
TaskType type = taskID.getTaskType();
DataStatistics statistics
= dataStatisticsForTask(taskID);
int completedTasksOfType
= type == TaskType.MAP
? job.getCompletedMaps() : job.getCompletedReduces();
int totalTasksOfType
= type == TaskType.MAP
? job.getTotalMaps() : job.getTotalReduces();
if (completedTasksOfType < MINIMUM_COMPLETE_NUMBER_TO_SPECULATE
|| (((float)completedTasksOfType) / totalTasksOfType)
< MINIMUM_COMPLETE_PROPORTION_TO_SPECULATE ) {
return Long.MAX_VALUE;
}
long result = statistics == null
? Long.MAX_VALUE
: (long)statistics.outlier(slowTaskRelativeTresholds.get(job));
return result;
}
@Override
public long estimatedNewAttemptRuntime(TaskId id) {
DataStatistics statistics = dataStatisticsForTask(id);
if (statistics == null) {
return -1L;
}
return (long)statistics.mean();
}
@Override
public void updateAttempt(TaskAttemptStatus status, long timestamp) {
TaskAttemptId attemptID = status.id;
TaskId taskID = attemptID.getTaskId();
JobId jobID = taskID.getJobId();
Job job = context.getJob(jobID);
if (job == null) {
return;
}
Task task = job.getTask(taskID);
if (task == null) {
return;
}
Long boxedStart = startTimes.get(attemptID);
long start = boxedStart == null ? Long.MIN_VALUE : boxedStart;
TaskAttempt taskAttempt = task.getAttempt(attemptID);
if (taskAttempt.getState() == TaskAttemptState.SUCCEEDED) {
boolean isNew = false;
// is this a new success?
synchronized (doneTasks) {
if (!doneTasks.contains(task)) {
doneTasks.add(task);
isNew = true;
}
}
// It's a new completion
// Note that if a task completes twice [because of a previous speculation
// and a race, or a success followed by loss of the machine with the
// local data] we only count the first one.
if (isNew) {
long finish = timestamp;
if (start > 1L && finish > 1L && start <= finish) {
long duration = finish - start;
DataStatistics statistics
= dataStatisticsForTask(taskID);
if (statistics != null) {
statistics.add(duration);
}
}
}
}
}
}

View File

@ -0,0 +1,90 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.speculate;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus;
public interface TaskRuntimeEstimator {
public void enrollAttempt(TaskAttemptStatus reportedStatus, long timestamp);
public long attemptEnrolledTime(TaskAttemptId attemptID);
public void updateAttempt(TaskAttemptStatus reportedStatus, long timestamp);
public void contextualize(Configuration conf, AppContext context);
/**
*
* Find a maximum reasonable execution wallclock time. Includes the time
* already elapsed.
*
* Find a maximum reasonable execution time. Includes the time
* already elapsed. If the projected total execution time for this task
* ever exceeds its reasonable execution time, we may speculate it.
*
* @param id the {@link TaskID} of the task we are asking about
* @return the task's maximum reasonable runtime, or MAX_VALUE if
* we don't have enough information to rule out any runtime,
* however long.
*
*/
public long thresholdRuntime(TaskId id);
/**
*
* Estimate a task attempt's total runtime. Includes the time already
* elapsed.
*
* @param id the {@link TaskAttemptID} of the attempt we are asking about
* @return our best estimate of the attempt's runtime, or {@code -1} if
* we don't have enough information yet to produce an estimate.
*
*/
public long estimatedRuntime(TaskAttemptId id);
/**
*
* Estimates how long a new attempt on this task will take if we start
* one now
*
* @param id the {@link TaskID} of the task we are asking about
* @return our best estimate of a new attempt's runtime, or {@code -1} if
* we don't have enough information yet to produce an estimate.
*
*/
public long estimatedNewAttemptRuntime(TaskId id);
/**
*
* Computes the width of the error band of our estimate of the task
* runtime as returned by {@link estimatedRuntime}
*
* @param id the {@link TaskAttemptID} of the attempt we are asking about
* @return our best estimate of the attempt's runtime, or {@code -1} if
* we don't have enough information yet to produce an estimate.
*
*/
public long runtimeEstimateVariance(TaskAttemptId id);
}

View File

@ -0,0 +1,39 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.speculate;
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.mapreduce.v2.app.job.Task;
public class TaskSpeculationPredicate {
boolean canSpeculate(AppContext context, TaskId taskID) {
// This class rejects speculating any task that already has speculations,
// or isn't running.
// Subclasses should call TaskSpeculationPredicate.canSpeculate(...) , but
// can be even more restrictive.
JobId jobID = taskID.getJobId();
Job job = context.getJob(jobID);
Task task = job.getTask(taskID);
return task.getAttempts().size() == 1;
}
}

View File

@ -0,0 +1,28 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.taskclean;
import org.apache.hadoop.yarn.event.EventHandler;
public interface TaskCleaner extends EventHandler<TaskCleanupEvent> {
enum EventType {
TASK_CLEAN
}
}

View File

@ -0,0 +1,108 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.taskclean;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapreduce.v2.app.AppContext;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.service.AbstractService;
public class TaskCleanerImpl extends AbstractService implements TaskCleaner {
private static final Log LOG = LogFactory.getLog(TaskCleanerImpl.class);
private final AppContext context;
private ThreadPoolExecutor launcherPool;
private Thread eventHandlingThread;
private BlockingQueue<TaskCleanupEvent> eventQueue =
new LinkedBlockingQueue<TaskCleanupEvent>();
public TaskCleanerImpl(AppContext context) {
super("TaskCleaner");
this.context = context;
}
public void start() {
launcherPool = new ThreadPoolExecutor(1, 5, 1,
TimeUnit.HOURS, new LinkedBlockingQueue<Runnable>());
eventHandlingThread = new Thread(new Runnable() {
@Override
public void run() {
TaskCleanupEvent event = null;
while (!Thread.currentThread().isInterrupted()) {
try {
event = eventQueue.take();
} catch (InterruptedException e) {
LOG.error("Returning, interrupted : " + e);
return;
}
// the events from the queue are handled in parallel
// using a thread pool
launcherPool.execute(new EventProcessor(event)); }
}
});
eventHandlingThread.start();
super.start();
}
public void stop() {
eventHandlingThread.interrupt();
launcherPool.shutdown();
super.stop();
}
private class EventProcessor implements Runnable {
private TaskCleanupEvent event;
EventProcessor(TaskCleanupEvent event) {
this.event = event;
}
@Override
public void run() {
LOG.info("Processing the event " + event.toString());
try {
event.getCommitter().abortTask(event.getAttemptContext());
} catch (Exception e) {
LOG.warn("Task cleanup failed for attempt " + event.getAttemptID(), e);
}
context.getEventHandler().handle(
new TaskAttemptEvent(event.getAttemptID(),
TaskAttemptEventType.TA_CLEANUP_DONE));
}
}
@Override
public void handle(TaskCleanupEvent event) {
try {
eventQueue.put(event);
} catch (InterruptedException e) {
throw new YarnException(e);
}
}
}

View File

@ -0,0 +1,56 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.v2.app.taskclean;
import org.apache.hadoop.mapreduce.OutputCommitter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.yarn.event.AbstractEvent;
/**
* This class encapsulates task cleanup event.
*
*/
public class TaskCleanupEvent extends AbstractEvent<TaskCleaner.EventType> {
private final TaskAttemptId attemptID;
private final OutputCommitter committer;
private final TaskAttemptContext attemptContext;
public TaskCleanupEvent(TaskAttemptId attemptID, OutputCommitter committer,
TaskAttemptContext attemptContext) {
super(TaskCleaner.EventType.TASK_CLEAN);
this.attemptID = attemptID;
this.committer = committer;
this.attemptContext = attemptContext;
}
public TaskAttemptId getAttemptID() {
return attemptID;
}
public OutputCommitter getCommitter() {
return committer;
}
public TaskAttemptContext getAttemptContext() {
return attemptContext;
}
}

Some files were not shown because too many files have changed in this diff Show More