Merge master in to 0.5 and make them in sync. Keep theirs
diff --git a/CHANGES.txt b/CHANGES.txt
index 6f5d06b..f5d942d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,26 +1,59 @@
 Apache Tez Change Log
 =====================
 
+Release 0.6.0: Unreleased
+
+INCOMPATIBLE CHANGES
+
+ALL CHANGES:
+
 Release 0.5.1: Unreleased
 
 INCOMPATIBLE CHANGES
+  TEZ-1488. Rename HashComparator to ProxyComparator and implement in TezBytesComparator
+  TEZ-1578. Remove TeraSort from Tez codebase.
+  TEZ-1499. Add SortMergeJoinExample to tez-examples
   TEZ-1539. Change InputInitializerEvent semantics to SEND_ONCE_ON_TASK_SUCCESS
+  TEZ-1571. Add create method for DataSinkDescriptor.
 
 ALL CHANGES
-  TEZ-1543. Shuffle Errors on heavy load (causing task retries)
+  TEZ-1544. Link to release artifacts for 0.5.0 does not point to a specific link for 0.5.0.
+  TEZ-1559. Add system tests for AM recovery.
+  TEZ-850. Recovery unit tests.
+  TEZ-853. Support counters recovery.
+  TEZ-1345. Add checks to guarantee all init events are written to recovery to consider vertex initialized.
+  TEZ-1575. MRRSleepJob does not pick MR settings for container size and java opts.
+  TEZ-1488. Rename HashComparator to ProxyComparator and implement in TezBytesComparator
+  TEZ-1578. Remove TeraSort from Tez codebase.
+  TEZ-1569. Add tests for preemption
+  TEZ-1580. Change TestOrderedWordCount to optionally use MR configs.
+  TEZ-1524. Resolve user group information only if ACLs are enabled.
+  TEZ-1581. GroupByOrderByMRRTest no longer functional.
+  TEZ-1157. Optimize broadcast shuffle to download data only once per host. 
+  TEZ-1607. support mr envs in mrrsleep and testorderedwordcount
+  TEZ-1499. Add SortMergeJoinExample to tez-examples
+  TEZ-1613. Decrease running time for TestAMRecovery
+  TEZ-1240. Add system test for propagation of diagnostics for errors
+  TEZ-1618. LocalTaskSchedulerService.getTotalResources() and getAvailableResources() can get 
+  negative if JVM memory is larger than 2GB
+  TEZ-1611. Change DataSource/Sink to be able to supply URIs for credentials
+  TEZ-1592. Vertex should wait for all initializers to finish before moving to INITED state
+  TEZ-1612. ShuffleVertexManager's EdgeManager should not hard code source num tasks
+  TEZ-1555. TestTezClientUtils.validateSetTezJarLocalResourcesDefinedButEmpty
+  failing on Windows
+  TEZ-1609. Add hostname to logIdentifiers of fetchers for easy debugging
   TEZ-1494. DAG hangs waiting for ShuffleManager.getNextInput()
   TEZ-1515. Remove usage of ResourceBundles in Counters.
   TEZ-1527. Fix indentation of Vertex status in DAGClient output.
   TEZ-1536. Fix spelling typo "configurartion" in TezClientUtils.
   TEZ-1310. Update website documentation framework
   TEZ-1447. Provide a mechanism for InputInitializers to know about Vertex state changes.
-  TEZ-1362. Remove DAG_COMPLETED in DAGEventType. 
+  TEZ-1362. Remove DAG_COMPLETED in DAGEventType.
   TEZ-1519. TezTaskRunner should not initialize TezConfiguration in TezChild.
   TEZ-1534. Make client side configs available to AM and tasks.
   TEZ-1574. Support additional formats for the tez deployed archive
   TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless
   of DAG submission
-  TEZ-1571. Add create method for DataSinkDescriptor.
   TEZ-1585. Memory leak in tez session mode.
   TEZ-1533. Request Events more often if a complete set of events is received by a task.
   TEZ-1587. Some tez-examples fail in local mode.
@@ -34,6 +67,8 @@
   Windows
   TEZ-1554. Failing tests in TestMRHelpers related to environment on Windows
   TEZ-978. Enhance auto parallelism tuning for queries having empty outputs or data skewness
+  TEZ-1433. Invalid credentials can be used when a DAG is submitted to a
+  session which has timed out
   TEZ-1624. Flaky tests in TestContainerReuse due to race condition in DelayedContainerManager thread
 
 Release 0.5.0: 2014-09-03
@@ -42,7 +77,7 @@
   TEZ-1038. Move TaskLocationHint outside of VertexLocationHint.
   TEZ-960. VertexManagerPluginContext::getTotalAVailableResource() changed to
   VertexManagerPluginContext::getTotalAvailableResource()
-  TEZ-1025. Rename tez.am.max.task.attempts to tez.am.task.max.failed.attempts 
+  TEZ-1025. Rename tez.am.max.task.attempts to tez.am.task.max.failed.attempts
   TEZ-1018. VertexManagerPluginContext should enable assigning locality to
   scheduled tasks
   TEZ-1169. Allow numPhysicalInputs to be specified for RootInputs.
@@ -52,8 +87,11 @@
   TEZ-692. Unify job submission in either TezClient or TezSession
   TEZ-1130. Replace confusing names on Vertex API
   TEZ-1213. Fix parameter naming in TezJobConfig.
-  TEZ-1080, TEZ-1272, TEZ-1279, TEZ-1266. Change YARNRunner to use EdgeConfigs. 
-    - Removes separation of runtime configs into input/ouput configs. Also 
+    - Details at https://issues.apache.org/jira/browse/TEZ-1213?focusedCommentId
+    =14039381&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpa
+    nel#comment-14039381
+  TEZ-1080, TEZ-1272, TEZ-1279, TEZ-1266. Change YARNRunner to use EdgeConfigs.
+    - Removes separation of runtime configs into input/ouput configs. Also
     refactors public methods used for this conversion.
   TEZ-696. Remove implicit copying of processor payload to input and output
   TEZ-1269. TaskScheduler prematurely releases containers
@@ -124,7 +162,6 @@
 
 ALL CHANGES
 
-  TEZ-1515. DAGAppMaster : Thread contentions due to org.apache.tez.common.counters.ResourceBundles (Rajesh Balamohan)
   TEZ-1516. Log transfer rates for broadcast fetch. (sseth)
   TEZ-1511. MROutputConfigBuilder sets OutputFormat as String class if OutputFormat is not provided (bikas)
   TEZ-1509. Set a useful default value for java opts (bikas)
diff --git a/docs/src/site/markdown/index_0_5_0.md b/docs/src/site/markdown/index_0_5_0.md
new file mode 100644
index 0000000..32f101b
--- /dev/null
+++ b/docs/src/site/markdown/index_0_5_0.md
@@ -0,0 +1,29 @@
+<!--
+   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.
+-->
+
+<head><title>Apache Tez 0.5.0</title></head>
+
+Apache Tez 0.5.0
+----------------
+
+- [Release Artifacts](http://www.apache.org/dyn/closer.cgi/tez/0.5.0/)
+- [Release Notes](releases/0.5.0/release-notes.txt)
+- Documentation
+    - [API Javadocs](releases/0.5.0/tez-api-javadocs/index.html) : Documentation for the Tez APIs
+    - [Runtime Library Javadocs](releases/0.5.0/tez-runtime-library-javadocs/index.html) : Documentation for built-in implementations of useful Inputs, Outputs, Processors etc. written based on the Tez APIs 
+    - [Tez Mapreduce Javadocs](releases/0.5.0/tez-mapreduce-javadocs/index.html) : Documentation for built-in implementations of Mapreduce compatible Inputs, Outputs, Processors etc. written based on the Tez APIs 
+
diff --git a/docs/src/site/markdown/install.md b/docs/src/site/markdown/install.md
index cb9b586..d31eeb9 100644
--- a/docs/src/site/markdown/install.md
+++ b/docs/src/site/markdown/install.md
@@ -17,87 +17,88 @@
 
 <head><title>Install and Deployment Instructions</title></head>
 
-[Install instructions for Tez-0.5.0-SNAPSHOT - master branch](./install_0_5_0.html)
------------------------------------------------------------------------------------
-
-Install/Deploy Instructions for the latest Tez release [(Tez-0.4.1 src)](http://www.apache.org/dyn/closer.cgi/incubator/tez/tez-0.4.1-incubating/)
---------------------------------------------------------------------------------------------------------------------------------------------------
+Install/Deploy Instructions for Tez
+---------------------------------------------------------------------------
+Replace x.y.z with the tez release number that you are using. E.g. 0.5.0
 
 1.  Deploy Apache Hadoop using either the 2.2.0 release or a compatible
     2.x version.
     -   One thing to note though when compiling Tez is that you will
         need to change the value of the hadoop.version property in the
-        toplevel pom.xml to match the version of the hadoop branch being
+        top-level pom.xml to match the version of the hadoop branch being
         used.
-2.  Build tez using `mvn clean install -DskipTests=true -Dmaven.javadoc.skip=true`
+2.  Build tez using `mvn clean package -DskipTests=true -Dmaven.javadoc.skip=true`
     -   This assumes that you have already installed JDK6 or later,
         Maven 3 or later and Protocol Buffers (protoc compiler) 2.5 or
         later
     -   If you prefer to run the unit tests, remove skipTests from the
         command above.
-    -   If you would like to create a tarball of the release, use `mvn
-        clean package -Dtar -DskipTests=true -Dmaven.javadoc.skip=true`
     -   If you use Eclipse IDE, you can import the projects using
         "Import/Maven/Existing Maven Projects". Eclipse does not
         automatically generate Java sources or include the generated
         sources into the projects. Please build using maven as described
         above and then use Project Properties to include
-        "target/generated-sources/java" as a source directory into the
+        "target/generatedsources/java" as a source directory into the
         "Java Build Path" for these projects: tez-api, tez-mapreduce,
         tez-runtime-internals and tez-runtime-library. This needs to be done
         just once after importing the project.
-3.  Copy the tez jars and their dependencies into HDFS.
-    -   The tez jars and dependencies will be found in
-        tez-dist/target/tez-0.4.1-incubating/tez-0.4.1-incubating if you run
-        the intial command mentioned in step 2.
+3.  Copy the relevant tez tarball into HDFS, and configure tez-site.xml
+    -   A tez tarball containing tez and hadoop libraries will be found
+        at tez-dist/target/tez-x.y.z-SNAPSHOT.tar.gz
     -   Assuming that the tez jars are put in /apps/ on HDFS, the
-        command would be `hadoop dfs -put
-        tez-dist/target/tez-0.4.1-incubating/tez-0.4.1-incubating /apps/`
-    -   Please do not upload the tarball to HDFS, upload only the jars.
-4.  Configure tez-site.xml to set tez.lib.uris to point to the paths in
-    HDFS containing the jars. Please note that the paths are not
-    searched recursively so for *basedir* and *basedir*/lib/, you will
-    need to configure the 2 paths as a comma-separated list. * Assuming
-    you followed step 3, the value would be:
-    "${fs.default.name}/apps/tez-0.4.1-incubating,${fs.default.name}/apps/tez-0.4.1-incubating/lib/"
-5.  Modify mapred-site.xml to change _mapreduce.framework.name_ property
-    from its default value of *yarn* to *yarn-tez*
-6.  Set HADOOP_CLASSPATH to have the following paths in it:
-    -   TEZ_CONF_DIR - location of tez-site.xml
-    -   TEZ_JARS and TEZ_JARS/libs - location of the tez jars and
-        dependencies.
-    -   The command to set up the classpath should be something like:
-        `export HADOOP_CLASSPATH=${TEZ_CONF_DIR}:${TEZ_JARS}/*:${TEZ_JARS}/lib/*`
-        Please note the "*" which is an important requirement when
+        command would be
+        ```
+            hadoop dfs -mkdir /apps/tez-x.y.z-SNAPSHOT
+            hadoop dfs -copyFromLocal tez-dist/target/tez-x.y.z-SNAPSHOT-archive.tar.gz /apps/tez-x.y.z-SNAPSHOT/
+        ```
+    -   tez-site.xml configuration.
+        -   Set tez.lib.uris to point to the tar.gz uploaded to HDFS.
+            Assuming the steps mentioned so far were followed,
+            ```
+            set tez.lib.uris to "${fs.defaultFS}/apps/tez-x.y.z-SNAPSHOT/tez-x.y.z-SNAPSHOT.tar.gz"
+            ```
+        -   Ensure tez.use.cluster.hadoop-libs is not set in tez-site.xml,
+            or if it is set, the value should be false
+4.  Optional: If running existing MapReduce jobs on Tez. Modify
+    mapred-site.xml to change "mapreduce.framework.name" property from
+    its default value of "yarn" to "yarn-tez"
+5.  Configure the client node to include the tez-libraries in the hadoop
+    classpath
+    -   Extract the tez minimal tarball created in step 2 to a local directory
+        (assuming TEZ_JARS is where the files will be decompressed for
+        the next steps)
+        ```
+        tar -xvzf tez-dist/target/tez-x.y.z-minimal.tar.gz -C $TEZ_JARS
+        ```
+    -   set TEZ_CONF_DIR to the location of tez-site.xml
+    -   Add $TEZ_CONF_DIR, ${TEZ_JARS}/* and ${TEZ_JARS}/lib/* to the application classpath.
+        For example, doing it via the standard Hadoop tool chain would use the following command 
+	to set up the application classpath:
+        ```
+        export HADOOP_CLASSPATH=${TEZ_CONF_DIR}:${TEZ_JARS}/*:${TEZ_JARS}/lib/*
+        ```
+    -   Please note the "*" which is an important requirement when
         setting up classpaths for directories containing jar files.
-7.  Submit a MR job as you normally would using something like:
+6.  There is a basic example of using an MRR job in the tez-examples.jar.
+    Refer to OrderedWordCount.java in the source code. To run this
+    example:
 
     ```
-    $HADOOP_PREFIX/bin/hadoop jar hadoop-mapreduce-client-jobclient-3.0.0-SNAPSHOT-tests.jar sleep -mt 1 -rt 1 -m 1 -r 1
-    ```
-
-    This will use the TEZ DAG ApplicationMaster to run the MR job. This
-    can be verified by looking at the AM’s logs from the YARN
-    ResourceManager UI.
-8.  There is a basic example of using an MRR job in the
-    tez-mapreduce-examples.jar. Refer to OrderedWordCount.java in the
-    source code. To run this example:
-
-    ``` 
-    $HADOOP_PREFIX/bin/hadoop jar tez-mapreduce-examples.jar orderedwordcount <input> <output>
+    $HADOOP_PREFIX/bin/hadoop jar tez-examples.jar orderedwordcount <input> <output>
     ```
 
     This will use the TEZ DAG ApplicationMaster to run the ordered word
     count job. This job is similar to the word count example except that
     it also orders all words based on the frequency of occurrence.
 
-    There are multiple variations to run orderedwordcount. You can use
-    it to run multiple DAGs serially on different inputs/outputs. These
-    DAGs could be run separately as different applications or serially
-    within a single TEZ session.
+    Tez DAGs could be run separately as different applications or
+    serially within a single TEZ session. There is a different variation
+    of orderedwordcount in tez-tests that supports the use of Sessions
+    and handling multiple input-output pairs. You can use it to run
+    multiple DAGs serially on different inputs/outputs.
 
     ```
-    $HADOOP_PREFIX/bin/hadoop jar tez-mapreduce-examples.jar orderedwordcount <input1> <output1> <input2> <output2> <input3> <output3> ...
+    $HADOOP_PREFIX/bin/hadoop jar tez-tests.jar testorderedwordcount <input1> <output1> <input2> <output2> <input3> <output3> ...
     ```
 
     The above will run multiple DAGs for each input-output pair.
@@ -105,5 +106,34 @@
     To use TEZ sessions, set -DUSE_TEZ_SESSION=true
 
     ```
-    $HADOOP_PREFIX/bin/hadoop jar tez-mapreduce-examples.jar orderedwordcount -DUSE_TEZ_SESSION=true <input1> <output1> <input2> <output2>
+    $HADOOP_PREFIX/bin/hadoop jar tez-tests.jar testorderedwordcount -DUSE_TEZ_SESSION=true <input1> <output1> <input2> <output2>
     ```
+7.  Submit a MR job as you normally would using something like:
+
+    ```
+    $HADOOP_PREFIX/bin/hadoop jar hadoop-mapreduce-client-jobclient-3.0.0-SNAPSHOT-tests.jar sleep -mt 1 -rt 1 -m 1 -r 1
+    ```
+
+    This will use the TEZ DAG ApplicationMaster to run the MR job. This
+    can be verified by looking at the AM’s logs from the YARN ResourceManager UI. 
+    This needs mapred-site.xml to have "mapreduce.framework.name" set to "yarn-tez"
+
+Hadoop Installation dependent Install/Deploy Instructions
+---------------------------------------------------------
+The above install instructions use Tez with pre-packaged Hadoop libraries included in the package and is the 
+recommended method for installation. If its needed to make Tez use the existing cluster Hadoop libraries then
+follow this alternate machanism to setup Tez to use Hadoop libraries from the cluster.
+Step 3 above changes as follows. Also subsequent steps would use tez-dist/target/tez-x.y.z-minimal.tar.gz instead of tez-dist/target/tez-x.y.z.tar.gz
+- A tez build without Hadoop dependencies will be available at tez-dist/target/tez-x.y.z-minimal.tar.gz
+- Assuming that the tez jars are put in /apps/ on HDFS, the command would be
+"hadoop fs -mkdir /apps/tez-x.y.z"
+"hadoop fs -copyFromLocal tez-dist/target/tez-x.y.z-minimal.tar.gz /apps/tez-x.y.z"
+- tez-site.xml configuration
+- Set tez.lib.uris to point to the paths in HDFS containing the tez jars. Assuming the steps mentioned so far were followed,
+set tez.lib.uris to "${fs.defaultFS}/apps/tez-x.y.z/tez-x.y.z-minimal.tar.gz
+- set tez.use.cluster.hadoop-libs to true
+
+
+[Install instructions for older versions of Tez (pre 0.5.0)](./install_pre_0_5_0.html)
+-----------------------------------------------------------------------------------
+
diff --git a/docs/src/site/markdown/install_0_5_0.md b/docs/src/site/markdown/install_0_5_0.md
deleted file mode 100644
index 2cf0e3d..0000000
--- a/docs/src/site/markdown/install_0_5_0.md
+++ /dev/null
@@ -1,117 +0,0 @@
-<!--
-   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.
--->
-
-<head><title>Install and Deployment Instructions</title></head>
-
-Install/Deploy Instructions for Tez-current (0.5.0-SNAPSHOT, branch master)
----------------------------------------------------------------------------
-
-1.  Deploy Apache Hadoop using either the 2.2.0 release or a compatible
-    2.x version.
-    -   One thing to note though when compiling Tez is that you will
-        need to change the value of the hadoop.version property in the
-        top-level pom.xml to match the version of the hadoop branch being
-        used.
-2.  Build tez using `mvn clean package -DskipTests=true -Dmaven.javadoc.skip=true`
-    -   This assumes that you have already installed JDK6 or later,
-        Maven 3 or later and Protocol Buffers (protoc compiler) 2.5 or
-        later
-    -   If you prefer to run the unit tests, remove skipTests from the
-        command above.
-    -   If you use Eclipse IDE, you can import the peojects using
-        "Import/Maven/Existing Maven Projects". Eclipse does not
-        automatically generate Java sources or include the generated
-        sources into the projects. Please build using maven as described
-        above and then use Project Properties to include
-        "target/generatedsources/java" as a source directory into the
-        "Java Build Path" for these projects: tez-api, tez-mapreduce,
-        tez-runtime-internals and tez-runtime-library. This needs to be done
-        just once after importing the project.
-3.  Copy the relevant tez tarball into HDFS, and configure tezsite.xml
-    -   A tez tarball containing tez and hadoop libraries will be found
-        at tez-dist/target/tez-0.5.0-SNAPSHOT.tar.gz
-    -   Assuming that the tez jars are put in /apps/ on HDFS, the
-        command would be
-        ```
-            hadoop dfs -mkdir /apps/tez-0.5.0-SNAPSHOT
-            hadoop dfs -copyFromLocal tez-dist/target/tez-0.5.0-SNAPSHOT-archive.tar.gz /apps/tez-0.5.0-SNAPSHOT/
-        ```
-    -   tez-site.xml configuration.
-        -   Set tez.lib.uris to point to the tar.gz uploaded to HDFS.
-            Assuming the steps mentioned so far were followed,
-            ```
-            set tez.lib.uris to "${fs.default.name}/apps/tez-0.5.0-SNAPSHOT/tez-0.5.0-SNAPSHOT.tar.gz"
-            ```
-        -   Ensure tez.use.cluster.hadoop-libs is not set in tez-site.xml,
-            or if it is set, the value should be false
-4.  Optional: If running existing MapReduce jobs on Tez. Modify
-    mapred-site.xml to change "mapreduce.framework.name" property from
-    its default value of "yarn" to "yarn-tez"
-5.  Configure the client node to include the tez-libraries in the hadoop
-    classpath
-    -   Extract the tez tarball created in step 2 to a local directory
-        (assuming TEZ_JARS is where the files will be decompressed for
-        the next steps)
-        ```
-        tar -xvzf tez-dist/target/tez-0.5.0-SNAPSHOT.tar.gz -C $TEZ_JARS
-        ```
-    -   set TEZ_CONF_DIR to the location of tez-site.xml
-    -   The command to set up the classpath should be something like:
-        ```
-        export HADOOP_CLASSPATH=${TEZ_CONF_DIR}:${TEZ_JARS}/*:${TEZ_JARS}/lib/*
-        ```
-    -   Please note the "*" which is an important requirement when
-        setting up classpaths for directories containing jar files.
-6.  Submit a MR job as you normally would using something like:
-
-    ```
-    $HADOOP_PREFIX/bin/hadoop jar hadoop-mapreduce-client-jobclient-3.0.0-SNAPSHOT-tests.jar sleep -mt 1 -rt 1 -m 1 -r 1
-    ```
-
-    This will use the TEZ DAG ApplicationMaster to run the MR job. This
-    can be verified by looking at the AM’s logs from the YARN
-    ResourceManager UI.
-
-7.  There is a basic example of using an MRR job in the tez-examples.jar.
-    Refer to OrderedWordCount.java in the source code. To run this
-    example:
-
-    ```
-    $HADOOP_PREFIX/bin/hadoop jar tez-examples.jar orderedwordcount <input> <output>
-    ```
-
-    This will use the TEZ DAG ApplicationMaster to run the ordered word
-    count job. This job is similar to the word count example except that
-    it also orders all words based on the frequency of occurrence.
-
-    Tez DAGs could be run separately as different applications or
-    serially within a single TEZ session. There is a different variation
-    of orderedwordcount in tez-tests that supports the use of Sessions
-    and handling multiple input-output pairs. You can use it to run
-    multiple DAGs serially on different inputs/outputs.
-
-    ```
-    $HADOOP_PREFIX/bin/hadoop jar tez-tests.jar testorderedwordcount <input1> <output1> <input2> <output2> <input3> <output3> ...
-    ```
-
-    The above will run multiple DAGs for each input-output pair.
-
-    To use TEZ sessions, set -DUSE_TEZ_SESSION=true
-
-    ```
-    $HADOOP_PREFIX/bin/hadoop jar tez-tests.jar testorderedwordcount -DUSE_TEZ_SESSION=true <input1> <output1> <input2> <output2>
-    ```
diff --git a/docs/src/site/markdown/install_pre_0_5_0.md b/docs/src/site/markdown/install_pre_0_5_0.md
new file mode 100644
index 0000000..494ff54
--- /dev/null
+++ b/docs/src/site/markdown/install_pre_0_5_0.md
@@ -0,0 +1,109 @@
+<!--
+   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.
+-->
+
+<head><title>Install and Deployment Instructions</title></head>
+
+[Install instructions for Tez (post 0.5.0)](./install.html)
+-----------------------------------------------------------------------------------
+
+Install/Deploy Instructions for Tez release pre-0.5.0 E.g. [(Tez-0.4.1)](http://archive.apache.org/dist/incubator/tez/tez-0.4.1-incubating/)
+--------------------------------------------------------------------------------------------------------------------------------------------------
+
+1.  Deploy Apache Hadoop using either the 2.2.0 release or a compatible
+    2.x version.
+    -   One thing to note though when compiling Tez is that you will
+        need to change the value of the hadoop.version property in the
+        toplevel pom.xml to match the version of the hadoop branch being
+        used.
+2.  Build tez using `mvn clean install -DskipTests=true -Dmaven.javadoc.skip=true`
+    -   This assumes that you have already installed JDK6 or later,
+        Maven 3 or later and Protocol Buffers (protoc compiler) 2.5 or
+        later
+    -   If you prefer to run the unit tests, remove skipTests from the
+        command above.
+    -   If you would like to create a tarball of the release, use `mvn
+        clean package -Dtar -DskipTests=true -Dmaven.javadoc.skip=true`
+    -   If you use Eclipse IDE, you can import the projects using
+        "Import/Maven/Existing Maven Projects". Eclipse does not
+        automatically generate Java sources or include the generated
+        sources into the projects. Please build using maven as described
+        above and then use Project Properties to include
+        "target/generated-sources/java" as a source directory into the
+        "Java Build Path" for these projects: tez-api, tez-mapreduce,
+        tez-runtime-internals and tez-runtime-library. This needs to be done
+        just once after importing the project.
+3.  Copy the tez jars and their dependencies into HDFS.
+    -   The tez jars and dependencies will be found in
+        tez-dist/target/tez-0.4.1-incubating/tez-0.4.1-incubating if you run
+        the intial command mentioned in step 2.
+    -   Assuming that the tez jars are put in /apps/ on HDFS, the
+        command would be `hadoop dfs -put
+        tez-dist/target/tez-0.4.1-incubating/tez-0.4.1-incubating /apps/`
+    -   Please do not upload the tarball to HDFS, upload only the jars.
+4.  Configure tez-site.xml to set tez.lib.uris to point to the paths in
+    HDFS containing the jars. Please note that the paths are not
+    searched recursively so for *basedir* and *basedir*/lib/, you will
+    need to configure the 2 paths as a comma-separated list. * Assuming
+    you followed step 3, the value would be:
+    "${fs.default.name}/apps/tez-0.4.1-incubating,${fs.default.name}/apps/tez-0.4.1-incubating/lib/"
+5.  Modify mapred-site.xml to change _mapreduce.framework.name_ property
+    from its default value of *yarn* to *yarn-tez*
+6.  Set HADOOP_CLASSPATH to have the following paths in it:
+    -   TEZ_CONF_DIR - location of tez-site.xml
+    -   TEZ_JARS and TEZ_JARS/libs - location of the tez jars and
+        dependencies.
+    -   The command to set up the classpath should be something like:
+        `export HADOOP_CLASSPATH=${TEZ_CONF_DIR}:${TEZ_JARS}/*:${TEZ_JARS}/lib/*`
+        Please note the "*" which is an important requirement when
+        setting up classpaths for directories containing jar files.
+7.  Submit a MR job as you normally would using something like:
+
+    ```
+    $HADOOP_PREFIX/bin/hadoop jar hadoop-mapreduce-client-jobclient-3.0.0-SNAPSHOT-tests.jar sleep -mt 1 -rt 1 -m 1 -r 1
+    ```
+
+    This will use the TEZ DAG ApplicationMaster to run the MR job. This
+    can be verified by looking at the AM’s logs from the YARN
+    ResourceManager UI.
+8.  There is a basic example of using an MRR job in the
+    tez-mapreduce-examples.jar. Refer to OrderedWordCount.java in the
+    source code. To run this example:
+
+    ``` 
+    $HADOOP_PREFIX/bin/hadoop jar tez-mapreduce-examples.jar orderedwordcount <input> <output>
+    ```
+
+    This will use the TEZ DAG ApplicationMaster to run the ordered word
+    count job. This job is similar to the word count example except that
+    it also orders all words based on the frequency of occurrence.
+
+    There are multiple variations to run orderedwordcount. You can use
+    it to run multiple DAGs serially on different inputs/outputs. These
+    DAGs could be run separately as different applications or serially
+    within a single TEZ session.
+
+    ```
+    $HADOOP_PREFIX/bin/hadoop jar tez-mapreduce-examples.jar orderedwordcount <input1> <output1> <input2> <output2> <input3> <output3> ...
+    ```
+
+    The above will run multiple DAGs for each input-output pair.
+
+    To use TEZ sessions, set -DUSE_TEZ_SESSION=true
+
+    ```
+    $HADOOP_PREFIX/bin/hadoop jar tez-mapreduce-examples.jar orderedwordcount -DUSE_TEZ_SESSION=true <input1> <output1> <input2> <output2>
+    ```
diff --git a/docs/src/site/markdown/talks.md b/docs/src/site/markdown/talks.md
index 8971088..a29d553 100644
--- a/docs/src/site/markdown/talks.md
+++ b/docs/src/site/markdown/talks.md
@@ -19,10 +19,10 @@
 
 Talks
 -----
--   Apache Tez : Accelerating Hadoop Query Processing by Arun Murthy and
-    Bikas Saha at [Hadoop Summit 2013, San Jose, CA, USA](http://hadoopsummit.org/san-jose/)
-    -   [Slides](http://www.slideshare.net/Hadoop_Summit/murhty-saha-june26255pmroom212)
-    -   [Video](http://www.youtube.com/watch?v=9ZLLzlsz7h8)
+-   Apache Tez : Accelerating Hadoop Query Processing by Bikas Saha and
+    Hitesh Shah at [Hadoop Summit 2014, San Jose, CA, USA](http://hadoopsummit.org/san-jose/)
+    -   [Slides](http://www.slideshare.net/Hadoop_Summit/w-1205phall1saha)
+    -   [Video](http://www.youtube.com/watch?v=yf_hBiZy3nk)
 
 User Meetup Recordings
 ----------------------
diff --git a/docs/src/site/site.xml b/docs/src/site/site.xml
index 1a915db..a3d629c 100644
--- a/docs/src/site/site.xml
+++ b/docs/src/site/site.xml
@@ -112,6 +112,7 @@
 
     <menu name="Releases">
       <item name="0.4.1-incubating" href="http://archive.apache.org/dist/incubator/tez/tez-0.4.1-incubating/"/>
+      <item name="0.5.0" href="index_0_5_0.html"/>
     </menu>
 
     <menu name="Contribute">
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
index 4fe421a..0955c20 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -123,7 +123,7 @@
   }
 
   @Private
-  TezClient(String name, TezConfiguration tezConf, boolean isSession,
+  protected TezClient(String name, TezConfiguration tezConf, boolean isSession,
             @Nullable Map<String, LocalResource> localResources,
             @Nullable Credentials credentials) {
     this.clientName = name;
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index 917fcff..f011f60 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -83,6 +83,8 @@
 import org.apache.tez.common.security.TokenCache;
 import org.apache.tez.dag.api.DAG;
 import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.DataSinkDescriptor;
+import org.apache.tez.dag.api.DataSourceDescriptor;
 import org.apache.tez.dag.api.SessionNotRunning;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezConstants;
@@ -334,17 +336,13 @@
    * @throws IOException
    */
   @Private
-  static void setupDAGCredentials(DAG dag, Credentials sessionCredentials,
+  static Credentials setupDAGCredentials(DAG dag, Credentials sessionCredentials,
       Configuration conf) throws IOException {
 
     Preconditions.checkNotNull(sessionCredentials);
     TezCommonUtils.logCredentials(LOG, sessionCredentials, "session");
 
-    Credentials dagCredentials = dag.getCredentials();
-    if (dagCredentials == null) {
-      dagCredentials = new Credentials();
-      dag.setCredentials(dagCredentials);
-    }
+    Credentials dagCredentials = new Credentials();
     // All session creds are required for the DAG.
     dagCredentials.mergeAll(sessionCredentials);
     
@@ -360,6 +358,18 @@
         for (LocalResource lr: v.getTaskLocalFiles().values()) {
           lrPaths.add(ConverterUtils.getPathFromYarnURL(lr.getResource()));
         }
+        List<DataSourceDescriptor> dataSources = v.getDataSources();
+        for (DataSourceDescriptor dataSource : dataSources) {
+          addFileSystemCredentialsFromURIs(dataSource.getURIsForCredentials(), dagCredentials, conf);
+        }
+        List<DataSinkDescriptor> dataSinks = v.getDataSinks();
+        for (DataSinkDescriptor dataSink : dataSinks) {
+          addFileSystemCredentialsFromURIs(dataSink.getURIsForCredentials(), dagCredentials, conf);
+        }
+      }
+      
+      for (LocalResource lr: dag.getTaskLocalFiles().values()) {
+        lrPaths.add(ConverterUtils.getPathFromYarnURL(lr.getResource()));
       }
 
       Path[] paths = lrPaths.toArray(new Path[lrPaths.size()]);
@@ -368,6 +378,8 @@
     } catch (URISyntaxException e) {
       throw new IOException(e);
     }
+    
+    return dagCredentials;
   }
 
   @Private
@@ -622,35 +634,12 @@
 
   }
   
-  static void updateDAGVertices(DAG dag, AMConfiguration amConfig,
-      Map<String, LocalResource> tezJarResources, boolean tezLrsAsArchive,
-      Credentials credentials) throws IOException {
-    setupDAGCredentials(dag, credentials, amConfig.getTezConfiguration());
-    for (Vertex v : dag.getVertices()) {
-      if (tezJarResources != null) {
-        v.getTaskLocalFiles().putAll(tezJarResources);
-      }
-      v.getTaskLocalFiles().put(TezConstants.TEZ_PB_BINARY_CONF_NAME,
-          amConfig.getBinaryConfLR());
-
-      Map<String, String> taskEnv = v.getTaskEnvironment();
-      TezYARNUtils.setupDefaultEnv(taskEnv, amConfig.getTezConfiguration(),
-          TezConfiguration.TEZ_TASK_LAUNCH_ENV,
-          TezConfiguration.TEZ_TASK_LAUNCH_ENV_DEFAULT, tezLrsAsArchive);
-
-      setDefaultLaunchCmdOpts(v, amConfig.getTezConfiguration());
-    }
-  }
-  
   static DAGPlan prepareAndCreateDAGPlan(DAG dag, AMConfiguration amConfig,
       Map<String, LocalResource> tezJarResources, boolean tezLrsAsArchive,
       Credentials credentials) throws IOException {
-    DAGPlan dagPB = dag.getCachedDAGPlan();
-    if (dagPB == null) {
-      updateDAGVertices(dag, amConfig, tezJarResources, tezLrsAsArchive, credentials);  
-      dagPB = dag.createDag(amConfig.getTezConfiguration());  
-    }    
-    return dagPB;
+    Credentials dagCredentials = setupDAGCredentials(dag, credentials, amConfig.getTezConfiguration());
+    return dag.createDag(amConfig.getTezConfiguration(), dagCredentials, tezJarResources,
+        amConfig.getBinaryConfLR(), true);
   }
   
   static void maybeAddDefaultLoggingJavaOpts(String logLevel, List<String> vargs) {
@@ -678,8 +667,8 @@
     return StringUtils.join(vargs, " ").trim();
   }
   
-  static void setDefaultLaunchCmdOpts(Vertex v, TezConfiguration conf) {
-    String vOpts = v.getTaskLaunchCmdOpts();
+  @Private
+  public static String addDefaultsToTaskLaunchCmdOpts(String vOpts, Configuration conf) {
     String vConfigOpts = conf.get(TezConfiguration.TEZ_TASK_LAUNCH_CMD_OPTS,
         TezConfiguration.TEZ_TASK_LAUNCH_CMD_OPTS_DEFAULT);
     if (vConfigOpts != null && vConfigOpts.length() > 0) {
@@ -689,7 +678,7 @@
     vOpts = maybeAddDefaultLoggingJavaOpts(conf.get(
         TezConfiguration.TEZ_TASK_LOG_LEVEL,
         TezConfiguration.TEZ_TASK_LOG_LEVEL_DEFAULT), vOpts);
-    v.setTaskLaunchCmdOpts(vOpts);
+    return vOpts;
   }
 
   @Private
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
index 3598572..b5bdffb 100644
--- a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
@@ -27,6 +27,7 @@
 public enum DAGCounter {
   NUM_FAILED_TASKS, 
   NUM_KILLED_TASKS,
+  NUM_SUCCEEDED_TASKS,
   TOTAL_LAUNCHED_TASKS,
   OTHER_LOCAL_TASKS,
   DATA_LOCAL_TASKS,
diff --git a/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java
index d8be327..77ab065 100644
--- a/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java
+++ b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java
@@ -18,6 +18,7 @@
 
 package org.apache.tez.common.security;
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -29,6 +30,7 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.tez.dag.api.TezConfiguration;
 
@@ -92,13 +94,19 @@
   }
 
   @VisibleForTesting
-  boolean checkAccess(String user, Collection<String> userGroups, ACLType aclType) {
+  boolean checkAccess(UserGroupInformation ugi, ACLType aclType) {
+
     if (!aclsEnabled) {
       return true;
     }
+
+    String user = ugi.getShortUserName();
+    Collection<String> userGroups = Arrays.asList(ugi.getGroupNames());
+
     if (amUser.equals(user)) {
       return true;
     }
+
     if (EnumSet.of(ACLType.DAG_MODIFY_ACL, ACLType.DAG_VIEW_ACL).contains(aclType)) {
       if (dagUser != null && dagUser.equals(user)) {
         return true;
@@ -129,22 +137,22 @@
     return false;
   }
 
-  public boolean checkAMViewAccess(String user, Collection<String> userGroups) {
-    return checkAccess(user, userGroups, ACLType.AM_VIEW_ACL);
+  public boolean checkAMViewAccess(UserGroupInformation ugi) {
+    return checkAccess(ugi, ACLType.AM_VIEW_ACL);
   }
 
-  public boolean checkAMModifyAccess(String user, Collection<String> userGroups) {
-    return checkAccess(user, userGroups, ACLType.AM_MODIFY_ACL);
+  public boolean checkAMModifyAccess(UserGroupInformation ugi) {
+    return checkAccess(ugi, ACLType.AM_MODIFY_ACL);
   }
 
-  public boolean checkDAGViewAccess(String user, Collection<String> userGroups) {
-    return checkAccess(user, userGroups, ACLType.AM_VIEW_ACL)
-        || checkAccess(user, userGroups, ACLType.DAG_VIEW_ACL);
+  public boolean checkDAGViewAccess(UserGroupInformation ugi) {
+    return checkAccess(ugi, ACLType.AM_VIEW_ACL)
+        || checkAccess(ugi, ACLType.DAG_VIEW_ACL);
   }
 
-  public boolean checkDAGModifyAccess(String user, Collection<String> userGroups) {
-    return checkAccess(user, userGroups, ACLType.AM_MODIFY_ACL)
-        || checkAccess(user, userGroups, ACLType.DAG_MODIFY_ACL);
+  public boolean checkDAGModifyAccess(UserGroupInformation ugi) {
+    return checkAccess(ugi, ACLType.AM_MODIFY_ACL)
+        || checkAccess(ugi, ACLType.DAG_MODIFY_ACL);
   }
 
   public Map<ApplicationAccessType, String> toYARNACls() {
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
index ffd2e83..9b428f0 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
@@ -39,10 +39,11 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.client.TezClientUtils;
 import org.apache.tez.common.security.DAGAccessControls;
 import org.apache.tez.common.TezCommonUtils;
+import org.apache.tez.common.TezYARNUtils;
 import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
 import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
@@ -53,7 +54,6 @@
 import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
 import org.apache.tez.dag.api.records.DAGProtos.PlanGroupInputEdgeInfo;
 import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
-import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResource;
 import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
 import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
 import org.apache.tez.dag.api.records.DAGProtos.PlanVertexGroupInfo;
@@ -89,7 +89,6 @@
   Map<String, LocalResource> commonTaskLocalFiles = Maps.newHashMap();
   
   private Stack<String> topologicalVertexStack = new Stack<String>();
-  private DAGPlan cachedDAGPlan;
 
   private DAG(String name) {
     this.name = name;
@@ -112,7 +111,7 @@
    *          elements of the map.
    * @return {@link DAG}
    */
-  public DAG addTaskLocalFiles(Map<String, LocalResource> localFiles) {
+  public synchronized DAG addTaskLocalFiles(Map<String, LocalResource> localFiles) {
     Preconditions.checkNotNull(localFiles);
     TezCommonUtils.addAdditionalLocalResources(localFiles, commonTaskLocalFiles);
     return this;
@@ -298,6 +297,11 @@
     return this.name;
   }
   
+  @Private
+  public Map<String, LocalResource> getTaskLocalFiles() {
+    return commonTaskLocalFiles;
+  }
+  
   void checkAndInferOneToOneParallelism() {
     // infer all 1-1 via dependencies
     // collect all 1-1 edges where the source parallelism is set
@@ -575,14 +579,11 @@
     }
   }
 
-  @Private
-  public DAGPlan getCachedDAGPlan() {
-    return cachedDAGPlan;
-  }
-
   // create protobuf message describing DAG
   @Private
-  public DAGPlan createDag(Configuration dagConf) {
+  public DAGPlan createDag(Configuration dagConf, Credentials extraCredentials,
+      Map<String, LocalResource> tezJarResources, LocalResource binaryConfig,
+      boolean tezLrsAsArchive) {
     verify(true);
 
     DAGPlan.Builder dagBuilder = DAGPlan.newBuilder();
@@ -608,6 +609,15 @@
       }
     }
 
+    Credentials dagCredentials = new Credentials();
+    if (extraCredentials != null) {
+      dagCredentials.mergeAll(extraCredentials);
+    }
+    dagCredentials.mergeAll(credentials);
+    if (!commonTaskLocalFiles.isEmpty()) {
+      dagBuilder.addAllLocalResource(DagTypeConverters.convertToDAGPlan(commonTaskLocalFiles));
+    }
+
     Preconditions.checkArgument(topologicalVertexStack.size() == vertices.size(),
         "size of topologicalVertexStack is:" + topologicalVertexStack.size() +
         " while size of vertices is:" + vertices.size() +
@@ -615,38 +625,56 @@
     while(!topologicalVertexStack.isEmpty()) {
       Vertex vertex = vertices.get(topologicalVertexStack.pop());
       // infer credentials, resources and parallelism from data source
-      if (vertex.getTaskResource() == null) {
-        vertex.setTaskResource(Resource.newInstance(dagConf.getInt(
+      Resource vertexTaskResource = vertex.getTaskResource();
+      if (vertexTaskResource == null) {
+        vertexTaskResource = Resource.newInstance(dagConf.getInt(
             TezConfiguration.TEZ_TASK_RESOURCE_MEMORY_MB,
             TezConfiguration.TEZ_TASK_RESOURCE_MEMORY_MB_DEFAULT), dagConf.getInt(
             TezConfiguration.TEZ_TASK_RESOURCE_CPU_VCORES,
-            TezConfiguration.TEZ_TASK_RESOURCE_CPU_VCORES_DEFAULT)));
+            TezConfiguration.TEZ_TASK_RESOURCE_CPU_VCORES_DEFAULT));
       }
+      Map<String, LocalResource> vertexLRs = Maps.newHashMap();
+      vertexLRs.putAll(vertex.getTaskLocalFiles());
       List<DataSourceDescriptor> dataSources = vertex.getDataSources();
       for (DataSourceDescriptor dataSource : dataSources) {
         if (dataSource.getCredentials() != null) {
-          credentials.addAll(dataSource.getCredentials());
+          dagCredentials.addAll(dataSource.getCredentials());
         }
-        vertex.addTaskLocalFiles(dataSource.getAdditionalLocalFiles());
+        if (dataSource.getAdditionalLocalFiles() != null) {
+          TezCommonUtils.addAdditionalLocalResources(dataSource.getAdditionalLocalFiles(), vertexLRs);
+        }
       }
+      if (tezJarResources != null) {
+        TezCommonUtils.addAdditionalLocalResources(tezJarResources, vertexLRs);
+      }
+      if (binaryConfig != null) {
+        vertexLRs.put(TezConstants.TEZ_PB_BINARY_CONF_NAME, binaryConfig);
+      }
+      int vertexParallelism = vertex.getParallelism();
+      VertexLocationHint vertexLocationHint = vertex.getLocationHint();
       if (dataSources.size() == 1) {
         DataSourceDescriptor dataSource = dataSources.get(0);
-        if (vertex.getParallelism() == -1 && dataSource.getNumberOfShards() > -1) {
-          vertex.setParallelism(dataSource.getNumberOfShards());
+        if (vertexParallelism == -1 && dataSource.getNumberOfShards() > -1) {
+          vertexParallelism = dataSource.getNumberOfShards();
         }
-        if (vertex.getLocationHint() == null && dataSource.getLocationHint() != null) {
-          vertex.setLocationHint(dataSource.getLocationHint());
+        if (vertexLocationHint == null && dataSource.getLocationHint() != null) {
+          vertexLocationHint = dataSource.getLocationHint();
         }
       }
+      if (vertexParallelism == -1) {
+        Preconditions.checkState(vertexLocationHint == null,
+            "Cannot specify vertex location hint without specifying vertex parallelism. Vertex: "
+                + vertex.getName());
+      } else if (vertexLocationHint != null) {
+        Preconditions.checkState(vertexParallelism == vertexLocationHint.getTaskLocationHints().size(),
+            "vertex task location hint must equal vertex parallelism. Vertex: " + vertex.getName());
+      }
       for (DataSinkDescriptor dataSink : vertex.getDataSinks()) {
         if (dataSink.getCredentials() != null) {
-          credentials.addAll(dataSink.getCredentials());
+          dagCredentials.addAll(dataSink.getCredentials());
         }
       }
       
-      // add common task files for this DAG
-      vertex.addTaskLocalFiles(commonTaskLocalFiles);
-        
       VertexPlan.Builder vertexBuilder = VertexPlan.newBuilder();
       vertexBuilder.setName(vertex.getName());
       vertexBuilder.setType(PlanVertexType.NORMAL); // vertex type is implicitly NORMAL until  TEZ-46.
@@ -665,48 +693,31 @@
 
       //task config
       PlanTaskConfiguration.Builder taskConfigBuilder = PlanTaskConfiguration.newBuilder();
-      Resource resource = vertex.getTaskResource();
-      taskConfigBuilder.setNumTasks(vertex.getParallelism());
-      taskConfigBuilder.setMemoryMb(resource.getMemory());
-      taskConfigBuilder.setVirtualCores(resource.getVirtualCores());
-      taskConfigBuilder.setJavaOpts(vertex.getTaskLaunchCmdOpts());
+      taskConfigBuilder.setNumTasks(vertexParallelism);
+      taskConfigBuilder.setMemoryMb(vertexTaskResource.getMemory());
+      taskConfigBuilder.setVirtualCores(vertexTaskResource.getVirtualCores());
+      taskConfigBuilder.setJavaOpts(
+          TezClientUtils.addDefaultsToTaskLaunchCmdOpts(vertex.getTaskLaunchCmdOpts(), dagConf));
 
       taskConfigBuilder.setTaskModule(vertex.getName());
-      PlanLocalResource.Builder localResourcesBuilder = PlanLocalResource.newBuilder();
-      localResourcesBuilder.clear();
-      for (Entry<String, LocalResource> entry :
-             vertex.getTaskLocalFiles().entrySet()) {
-        String key = entry.getKey();
-        LocalResource lr = entry.getValue();
-        localResourcesBuilder.setName(key);
-        localResourcesBuilder.setUri(
-          DagTypeConverters.convertToDAGPlan(lr.getResource()));
-        localResourcesBuilder.setSize(lr.getSize());
-        localResourcesBuilder.setTimeStamp(lr.getTimestamp());
-        localResourcesBuilder.setType(
-          DagTypeConverters.convertToDAGPlan(lr.getType()));
-        localResourcesBuilder.setVisibility(
-          DagTypeConverters.convertToDAGPlan(lr.getVisibility()));
-        if (lr.getType() == LocalResourceType.PATTERN) {
-          if (lr.getPattern() == null || lr.getPattern().isEmpty()) {
-            throw new TezUncheckedException("LocalResource type set to pattern"
-              + " but pattern is null or empty");
-          }
-          localResourcesBuilder.setPattern(lr.getPattern());
-        }
-        taskConfigBuilder.addLocalResource(localResourcesBuilder);
+      if (!vertexLRs.isEmpty()) {
+        taskConfigBuilder.addAllLocalResource(DagTypeConverters.convertToDAGPlan(vertexLRs));
       }
-      
-      for (String key : vertex.getTaskEnvironment().keySet()) {
+
+      Map<String, String> taskEnv = Maps.newHashMap(vertex.getTaskEnvironment());
+      TezYARNUtils.setupDefaultEnv(taskEnv, dagConf,
+          TezConfiguration.TEZ_TASK_LAUNCH_ENV,
+          TezConfiguration.TEZ_TASK_LAUNCH_ENV_DEFAULT, tezLrsAsArchive);
+      for (Map.Entry<String, String> entry : taskEnv.entrySet()) {
         PlanKeyValuePair.Builder envSettingBuilder = PlanKeyValuePair.newBuilder();
-        envSettingBuilder.setKey(key);
-        envSettingBuilder.setValue(vertex.getTaskEnvironment().get(key));
+        envSettingBuilder.setKey(entry.getKey());
+        envSettingBuilder.setValue(entry.getValue());
         taskConfigBuilder.addEnvironmentSetting(envSettingBuilder);
       }
 
-      if (vertex.getLocationHint() != null) {
-        if (vertex.getLocationHint().getTaskLocationHints() != null) {
-          for (TaskLocationHint hint : vertex.getLocationHint().getTaskLocationHints()) {
+      if (vertexLocationHint != null) {
+        if (vertexLocationHint.getTaskLocationHints() != null) {
+          for (TaskLocationHint hint : vertexLocationHint.getTaskLocationHints()) {
             PlanTaskLocationHint.Builder taskLocationHintBuilder = PlanTaskLocationHint.newBuilder();
 
             if (hint.getAffinitizedContainer() != null) {
@@ -788,12 +799,11 @@
     dagBuilder.setDagKeyValues(confProtoBuilder); // This does not seem to be used anywhere
     // should this replace BINARY_PB_CONF???
 
-    if (credentials != null) {
-      dagBuilder.setCredentialsBinary(DagTypeConverters.convertCredentialsToProto(credentials));
-      TezCommonUtils.logCredentials(LOG, credentials, "dag");
+    if (dagCredentials != null) {
+      dagBuilder.setCredentialsBinary(DagTypeConverters.convertCredentialsToProto(dagCredentials));
+      TezCommonUtils.logCredentials(LOG, dagCredentials, "dag");
     }
     
-    cachedDAGPlan = dagBuilder.build();
-    return cachedDAGPlan;
+    return dagBuilder.build();
   }
 }
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
index 6594f02..179f3cc 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
@@ -28,6 +28,7 @@
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.Map.Entry;
 
 import javax.annotation.Nullable;
 
@@ -72,6 +73,7 @@
 import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
 import org.apache.tez.dag.api.records.DAGProtos.VertexLocationHintProto;
 
+import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ByteString.Output;
 
@@ -86,6 +88,33 @@
       default : throw new RuntimeException("unknown 'visibility': " + visibility);
     }
   }
+  
+  public static List<PlanLocalResource> convertToDAGPlan(Map<String, LocalResource> lrs) {
+    List<PlanLocalResource> planLrs = Lists.newArrayListWithCapacity(lrs.size());
+    for (Entry<String, LocalResource> entry : lrs.entrySet()) {
+      PlanLocalResource.Builder localResourcesBuilder = PlanLocalResource.newBuilder();
+      String key = entry.getKey();
+      LocalResource lr = entry.getValue();
+      localResourcesBuilder.setName(key);
+      localResourcesBuilder.setUri(
+        DagTypeConverters.convertToDAGPlan(lr.getResource()));
+      localResourcesBuilder.setSize(lr.getSize());
+      localResourcesBuilder.setTimeStamp(lr.getTimestamp());
+      localResourcesBuilder.setType(
+        DagTypeConverters.convertToDAGPlan(lr.getType()));
+      localResourcesBuilder.setVisibility(
+        DagTypeConverters.convertToDAGPlan(lr.getVisibility()));
+      if (lr.getType() == LocalResourceType.PATTERN) {
+        if (lr.getPattern() == null || lr.getPattern().isEmpty()) {
+          throw new TezUncheckedException("LocalResource type set to pattern"
+            + " but pattern is null or empty");
+        }
+        localResourcesBuilder.setPattern(lr.getPattern());
+      }
+      planLrs.add(localResourcesBuilder.build());
+    }
+    return planLrs;
+  }
 
   public static LocalResourceVisibility convertFromDAGPlan(PlanLocalResourceVisibility visibility){
     switch(visibility){
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java
index bc43c88..4d0d615 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java
@@ -18,11 +18,18 @@
 
 package org.apache.tez.dag.api;
 
+import java.net.URI;
+import java.util.Collection;
+import java.util.Collections;
+
 import javax.annotation.Nullable;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.security.Credentials;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+
 /**
  * Defines the output and output committer for a data sink 
  *
@@ -33,6 +40,7 @@
   private final OutputCommitterDescriptor committerDescriptor;
   
   private final Credentials credentials;
+  private final Collection<URI> urisForCredentials = Sets.newHashSet();
 
   /**
    * Create a {@link DataSinkDescriptor}
@@ -83,14 +91,54 @@
     return new DataSinkDescriptor(outputDescriptor, committerDescriptor, credentials);
   }
   
+  /**
+   * Get the {@link OutputDescriptor} for this {@link DataSinkDescriptor}
+   * @return {@link OutputDescriptor}
+   */
   public OutputDescriptor getOutputDescriptor() {
     return outputDescriptor;
   }
   
+  /**
+   * Get the {@link OutputCommitterDescriptor} for this {@link DataSinkDescriptor}
+   * @return {@link OutputCommitterDescriptor}
+   */
   public @Nullable OutputCommitterDescriptor getOutputCommitterDescriptor() {
     return committerDescriptor;
   }
   
+  /** 
+  * This method can be used to specify a list of URIs for which Credentials
+  * need to be obtained so that the job can run. An incremental list of URIs
+  * can be provided by making multiple calls to the method.
+  * 
+  * Currently, @{link credentials} can only be fetched for HDFS and other
+  * {@link org.apache.hadoop.fs.FileSystem} implementations that support
+  * credentials.
+  * 
+  * @param uris
+  *          a list of {@link URI}s
+  * @return this
+  */
+  public synchronized DataSinkDescriptor addURIsForCredentials(Collection<URI> uris) {
+    Preconditions.checkNotNull(uris, "URIs cannot be null");
+    urisForCredentials.addAll(uris);
+    return this;
+  }
+  
+  /**
+   * Get the URIs for which credentials will be obtained
+   * @return an unmodifiable list representing the URIs for which credentials
+   *         are required.
+   */
+  public Collection<URI> getURIsForCredentials() {
+    return Collections.unmodifiableCollection(urisForCredentials);
+  }
+
+  /**
+   * Get the {@link Credentials} for this {@link DataSinkDescriptor}
+   * @return {@link Credentials}
+   */
   public @Nullable Credentials getCredentials() {
     return credentials;
   }
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DataSourceDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/DataSourceDescriptor.java
index 78ddefa..1c5c16d 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DataSourceDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DataSourceDescriptor.java
@@ -20,9 +20,13 @@
 
 import javax.annotation.Nullable;
 
+import java.net.URI;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.LocalResource;
@@ -30,6 +34,9 @@
 import org.apache.tez.runtime.api.InputInitializer;
 import org.apache.tez.runtime.api.events.InputDataInformationEvent;
 
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+
 /**
  * Defines the input and input initializer for a data source 
  *
@@ -43,6 +50,7 @@
   private final int numShards;
   private final VertexLocationHint locationHint;
   private final Map<String, LocalResource> additionalLocalFiles;
+  private final Collection<URI> urisForCredentials = Sets.newHashSet();
 
   private DataSourceDescriptor(InputDescriptor inputDescriptor,
                                @Nullable InputInitializerDescriptor initializerDescriptor,
@@ -119,14 +127,50 @@
         locationHint, additionalLocalFiles);
   }
 
+  /**
+   * Get the {@link InputDescriptor} for this {@link DataSourceDescriptor} 
+   * @return {@link InputDescriptor}
+   */
   public InputDescriptor getInputDescriptor() {
     return inputDescriptor;
   }
   
+  /**
+   * Get the {@link InputInitializerDescriptor} for this {@link DataSourceDescriptor}
+   * @return {@link InputInitializerDescriptor}
+   */
   public @Nullable InputInitializerDescriptor getInputInitializerDescriptor() {
     return initializerDescriptor;
   }
   
+  /** 
+  * This method can be used to specify a list of URIs for which Credentials
+  * need to be obtained so that the job can run. An incremental list of URIs
+  * can be provided by making multiple calls to the method.
+  * 
+  * Currently, @{link credentials} can only be fetched for HDFS and other
+  * {@link org.apache.hadoop.fs.FileSystem} implementations that support
+  * credentials.
+  * 
+  * @param uris
+  *          a list of {@link URI}s
+  * @return this
+  */
+  public synchronized DataSourceDescriptor addURIsForCredentials(Collection<URI> uris) {
+    Preconditions.checkNotNull(uris, "URIs cannot be null");
+    urisForCredentials.addAll(uris);
+    return this;
+  }
+  
+  /**
+   * Get the URIs for which credentials will be obtained
+   * @return an unmodifiable list representing the URIs for which credentials
+   *         are required.
+   */
+  public Collection<URI> getURIsForCredentials() {
+    return Collections.unmodifiableCollection(urisForCredentials);
+  }
+  
   /**
    * Number of shards for this data source. If a vertex has only one
    * data source this the number of tasks in the vertex should be set to 
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
index 0dffbef..04acdaf 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
@@ -25,6 +25,7 @@
 import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -440,11 +441,13 @@
     this.taskResource = resource;
   }
 
-  List<DataSourceDescriptor> getDataSources() {
+  @Private
+  public List<DataSourceDescriptor> getDataSources() {
     return dataSources;
   }
   
-  List<DataSinkDescriptor> getDataSinks() {
+  @Private
+  public List<DataSinkDescriptor> getDataSinks() {
     return dataSinks;
   }
 
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
index 297c6f2..7c51c42 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
@@ -170,7 +170,7 @@
       + ", diagnostics="
       + StringUtils.join(getDiagnostics(), LINE_SEPARATOR)
       + ", counters="
-      + (dagCounters == null ? "null" : dagCounters.toString()));
+      + (getDAGCounters() == null ? "null" : getDAGCounters().toString()));
     return sb.toString();
   }
 
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java
index f9045ca..7b22b62 100644
--- a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java
@@ -91,7 +91,7 @@
    * Receive notifications on vertex state changes.
    * <p/>
    * State changes will be received based on the registration via {@link
-   * org.apache.tez.runtime.api.InputInitializerContext#registerForVertexStatusUpdates(String,
+   * org.apache.tez.runtime.api.InputInitializerContext#registerForVertexStateUpdates(String,
    * java.util.Set)}. Notifications will be received for all registered state changes, and not just
    * for the latest state update. They will be in order in which the state change occurred. </p>
    *
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java
index fe82b54..6a123cf 100644
--- a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializerContext.java
@@ -115,6 +115,6 @@
    * @param vertexName the vertex name for which notifications are required.
    * @param stateSet   the set of states for which notifications are required. null implies all
    */
-  void registerForVertexStatusUpdates(String vertexName, @Nullable Set<VertexState> stateSet);
+  void registerForVertexStateUpdates(String vertexName, @Nullable Set<VertexState> stateSet);
 
 }
diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto
index 54cd191..5b1f518 100644
--- a/tez-api/src/main/proto/DAGApiRecords.proto
+++ b/tez-api/src/main/proto/DAGApiRecords.proto
@@ -159,6 +159,7 @@
   optional ConfigurationProto dagKeyValues = 4;
   optional bytes credentials_binary = 5;
   repeated PlanVertexGroupInfo vertex_groups = 6;
+  repeated PlanLocalResource local_resource = 7;
 }
 
 // DAG monitoring messages
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
index 163b042..0ce6dff 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
@@ -161,11 +161,7 @@
         Resource.newInstance(1, 1));
     DAG dag = DAG.create("DAG").addVertex(vertex).addTaskLocalFiles(lrDAG);
     DAGClient dagClient = client.submitDAG(dag);
-    
-    // verify that both DAG and TezClient localResources are added to the vertex
-    Map<String, LocalResource> vertexLR = vertex.getTaskLocalFiles();
-    Assert.assertTrue(vertexLR.containsKey(mockLR1Name));
-    
+        
     Assert.assertTrue(dagClient.getExecutionContext().contains(client.mockAppId.toString()));
     
     if (isSession) {
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
index ddd5963..a802041 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClientUtils.java
@@ -24,6 +24,7 @@
 
 import java.io.File;
 import java.io.FileNotFoundException;
+import java.net.URI;
 import java.net.URL;
 import java.net.URLClassLoader;
 import java.util.HashMap;
@@ -73,9 +74,8 @@
     File emptyDir = new File(TEST_ROOT_DIR, "emptyDir");
     emptyDir.deleteOnExit();
     Assert.assertTrue(emptyDir.mkdirs());
-    Path emptyDirPath = new Path(emptyDir.getAbsolutePath());
     TezConfiguration conf = new TezConfiguration();
-    conf.set(TezConfiguration.TEZ_LIB_URIS, "file://" + emptyDirPath.toString());
+    conf.set(TezConfiguration.TEZ_LIB_URIS, emptyDir.toURI().toURL().toString());
     Credentials credentials = new Credentials();
     try {
       TezClientUtils.setupTezJarsLocalResources(conf, credentials);
diff --git a/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java
index 7ad4ede..bc35b51 100644
--- a/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java
+++ b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java
@@ -23,6 +23,7 @@
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezConstants;
@@ -33,138 +34,141 @@
 
 public class TestACLManager {
 
-  private static final Set<String> noGroups = Sets.newHashSet();
+  private static final String[] noGroups = new String[0];
 
   @Test
   public void testCurrentUserACLChecks() {
-    String currentUser = "currentUser";
-    ACLManager aclManager = new ACLManager(currentUser);
+    UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+    UserGroupInformation dagUser = UserGroupInformation.createUserForTesting("dagUser", noGroups);
+    UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", noGroups);
 
-    String user = "user1";
-    Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+    ACLManager aclManager = new ACLManager(currentUser.getShortUserName());
+
+    UserGroupInformation user = user1;
+
+    Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
 
     user = currentUser;
-    Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
 
-    aclManager = new ACLManager(currentUser, new Configuration(false));
+    aclManager = new ACLManager(currentUser.getShortUserName(), new Configuration(false));
 
-    user = "user1";
-    Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+    user = user1;
+    Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
 
     user = currentUser;
-    Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
 
-    String dagUser = "dagUser";
-    ACLManager dagAclManager = new ACLManager(aclManager, dagUser, new Configuration(false));
+    ACLManager dagAclManager = new ACLManager(aclManager, dagUser.getShortUserName(), new Configuration(false));
     user = dagUser;
-    Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
-    Assert.assertTrue(dagAclManager.checkAccess(user, null, ACLType.DAG_VIEW_ACL));
-    Assert.assertTrue(dagAclManager.checkAccess(user, null, ACLType.DAG_MODIFY_ACL));
-    user = "user1";
-    Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.DAG_VIEW_ACL));
-    Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.DAG_MODIFY_ACL));
+    Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(dagAclManager.checkAccess(user, ACLType.DAG_VIEW_ACL));
+    Assert.assertTrue(dagAclManager.checkAccess(user, ACLType.DAG_MODIFY_ACL));
+    user = user1;
+    Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.DAG_VIEW_ACL));
+    Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.DAG_MODIFY_ACL));
   }
 
   @Test
   public void testOtherUserACLChecks() throws IOException {
-    Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
-    Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
-    Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
+    String[] groups1 = new String[] {"grp1", "grp2"};
+    String[] groups2 = new String[] {"grp3", "grp4"};
+    String[] groups3 = new String[] {"grp5", "grp6"};
 
-    String currentUser = "currentUser";
-    String user1 = "user1"; // belongs to grp1 and grp2
-    String user2 = "user2"; // belongs to grp3 and grp4
-    String user3 = "user3";
-    String user4 = "user4";
-    String user5 = "user5"; // belongs to grp5 and grp6
-    String user6 = "user6";
+    UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+    UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+    UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+    UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+    UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+    UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+    UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
 
     Configuration conf = new Configuration(false);
     // View ACLs: user1, user4, grp3, grp4.
-    String viewACLs = user1 + "," + user4
+    String viewACLs = user1.getShortUserName() + "," + user4.getShortUserName()
         + "   " + "grp3,grp4  ";
     // Modify ACLs: user3, grp6, grp7
-    String modifyACLs = user3 + "  " + "grp6,grp7";
+    String modifyACLs = user3.getShortUserName() + "  " + "grp6,grp7";
     conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
     conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
 
-    ACLManager aclManager = new ACLManager(currentUser, conf);
+    ACLManager aclManager = new ACLManager(currentUser.getShortUserName(), conf);
 
-    Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user1, groups1, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user2, groups2, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user3, noGroups, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user4, noGroups, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user5, groups3, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user1, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user2, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user3, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user4, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user5,  ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_VIEW_ACL));
 
-    Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user1, groups1, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user2, groups2, ACLType.AM_MODIFY_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user3, noGroups, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user4, noGroups, ACLType.AM_MODIFY_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user5, groups3, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user1, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user3, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user4, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user5, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_MODIFY_ACL));
   }
 
   @Test
   public void testNoGroupsACLChecks() throws IOException {
-    Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
-    Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
-    Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
+    String[] groups1 = new String[] {"grp1", "grp2"};
+    String[] groups2 = new String[] {"grp3", "grp4"};
+    String[] groups3 = new String[] {"grp5", "grp6"};
 
-    String currentUser = "currentUser";
-    String user1 = "user1"; // belongs to grp1 and grp2
-    String user2 = "user2"; // belongs to grp3 and grp4
-    String user3 = "user3";
-    String user4 = "user4";
-    String user5 = "user5"; // belongs to grp5 and grp6
-    String user6 = "user6";
+    UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+    UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+    UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+    UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+    UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+    UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+    UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
 
     Configuration conf = new Configuration(false);
     // View ACLs: user1, user4
-    String viewACLs = user1 + "," + user4 + " ";
+    String viewACLs = user1.getShortUserName() + "," + user4.getShortUserName() + " ";
     // Modify ACLs: user3
-    String modifyACLs = "user3  ";
+    String modifyACLs = user3.getShortUserName() + " ";
     conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
     conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
 
-    ACLManager aclManager = new ACLManager(currentUser, conf);
-    Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user1, groups1, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user2, groups2, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user3, noGroups, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user4, noGroups, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user5, groups3, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_VIEW_ACL));
+    ACLManager aclManager = new ACLManager(currentUser.getShortUserName(), conf);
+    Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user1, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user3, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user4, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user5, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_VIEW_ACL));
 
-    Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user1, groups1, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user2, groups2, ACLType.AM_MODIFY_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user3, noGroups, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user4, noGroups, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user5, groups3, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user1, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user3, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user4, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user5, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_MODIFY_ACL));
   }
 
   @Test
   public void checkAMACLs() throws IOException {
-    Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
-    Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
-    Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
+    String[] groups1 = new String[] {"grp1", "grp2"};
+    String[] groups2 = new String[] {"grp3", "grp4"};
+    String[] groups3 = new String[] {"grp5", "grp6"};
 
-    String currentUser = "currentUser";
-    String user1 = "user1"; // belongs to grp1 and grp2
-    String user2 = "user2"; // belongs to grp3 and grp4
-    String user3 = "user3";
-    String user4 = "user4";
-    String user5 = "user5"; // belongs to grp5 and grp6
-    String user6 = "user6";
+    UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+    UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+    UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+    UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+    UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+    UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+    UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
 
     Configuration conf = new Configuration(false);
     // View ACLs: user1, user4, grp3, grp4.
@@ -174,55 +178,55 @@
     conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
     conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
 
-    ACLManager aclManager = new ACLManager(currentUser, conf);
+    ACLManager aclManager = new ACLManager(currentUser.getShortUserName(), conf);
 
-    Assert.assertTrue(aclManager.checkAMViewAccess(currentUser, null));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user1, groups1));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user2, groups2));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user3, noGroups));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user4, noGroups));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user6, noGroups));
+    Assert.assertTrue(aclManager.checkAMViewAccess(currentUser));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user1));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user2));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user3));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user4));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user5));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user6));
 
-    Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser, null));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user1, groups1));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user2, groups2));
-    Assert.assertTrue(aclManager.checkAMModifyAccess(user3, noGroups));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user4, noGroups));
-    Assert.assertTrue(aclManager.checkAMModifyAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user6, noGroups));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user1));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user2));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(user3));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user4));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(user5));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user6));
 
-    Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser, null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user1, groups1));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user2, groups2));
-    Assert.assertFalse(aclManager.checkDAGViewAccess(user3, noGroups));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user4, noGroups));
-    Assert.assertFalse(aclManager.checkDAGViewAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkDAGViewAccess(user6, noGroups));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user2));
+    Assert.assertFalse(aclManager.checkDAGViewAccess(user3));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user4));
+    Assert.assertFalse(aclManager.checkDAGViewAccess(user5));
+    Assert.assertFalse(aclManager.checkDAGViewAccess(user6));
 
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser, null));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user1, groups1));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user2, groups2));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(user3, noGroups));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user4, noGroups));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user6, noGroups));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user1));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user2));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(user3));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user4));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(user5));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user6));
 
   }
 
   @Test
   public void checkDAGACLs() throws IOException {
-    Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
-    Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
-    Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
+    String[] groups1 = new String[] {"grp1", "grp2"};
+    String[] groups2 = new String[] {"grp3", "grp4"};
+    String[] groups3 = new String[] {"grp5", "grp6"};
 
-    String currentUser = "currentUser";
-    String user1 = "user1"; // belongs to grp1 and grp2
-    String user2 = "user2"; // belongs to grp3 and grp4
-    String user3 = "user3";
-    String user4 = "user4";
-    String user5 = "user5"; // belongs to grp5 and grp6
-    String user6 = "user6";
+    UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+    UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+    UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+    UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+    UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+    UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+    UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
 
     Configuration conf = new Configuration(false);
     // View ACLs: user1, user4, grp3, grp4.
@@ -239,46 +243,46 @@
     conf.set(TezConstants.TEZ_DAG_VIEW_ACLS, dagViewACLs);
     conf.set(TezConstants.TEZ_DAG_MODIFY_ACLS, dagModifyACLs);
 
-    String dagUser = "dagUser";
+    UserGroupInformation dagUser = UserGroupInformation.createUserForTesting("dagUser", noGroups);
 
-    ACLManager amAclManager = new ACLManager(currentUser, conf);
-    ACLManager aclManager = new ACLManager(amAclManager, dagUser, conf);
+    ACLManager amAclManager = new ACLManager(currentUser.getShortUserName(), conf);
+    ACLManager aclManager = new ACLManager(amAclManager, dagUser.getShortUserName(), conf);
 
-    Assert.assertTrue(aclManager.checkAMViewAccess(currentUser, null));
-    Assert.assertFalse(aclManager.checkAMViewAccess(dagUser, null));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user1, groups1));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user2, groups2));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user3, noGroups));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user4, noGroups));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user6, noGroups));
+    Assert.assertTrue(aclManager.checkAMViewAccess(currentUser));
+    Assert.assertFalse(aclManager.checkAMViewAccess(dagUser));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user1));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user2));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user3));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user4));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user5));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user6));
 
-    Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser, null));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(dagUser, null));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user1, groups1));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user2, groups2));
-    Assert.assertTrue(aclManager.checkAMModifyAccess(user3, noGroups));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user4, noGroups));
-    Assert.assertTrue(aclManager.checkAMModifyAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user6, noGroups));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(dagUser));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user1));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user2));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(user3));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user4));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(user5));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user6));
 
-    Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser, null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(dagUser, null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user1, groups1));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user2, groups2));
-    Assert.assertFalse(aclManager.checkDAGViewAccess(user3, noGroups));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user4, noGroups));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user5, groups3));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user6, noGroups));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(dagUser));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user2));
+    Assert.assertFalse(aclManager.checkDAGViewAccess(user3));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user4));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user5));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user6));
 
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser, null));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(dagUser, null));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user1, groups1));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user2, groups2));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(user3, noGroups));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user4, noGroups));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(user5, groups3));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(user6, noGroups));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(dagUser));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user1));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user2));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(user3));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user4));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(user5));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(user6));
 
   }
 
@@ -290,15 +294,18 @@
     conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
     conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
 
-    ACLManager aclManager = new ACLManager("a1", conf);
-    Assert.assertTrue(aclManager.checkAMViewAccess("a1", null));
-    Assert.assertTrue(aclManager.checkAMViewAccess("u1", null));
-    Assert.assertTrue(aclManager.checkAMModifyAccess("a1", null));
-    Assert.assertTrue(aclManager.checkAMModifyAccess("u1", null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess("a1", null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess("u1", null));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess("a1", null));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess("u1", null));
+    UserGroupInformation a1 = UserGroupInformation.createUserForTesting("a1", noGroups);
+    UserGroupInformation u1 = UserGroupInformation.createUserForTesting("u1", noGroups);
+
+    ACLManager aclManager = new ACLManager(a1.getShortUserName(), conf);
+    Assert.assertTrue(aclManager.checkAMViewAccess(a1));
+    Assert.assertTrue(aclManager.checkAMViewAccess(u1));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(a1));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(u1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(a1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(u1));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(a1));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(u1));
   }
 
   @Test
@@ -309,25 +316,29 @@
     String modifyACLs = "a2,u2 ";
     conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
     conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
-    ACLManager aclManager = new ACLManager("a1", conf);
-    Assert.assertTrue(aclManager.checkAMViewAccess("a1", null));
-    Assert.assertTrue(aclManager.checkAMViewAccess("u1", null));
-    Assert.assertTrue(aclManager.checkAMModifyAccess("a1", null));
-    Assert.assertTrue(aclManager.checkAMModifyAccess("u1", null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess("a1", null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess("u1", null));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess("a1", null));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess("u1", null));
+
+    UserGroupInformation a1 = UserGroupInformation.createUserForTesting("a1", noGroups);
+    UserGroupInformation u1 = UserGroupInformation.createUserForTesting("u1", noGroups);
+
+    ACLManager aclManager = new ACLManager(a1.getShortUserName(), conf);
+    Assert.assertTrue(aclManager.checkAMViewAccess(a1));
+    Assert.assertTrue(aclManager.checkAMViewAccess(u1));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(a1));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(u1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(a1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(u1));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(a1));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(u1));
 
     ACLManager dagAclManager = new ACLManager(aclManager, "dagUser", null);
-    Assert.assertTrue(dagAclManager.checkAMViewAccess("a1", null));
-    Assert.assertTrue(dagAclManager.checkAMViewAccess("u1", null));
-    Assert.assertTrue(dagAclManager.checkAMModifyAccess("a1", null));
-    Assert.assertTrue(dagAclManager.checkAMModifyAccess("u1", null));
-    Assert.assertTrue(dagAclManager.checkDAGViewAccess("a1", null));
-    Assert.assertTrue(dagAclManager.checkDAGViewAccess("u1", null));
-    Assert.assertTrue(dagAclManager.checkDAGModifyAccess("a1", null));
-    Assert.assertTrue(dagAclManager.checkDAGModifyAccess("u1", null));
+    Assert.assertTrue(dagAclManager.checkAMViewAccess(a1));
+    Assert.assertTrue(dagAclManager.checkAMViewAccess(u1));
+    Assert.assertTrue(dagAclManager.checkAMModifyAccess(a1));
+    Assert.assertTrue(dagAclManager.checkAMModifyAccess(u1));
+    Assert.assertTrue(dagAclManager.checkDAGViewAccess(a1));
+    Assert.assertTrue(dagAclManager.checkDAGViewAccess(u1));
+    Assert.assertTrue(dagAclManager.checkDAGModifyAccess(a1));
+    Assert.assertTrue(dagAclManager.checkDAGModifyAccess(u1));
   }
 
   @Test
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
index 22fe3a1..8cbd611 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
@@ -123,7 +123,7 @@
 
     dag.addVertex(v1).addVertex(v2).addEdge(edge);
 
-    DAGPlan dagProto = dag.createDag(new TezConfiguration());
+    DAGPlan dagProto = dag.createDag(new TezConfiguration(), null, null, null, true);
 
     EdgeProperty edgeProperty = DagTypeConverters.createEdgePropertyMapFromDAGPlan(dagProto
         .getEdgeList().get(0));
@@ -158,7 +158,7 @@
 
     dag.addVertex(v1).addVertex(v2).addEdge(edge);
 
-    DAGPlan dagProto = dag.createDag(new TezConfiguration());
+    DAGPlan dagProto = dag.createDag(new TezConfiguration(), null, null, null, true);
 
     assertEquals(2, dagProto.getVertexCount());
     assertEquals(1, dagProto.getEdgeCount());
@@ -224,7 +224,7 @@
 
     dag.addVertex(v1).addVertex(v2).addEdge(edge).addVertex(v3);
 
-    DAGPlan dagProto = dag.createDag(new TezConfiguration());
+    DAGPlan dagProto = dag.createDag(new TezConfiguration(), null, null, null, true);
 
     assertEquals(3, dagProto.getVertexCount());
     assertEquals(1, dagProto.getEdgeCount());
@@ -300,7 +300,7 @@
     
     dag.setCredentials(dagCredentials);
 
-    DAGPlan dagProto = dag.createDag(new TezConfiguration());
+    DAGPlan dagProto = dag.createDag(new TezConfiguration(), null, null, null, true);
 
     assertTrue(dagProto.hasCredentialsBinary());
     
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
index ba89a47..0697584 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
@@ -19,7 +19,9 @@
 package org.apache.tez.dag.api;
 
 import java.util.Arrays;
+import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.nio.ByteBuffer;
 
 import org.apache.hadoop.conf.Configuration;
@@ -35,10 +37,14 @@
 import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
 import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
 import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
+import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 public class TestDAGVerify {
 
@@ -901,51 +907,51 @@
 
     dag.addVertex(v1);
 
-    dag.createDag(new TezConfiguration());
+    dag.createDag(new TezConfiguration(), null, null, null, true);
   }
   
   
   @Test(timeout = 5000)
-  public void testVerifyCommonFiles() {
-    Vertex v1 = Vertex.create("v1",
-        ProcessorDescriptor.create(dummyProcessorClassName),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = Vertex.create("v2",
-        ProcessorDescriptor.create("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = Edge.create(v1, v2,
-        EdgeProperty.create(DataMovementType.SCATTER_GATHER,
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL,
-            OutputDescriptor.create(dummyOutputClassName),
-            InputDescriptor.create(dummyInputClassName)));
-    Map<String, LocalResource> lrs = Maps.newHashMap();
+  public void testDAGCreateDataInference() {
+    Vertex v1 = Vertex.create("v1", ProcessorDescriptor.create(dummyProcessorClassName));
+    Map<String, LocalResource> lrs1 = Maps.newHashMap();
     String lrName1 = "LR1";
-    lrs.put(lrName1, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test"),
+    lrs1.put(lrName1, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test"),
+        LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1));
+    Map<String, LocalResource> lrs2 = Maps.newHashMap();
+    String lrName2 = "LR2";
+    lrs2.put(lrName2, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test1"),
         LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1));
     
+    Set<String> hosts = Sets.newHashSet();
+    hosts.add("h1");
+    hosts.add("h2");
+    List<TaskLocationHint> taskLocationHints = Lists.newLinkedList();
+    taskLocationHints.add(TaskLocationHint.createTaskLocationHint(hosts, null));
+    taskLocationHints.add(TaskLocationHint.createTaskLocationHint(hosts, null));
+    VertexLocationHint vLoc = VertexLocationHint.create(taskLocationHints);
+    DataSourceDescriptor ds = DataSourceDescriptor.create(InputDescriptor.create("I.class"), 
+        null, dummyTaskCount, null, vLoc, lrs2);
+    v1.addDataSource("i1", ds);
+        
     DAG dag = DAG.create("testDag");
     dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addEdge(e1);
-    dag.addTaskLocalFiles(lrs);
-    dag.createDag(new TezConfiguration());
-    Assert.assertTrue(v1.getTaskLocalFiles().containsKey(lrName1));
-    Assert.assertTrue(v2.getTaskLocalFiles().containsKey(lrName1));
+    dag.addTaskLocalFiles(lrs1);
+    DAGPlan dagPlan = dag.createDag(new TezConfiguration(), null, null, null, true);
+    Assert.assertEquals(lrName1, dagPlan.getLocalResource(0).getName());
+    VertexPlan vPlan = dagPlan.getVertex(0);
+    PlanTaskConfiguration taskPlan = vPlan.getTaskConfig();
+    Assert.assertEquals(dummyTaskCount, taskPlan.getNumTasks());
+    Assert.assertEquals(TezConfiguration.TEZ_TASK_RESOURCE_MEMORY_MB_DEFAULT, taskPlan.getMemoryMb());
+    Assert.assertEquals(lrName2, taskPlan.getLocalResource(0).getName());
+    Assert.assertEquals(dummyTaskCount, vPlan.getTaskLocationHintCount());
   }
 
   @Test(timeout = 5000)
-  public void testVerifyCommonFilesFail() {
+  public void testInferredFilesFail() {
     Vertex v1 = Vertex.create("v1",
         ProcessorDescriptor.create(dummyProcessorClassName),
         dummyTaskCount, dummyTaskResource);
-    Vertex v2 = Vertex.create("v2",
-        ProcessorDescriptor.create("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = Edge.create(v1, v2,
-        EdgeProperty.create(DataMovementType.SCATTER_GATHER,
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL,
-            OutputDescriptor.create(dummyOutputClassName),
-            InputDescriptor.create(dummyInputClassName)));
     Map<String, LocalResource> lrs = Maps.newHashMap();
     String lrName1 = "LR1";
     lrs.put(lrName1, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test"),
@@ -957,10 +963,13 @@
     } catch (TezUncheckedException e) {
       Assert.assertTrue(e.getMessage().contains("Attempting to add duplicate resource"));
     }
+
+    DataSourceDescriptor ds = DataSourceDescriptor.create(InputDescriptor.create("I.class"), 
+        null, -1, null, null, lrs);
+    v1.addDataSource("i1", ds);
+    
     DAG dag = DAG.create("testDag");
     dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addEdge(e1);
     dag.addTaskLocalFiles(lrs);
     try {
       dag.addTaskLocalFiles(lrs);
@@ -969,8 +978,8 @@
       Assert.assertTrue(e.getMessage().contains("Attempting to add duplicate resource"));
     }
     try {
-      // dag will add duplicate common files to vertex
-      dag.createDag(new TezConfiguration());
+      // data source will add duplicate common files to vertex
+      dag.createDag(new TezConfiguration(), null, null, null, true);
       Assert.fail();
     } catch (TezUncheckedException e) {
       Assert.assertTrue(e.getMessage().contains("Attempting to add duplicate resource"));
@@ -993,7 +1002,7 @@
     dag.setAccessControls(dagAccessControls);
 
     Configuration conf = new Configuration(false);
-    DAGPlan dagPlan = dag.createDag(conf);
+    DAGPlan dagPlan = dag.createDag(conf, null, null, null, true);
     Assert.assertNull(conf.get(TezConstants.TEZ_DAG_VIEW_ACLS));
     Assert.assertNull(conf.get(TezConstants.TEZ_DAG_MODIFY_ACLS));
 
diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
index 1ce6c55..bd7089f 100644
--- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
+++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
@@ -30,6 +30,7 @@
 import org.apache.hadoop.yarn.client.api.YarnClientApplication;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -54,6 +55,8 @@
 import org.apache.tez.dag.app.DAGAppMasterState;
 import org.apache.tez.dag.app.dag.DAG;
 
+import com.google.common.annotations.VisibleForTesting;
+
 public class LocalClient extends FrameworkClient {
   public static final Logger LOG = Logger.getLogger(LocalClient.class);
 
@@ -286,8 +289,7 @@
           int nmHttpPort = YarnConfiguration.DEFAULT_NM_WEBAPP_PORT;
           long appSubmitTime = System.currentTimeMillis();
 
-          dagAppMaster =
-              new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
+          dagAppMaster = createDAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
                   new SystemClock(),
                   appSubmitTime, isSession, userDir.toUri().getPath());
           clientHandler = new DAGClientHandler(dagAppMaster);
@@ -305,4 +307,13 @@
 
     return thread;
   }
+  
+  // this can be overridden by test code to create a mock app
+  @VisibleForTesting
+  protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemptId,
+      ContainerId cId, String currentHost, int nmPort, int nmHttpPort,
+      Clock clock, long appSubmitTime, boolean isSession, String userDir) {
+    return new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
+        new SystemClock(), appSubmitTime, isSession, userDir);
+  }
 }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
index 6381b71..c054305 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
@@ -62,17 +62,9 @@
     this.real = real;
   }
 
-  private String getRPCUserName() throws ServiceException {
+  private UserGroupInformation getRPCUser() throws ServiceException {
     try {
-      return UserGroupInformation.getCurrentUser().getShortUserName();
-    } catch (IOException e) {
-      throw wrapException(e);
-    }
-  }
-
-  private List<String> getRPCUserGroups() throws ServiceException {
-    try {
-      return Arrays.asList(UserGroupInformation.getCurrentUser().getGroupNames());
+      return UserGroupInformation.getCurrentUser();
     } catch (IOException e) {
       throw wrapException(e);
     }
@@ -81,8 +73,8 @@
   @Override
   public GetAllDAGsResponseProto getAllDAGs(RpcController controller,
       GetAllDAGsRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
-    if (!real.getACLManager().checkAMViewAccess(user, getRPCUserGroups())) {
+    UserGroupInformation user = getRPCUser();
+    if (!real.getACLManager().checkAMViewAccess(user)) {
       throw new AccessControlException("User " + user + " cannot perform AM view operation");
     }
     try{
@@ -96,10 +88,10 @@
   @Override
   public GetDAGStatusResponseProto getDAGStatus(RpcController controller,
       GetDAGStatusRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
+    UserGroupInformation user = getRPCUser();
     try {
       String dagId = request.getDagId();
-      if (!real.getACLManager(dagId).checkDAGViewAccess(user, getRPCUserGroups())) {
+      if (!real.getACLManager(dagId).checkDAGViewAccess(user)) {
         throw new AccessControlException("User " + user + " cannot perform DAG view operation");
       }
       DAGStatus status;
@@ -118,10 +110,10 @@
   @Override
   public GetVertexStatusResponseProto getVertexStatus(RpcController controller,
       GetVertexStatusRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
+    UserGroupInformation user = getRPCUser();
     try {
       String dagId = request.getDagId();
-      if (!real.getACLManager(dagId).checkDAGViewAccess(user, getRPCUserGroups())) {
+      if (!real.getACLManager(dagId).checkDAGViewAccess(user)) {
         throw new AccessControlException("User " + user + " cannot perform DAG view operation");
       }
       String vertexName = request.getVertexName();
@@ -140,10 +132,10 @@
   @Override
   public TryKillDAGResponseProto tryKillDAG(RpcController controller,
       TryKillDAGRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
+    UserGroupInformation user = getRPCUser();
     try {
       String dagId = request.getDagId();
-      if (!real.getACLManager(dagId).checkDAGModifyAccess(user, getRPCUserGroups())) {
+      if (!real.getACLManager(dagId).checkDAGModifyAccess(user)) {
         throw new AccessControlException("User " + user + " cannot perform DAG modify operation");
       }
       real.tryKillDAG(dagId);
@@ -156,8 +148,8 @@
   @Override
   public SubmitDAGResponseProto submitDAG(RpcController controller,
       SubmitDAGRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
-    if (!real.getACLManager().checkAMModifyAccess(user, getRPCUserGroups())) {
+    UserGroupInformation user = getRPCUser();
+    if (!real.getACLManager().checkAMModifyAccess(user)) {
       throw new AccessControlException("User " + user + " cannot perform AM modify operation");
     }
     try{
@@ -181,8 +173,8 @@
   @Override
   public ShutdownSessionResponseProto shutdownSession(RpcController arg0,
       ShutdownSessionRequestProto arg1) throws ServiceException {
-    String user = getRPCUserName();
-    if (!real.getACLManager().checkAMModifyAccess(user, getRPCUserGroups())) {
+    UserGroupInformation user = getRPCUser();
+    if (!real.getACLManager().checkAMModifyAccess(user)) {
       throw new AccessControlException("User " + user + " cannot perform AM modify operation");
     }
     real.shutdownAM();
@@ -192,8 +184,8 @@
   @Override
   public GetAMStatusResponseProto getAMStatus(RpcController controller,
       GetAMStatusRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
-    if (!real.getACLManager().checkAMViewAccess(user, getRPCUserGroups())) {
+    UserGroupInformation user = getRPCUser();
+    if (!real.getACLManager().checkAMViewAccess(user)) {
       throw new AccessControlException("User " + user + " cannot perform AM view operation");
     }
     try {
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 1b4d18d..e6a1d9c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -156,12 +156,12 @@
 import com.google.common.collect.Maps;
 
 /**
- * The Map-Reduce Application Master.
+ * The Tez DAG 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
+ * Tez DAG 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.
@@ -444,6 +444,11 @@
       System.exit(0);
     }
   }
+  
+  @VisibleForTesting
+  protected TaskSchedulerEventHandler getTaskSchedulerEventHandler() {
+    return taskSchedulerEventHandler;
+  }
 
   private synchronized void handle(DAGAppMasterEvent event) {
     switch (event.getType()) {
@@ -1447,14 +1452,14 @@
     }
     return null;
   }
-
+  
   @Override
   public synchronized void serviceStart() throws Exception {
 
     //start all the components
     startServices();
     super.serviceStart();
-
+    
     // metrics system init is really init & start.
     // It's more test friendly to put it here.
     DefaultMetricsSystem.initialize("DAGAppMaster");
@@ -1896,7 +1901,7 @@
 
     UserGroupInformation.setConfiguration(conf);
     Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
-
+    
     appMaster.appMasterUgi = UserGroupInformation
         .createRemoteUser(jobUserName);
     appMaster.appMasterUgi.addCredentials(credentials);
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
index 4e6a3a1..220b5b5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
@@ -235,18 +235,18 @@
 
 
 
-  private static void parseDAGRecoveryFile(FSDataInputStream inputStream)
+  public static List<HistoryEvent> parseDAGRecoveryFile(FSDataInputStream inputStream)
       throws IOException {
+    List<HistoryEvent> historyEvents = new ArrayList<HistoryEvent>();
     while (true) {
       HistoryEvent historyEvent = getNextEvent(inputStream);
       if (historyEvent == null) {
         LOG.info("Reached end of stream");
         break;
       }
-      LOG.info("Parsed event from recovery stream"
-          + ", eventType=" + historyEvent.getEventType()
-          + ", event=" + historyEvent);
+      historyEvents.add(historyEvent);
     }
+    return historyEvents;
   }
 
   public static void main(String argv[]) throws IOException {
@@ -264,7 +264,12 @@
     parseSummaryFile(fs.open(new Path(summaryPath)));
     for (String dagPath : dagPaths) {
       LOG.info("Parsing DAG recovery file " + dagPath);
-      parseDAGRecoveryFile(fs.open(new Path(dagPath)));
+      List<HistoryEvent> historyEvents = parseDAGRecoveryFile(fs.open(new Path(dagPath)));
+      for (HistoryEvent historyEvent : historyEvents) {
+        LOG.info("Parsed event from recovery stream"
+            + ", eventType=" + historyEvent.getEventType()
+            + ", event=" + historyEvent);
+      }
     }
   }
 
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
index 2fae860..a2f04ab 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
@@ -25,6 +25,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.client.DAGStatusBuilder;
 import org.apache.tez.dag.api.client.StatusGetOpts;
@@ -41,6 +42,7 @@
 public interface DAG {
 
   TezDAGID getID();
+  Map<String, LocalResource> getLocalResources();
   String getName();
   DAGState getState();
   DAGReport getReport();
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
index 24d2e6b..c4e16e2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
@@ -44,6 +44,7 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
@@ -105,6 +106,7 @@
 import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
 import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
 import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TaskSpecificLaunchCmdOption;
 import org.apache.tez.dag.utils.RelocalizationUtils;
@@ -139,7 +141,7 @@
   private final TaskAttemptListener taskAttemptListener;
   private final TaskHeartbeatHandler taskHeartbeatHandler;
   private final Object tasksSyncHandle = new Object();
-  
+
   private volatile boolean committedOrAborted = false;
   private volatile boolean allOutputsCommitted = false;
   boolean commitAllOutputsOnSuccess = true;
@@ -157,7 +159,8 @@
   private final StateChangeNotifier entityUpdateTracker;
 
   volatile Map<TezVertexID, Vertex> vertices = new HashMap<TezVertexID, Vertex>();
-  private Map<String, Edge> edges = new HashMap<String, Edge>();
+  @VisibleForTesting
+  Map<String, Edge> edges = new HashMap<String, Edge>();
   private TezCounters dagCounters = new TezCounters();
   private Object fullCountersLock = new Object();
   private TezCounters fullCounters = null;
@@ -165,6 +168,8 @@
 
   public final Configuration conf;
   private final DAGPlan jobPlan;
+  
+  Map<String, LocalResource> localResources;
 
   private final List<String> diagnostics = new ArrayList<String>();
 
@@ -359,14 +364,18 @@
   private DAGTerminationCause terminationCause;
   private Credentials credentials;
 
-  private long initTime;
-  private long startTime;
-  private long finishTime;
-  
+  @VisibleForTesting
+  long initTime;
+  @VisibleForTesting
+  long startTime;
+  @VisibleForTesting
+  long finishTime;
+
   Map<String, VertexGroupInfo> vertexGroups = Maps.newHashMap();
   Map<String, List<VertexGroupInfo>> vertexGroupInfo = Maps.newHashMap();
   private DAGState recoveredState = DAGState.NEW;
-  private boolean recoveryCommitInProgress = false;
+  @VisibleForTesting
+  boolean recoveryCommitInProgress = false;
   Map<String, Boolean> recoveredGroupCommits = new HashMap<String, Boolean>();
 
   static class VertexGroupInfo {
@@ -381,7 +390,7 @@
       groupMembers = Sets.newHashSet(groupInfo.getGroupMembersList());
       edgeMergedInputs = Maps.newHashMapWithExpectedSize(groupInfo.getEdgeMergedInputsCount());
       for (PlanGroupInputEdgeInfo edgInfo : groupInfo.getEdgeMergedInputsList()) {
-        edgeMergedInputs.put(edgInfo.getDestVertexName(), 
+        edgeMergedInputs.put(edgInfo.getDestVertexName(),
             DagTypeConverters.convertInputDescriptorFromDAGPlan(edgInfo.getMergedInput()));
       }
       outputs = Sets.newHashSet(groupInfo.getOutputsList());
@@ -415,6 +424,9 @@
     ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
     this.readLock = readWriteLock.readLock();
     this.writeLock = readWriteLock.writeLock();
+    
+    this.localResources = DagTypeConverters.createLocalResourceMapFromDAGPlan(jobPlan
+        .getLocalResourceList());
 
     this.credentials = dagCredentials;
     if (this.credentials == null) {
@@ -446,6 +458,11 @@
   public TezDAGID getID() {
     return dagId;
   }
+  
+  @Override
+  public Map<String, LocalResource> getLocalResources() {
+    return localResources;
+  }
 
   // TODO maybe removed after TEZ-74
   @Override
@@ -685,6 +702,11 @@
     }
     return vertex.getVertexStatus(statusOptions);
   }
+  
+  public TaskAttemptImpl getTaskAttempt(TezTaskAttemptID taId) {
+    return (TaskAttemptImpl) getVertex(taId.getTaskID().getVertexID()).getTask(taId.getTaskID())
+        .getAttempt(taId);
+  }
 
   protected void initializeVerticesAndStart() {
     for (Vertex v : vertices.values()) {
@@ -706,7 +728,7 @@
       }
     }
   }
-  
+
   private boolean commitOutput(String outputName, OutputCommitter outputCommitter) {
     final OutputCommitter committer = outputCommitter;
     try {
@@ -723,7 +745,7 @@
     }
     return false;
   }
-  
+
   private synchronized boolean commitOrAbortOutputs(boolean dagSucceeded) {
     if (this.committedOrAborted) {
       LOG.info("Ignoring multiple output commit/abort");
@@ -731,7 +753,7 @@
     }
     LOG.info("Calling DAG commit/abort for dag: " + getID());
     this.committedOrAborted = true;
-    
+
     boolean successfulOutputsAlreadyCommitted = !commitAllOutputsOnSuccess;
     boolean failedWhileCommitting = false;
     if (dagSucceeded && !successfulOutputsAlreadyCommitted) {
@@ -772,7 +794,7 @@
           LOG.info("No output committers for vertex: " + vertex.getName());
           continue;
         }
-        Map<String, OutputCommitter> outputCommitters = 
+        Map<String, OutputCommitter> outputCommitters =
             new HashMap<String, OutputCommitter>(vertex.getOutputCommitters());
         Set<String> sharedOutputs = vertex.getSharedOutputs();
         // remove shared outputs
@@ -793,7 +815,7 @@
           LOG.info("Committing output: " + entry.getKey() + " for vertex: "
               + vertex.getVertexId());
           if (vertex.getState() != VertexState.SUCCEEDED) {
-            throw new TezUncheckedException("Vertex: " + vertex.getName() + 
+            throw new TezUncheckedException("Vertex: " + vertex.getName() +
                 " not in SUCCEEDED state. State= " + vertex.getState());
           }
           if (!commitOutput(entry.getKey(), entry.getValue())) {
@@ -803,11 +825,11 @@
         }
       }
     }
-    
+
     if (failedWhileCommitting) {
       LOG.info("DAG: " + getID() + " failed while committing");
     }
-        
+
     if (!dagSucceeded || failedWhileCommitting) {
       // come here because dag failed or
       // dag succeeded and all or none semantics were on and a commit failed
@@ -1026,9 +1048,9 @@
     if (finishTime == 0) {
       setFinishTime();
     }
-    
+
     boolean allOutputsCommitted = commitOrAbortOutputs(finalState == DAGState.SUCCEEDED);
-    
+
     if (finalState == DAGState.SUCCEEDED && !allOutputsCommitted) {
       finalState = DAGState.FAILED;
       trySetTerminationCause(DAGTerminationCause.COMMIT_FAILURE);
@@ -1057,7 +1079,7 @@
     LOG.info("DAG: " + getID() + " finished with state: " + finalState);
     return finalState;
   }
-  
+
   private DAGStatus.State getDAGStatusFromState(DAGState finalState) {
     switch (finalState) {
       case NEW:
@@ -1631,7 +1653,7 @@
           + ", numFailedVertices=" + job.numFailedVertices
           + ", numKilledVertices=" + job.numKilledVertices
           + ", numVertices=" + job.numVertices);
-      
+
       if (failed) {
         return DAGState.TERMINATING;
       }
@@ -1724,7 +1746,7 @@
     addDiagnostic("Vertex re-running"
       + ", vertexName=" + vertex.getName()
       + ", vertexId=" + vertex.getVertexId());
-    
+
     if (!commitAllOutputsOnSuccess) {
       // partial output may already have been committed. fail if so
       List<VertexGroupInfo> groupList = vertexGroupInfo.get(vertex.getName());
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
index e63dbf5..b8430cc 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
@@ -137,7 +137,8 @@
   private String nodeHttpAddress;
   private String nodeRackName;
 
-  private TaskAttemptStatus reportedStatus;
+  @VisibleForTesting
+  TaskAttemptStatus reportedStatus;
   private DAGCounter localityCounter;
 
   // Used to store locality information when
@@ -790,6 +791,7 @@
         this.launchTime = tEvent.getStartTime();
         recoveryStartEventSeen = true;
         recoveredState = TaskAttemptState.RUNNING;
+        sendEvent(createDAGCounterUpdateEventTALaunched(this));
         return recoveredState;
       }
       case TASK_ATTEMPT_FINISHED:
@@ -805,6 +807,7 @@
         this.reportedStatus.state = tEvent.getState();
         this.diagnostics.add(tEvent.getDiagnostics());
         this.recoveredState = tEvent.getState();
+        sendEvent(createDAGCounterUpdateEventTAFinished(this, tEvent.getState()));
         return recoveredState;
       }
       default:
@@ -828,47 +831,29 @@
   }
 
   // TOOD Merge some of these JobCounter events.
-  private static DAGEventCounterUpdate createJobCounterUpdateEventTALaunched(
+  private static DAGEventCounterUpdate createDAGCounterUpdateEventTALaunched(
       TaskAttemptImpl ta) {
-    DAGEventCounterUpdate jce =
+    DAGEventCounterUpdate dagCounterEvent =
         new DAGEventCounterUpdate(
             ta.getDAGID()
             );
-    jce.addCounterUpdate(DAGCounter.TOTAL_LAUNCHED_TASKS, 1);
-    return jce;
+    dagCounterEvent.addCounterUpdate(DAGCounter.TOTAL_LAUNCHED_TASKS, 1);
+    return dagCounterEvent;
   }
 
-  private static DAGEventCounterUpdate createJobCounterUpdateEventSlotMillis(
-      TaskAttemptImpl ta) {
+  private static DAGEventCounterUpdate createDAGCounterUpdateEventTAFinished(
+      TaskAttemptImpl taskAttempt, TaskAttemptState taState) {
     DAGEventCounterUpdate jce =
-        new DAGEventCounterUpdate(
-            ta.getDAGID()
-            );
+        new DAGEventCounterUpdate(taskAttempt.getDAGID());
 
-//    long slotMillis = computeSlotMillis(ta);
-//    jce.addCounterUpdate(DAGCounter.SLOTS_MILLIS_TASKS, slotMillis);
-    return jce;
-  }
-
-  private static DAGEventCounterUpdate createJobCounterUpdateEventTATerminated(
-      TaskAttemptImpl taskAttempt, boolean taskAlreadyCompleted,
-      TaskAttemptStateInternal taState) {
-    DAGEventCounterUpdate jce =
-        new DAGEventCounterUpdate(
-            taskAttempt.getDAGID());
-
-    if (taState == TaskAttemptStateInternal.FAILED) {
+    if (taState == TaskAttemptState.FAILED) {
       jce.addCounterUpdate(DAGCounter.NUM_FAILED_TASKS, 1);
-    } else if (taState == TaskAttemptStateInternal.KILLED) {
+    } else if (taState == TaskAttemptState.KILLED) {
       jce.addCounterUpdate(DAGCounter.NUM_KILLED_TASKS, 1);
+    } else if (taState == TaskAttemptState.SUCCEEDED ) {
+      jce.addCounterUpdate(DAGCounter.NUM_SUCCEEDED_TASKS, 1);
     }
 
-//    long slotMillisIncrement = computeSlotMillis(taskAttempt);
-//    if (!taskAlreadyCompleted) {
-//      // dont double count the elapsed time
-//      jce.addCounterUpdate(DAGCounter.SLOTS_MILLIS_TASKS, slotMillisIncrement);
-//    }
-
     return jce;
   }
 
@@ -1141,8 +1126,8 @@
         ta.addDiagnosticInfo(((DiagnosableEvent) event).getDiagnosticInfo());
       }
 
-      ta.sendEvent(createJobCounterUpdateEventTATerminated(ta, false,
-          helper.getTaskAttemptStateInternal()));
+      ta.sendEvent(createDAGCounterUpdateEventTAFinished(ta,
+          helper.getTaskAttemptState()));
       if (ta.getLaunchTime() != 0) {
         // TODO For cases like this, recovery goes for a toss, since the the
         // attempt will not exist in the history file.
@@ -1183,7 +1168,7 @@
           .createSocketAddr(ta.nodeHttpAddress); // TODO: Costly?
       ta.trackerName = StringInterner.weakIntern(nodeHttpInetAddr.getHostName());
       ta.httpPort = nodeHttpInetAddr.getPort();
-      ta.sendEvent(createJobCounterUpdateEventTALaunched(ta));
+      ta.sendEvent(createDAGCounterUpdateEventTALaunched(ta));
 
       LOG.info("TaskAttempt: [" + ta.attemptId + "] started."
           + " Is using containerId: [" + ta.containerId + "]" + " on NM: ["
@@ -1315,7 +1300,8 @@
       ta.setFinishTime();
       // Send out history event.
       ta.logJobHistoryAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED);
-      ta.sendEvent(createJobCounterUpdateEventSlotMillis(ta));
+      ta.sendEvent(createDAGCounterUpdateEventTAFinished(ta,
+          TaskAttemptState.SUCCEEDED));
 
       // Inform the Scheduler.
       ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId,
@@ -1414,6 +1400,8 @@
           // TODO abort taskattempt
           taskAttempt.sendEvent(new TaskEventTAUpdate(taskAttempt.attemptId,
               TaskEventType.T_ATTEMPT_KILLED));
+          taskAttempt.sendEvent(createDAGCounterUpdateEventTAFinished(taskAttempt,
+              getExternalState(TaskAttemptStateInternal.KILLED)));
           endState = TaskAttemptStateInternal.KILLED;
           break;
         case SUCCEEDED:
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
index afc94e3..976f10f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
@@ -119,7 +119,8 @@
   protected final AppContext appContext;
   private final Resource taskResource;
   private final ContainerContext containerContext;
-  private long scheduledTime;
+  @VisibleForTesting
+  long scheduledTime;
   final StateChangeNotifier stateChangeNotifier;
 
   private final List<TezEvent> tezEventsForTaskAttempts = new ArrayList<TezEvent>();
@@ -128,7 +129,8 @@
 
   // 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;
+  @VisibleForTesting
+  int numberUncompletedAttempts = 0;
 
   private boolean historyTaskStartGenerated = false;
 
@@ -302,11 +304,13 @@
   //saying COMMIT_PENDING
   private TezTaskAttemptID commitAttempt;
 
-  private TezTaskAttemptID successfulAttempt;
+  @VisibleForTesting
+  TezTaskAttemptID successfulAttempt;
 
   @VisibleForTesting
   int failedAttempts;
-  private int finishedAttempts;//finish are total of success, failed and killed
+  @VisibleForTesting
+  int finishedAttempts;//finish are total of success, failed and killed
 
   private final boolean leafVertex;
   private TaskState recoveredState = TaskState.NEW;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java
index 846d208..d4ef4d5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TezRootInputInitializerContextImpl.java
@@ -115,7 +115,7 @@
   }
 
   @Override
-  public void registerForVertexStatusUpdates(String vertexName, Set<VertexState> stateSet) {
+  public void registerForVertexStateUpdates(String vertexName, Set<VertexState> stateSet) {
     manager.registerForVertexUpdates(vertexName, input.getName(), stateSet);
   }
 
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
index 1f6b556..34fffd8 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
@@ -157,7 +157,6 @@
 import org.apache.tez.runtime.api.impl.InputSpec;
 import org.apache.tez.runtime.api.impl.OutputSpec;
 import org.apache.tez.runtime.api.impl.TezEvent;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.HashMultiset;
@@ -206,9 +205,12 @@
 
   //fields initialized in init
 
-  private int numStartedSourceVertices = 0;
-  private int numInitedSourceVertices = 0;
-  private int numRecoveredSourceVertices = 0;
+  @VisibleForTesting
+  int numStartedSourceVertices = 0;
+  @VisibleForTesting
+  int numInitedSourceVertices = 0;
+  @VisibleForTesting
+  int numRecoveredSourceVertices = 0;
 
   private int distanceFromRoot = 0;
 
@@ -239,7 +241,8 @@
       new VertexStateChangedCallback();
 
   private VertexState recoveredState = VertexState.NEW;
-  private List<TezEvent> recoveredEvents = new ArrayList<TezEvent>();
+  @VisibleForTesting
+  List<TezEvent> recoveredEvents = new ArrayList<TezEvent>();
   private boolean vertexAlreadyInitialized = false;
 
   @VisibleForTesting
@@ -258,7 +261,7 @@
                       VertexState.INITIALIZING, VertexState.FAILED),
                   VertexEventType.V_INIT,
                   new InitTransition())
-          .addTransition(VertexState.NEW, 
+          .addTransition(VertexState.NEW,
                 EnumSet.of(VertexState.NEW),
                   VertexEventType.V_NULL_EDGE_INITIALIZED,
                   new NullEdgeInitializedTransition())
@@ -318,7 +321,7 @@
 
           // Transitions from INITIALIZING state
           .addTransition(VertexState.INITIALIZING,
-              EnumSet.of(VertexState.INITIALIZING, VertexState.INITED, 
+              EnumSet.of(VertexState.INITIALIZING, VertexState.INITED,
                   VertexState.FAILED),
               VertexEventType.V_ROOT_INPUT_INITIALIZED,
               new RootInputInitializedTransition())
@@ -351,14 +354,14 @@
           .addTransition(VertexState.INITIALIZING, VertexState.ERROR,
               VertexEventType.V_INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
-          .addTransition(VertexState.INITIALIZING, 
+          .addTransition(VertexState.INITIALIZING,
               EnumSet.of(VertexState.INITIALIZING, VertexState.INITED,
                   VertexState.FAILED),
                   VertexEventType.V_NULL_EDGE_INITIALIZED,
                   new NullEdgeInitializedTransition())
 
           // Transitions from INITED state
-          // SOURCE_VERTEX_STARTED - for sources which determine parallelism, 
+          // SOURCE_VERTEX_STARTED - for sources which determine parallelism,
           // they must complete before this vertex can start.
           .addTransition
               (VertexState.INITED,
@@ -368,14 +371,14 @@
           .addTransition(VertexState.INITED, VertexState.INITED,
               VertexEventType.V_SOURCE_VERTEX_STARTED,
               new SourceVertexStartedTransition())
-          .addTransition(VertexState.INITED, 
+          .addTransition(VertexState.INITED,
               EnumSet.of(VertexState.INITED),
               VertexEventType.V_ONE_TO_ONE_SOURCE_SPLIT,
               new OneToOneSourceSplitTransition())
           .addTransition(VertexState.INITED,  VertexState.INITED,
               VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED,
               SOURCE_TASK_ATTEMPT_COMPLETED_EVENT_TRANSITION)
-          .addTransition(VertexState.INITED, 
+          .addTransition(VertexState.INITED,
               EnumSet.of(VertexState.RUNNING, VertexState.INITED),
               VertexEventType.V_START,
               new StartTransition())
@@ -403,7 +406,7 @@
                   VertexState.ERROR),
               VertexEventType.V_TASK_COMPLETED,
               new TaskCompletedTransition())
-          .addTransition(VertexState.RUNNING, 
+          .addTransition(VertexState.RUNNING,
               EnumSet.of(VertexState.RUNNING),
               VertexEventType.V_ONE_TO_ONE_SOURCE_SPLIT,
               new OneToOneSourceSplitTransition())
@@ -465,7 +468,7 @@
               VertexEventType.V_ROUTE_EVENT,
               ROUTE_EVENT_TRANSITION)
           .addTransition(
-              VertexState.SUCCEEDED, 
+              VertexState.SUCCEEDED,
               EnumSet.of(VertexState.FAILED, VertexState.ERROR),
               VertexEventType.V_TASK_COMPLETED,
               new TaskCompletedAfterVertexSuccessTransition())
@@ -561,17 +564,27 @@
   private final StateMachineTez<VertexState, VertexEventType, VertexEvent, VertexImpl> stateMachine;
 
   //changing fields while the vertex is running
-  private int numTasks;
-  private int completedTaskCount = 0;
-  private int succeededTaskCount = 0;
-  private int failedTaskCount = 0;
-  private int killedTaskCount = 0;
+  @VisibleForTesting
+  int numTasks;
+  @VisibleForTesting
+  int completedTaskCount = 0;
+  @VisibleForTesting
+  int succeededTaskCount = 0;
+  @VisibleForTesting
+  int failedTaskCount = 0;
+  @VisibleForTesting
+  int killedTaskCount = 0;
 
-  private long initTimeRequested; // Time at which INIT request was received.
-  private long initedTime; // Time when entering state INITED
-  private long startTimeRequested; // Time at which START request was received.
-  private long startedTime; // Time when entering state STARTED
-  private long finishTime;
+  @VisibleForTesting
+  long initTimeRequested; // Time at which INIT request was received.
+  @VisibleForTesting
+  long initedTime; // Time when entering state INITED
+  @VisibleForTesting
+  long startTimeRequested; // Time at which START request was received.
+  @VisibleForTesting
+  long startedTime; // Time when entering state STARTED
+  @VisibleForTesting
+  long finishTime;
   private float progress;
 
   private final TezVertexID vertexId;  //runtime assigned id.
@@ -586,14 +599,14 @@
   private Map<Vertex, Edge> targetVertices;
   Set<Edge> uninitializedEdges = Sets.newHashSet();
 
-  private Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
+  private Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
     rootInputDescriptors;
-  private Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>> 
+  private Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>>
     additionalOutputs;
   private Map<String, OutputCommitter> outputCommitters;
   private Map<String, InputSpecUpdate> rootInputSpecs = new HashMap<String, InputSpecUpdate>();
   private static final InputSpecUpdate DEFAULT_ROOT_INPUT_SPECS = InputSpecUpdate
-      .getDefaultSinglePhysicalInputSpecUpdate(); 
+      .getDefaultSinglePhysicalInputSpecUpdate();
   private final List<OutputSpec> additionalOutputSpecs = new ArrayList<OutputSpec>();
   private Set<String> inputsWithInitializers;
   private int numInitializedInputs;
@@ -608,7 +621,7 @@
   private RootInputInitializerManager rootInputInitializerManager;
 
   VertexManager vertexManager;
-  
+
   private final UserGroupInformation dagUgi;
 
   private boolean parallelismSet = false;
@@ -617,20 +630,22 @@
   private AtomicBoolean committed = new AtomicBoolean(false);
   private AtomicBoolean aborted = new AtomicBoolean(false);
   private boolean commitVertexOutputs = false;
-  
+
   private Map<String, VertexGroupInfo> dagVertexGroups;
-  
+
   private TaskLocationHint taskLocationHints[];
   private Map<String, LocalResource> localResources;
   private Map<String, String> environment;
   private final String javaOpts;
   private final ContainerContext containerContext;
   private VertexTerminationCause terminationCause;
-  
+
   private String logIdentifier;
-  private boolean recoveryCommitInProgress = false;
+  @VisibleForTesting
+  boolean recoveryCommitInProgress = false;
   private boolean summaryCompleteSeen = false;
-  private boolean hasCommitter = false;
+  @VisibleForTesting
+  boolean hasCommitter = false;
   private boolean vertexCompleteSeen = false;
   private Map<String,EdgeManagerPluginDescriptor> recoveredSourceEdgeManagers = null;
   private Map<String, InputSpecUpdate> recoveredRootInputSpecUpdates = null;
@@ -668,7 +683,7 @@
       logLocationHints(this.vertexName, vertexLocationHint);
     }
     setTaskLocationHints(vertexLocationHint);
-    
+
     this.dagUgi = appContext.getCurrentDAG().getDagUGI();
 
     this.taskResource = DagTypeConverters
@@ -907,12 +922,12 @@
       this.readLock.unlock();
     }
   }
-  
+
   @Override
   public TaskLocationHint getTaskLocationHint(TezTaskID taskId) {
     this.readLock.lock();
     try {
-      if (taskLocationHints == null || 
+      if (taskLocationHints == null ||
           taskLocationHints.length <= taskId.getId()) {
         return null;
       }
@@ -1091,8 +1106,8 @@
   }
 
   private void setTaskLocationHints(VertexLocationHint vertexLocationHint) {
-    if (vertexLocationHint != null && 
-        vertexLocationHint.getTaskLocationHints() != null && 
+    if (vertexLocationHint != null &&
+        vertexLocationHint.getTaskLocationHints() != null &&
         !vertexLocationHint.getTaskLocationHints().isEmpty()) {
       List<TaskLocationHint> locHints = vertexLocationHint.getTaskLocationHints();
       taskLocationHints = locHints.toArray(new TaskLocationHint[locHints.size()]);
@@ -1164,7 +1179,7 @@
             }
           }
         }
-        
+
         // Restore any rootInputSpecUpdates which may have been registered during a parallelism
         // update.
         if (rootInputSpecUpdates != null) {
@@ -1176,7 +1191,7 @@
         writeLock.unlock();
       }
     }
-    Preconditions.checkArgument(parallelism >= 0, "Parallelism must be >=0. Value: " 
+    Preconditions.checkArgument(parallelism >= 0, "Parallelism must be >=0. Value: "
     + parallelism + " for vertex: " + logIdentifier);
     setVertexLocationHint(vertexLocationHint);
     writeLock.lock();
@@ -1185,7 +1200,7 @@
         LOG.info("Parallelism can only be set dynamically once per vertex: " + logIdentifier);
         return false;
       }
-      
+
       parallelismSet = true;
 
       // Input initializer/Vertex Manager/1-1 split expected to set parallelism.
@@ -1195,7 +1210,7 @@
               "Vertex state is not Initializing. Value: " + getState()
                   + " for vertex: " + logIdentifier);
         }
-        
+
         if(sourceEdgeManagers != null) {
           for(Map.Entry<String, EdgeManagerPluginDescriptor> entry : sourceEdgeManagers.entrySet()) {
             LOG.info("Replacing edge manager for source:"
@@ -1233,7 +1248,7 @@
         stateChangeNotifier.stateChanged(vertexId,
             new VertexStateUpdateParallelismUpdated(vertexName, numTasks, oldNumTasks));
         this.createTasks();
-        LOG.info("Vertex " + getVertexId() + 
+        LOG.info("Vertex " + getVertexId() +
             " parallelism set to " + parallelism);
         if (canInitVertex()) {
           getEventHandler().handle(new VertexEvent(getVertexId(), VertexEventType.V_READY_TO_INIT));
@@ -1254,7 +1269,7 @@
           return false;
         }
         if (parallelism == numTasks) {
-          LOG.info("setParallelism same as current value: " + parallelism + 
+          LOG.info("setParallelism same as current value: " + parallelism +
               " for vertex: " + logIdentifier);
           Preconditions.checkArgument(sourceEdgeManagers != null,
               "Source edge managers or RootInputSpecs must be set when not changing parallelism");
@@ -1268,7 +1283,7 @@
         for (Edge edge : sourceVertices.values()) {
           edge.startEventBuffering();
         }
-  
+
         // assign to local variable of LinkedHashMap to make sure that changing
         // type of task causes compile error. We depend on LinkedHashMap for order
         LinkedHashMap<TezTaskID, Task> currentTasks = this.tasks;
@@ -1291,14 +1306,14 @@
           LOG.info("Removing task: " + entry.getKey());
           iter.remove();
         }
-        LOG.info("Vertex " + logIdentifier + 
+        LOG.info("Vertex " + logIdentifier +
             " parallelism set to " + parallelism + " from " + numTasks);
         int oldNumTasks = numTasks;
         this.numTasks = parallelism;
         stateChangeNotifier.stateChanged(vertexId,
             new VertexStateUpdateParallelismUpdated(vertexName, numTasks, oldNumTasks));
         assert tasks.size() == numTasks;
-  
+
         // set new edge managers
         if(sourceEdgeManagers != null) {
           for(Map.Entry<String, EdgeManagerPluginDescriptor> entry : sourceEdgeManagers.entrySet()) {
@@ -1330,17 +1345,17 @@
           edge.stopEventBuffering();
         }
       }
-      
+
       for (Map.Entry<Vertex, Edge> entry : targetVertices.entrySet()) {
         Edge edge = entry.getValue();
-        if (edge.getEdgeProperty().getDataMovementType() 
+        if (edge.getEdgeProperty().getDataMovementType()
             == DataMovementType.ONE_TO_ONE) {
           // inform these target vertices that we have changed parallelism
-          VertexEventOneToOneSourceSplit event = 
+          VertexEventOneToOneSourceSplit event =
               new VertexEventOneToOneSourceSplit(entry.getKey().getVertexId(),
                   getVertexId(),
-                  ((originalOneToOneSplitSource!=null) ? 
-                      originalOneToOneSplitSource : getVertexId()), 
+                  ((originalOneToOneSplitSource!=null) ?
+                      originalOneToOneSplitSource : getVertexId()),
                   numTasks);
           getEventHandler().handle(event);
         }
@@ -1349,7 +1364,7 @@
     } finally {
       writeLock.unlock();
     }
-    
+
     return true;
   }
 
@@ -1749,7 +1764,8 @@
     // no code, for now
   }
 
-  private ContainerContext getContainerContext(int taskIdx) {
+  @VisibleForTesting
+  ContainerContext getContainerContext(int taskIdx) {
     if (taskSpecificLaunchCmdOpts.addTaskSpecificLaunchCmdOption(vertexName, taskIdx)) {
       String jvmOpts = taskSpecificLaunchCmdOpts.getTaskSpecificOption(javaOpts, vertexName, taskIdx);
       ContainerContext context = new ContainerContext(this.localResources,
@@ -1823,9 +1839,9 @@
       if (rootInputDescriptors != null) {
         LOG.info("Root Inputs exist for Vertex: " + getName() + " : "
             + rootInputDescriptors);
-        for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input 
+        for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input
             : rootInputDescriptors.values()) {
-          if (input.getControllerDescriptor() != null && 
+          if (input.getControllerDescriptor() != null &&
               input.getControllerDescriptor().getClassName() != null) {
             if (inputsWithInitializers == null) {
               inputsWithInitializers = Sets.newHashSet();
@@ -1892,7 +1908,7 @@
     checkTaskLimits();
     return VertexState.INITED;
   }
-  
+
   private void assignVertexManager() {
     boolean hasBipartite = false;
     boolean hasOneToOne = false;
@@ -1912,12 +1928,12 @@
           hasCustom = true;
           break;
         default:
-          throw new TezUncheckedException("Unknown data movement type: " + 
+          throw new TezUncheckedException("Unknown data movement type: " +
               edge.getEdgeProperty().getDataMovementType());
         }
       }
     }
-    
+
     boolean hasUserVertexManager = vertexPlan.hasVertexManagerPlugin();
 
     if (hasUserVertexManager) {
@@ -2025,6 +2041,8 @@
           vertex.finished(VertexState.ERROR);
       }
 
+      // recover from recover log, should recover to running
+      // desiredState must be RUNNING based on above code
       VertexState endState;
       switch (vertex.recoveredState) {
         case NEW:
@@ -2103,6 +2121,7 @@
             vertex.vertexManager.onVertexStarted(vertex.pendingReportedSrcCompletions);
             endState = VertexState.RUNNING;
           } else {
+            // why succeeded here
             endState = VertexState.SUCCEEDED;
             vertex.finished(endState);
           }
@@ -2274,7 +2293,7 @@
           (vertex.sourceVertices == null || vertex.sourceVertices.containsKey(otherVertex) ||
           vertex.targetVertices == null || vertex.targetVertices.containsKey(otherVertex)),
           "Not connected to vertex " + otherVertex.getName() + " from vertex: " + vertex.logIdentifier);
-      LOG.info("Edge initialized for connection to vertex " + otherVertex.getName() + 
+      LOG.info("Edge initialized for connection to vertex " + otherVertex.getName() +
           " at vertex : " + vertex.logIdentifier);
       vertex.uninitializedEdges.remove(edge);
       if(vertex.getState() == VertexState.INITIALIZING && vertex.canInitVertex()) {
@@ -2621,7 +2640,7 @@
       if (state.equals(VertexState.FAILED)) {
         return state;
       }
-      // TODO move before to handle NEW state 
+      // TODO move before to handle NEW state
       if (vertex.targetVertices != null) {
         for (Edge e : vertex.targetVertices.values()) {
           if (e.getEdgeManager() == null) {
@@ -2644,37 +2663,19 @@
           }
         }
       }
-      
+
       // Create tasks based on initial configuration, but don't start them yet.
       if (vertex.numTasks == -1) {
         LOG.info("Num tasks is -1. Expecting VertexManager/InputInitializers/1-1 split"
             + " to set #tasks for the vertex " + vertex.getVertexId());
 
         if (vertex.inputsWithInitializers != null) {
-          // Use DAGScheduler to arbitrate resources among vertices later
-          vertex.rootInputInitializerManager = vertex.createRootInputInitializerManager(
-              vertex.getDAG().getName(), vertex.getName(), vertex.getVertexId(),
-              vertex.eventHandler, -1,
-              vertex.appContext.getTaskScheduler().getNumClusterNodes(),
-              vertex.getTaskResource(),
-              vertex.appContext.getTaskScheduler().getTotalResources());
-          List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
-              inputList = Lists.newArrayListWithCapacity(vertex.inputsWithInitializers.size());
-          for (String inputName : vertex.inputsWithInitializers) {
-            inputList.add(vertex.rootInputDescriptors.get(inputName));
-          }
-          LOG.info("Vertex will initialize via inputInitializers "
-              + vertex.logIdentifier + ". Starting root input initializers: "
-              + vertex.inputsWithInitializers.size());
-          vertex.rootInputInitializerManager.runInputInitializers(inputList);
-          // Send pending rootInputInitializerEvents
-          vertex.rootInputInitializerManager.handleInitializerEvents(vertex.pendingInitializerEvents);
-          vertex.pendingInitializerEvents.clear();
+          vertex.setupInputInitializerManager();
           return VertexState.INITIALIZING;
         } else {
           boolean hasOneToOneUninitedSource = false;
           for (Map.Entry<Vertex, Edge> entry : vertex.sourceVertices.entrySet()) {
-            if (entry.getValue().getEdgeProperty().getDataMovementType() == 
+            if (entry.getValue().getEdgeProperty().getDataMovementType() ==
                 DataMovementType.ONE_TO_ONE) {
               if (entry.getKey().getTotalTasks() == -1) {
                 hasOneToOneUninitedSource = true;
@@ -2690,34 +2691,16 @@
             LOG.info("Vertex will initialize via custom vertex manager. " + vertex.logIdentifier);
             return VertexState.INITIALIZING;
           }
-          throw new TezUncheckedException(vertex.getVertexId() + 
+          throw new TezUncheckedException(vertex.getVertexId() +
           " has -1 tasks but does not have input initializers, " +
           "1-1 uninited sources or custom vertex manager to set it at runtime");
         }
       } else {
         LOG.info("Creating " + vertex.numTasks + " for vertex: " + vertex.logIdentifier);
         vertex.createTasks();
+
         if (vertex.inputsWithInitializers != null) {
-          vertex.rootInputInitializerManager = vertex.createRootInputInitializerManager(
-              vertex.getDAG().getName(), vertex.getName(), vertex.getVertexId(),
-              vertex.eventHandler, vertex.getTotalTasks(),
-              vertex.appContext.getTaskScheduler().getNumClusterNodes(),
-              vertex.getTaskResource(),
-              vertex.appContext.getTaskScheduler().getTotalResources());
-          List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
-          inputList = Lists.newArrayListWithCapacity(vertex.inputsWithInitializers.size());
-          for (String inputName : vertex.inputsWithInitializers) {
-            inputList.add(vertex.rootInputDescriptors.get(inputName));
-          }
-          LOG.info("Starting root input initializers: "
-              + vertex.inputsWithInitializers.size());
-          // special case when numTasks>0 and still we want to stay in initializing 
-          // state. This is handled in RootInputInitializedTransition specially.
-          vertex.initWaitsForRootInitializers = true;
-          vertex.rootInputInitializerManager.runInputInitializers(inputList);
-          // Send pending rootInputInitializerEvents
-          vertex.rootInputInitializerManager.handleInitializerEvents(vertex.pendingInitializerEvents);
-          vertex.pendingInitializerEvents.clear();
+          vertex.setupInputInitializerManager();
           return VertexState.INITIALIZING;
         }
         if (!vertex.uninitializedEdges.isEmpty()) {
@@ -2742,7 +2725,7 @@
       Resource vertexTaskResource, Resource totalResource) {
     return new RootInputInitializerManager(this, appContext, this.dagUgi, this.stateChangeNotifier);
   }
-  
+
   private boolean initializeVertexInInitializingState() {
     boolean isInitialized = initializeVertex();
     if (!isInitialized) {
@@ -2752,7 +2735,7 @@
 
     return true;
   }
-  
+
   void startIfPossible() {
     if (startSignalPending) {
       // Trigger a start event to ensure route events are seen before
@@ -2766,7 +2749,7 @@
 
   public static class VertexInitializedTransition implements
       MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
-    
+
     static VertexState doTransition(VertexImpl vertex) {
       Preconditions.checkState(vertex.canInitVertex(), "Vertex: " + vertex.logIdentifier);
       boolean isInitialized = vertex.initializeVertexInInitializingState();
@@ -2775,15 +2758,15 @@
       }
 
       vertex.startIfPossible();
-      return VertexState.INITED;      
+      return VertexState.INITED;
     }
-    
+
     @Override
     public VertexState transition(VertexImpl vertex, VertexEvent event) {
       return doTransition(vertex);
     }
   }
-  
+
   // present in most transitions so that the initializer thread can be shutdown properly
   public static class RootInputInitializedTransition implements
       MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
@@ -2793,10 +2776,14 @@
       VertexEventRootInputInitialized liInitEvent = (VertexEventRootInputInitialized) event;
       VertexState state = vertex.getState();
       if (state == VertexState.INITIALIZING) {
-        vertex.vertexManager.onRootVertexInitialized(
+        List<TezEvent> inputInfoEvents =
+            vertex.vertexManager.onRootVertexInitialized(
             liInitEvent.getInputName(),
             vertex.getAdditionalInputs().get(liInitEvent.getInputName())
                 .getIODescriptor(), liInitEvent.getEvents());
+        if (inputInfoEvents != null && !inputInfoEvents.isEmpty()) {
+          ROUTE_EVENT_TRANSITION.transition(vertex, new VertexEventRouteEvent(vertex.vertexId, inputInfoEvents));
+        }
       }
 
       vertex.numInitializedInputs++;
@@ -2805,15 +2792,17 @@
         vertex.rootInputInitializerManager.shutdown();
         vertex.rootInputInitializerManager = null;
       }
-      
+
       // done. check if we need to do the initialization
-      if (vertex.getState() == VertexState.INITIALIZING && 
+      if (vertex.getState() == VertexState.INITIALIZING &&
           vertex.initWaitsForRootInitializers) {
-        // set the wait flag to false
-        vertex.initWaitsForRootInitializers = false;
+        if (vertex.numInitializedInputs == vertex.inputsWithInitializers.size()) {
+          // set the wait flag to false if all initializers are done
+          vertex.initWaitsForRootInitializers = false;
+        }
         // initialize vertex if possible and needed
         if (vertex.canInitVertex()) {
-          Preconditions.checkState(vertex.numTasks >= 0, 
+          Preconditions.checkState(vertex.numTasks >= 0,
               "Parallelism should have been set by now for vertex: " + vertex.logIdentifier);
           return VertexInitializedTransition.doTransition(vertex);
         }
@@ -2827,10 +2816,10 @@
 
     @Override
     public VertexState transition(VertexImpl vertex, VertexEvent event) {
-      VertexEventOneToOneSourceSplit splitEvent = 
+      VertexEventOneToOneSourceSplit splitEvent =
           (VertexEventOneToOneSourceSplit)event;
       TezVertexID originalSplitSource = splitEvent.getOriginalSplitSource();
-      
+
       if (vertex.originalOneToOneSplitSource != null) {
         VertexState state = vertex.getState();
         Preconditions
@@ -2845,25 +2834,25 @@
         if (vertex.originalOneToOneSplitSource.equals(originalSplitSource)) {
           // ignore another split event that may have come from a different
           // path in the DAG. We have already split because of that source
-          LOG.info("Ignoring split of vertex " + vertex.getVertexId() + 
-              " because of split in vertex " + originalSplitSource + 
+          LOG.info("Ignoring split of vertex " + vertex.getVertexId() +
+              " because of split in vertex " + originalSplitSource +
               " sent by vertex " + splitEvent.getSenderVertex() +
               " numTasks " + splitEvent.getNumTasks());
           return state;
         }
         // cannot split from multiple sources
-        throw new TezUncheckedException("Vertex: " + vertex.getVertexId() + 
-            " asked to split by: " + originalSplitSource + 
+        throw new TezUncheckedException("Vertex: " + vertex.getVertexId() +
+            " asked to split by: " + originalSplitSource +
             " but was already split by:" + vertex.originalOneToOneSplitSource);
       }
-      
-      LOG.info("Splitting vertex " + vertex.getVertexId() + 
-          " because of split in vertex " + originalSplitSource + 
+
+      LOG.info("Splitting vertex " + vertex.getVertexId() +
+          " because of split in vertex " + originalSplitSource +
           " sent by vertex " + splitEvent.getSenderVertex() +
           " numTasks " + splitEvent.getNumTasks());
       vertex.originalOneToOneSplitSource = originalSplitSource;
       vertex.setParallelism(splitEvent.getNumTasks(), null, null, null);
-      if (vertex.getState() == VertexState.RUNNING || 
+      if (vertex.getState() == VertexState.RUNNING ||
           vertex.getState() == VertexState.INITED) {
         return vertex.getState();
       } else {
@@ -2893,19 +2882,19 @@
       }
       vertex.numStartedSourceVertices++;
       LOG.info("Source vertex started: " + startEvent.getSourceVertexId() +
-          " for vertex: " + vertex.getVertexId() + " numStartedSources: " + 
+          " for vertex: " + vertex.getVertexId() + " numStartedSources: " +
           vertex.numStartedSourceVertices + " numSources: " + vertex.sourceVertices.size());
-      
+
       if (vertex.numStartedSourceVertices < vertex.sourceVertices.size()) {
         LOG.info("Cannot start vertex: " + vertex.logIdentifier + " numStartedSources: "
             + vertex.numStartedSourceVertices + " numSources: " + vertex.sourceVertices.size());
         return;
       }
-      
-      // vertex meets external start dependency conditions. Save this signal in 
+
+      // vertex meets external start dependency conditions. Save this signal in
       // case we are not ready to start now and need to start later
       vertex.startSignalPending = true;
-      
+
       if (vertex.getState() != VertexState.INITED) {
         // vertex itself is not ready to start. External dependencies have already
         // notified us.
@@ -2915,14 +2904,14 @@
             + vertex.uninitializedEdges.size());
         return;
       }
-      
+
       // vertex is inited and all dependencies are ready. Inited vertex means
       // parallelism must be set already and edges defined
       Preconditions.checkState(
           (vertex.numTasks >= 0 && vertex.uninitializedEdges.isEmpty()),
           "Cannot start vertex that is not completely defined. Vertex: "
               + vertex.logIdentifier + " numTasks: " + vertex.numTasks);
-      
+
       vertex.startIfPossible();
     }
   }
@@ -2938,14 +2927,14 @@
         + " initWaitsForRootInitializers: " + initWaitsForRootInitializers);
     return false;
   }
-  
-  public static class StartWhileInitializingTransition implements 
+
+  public static class StartWhileInitializingTransition implements
     SingleArcTransition<VertexImpl, VertexEvent> {
 
     @Override
     public void transition(VertexImpl vertex, VertexEvent event) {
       // vertex state machine does not start itself in the initializing state
-      // this start event can only come directly from the DAG. That means this 
+      // this start event can only come directly from the DAG. That means this
       // is a top level vertex of the dag
       Preconditions.checkState(
           (vertex.sourceVertices == null || vertex.sourceVertices.isEmpty()),
@@ -2958,10 +2947,10 @@
 
   public static class StartTransition implements
     MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
-  
+
   @Override
   public VertexState transition(VertexImpl vertex, VertexEvent event) {
-      Preconditions.checkState(vertex.getState() == VertexState.INITED, 
+      Preconditions.checkState(vertex.getState() == VertexState.INITED,
           "Unexpected state " + vertex.getState() + " for " + vertex.logIdentifier);
       vertex.startTimeRequested = vertex.clock.getTime();
       return vertex.startVertex();
@@ -2969,7 +2958,7 @@
   }
 
   private VertexState startVertex() {
-    Preconditions.checkState(getState() == VertexState.INITED, 
+    Preconditions.checkState(getState() == VertexState.INITED,
         "Vertex must be inited " + logIdentifier);
 
     startedTime = clock.getTime();
@@ -3129,7 +3118,7 @@
         case OWN_TASK_FAILURE: vertex.tryEnactKill(trigger, TaskTerminationCause.OTHER_TASK_FAILURE); break;
         case ROOT_INPUT_INIT_FAILURE:
         case COMMIT_FAILURE:
-        case INVALID_NUM_OF_TASKS: 
+        case INVALID_NUM_OF_TASKS:
         case INIT_FAILURE:
         case INTERNAL_ERROR:
         case OTHER_VERTEX_FAILURE: vertex.tryEnactKill(trigger, TaskTerminationCause.OTHER_VERTEX_FAILURE); break;
@@ -3208,7 +3197,7 @@
       if (taskEvent.getState() == TaskState.SUCCEEDED) {
         taskSucceeded(vertex, task);
       } else if (taskEvent.getState() == TaskState.FAILED) {
-        LOG.info("Failing vertex: " + vertex.logIdentifier + 
+        LOG.info("Failing vertex: " + vertex.logIdentifier +
             " because task failed: " + taskEvent.getTaskID());
         vertex.tryEnactKill(VertexTerminationCause.OWN_TASK_FAILURE, TaskTerminationCause.OTHER_TASK_FAILURE);
         forceTransitionToKillWait = true;
@@ -3256,7 +3245,7 @@
       vertex.succeededTaskCount--;
     }
   }
-  
+
   private static class VertexNoTasksCompletedTransition implements
       MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
 
@@ -3265,7 +3254,7 @@
       return VertexImpl.checkVertexForCompletion(vertex);
     }
   }
-  
+
   private static class TaskCompletedAfterVertexSuccessTransition implements
     MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
     @Override
@@ -3277,12 +3266,12 @@
       if (vEvent.getState() == TaskState.FAILED) {
         finalState = VertexState.FAILED;
         finalStatus = VertexStatus.State.FAILED;
-        diagnosticMsg = "Vertex " + vertex.logIdentifier +" failed as task " + vEvent.getTaskID() + 
+        diagnosticMsg = "Vertex " + vertex.logIdentifier +" failed as task " + vEvent.getTaskID() +
           " failed after vertex succeeded.";
       } else {
         finalState = VertexState.ERROR;
         finalStatus = VertexStatus.State.ERROR;
-        diagnosticMsg = "Vertex " + vertex.logIdentifier + " error as task " + vEvent.getTaskID() + 
+        diagnosticMsg = "Vertex " + vertex.logIdentifier + " error as task " + vEvent.getTaskID() +
             " completed with state " + vEvent.getState() + " after vertex succeeded.";
       }
       LOG.info(diagnosticMsg);
@@ -3549,6 +3538,28 @@
     }
   }
 
+  private void setupInputInitializerManager() {
+    rootInputInitializerManager = createRootInputInitializerManager(
+        getDAG().getName(), getName(), getVertexId(),
+        eventHandler, getTotalTasks(),
+        appContext.getTaskScheduler().getNumClusterNodes(),
+        getTaskResource(),
+        appContext.getTaskScheduler().getTotalResources());
+    List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
+        inputList = Lists.newArrayListWithCapacity(inputsWithInitializers.size());
+    for (String inputName : inputsWithInitializers) {
+      inputList.add(rootInputDescriptors.get(inputName));
+    }
+    LOG.info("Vertex will initialize via inputInitializers "
+        + logIdentifier + ". Starting root input initializers: "
+        + inputsWithInitializers.size());
+    initWaitsForRootInitializers = true;
+    rootInputInitializerManager.runInputInitializers(inputList);
+    // Send pending rootInputInitializerEvents
+    rootInputInitializerManager.handleInitializerEvents(pendingInitializerEvents);
+    pendingInitializerEvents.clear();
+  }
+
   private static class VertexStateChangedCallback
       implements OnStateChangedCallback<VertexState, VertexImpl> {
 
@@ -3653,14 +3664,14 @@
 
   @Nullable
   @Override
-  public Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
+  public Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
     getAdditionalInputs() {
     return this.rootInputDescriptors;
   }
 
   @Nullable
   @Override
-  public Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>> 
+  public Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>>
     getAdditionalOutputs() {
     return this.additionalOutputs;
   }
@@ -3750,7 +3761,7 @@
     inputSpecList = new ArrayList<InputSpec>(this.getInputVerticesCount()
         + (rootInputDescriptors == null ? 0 : rootInputDescriptors.size()));
     if (rootInputDescriptors != null) {
-      for (Entry<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
+      for (Entry<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
            rootInputDescriptorEntry : rootInputDescriptors.entrySet()) {
         inputSpecList.add(new InputSpec(rootInputDescriptorEntry.getKey(),
             rootInputDescriptorEntry.getValue().getIODescriptor(), rootInputSpecs.get(
@@ -3783,18 +3794,18 @@
     }
     return outputSpecList;
   }
-  
+
   //TODO Eventually remove synchronization.
   @Override
   public synchronized List<GroupInputSpec> getGroupInputSpecList(int taskIndex) {
     return groupInputSpecList;
   }
-  
+
   @Override
   public synchronized void addSharedOutputs(Set<String> outputs) {
     this.sharedOutputs.addAll(outputs);
   }
-  
+
   @Override
   public synchronized Set<String> getSharedOutputs() {
     return this.sharedOutputs;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
index 177b946..c2ff660 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
@@ -24,6 +24,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 
 import javax.annotation.Nullable;
 
@@ -44,7 +45,6 @@
 import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.dag.TaskAttempt;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.runtime.api.Event;
@@ -68,10 +68,11 @@
   VertexManagerPluginContextImpl pluginContext;
   UserPayload payload = null;
   AppContext appContext;
-    
+  ConcurrentHashMap<String, List<TezEvent>> cachedRootInputEventMap;
+
   class VertexManagerPluginContextImpl implements VertexManagerPluginContext {
     // TODO Add functionality to allow VertexManagers to send VertexManagerEvents
-    
+
     private EventMetaData rootEventSourceMetadata = new EventMetaData(EventProducerConsumerType.INPUT,
         managedVertex.getName(), "NULL_VERTEX", null);
     private Map<String, EventMetaData> destinationEventMetadataMap = Maps.newHashMap();
@@ -80,7 +81,7 @@
     public Map<String, EdgeProperty> getInputVertexEdgeProperties() {
       // TODO Something similar for Initial Inputs - payload etc visible
       Map<Vertex, Edge> inputs = managedVertex.getInputVertices();
-      Map<String, EdgeProperty> vertexEdgeMap = 
+      Map<String, EdgeProperty> vertexEdgeMap =
                           Maps.newHashMapWithExpectedSize(inputs.size());
       for (Map.Entry<Vertex, Edge> entry : inputs.entrySet()) {
         vertexEdgeMap.put(entry.getKey().getName(), entry.getValue().getEdgeProperty());
@@ -115,7 +116,7 @@
     @Override
     public Set<String> getVertexInputNames() {
       Set<String> inputNames = null;
-      Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
+      Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
           inputs = managedVertex.getAdditionalInputs();
       if (inputs != null) {
         inputNames = inputs.keySet();
@@ -128,7 +129,6 @@
       return payload;
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     public void addRootInputEvents(final String inputName,
         Collection<InputDataInformationEvent> events) {
@@ -142,8 +142,8 @@
               return tezEvent;
             }
           });
-      appContext.getEventHandler().handle(
-          new VertexEventRouteEvent(managedVertex.getVertexId(), Lists.newArrayList(tezEvents)));
+
+      cachedRootInputEventMap.put(inputName,Lists.newArrayList(tezEvents));
       // Recovery handling is taken care of by the Vertex.
     }
 
@@ -201,7 +201,7 @@
     }
   }
 
-  public VertexManager(VertexManagerPluginDescriptor pluginDesc, 
+  public VertexManager(VertexManagerPluginDescriptor pluginDesc,
       Vertex managedVertex, AppContext appContext) {
     checkNotNull(pluginDesc, "pluginDesc is null");
     checkNotNull(managedVertex, "managedVertex is null");
@@ -209,12 +209,13 @@
     this.pluginDesc = pluginDesc;
     this.managedVertex = managedVertex;
     this.appContext = appContext;
+    this.cachedRootInputEventMap = new ConcurrentHashMap<String, List<TezEvent>>();
   }
-  
+
   public VertexManagerPlugin getPlugin() {
     return plugin;
   }
-  
+
   public void initialize() {
     pluginContext = new VertexManagerPluginContextImpl();
     if (pluginDesc != null) {
@@ -246,7 +247,7 @@
 
   public void onSourceTaskCompleted(TezTaskID tezTaskId) {
     Integer taskId = new Integer(tezTaskId.getId());
-    String vertexName = 
+    String vertexName =
         appContext.getCurrentDAG().getVertex(tezTaskId.getVertexID()).getName();
     plugin.onSourceTaskCompleted(vertexName, taskId);
   }
@@ -255,8 +256,9 @@
     plugin.onVertexManagerEventReceived(vmEvent);
   }
 
-  public void onRootVertexInitialized(String inputName, 
+  public List<TezEvent> onRootVertexInitialized(String inputName,
       InputDescriptor inputDescriptor, List<Event> events) {
     plugin.onRootVertexInitialized(inputName, inputDescriptor, events);
+    return cachedRootInputEventMap.get(inputName);
   }
 }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
index 490ffeb..026ed7d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/LocalTaskSchedulerService.java
@@ -27,6 +27,7 @@
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 
+import com.google.common.primitives.Ints;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -91,10 +92,17 @@
 
   @Override
   public Resource getAvailableResources() {
-    Resource freeResources = Resource.newInstance(
-        (int)Runtime.getRuntime().freeMemory()/(1024*1024),
-        Runtime.getRuntime().availableProcessors());
-    return freeResources;
+    long memory = Runtime.getRuntime().freeMemory();
+    int cores = Runtime.getRuntime().availableProcessors();
+    return createResource(memory, cores);
+  }
+
+  static Resource createResource(long runtimeMemory, int core) {
+    if (runtimeMemory < 0 || core < 0) {
+      throw new IllegalArgumentException("Negative Memory or Core provided!"
+          + "mem: "+runtimeMemory+" core:"+core);
+    }
+    return Resource.newInstance(Ints.checkedCast(runtimeMemory/(1024*1024)), core);
   }
 
   @Override
@@ -108,10 +116,9 @@
 
   @Override
   public Resource getTotalResources() {
-    Resource totalResources = Resource.newInstance(
-        (int)Runtime.getRuntime().maxMemory()/(1024*1024),
-        Runtime.getRuntime().availableProcessors());
-    return totalResources;
+    long memory = Runtime.getRuntime().maxMemory();
+    int cores = Runtime.getRuntime().availableProcessors();
+    return createResource(memory, cores);
   }
 
   @Override
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java
index 5eb1e67..b776349 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java
@@ -90,11 +90,7 @@
    */
   private static ContainerLaunchContext createCommonContainerLaunchContext(
       Map<ApplicationAccessType, String> applicationACLs,
-      Credentials credentials) {
-
-    // Application resources
-    Map<String, LocalResource> localResources =
-        new HashMap<String, LocalResource>();
+      Credentials credentials, Map<String, LocalResource> localResources) {
 
     // Application environment
     Map<String, String> environment = new HashMap<String, String>();
@@ -141,6 +137,7 @@
   @VisibleForTesting
   public static ContainerLaunchContext createContainerLaunchContext(
       TezDAGID tezDAGID,
+      Map<String, LocalResource> commonDAGLRs,
       Map<ApplicationAccessType, String> acls,
       ContainerId containerId,
       Map<String, LocalResource> localResources,
@@ -154,7 +151,7 @@
     synchronized (commonContainerSpecLock) {
       if (!commonContainerSpecs.containsKey(tezDAGID)) {
         commonContainerSpec =
-            createCommonContainerLaunchContext(acls, credentials);
+            createCommonContainerLaunchContext(acls, credentials, commonDAGLRs);
         commonContainerSpecs.put(tezDAGID, commonContainerSpec);
       } else {
         commonContainerSpec = commonContainerSpecs.get(tezDAGID);
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index df1b65d..a0f9cb7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -496,8 +496,14 @@
       container.credentials = containerContext.getCredentials();
       container.credentialsChanged = true;
 
+      TezDAGID dagId = null;
+      Map<String, LocalResource> dagLocalResources = null;
+      if (container.appContext.getCurrentDAG() != null) {
+        dagId = container.appContext.getCurrentDAG().getID();
+        dagLocalResources = container.appContext.getCurrentDAG().getLocalResources();
+      }
       ContainerLaunchContext clc = AMContainerHelpers.createContainerLaunchContext(
-          container.appContext.getCurrentDAGID(),
+          dagId, dagLocalResources,
           container.appContext.getApplicationACLs(),
           container.getContainerId(),
           containerContext.getLocalResources(),
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
index c60ef58..9323270 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
@@ -25,7 +25,6 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
@@ -45,7 +44,7 @@
   private TezCounters tezCounters;
   private TezTaskAttemptID successfulAttemptID;
   private String diagnostics;
-  
+
   public TaskFinishedEvent(TezTaskID taskID,
       String vertexName, long startTime, long finishTime,
       TezTaskAttemptID successfulAttemptID,
@@ -85,9 +84,6 @@
     if (diagnostics != null) {
       builder.setDiagnostics(diagnostics);
     }
-    if (tezCounters != null) {
-      builder.setCounters(DagTypeConverters.convertTezCountersToProto(tezCounters));
-    }
     if (successfulAttemptID != null) {
       builder.setSuccessfulTaskAttemptId(successfulAttemptID.toString());
     }
@@ -101,10 +97,6 @@
     if (proto.hasDiagnostics()) {
       this.diagnostics = proto.getDiagnostics();
     }
-    if (proto.hasCounters()) {
-      this.tezCounters = DagTypeConverters.convertTezCountersFromProto(
-          proto.getCounters());
-    }
     if (proto.hasSuccessfulTaskAttemptId()) {
       this.successfulAttemptID =
           TezTaskAttemptID.fromString(proto.getSuccessfulTaskAttemptId());
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
index 075922d..d9cafc7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
@@ -26,7 +26,6 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.app.dag.VertexState;
 import org.apache.tez.dag.app.dag.impl.VertexStats;
 import org.apache.tez.dag.history.HistoryEvent;
@@ -102,9 +101,6 @@
     if (diagnostics != null) {
       builder.setDiagnostics(diagnostics);
     }
-    if (tezCounters != null) {
-      builder.setCounters(DagTypeConverters.convertTezCountersToProto(tezCounters));
-    }
     return builder.build();
   }
 
@@ -116,10 +112,6 @@
     if (proto.hasDiagnostics())  {
       this.diagnostics = proto.getDiagnostics();
     }
-    if (proto.hasCounters()) {
-      this.tezCounters = DagTypeConverters.convertTezCountersFromProto(
-          proto.getCounters());
-    }
   }
 
   @Override
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
index 737fde8..0034feb 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
@@ -45,6 +45,8 @@
 import org.apache.tez.dag.history.events.DAGSubmittedEvent;
 import org.apache.tez.dag.records.TezDAGID;
 
+import com.google.common.annotations.VisibleForTesting;
+
 public class RecoveryService extends AbstractService {
 
   private static final Log LOG = LogFactory.getLog(RecoveryService.class);
@@ -53,6 +55,21 @@
   public static final String RECOVERY_FATAL_OCCURRED_DIR =
       "RecoveryFatalErrorOccurred";
 
+  /**
+   * whether to handle remaining event in the eventqueue when AM is stopped
+   */
+  @VisibleForTesting
+  public static final String TEZ_AM_RECOVERY_HANDLE_REMAINING_EVENT_WHEN_STOPPED =
+      TezConfiguration.TEZ_AM_PREFIX + "recovery.handle_remaining_event_when_stopped";
+
+  /**
+   * by default do not handle remaining event when AM is stopped.
+   * Most of time, true is for recovery unit test
+   */
+  @VisibleForTesting
+  public static final boolean TEZ_AM_RECOVERY_HANDLE_REMAINING_EVENT_WHEN_STOPPED_DEFAULT = false;
+
+
   private LinkedBlockingQueue<DAGHistoryEvent> eventQueue =
       new LinkedBlockingQueue<DAGHistoryEvent>();
   private Set<TezDAGID> completedDAGs = new HashSet<TezDAGID>();
@@ -75,6 +92,7 @@
   private int maxUnflushedEvents;
   private int flushInterval;
   private AtomicBoolean recoveryFatalErrorOccurred = new AtomicBoolean(false);
+  private boolean handleRemainingEventWhenStopped;
 
   public RecoveryService(AppContext appContext) {
     super(RecoveryService.class.getName());
@@ -93,6 +111,10 @@
         TezConfiguration.DAG_RECOVERY_FLUSH_INTERVAL_SECS_DEFAULT);
     maxUnflushedEvents = conf.getInt(TezConfiguration.DAG_RECOVERY_MAX_UNFLUSHED_EVENTS,
         TezConfiguration.DAG_RECOVERY_MAX_UNFLUSHED_EVENTS_DEFAULT);
+
+    handleRemainingEventWhenStopped = conf.getBoolean(
+        TEZ_AM_RECOVERY_HANDLE_REMAINING_EVENT_WHEN_STOPPED,
+        TEZ_AM_RECOVERY_HANDLE_REMAINING_EVENT_WHEN_STOPPED_DEFAULT);
   }
 
   @Override
@@ -150,11 +172,28 @@
   @Override
   public void serviceStop() {
     LOG.info("Stopping RecoveryService");
+
     stopped.set(true);
     if (eventHandlingThread != null) {
       eventHandlingThread.interrupt();
     }
 
+    if (handleRemainingEventWhenStopped) {
+      LOG.info("Handle the remaining events in queue, queue size=" + eventQueue.size());
+      while(!eventQueue.isEmpty()) {
+        synchronized (lock) {
+          try {
+            DAGHistoryEvent event = eventQueue.take();
+            handleRecoveryEvent(event);
+          } catch (Exception e) {
+            // For now, ignore any such errors as these are non-critical
+            // All summary event related errors are handled as critical
+            LOG.warn("Error handling recovery event", e);
+          }
+        }
+      }
+    }
+
     if (summaryStream != null) {
       try {
         LOG.info("Closing Summary Stream");
diff --git a/tez-dag/src/main/proto/HistoryEvents.proto b/tez-dag/src/main/proto/HistoryEvents.proto
index e7ba3b4..93f217f 100644
--- a/tez-dag/src/main/proto/HistoryEvents.proto
+++ b/tez-dag/src/main/proto/HistoryEvents.proto
@@ -136,7 +136,6 @@
   optional int64 finish_time = 3;
   optional int32 state = 4;
   optional string diagnostics = 5;
-  optional TezCountersProto counters = 6;
 }
 
 message TaskStartedProto {
@@ -150,8 +149,7 @@
   optional int64 finish_time = 2;
   optional int32 state = 3;
   optional string diagnostics = 4;
-  optional TezCountersProto counters = 5;
-  optional string successful_task_attempt_id = 6;
+  optional string successful_task_attempt_id = 5;
 }
 
 message TaskAttemptStartedProto {
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
new file mode 100644
index 0000000..7c7e091
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -0,0 +1,259 @@
+/**
+* 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.tez.dag.app;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
+import org.apache.tez.dag.app.launcher.ContainerLauncher;
+import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
+import org.apache.tez.dag.app.rm.container.AMContainerEventType;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+
+import com.google.common.collect.Maps;
+
+@SuppressWarnings("unchecked")
+public class MockDAGAppMaster extends DAGAppMaster {
+  
+  MockContainerLauncher containerLauncher;
+  
+  // mock container launcher does not launch real tasks.
+  // Upon, launch of a container is simulates the container asking for tasks
+  // Upon receiving a task it simulates completion of the tasks
+  // It can be used to preempt the container for a given task
+  public class MockContainerLauncher extends AbstractService implements ContainerLauncher, Runnable {
+
+    BlockingQueue<NMCommunicatorEvent> eventQueue = new LinkedBlockingQueue<NMCommunicatorEvent>();
+    Thread eventHandlingThread;
+    
+    Map<ContainerId, ContainerData> containers = Maps.newConcurrentMap();
+    TaskAttemptListenerImpTezDag taListener;
+    
+    AtomicBoolean startScheduling = new AtomicBoolean(true);
+    AtomicBoolean goFlag;
+    
+    Map<TezTaskID, Integer> preemptedTasks = Maps.newConcurrentMap();
+    
+    public MockContainerLauncher(AtomicBoolean goFlag) {
+      super("MockContainerLauncher");
+      this.goFlag = goFlag;
+    }
+
+    public class ContainerData {
+      ContainerId cId;
+      TezTaskAttemptID taId;
+      String vName;
+      ContainerLaunchContext launchContext;
+      boolean completed;
+      
+      public ContainerData(ContainerId cId, ContainerLaunchContext context) {
+        this.cId = cId;
+        this.launchContext = context;
+      }
+      
+      void clear() {
+        taId = null;
+        vName = null;
+        completed = false;
+        launchContext = null;
+      }
+    }
+    
+    @Override
+    public void serviceStart() throws Exception {
+      taListener = (TaskAttemptListenerImpTezDag) getTaskAttemptListener();
+      eventHandlingThread = new Thread(this);
+      eventHandlingThread.start();
+    }
+
+    @Override
+    public void serviceStop() throws Exception {
+      if (eventHandlingThread != null) {
+        eventHandlingThread.interrupt();
+        eventHandlingThread.join(2000l);
+      }
+    }
+    
+    @Override
+    public void handle(NMCommunicatorEvent event) {
+      switch (event.getType()) {
+      case CONTAINER_LAUNCH_REQUEST:
+        launch((NMCommunicatorLaunchRequestEvent) event);
+        break;
+      case CONTAINER_STOP_REQUEST:
+        stop((NMCommunicatorStopRequestEvent)event);
+        break;
+      }
+    }
+    
+    
+    void waitToGo() {
+      if (goFlag == null) {
+        return;
+      }
+      synchronized (goFlag) {
+        goFlag.set(true);
+        goFlag.notify();
+        try {
+          goFlag.wait();
+        } catch (InterruptedException e) {
+          throw new TezUncheckedException(e);
+        }
+      }
+    }
+    
+    public void startScheduling(boolean value) {
+      startScheduling.set(value);
+    }
+
+    public Map<ContainerId, ContainerData> getContainers() {
+      return containers;
+    }
+    
+    public void preemptContainerForTask(TezTaskID tId, int uptoVersion) {
+      preemptedTasks.put(tId, uptoVersion);
+    }
+    
+    public void preemptContainer(ContainerData cData) {
+      getTaskSchedulerEventHandler().containerCompleted(null, 
+          ContainerStatus.newInstance(cData.cId, null, "Preempted", ContainerExitStatus.PREEMPTED));
+      cData.clear();
+    }
+    
+    void stop(NMCommunicatorStopRequestEvent event) {
+      // remove from simulated container list
+      containers.remove(event.getContainerId());
+      getContext().getEventHandler().handle(
+          new AMContainerEvent(event.getContainerId(), AMContainerEventType.C_NM_STOP_SENT));
+    }
+
+    void launch(NMCommunicatorLaunchRequestEvent event) {
+      // launch container by putting it in simulated container list
+      containers.put(event.getContainerId(), new ContainerData(event.getContainerId(), 
+          event.getContainerLaunchContext()));
+      getContext().getEventHandler().handle(new AMContainerEventLaunched(event.getContainerId()));      
+    }
+    
+    public void waitTillContainersLaunched() throws InterruptedException {
+      while (containers.isEmpty()) {
+        Thread.sleep(50);
+      }
+    }
+
+    @Override
+    public void run() {
+      // wait for test to sync with us and get a reference to us. Go when sync is done
+      waitToGo();
+      while(true) {
+        if (!startScheduling.get()) { // schedule when asked to do so by the test code
+          continue;
+        }
+        for (Map.Entry<ContainerId, ContainerData> entry : containers.entrySet()) {
+          ContainerData cData = entry.getValue();
+          ContainerId cId = entry.getKey();
+          if (cData.taId == null) {
+            // if container is not assigned a task, ask for a task
+            try {
+              ContainerTask cTask = taListener.getTask(new ContainerContext(cId.toString()));
+              if (cTask == null) {
+                continue;
+              }
+              if (cTask.shouldDie()) {
+                containers.remove(cId);
+              } else {
+                cData.taId = cTask.getTaskSpec().getTaskAttemptID();
+                cData.vName = cTask.getTaskSpec().getVertexName();
+              }
+            } catch (IOException e) {
+              e.printStackTrace();
+            }
+          } else if (!cData.completed) {
+            // container is assigned a task and task is not completed
+            // complete the task or preempt the task
+            Integer version = preemptedTasks.get(cData.taId.getTaskID()); 
+            if (version != null && cData.taId.getId() <= version.intValue()) {
+              preemptContainer(cData);
+            } else {
+              // send a done notification
+              TezVertexID vertexId = cData.taId.getTaskID().getVertexID();
+              cData.completed = true;
+              getContext().getEventHandler().handle(
+                  new VertexEventRouteEvent(vertexId, Collections.singletonList(new TezEvent(
+                      new TaskAttemptCompletedEvent(), new EventMetaData(
+                          EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId)))));
+              cData.clear();
+            }
+          }
+        }
+        try {
+          Thread.sleep(10);
+        } catch (InterruptedException e) {
+          System.out.println("Interrupted in mock container launcher thread");
+          break;
+        }
+      }
+    }
+    
+  }
+
+  public MockDAGAppMaster(ApplicationAttemptId applicationAttemptId, ContainerId containerId,
+      String nmHost, int nmPort, int nmHttpPort, Clock clock, long appSubmitTime,
+      boolean isSession, String workingDirectory, AtomicBoolean launcherGoFlag) {
+    super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime,
+        isSession, workingDirectory);
+    containerLauncher = new MockContainerLauncher(launcherGoFlag);
+  }
+  
+  // use mock container launcher for tests
+  @Override
+  protected ContainerLauncher createContainerLauncher(final AppContext context)
+      throws UnknownHostException {
+    return containerLauncher;
+  }
+  
+  public MockContainerLauncher getContainerLauncher() {
+    return containerLauncher;
+  }
+}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java
new file mode 100644
index 0000000..7e408e1
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java
@@ -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.tez.dag.app;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tez.client.LocalClient;
+
+public class MockLocalClient extends LocalClient {
+  MockDAGAppMaster mockApp;
+  AtomicBoolean mockAppLauncherGoFlag;
+  
+  public MockLocalClient(AtomicBoolean mockAppLauncherGoFlag) {
+    this.mockAppLauncherGoFlag = mockAppLauncherGoFlag;
+  }
+  
+  protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemptId,
+      ContainerId cId, String currentHost, int nmPort, int nmHttpPort,
+      Clock clock, long appSubmitTime, boolean isSession, String userDir) {
+    mockApp = new MockDAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
+        new SystemClock(), appSubmitTime, isSession, userDir, mockAppLauncherGoFlag);
+    return mockApp;
+  }
+  
+  public MockDAGAppMaster getMockApp() {
+    return mockApp;
+  }
+}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockTezClient.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockTezClient.java
new file mode 100644
index 0000000..617415e
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockTezClient.java
@@ -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.tez.dag.app;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.client.FrameworkClient;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.dag.api.TezConfiguration;
+
+public class MockTezClient extends TezClient {
+  MockLocalClient client;
+  
+  MockTezClient(String name, TezConfiguration tezConf, boolean isSession,
+      Map<String, LocalResource> localResources, Credentials credentials,
+      AtomicBoolean mockAppLauncherGoFlag) {
+    super(name, tezConf, isSession, localResources, credentials);
+    this.client = new MockLocalClient(mockAppLauncherGoFlag);
+  }
+  
+  protected FrameworkClient createFrameworkClient() {
+    return client;
+  }
+  
+  public MockLocalClient getLocalClient() {
+    return client;
+  }
+
+}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java
new file mode 100644
index 0000000..1bab0d2
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestMockDAGAppMaster.java
@@ -0,0 +1,138 @@
+/**
+* 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.tez.dag.app;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.dag.app.MockDAGAppMaster.MockContainerLauncher;
+import org.apache.tez.dag.app.MockDAGAppMaster.MockContainerLauncher.ContainerData;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+@SuppressWarnings("deprecation")
+public class TestMockDAGAppMaster {
+  static Configuration defaultConf;
+  static FileSystem localFs;
+  static Path workDir;
+  
+  static {
+    try {
+      defaultConf = new Configuration(false);
+      defaultConf.set("fs.defaultFS", "file:///");
+      defaultConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true);
+      localFs = FileSystem.getLocal(defaultConf);
+      workDir = new Path(new Path(System.getProperty("test.build.data", "/tmp")),
+          "TestDAGAppMaster").makeQualified(localFs);
+    } catch (IOException e) {
+      throw new RuntimeException("init failure", e);
+    }
+  }
+  
+  @Test (timeout = 5000)
+  public void testLocalResourceSetup() throws Exception {
+    TezConfiguration tezconf = new TezConfiguration(defaultConf);
+    
+    MockTezClient tezClient = new MockTezClient("testMockAM", tezconf, true, null, null, null);
+    tezClient.start();
+    
+    MockDAGAppMaster mockApp = tezClient.getLocalClient().getMockApp();
+    MockContainerLauncher mockLauncher = mockApp.getContainerLauncher();
+    mockLauncher.startScheduling(false);
+    
+    Map<String, LocalResource> lrDAG = Maps.newHashMap();
+    String lrName1 = "LR1";
+    lrDAG.put(lrName1, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test"),
+        LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1));
+    Map<String, LocalResource> lrVertex = Maps.newHashMap();
+    String lrName2 = "LR2";
+    lrVertex.put(lrName2, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test1"),
+        LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1));
+
+    DAG dag = DAG.create("test").addTaskLocalFiles(lrDAG);
+    Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5).addTaskLocalFiles(lrVertex);
+    dag.addVertex(vA);
+
+    DAGClient dagClient = tezClient.submitDAG(dag);
+    mockLauncher.waitTillContainersLaunched();
+    ContainerData cData = mockLauncher.getContainers().values().iterator().next();
+    ContainerLaunchContext launchContext = cData.launchContext;
+    Map<String, LocalResource> taskLR = launchContext.getLocalResources();
+    // verify tasks are launched with both DAG and task resources.
+    Assert.assertTrue(taskLR.containsKey(lrName1));
+    Assert.assertTrue(taskLR.containsKey(lrName2));
+    
+    mockLauncher.startScheduling(true);
+    dagClient.waitForCompletion();
+    Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState());
+    tezClient.stop();
+  }
+  
+  @Test (timeout = 10000)
+  public void testMultipleSubmissions() throws Exception {
+    Map<String, LocalResource> lrDAG = Maps.newHashMap();
+    String lrName1 = "LR1";
+    lrDAG.put(lrName1, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test"),
+        LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1));
+    Map<String, LocalResource> lrVertex = Maps.newHashMap();
+    String lrName2 = "LR2";
+    lrVertex.put(lrName2, LocalResource.newInstance(URL.newInstance("file", "localhost", 0, "/test1"),
+        LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1));
+
+    DAG dag = DAG.create("test").addTaskLocalFiles(lrDAG);
+    Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5).addTaskLocalFiles(lrVertex);
+    dag.addVertex(vA);
+
+    TezConfiguration tezconf = new TezConfiguration(defaultConf);
+    
+    MockTezClient tezClient = new MockTezClient("testMockAM", tezconf, true, null, null, null);
+    tezClient.start();
+    DAGClient dagClient = tezClient.submitDAG(dag);
+    dagClient.waitForCompletion();
+    Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState());
+    tezClient.stop();
+    
+    // submit the same DAG again to verify it can be done.
+    tezClient = new MockTezClient("testMockAM", tezconf, true, null, null, null);
+    tezClient.start();
+    dagClient = tezClient.submitDAG(dag);
+    dagClient.waitForCompletion();
+    Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState());
+    tezClient.stop();
+
+  }
+  
+
+}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
new file mode 100644
index 0000000..cfb1c9f
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
@@ -0,0 +1,203 @@
+/**
+* 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.tez.dag.app;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.Edge;
+import org.apache.tez.dag.api.EdgeProperty;
+import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
+import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
+import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.dag.app.MockDAGAppMaster.MockContainerLauncher;
+import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
+import org.apache.tez.dag.app.dag.impl.DAGImpl;
+import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.junit.Assert;
+import org.junit.Test;
+
+@SuppressWarnings("deprecation")
+public class TestPreemption {
+  
+  static Configuration defaultConf;
+  static FileSystem localFs;
+  static Path workDir;
+  
+  static {
+    try {
+      defaultConf = new Configuration(false);
+      defaultConf.set("fs.defaultFS", "file:///");
+      defaultConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true);
+      localFs = FileSystem.getLocal(defaultConf);
+      workDir = new Path(new Path(System.getProperty("test.build.data", "/tmp")),
+          "TestDAGAppMaster").makeQualified(localFs);
+    } catch (IOException e) {
+      throw new RuntimeException("init failure", e);
+    }
+  }
+  
+  MockDAGAppMaster mockApp;    
+  MockContainerLauncher mockLauncher;
+  
+  int dagCount = 0;
+  
+  DAG createDAG(DataMovementType dmType) {
+    DAG dag = DAG.create("test-" + dagCount++);
+    Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5);
+    Vertex vB = Vertex.create("B", ProcessorDescriptor.create("Proc.class"), 5);
+    Edge eAB = Edge.create(vA, vB, 
+    EdgeProperty.create(dmType, DataSourceType.PERSISTED,
+        SchedulingType.SEQUENTIAL, OutputDescriptor.create("O.class"),
+        InputDescriptor.create("I.class")));
+    
+    dag.addVertex(vA).addVertex(vB).addEdge(eAB);
+    return dag;
+  }
+  
+  @Test (timeout = 5000)
+  public void testPreemptionWithoutSession() throws Exception {
+    System.out.println("TestPreemptionWithoutSession");
+    TezConfiguration tezconf = new TezConfiguration(defaultConf);
+    tezconf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 0);
+    AtomicBoolean mockAppLauncherGoFlag = new AtomicBoolean(false);
+    MockTezClient tezClient = new MockTezClient("testPreemption", tezconf, false, null, null,
+        mockAppLauncherGoFlag);
+    tezClient.start();
+    
+    DAGClient dagClient = tezClient.submitDAG(createDAG(DataMovementType.SCATTER_GATHER));
+    // now the MockApp has been started. sync with it to get the launcher
+    syncWithMockAppLauncher(false, mockAppLauncherGoFlag, tezClient);
+
+    DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG();
+    int vertexIndex = 0;
+    int upToTaskVersion = 3;
+    TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), vertexIndex);
+    TezTaskAttemptID taId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0);
+
+    mockLauncher.preemptContainerForTask(taId.getTaskID(), upToTaskVersion);
+    mockLauncher.startScheduling(true);
+    
+    dagClient.waitForCompletion();
+    Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState());
+
+    for (int i=0; i<=upToTaskVersion; ++i) {
+      TezTaskAttemptID testTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), i);      
+      TaskAttemptImpl taImpl = dagImpl.getTaskAttempt(testTaId);
+      Assert.assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState());
+    }
+    
+    tezClient.stop();
+  }
+  
+  @Test (timeout = 30000)
+  public void testPreemptionWithSession() throws Exception {
+    System.out.println("TestPreemptionWithSession");
+    MockTezClient tezClient = createTezSession();
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 0, "Scatter-Gather");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 0, "Scatter-Gather");
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.BROADCAST), 0, "Broadcast");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.BROADCAST), 0, "Broadcast");
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 0, "1-1");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 0, "1-1");
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 1, "Scatter-Gather");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 1, "Scatter-Gather");
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.BROADCAST), 1, "Broadcast");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.BROADCAST), 1, "Broadcast");
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 1, "1-1");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 1, "1-1");
+    tezClient.stop();
+  }
+  
+  MockTezClient createTezSession() throws Exception {
+    TezConfiguration tezconf = new TezConfiguration(defaultConf);
+    tezconf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 0);
+    AtomicBoolean mockAppLauncherGoFlag = new AtomicBoolean(false);
+    MockTezClient tezClient = new MockTezClient("testPreemption", tezconf, true, null, null,
+        mockAppLauncherGoFlag);
+    tezClient.start();
+    syncWithMockAppLauncher(false, mockAppLauncherGoFlag, tezClient);
+    return tezClient;
+  }
+  
+  void syncWithMockAppLauncher(boolean allowScheduling, AtomicBoolean mockAppLauncherGoFlag, 
+      MockTezClient tezClient) throws Exception {
+    synchronized (mockAppLauncherGoFlag) {
+      while (!mockAppLauncherGoFlag.get()) {
+        mockAppLauncherGoFlag.wait();
+      }
+      mockApp = tezClient.getLocalClient().getMockApp();
+      mockLauncher = mockApp.getContainerLauncher();
+      mockLauncher.startScheduling(allowScheduling);
+      mockAppLauncherGoFlag.notify();
+    }     
+  }
+  
+  void testPreemptionSingle(MockTezClient tezClient, DAG dag, int vertexIndex, String info)
+      throws Exception {
+    testPreemptionJob(tezClient, dag, vertexIndex, 0, info + "-Single");
+  }
+
+  void testPreemptionMultiple(MockTezClient tezClient, DAG dag, int vertexIndex, String info)
+      throws Exception {
+    testPreemptionJob(tezClient, dag, vertexIndex, 3, info + "-Multiple");
+  }
+
+  void testPreemptionJob(MockTezClient tezClient, DAG dag, int vertexIndex,
+      int upToTaskVersion, String info) throws Exception {
+    System.out.println("TestPreemption - Running - " + info);
+    TezConfiguration tezconf = new TezConfiguration(defaultConf);
+    tezconf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 0);
+    
+    mockLauncher.startScheduling(false); // turn off scheduling to block DAG before submitting it
+    DAGClient dagClient = tezClient.submitDAG(dag);
+    
+    DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG();
+    TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), vertexIndex);
+    TezTaskAttemptID taId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0);
+
+    mockLauncher.preemptContainerForTask(taId.getTaskID(), upToTaskVersion);
+    mockLauncher.startScheduling(true);
+    
+    dagClient.waitForCompletion();
+    Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState());
+    
+    for (int i=0; i<=upToTaskVersion; ++i) {
+      TezTaskAttemptID testTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), i);      
+      TaskAttemptImpl taImpl = dagImpl.getTaskAttempt(testTaId);
+      Assert.assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState());
+    }
+    
+    System.out.println("TestPreemption - Done running - " + info);
+  }
+}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
index d10a8b3..2d7cf65 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
@@ -111,7 +111,7 @@
   private static final Log LOG = LogFactory.getLog(TestDAGImpl.class);
   private DAGPlan dagPlan;
   private TezDAGID dagId;
-  private Configuration conf;
+  private static Configuration conf;
   private DrainDispatcher dispatcher;
   private Credentials fsTokens;
   private AppContext appContext;
@@ -344,7 +344,7 @@
   }
 
   // Create a plan with 3 vertices: A, B, C. Group(A,B)->C
-  private DAGPlan createGroupDAGPlan() {
+  static DAGPlan createGroupDAGPlan() {
     LOG.info("Setting up group dag plan");
     int dummyTaskCount = 1;
     Resource dummyTaskResource = Resource.newInstance(1, 1);
@@ -378,10 +378,10 @@
     dag.addVertex(v2);
     dag.addVertex(v3);
     dag.addEdge(e1);
-    return dag.createDag(conf);
+    return dag.createDag(conf, null, null, null, true);
   }
 
-  private DAGPlan createTestDAGPlan() {
+  public static DAGPlan createTestDAGPlan() {
     LOG.info("Setting up dag plan");
     DAGPlan dag = DAGPlan.newBuilder()
         .setName("testverteximpl")
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
new file mode 100644
index 0000000..da0186e
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
@@ -0,0 +1,514 @@
+/**
+ * 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.tez.dag.app.dag.impl;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.DAGState;
+import org.apache.tez.dag.app.dag.VertexState;
+import org.apache.tez.dag.app.dag.event.DAGAppMasterEventDAGFinished;
+import org.apache.tez.dag.app.dag.event.DAGEvent;
+import org.apache.tez.dag.app.dag.event.DAGEventRecoverEvent;
+import org.apache.tez.dag.app.dag.event.DAGEventType;
+import org.apache.tez.dag.app.dag.event.VertexEvent;
+import org.apache.tez.dag.app.dag.event.VertexEventRecoverVertex;
+import org.apache.tez.dag.history.events.DAGCommitStartedEvent;
+import org.apache.tez.dag.history.events.DAGFinishedEvent;
+import org.apache.tez.dag.history.events.DAGInitializedEvent;
+import org.apache.tez.dag.history.events.DAGStartedEvent;
+import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
+import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
+import org.apache.tez.dag.records.TezDAGID;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+@SuppressWarnings({ "unchecked", "rawtypes" })
+public class TestDAGRecovery {
+
+  private DAGImpl dag;
+  private EventHandler mockEventHandler;
+
+  private String user = "root";
+  private String dagName = "dag1";
+
+  private AppContext mockAppContext;
+  private ApplicationId appId = ApplicationId.newInstance(
+      System.currentTimeMillis(), 1);
+  private TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+  private long initTime = 100L;
+  private long startTime = initTime + 200L;
+  private long commitStartTime = startTime + 200L;
+  private long finishTime = commitStartTime + 200L;
+
+  @Before
+  public void setUp() {
+
+    mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+    when(mockAppContext.getCurrentDAG().getDagUGI()).thenReturn(null);
+    mockEventHandler = mock(EventHandler.class);
+
+    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+    dag =
+        new DAGImpl(dagId, new Configuration(), dagPlan, mockEventHandler,
+            mock(TaskAttemptListener.class), new Credentials(),
+            new SystemClock(), user, mock(TaskHeartbeatHandler.class),
+            mockAppContext);
+  }
+
+  private void assertNewState() {
+    assertEquals(0, dag.getVertices().size());
+    assertEquals(0, dag.edges.size());
+    assertNull(dag.dagScheduler);
+    assertFalse(dag.recoveryCommitInProgress);
+    assertEquals(0, dag.recoveredGroupCommits.size());
+  }
+
+  private void restoreFromDAGInitializedEvent() {
+    DAGState recoveredState =
+        dag.restoreFromEvent(new DAGInitializedEvent(dagId, initTime, user,
+            dagName));
+    assertEquals(DAGState.INITED, recoveredState);
+    assertEquals(initTime, dag.initTime);
+    assertEquals(6, dag.getVertices().size());
+    assertEquals(6, dag.edges.size());
+    assertNotNull(dag.dagScheduler);
+  }
+
+  private void restoreFromDAGStartedEvent() {
+    DAGState recoveredState =
+        dag.restoreFromEvent(new DAGStartedEvent(dagId, startTime, user,
+            dagName));
+    assertEquals(startTime, dag.startTime);
+    assertEquals(DAGState.RUNNING, recoveredState);
+  }
+
+  private void restoreFromDAGCommitStartedEvent() {
+    DAGState recoveredState =
+        dag.restoreFromEvent(new DAGCommitStartedEvent(dagId, commitStartTime));
+    assertTrue(dag.recoveryCommitInProgress);
+    assertEquals(DAGState.RUNNING, recoveredState);
+  }
+
+  private void restoreFromVertexGroupCommitStartedEvent() {
+    DAGState recoveredState =
+        dag.restoreFromEvent(new VertexGroupCommitStartedEvent(dagId, "g1",
+            commitStartTime));
+    assertEquals(1, dag.recoveredGroupCommits.size());
+    assertFalse(dag.recoveredGroupCommits.get("g1").booleanValue());
+    assertEquals(DAGState.RUNNING, recoveredState);
+  }
+
+  private void restoreFromVertexGroupCommitFinishedEvent() {
+    DAGState recoveredState =
+        dag.restoreFromEvent(new VertexGroupCommitFinishedEvent(dagId, "g1",
+            commitStartTime + 100L));
+    assertEquals(1, dag.recoveredGroupCommits.size());
+    assertTrue(dag.recoveredGroupCommits.get("g1").booleanValue());
+    assertEquals(DAGState.RUNNING, recoveredState);
+  }
+
+  private void restoreFromDAGFinishedEvent(DAGState finalState) {
+    DAGState recoveredState =
+        dag.restoreFromEvent(new DAGFinishedEvent(dagId, startTime, finishTime,
+            finalState, "", new TezCounters(), user, dagName));
+    assertEquals(finishTime, dag.finishTime);
+    assertFalse(dag.recoveryCommitInProgress);
+    assertEquals(finalState, recoveredState);
+  }
+
+  /**
+   * New -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_FromNew() {
+    assertNewState();
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+
+    ArgumentCaptor<DAGEvent> eventCaptor =
+        ArgumentCaptor.forClass(DAGEvent.class);
+    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+    List<DAGEvent> events = eventCaptor.getAllValues();
+    assertEquals(2, events.size());
+    assertEquals(DAGEventType.DAG_INIT, events.get(0).getType());
+    assertEquals(DAGEventType.DAG_START, events.get(1).getType());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_FromInited() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+
+    assertEquals(DAGState.RUNNING, dag.getState());
+    // send recover event to 2 root vertex
+    ArgumentCaptor<VertexEvent> eventCaptor =
+        ArgumentCaptor.forClass(VertexEvent.class);
+    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+    List<VertexEvent> vertexEvents = eventCaptor.getAllValues();
+    assertEquals(2, vertexEvents.size());
+    for (VertexEvent vEvent : vertexEvents) {
+      assertTrue(vEvent instanceof VertexEventRecoverVertex);
+      VertexEventRecoverVertex recoverEvent = (VertexEventRecoverVertex) vEvent;
+      assertEquals(VertexState.RUNNING, recoverEvent.getDesiredState());
+    }
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_FromStarted() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+
+    assertEquals(DAGState.RUNNING, dag.getState());
+    // send recover event to 2 root vertex
+    ArgumentCaptor<VertexEvent> eventCaptor =
+        ArgumentCaptor.forClass(VertexEvent.class);
+    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+    List<VertexEvent> vertexEvents = eventCaptor.getAllValues();
+    assertEquals(2, vertexEvents.size());
+    for (VertexEvent vEvent : vertexEvents) {
+      assertTrue(vEvent instanceof VertexEventRecoverVertex);
+      VertexEventRecoverVertex recoverEvent = (VertexEventRecoverVertex) vEvent;
+      assertEquals(VertexState.RUNNING, recoverEvent.getDesiredState());
+    }
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromDAGFinishedEvent (SUCCEEDED) -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_Finished_SUCCEEDED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    restoreFromDAGFinishedEvent(DAGState.SUCCEEDED);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.SUCCEEDED, dag.getState());
+
+    // recover all the vertices to SUCCEED
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    int i = 0;
+    for (; i < 6; ++i) {
+      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+      VertexEventRecoverVertex recoverEvent =
+          (VertexEventRecoverVertex) events.get(i);
+      assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.SUCCEEDED, dagFinishedEvent.getDAGState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromDAGFinishedEvent(FAILED) -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_Finished_FAILED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    restoreFromDAGFinishedEvent(DAGState.FAILED);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.FAILED, dag.getState());
+
+    // recover all the vertices to FAILED
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    int i = 0;
+    for (; i < 6; ++i) {
+      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+      VertexEventRecoverVertex recoverEvent =
+          (VertexEventRecoverVertex) events.get(i);
+      assertEquals(VertexState.FAILED, recoverEvent.getDesiredState());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent -> ->
+   * restoreFromDAGFinishedEvent -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_Finished_KILLED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    restoreFromDAGFinishedEvent(DAGState.KILLED);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.KILLED, dag.getState());
+
+    // recover all the vertices to KILLED
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    int i = 0;
+    for (; i < 6; ++i) {
+      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+      VertexEventRecoverVertex recoverEvent =
+          (VertexEventRecoverVertex) events.get(i);
+      assertEquals(VertexState.KILLED, recoverEvent.getDesiredState());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.KILLED, dagFinishedEvent.getDAGState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent -> ->
+   * restoreFromDAGFinishedEvent -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_Finished_ERROR() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    restoreFromDAGFinishedEvent(DAGState.ERROR);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.ERROR, dag.getState());
+
+    // recover all the vertices to KILLED
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    int i = 0;
+    for (; i < 6; ++i) {
+      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+      VertexEventRecoverVertex recoverEvent =
+          (VertexEventRecoverVertex) events.get(i);
+      assertEquals(VertexState.FAILED, recoverEvent.getDesiredState());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.ERROR, dagFinishedEvent.getDAGState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromDAG_COMMIT_STARTED -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_COMMIT_STARTED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    restoreFromDAGCommitStartedEvent();
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.FAILED, dag.getState());
+
+    // recover all the vertices to SUCCEEDED
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    int i = 0;
+    for (; i < 6; ++i) {
+      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+      VertexEventRecoverVertex recoverEvent =
+          (VertexEventRecoverVertex) events.get(i);
+      assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromDAG_COMMIT_STARTED -> -> restoreFromDAGFinished (SUCCEEDED)->
+   * RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_COMMIT_STARTED_Finished_SUCCEEDED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+
+    restoreFromDAGCommitStartedEvent();
+    restoreFromDAGFinishedEvent(DAGState.SUCCEEDED);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.SUCCEEDED, dag.getState());
+
+    // recover all the vertices to SUCCEED
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    int i = 0;
+    for (; i < 6; ++i) {
+      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+      VertexEventRecoverVertex recoverEvent =
+          (VertexEventRecoverVertex) events.get(i);
+      assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.SUCCEEDED, dagFinishedEvent.getDAGState());
+
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromVERTEX_GROUP_COMMIT_STARTED -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_GROUP_COMMIT_STARTED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    restoreFromVertexGroupCommitStartedEvent();
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.FAILED, dag.getState());
+
+    // recover all the vertices to SUCCEEDED
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    int i = 0;
+    for (; i < 6; ++i) {
+      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+      VertexEventRecoverVertex recoverEvent =
+          (VertexEventRecoverVertex) events.get(i);
+      assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromVERTEX_GROUP_COMMIT_STARTED -> VERTEX_GROUP_COMMIT_FINISHED ->
+   * RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_GROUP_COMMIT_STARTED_FINISHED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+
+    restoreFromVertexGroupCommitStartedEvent();
+    restoreFromVertexGroupCommitFinishedEvent();
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.RUNNING, dag.getState());
+
+    // send recover event to 2 root vertex
+    verify(mockEventHandler, times(2)).handle(
+        any(VertexEventRecoverVertex.class));
+    assertEquals(DAGState.RUNNING, dag.getState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromVERTEX_GROUP_COMMIT_STARTED -> VERTEX_GROUP_COMMIT_FINISHED ->
+   * restoreFromDAG_Finished -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_GROUP_COMMIT_Finished() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+
+    restoreFromVertexGroupCommitStartedEvent();
+    restoreFromVertexGroupCommitFinishedEvent();
+    restoreFromDAGFinishedEvent(DAGState.SUCCEEDED);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+
+    // recover all the vertices to SUCCEEDED
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    int i = 0;
+    for (; i < 6; ++i) {
+      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
+      VertexEventRecoverVertex recoverEvent =
+          (VertexEventRecoverVertex) events.get(i);
+      assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.SUCCEEDED, dagFinishedEvent.getDAGState());
+  }
+
+}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
index 3633a85..8e134f2 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
@@ -67,6 +67,7 @@
 import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.event.DAGEvent;
+import org.apache.tez.dag.app.dag.event.DAGEventCounterUpdate;
 import org.apache.tez.dag.app.dag.event.DAGEventType;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventContainerTerminated;
@@ -423,6 +424,9 @@
     verifyEventType(
         arg.getAllValues().subList(expectedEventsAtRunning,
             expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+    verifyEventType(
+        arg.getAllValues().subList(expectedEventsAtRunning,
+            expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
 
     taImpl.handle(new TaskAttemptEventContainerTerminated(taskAttemptID,
         "Terminated"));
@@ -571,6 +575,9 @@
     verifyEventType(
         arg.getAllValues().subList(expectedEventsAtRunning,
             expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+    verifyEventType(
+        arg.getAllValues().subList(expectedEventsAtRunning,
+            expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
 
     taImpl.handle(new TaskAttemptEventContainerTerminated(taskAttemptID,
         "Terminated"));
@@ -658,6 +665,9 @@
     verifyEventType(
         arg.getAllValues().subList(expectedEventsAtRunning,
             expectedEventsAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+    verifyEventType(
+        arg.getAllValues().subList(expectedEventsAtRunning,
+            expectedEventsAfterTerminating), DAGEventCounterUpdate.class, 1);
 
     taImpl.handle(new TaskAttemptEvent(taskAttemptID,
         TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM));
@@ -744,6 +754,9 @@
     verifyEventType(
         arg.getAllValues().subList(expectedEventsAtRunning,
             expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+    verifyEventType(
+        arg.getAllValues().subList(expectedEventsAtRunning,
+            expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
 
     // Send out a Node Failure.
     taImpl.handle(new TaskAttemptEventNodeFailed(taskAttemptID, "NodeDecomissioned"));
@@ -838,6 +851,9 @@
     verifyEventType(
         arg.getAllValues().subList(expectedEventsAtRunning,
             expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+    verifyEventType(
+        arg.getAllValues().subList(expectedEventsAtRunning,
+            expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
 
     // Send out a Node Failure.
     taImpl.handle(new TaskAttemptEventNodeFailed(taskAttemptID, "NodeDecomissioned"));
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
new file mode 100644
index 0000000..a443a35
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
@@ -0,0 +1,224 @@
+/**
+ * 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.tez.dag.app.dag.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.ContainerContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
+import org.apache.tez.dag.app.dag.event.DAGEventCounterUpdate;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
+import org.apache.tez.dag.app.dag.event.TaskEventTAUpdate;
+import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+@SuppressWarnings({ "unchecked", "rawtypes" })
+public class TestTaskAttemptRecovery {
+
+  private TaskAttemptImpl ta;
+  private EventHandler mockEventHandler;
+  private long startTime = System.currentTimeMillis();
+  private long finishTime = startTime + 5000;
+
+  private TezTaskAttemptID taId = mock(TezTaskAttemptID.class);
+  private String vertexName = "v1";
+
+  @Before
+  public void setUp() {
+    mockEventHandler = mock(EventHandler.class);
+    TezTaskID taskId =
+        TezTaskID.fromString("task_1407371892933_0001_1_00_000000");
+    ta =
+        new TaskAttemptImpl(taskId, 0, mockEventHandler,
+            mock(TaskAttemptListener.class), new Configuration(),
+            new SystemClock(), mock(TaskHeartbeatHandler.class),
+            mock(AppContext.class), false, Resource.newInstance(1, 1),
+            mock(ContainerContext.class), false);
+  }
+
+  private void restoreFromTAStartEvent() {
+    TaskAttemptState recoveredState =
+        ta.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName,
+            startTime, mock(ContainerId.class), mock(NodeId.class), "", ""));
+    assertEquals(startTime, ta.getLaunchTime());
+    assertEquals(TaskAttemptState.RUNNING, recoveredState);
+  }
+
+  private void restoreFromTAFinishedEvent(TaskAttemptState state) {
+    String diag = "test_diag";
+    TezCounters counters = mock(TezCounters.class);
+
+    TaskAttemptState recoveredState =
+        ta.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+            startTime, finishTime, state, diag, counters));
+    assertEquals(startTime, ta.getLaunchTime());
+    assertEquals(finishTime, ta.getFinishTime());
+    assertEquals(counters, ta.reportedStatus.counters);
+    assertEquals(1.0f, ta.reportedStatus.progress, 1e-6);
+    assertEquals(state, ta.reportedStatus.state);
+    assertEquals(1, ta.getDiagnostics().size());
+    assertEquals(diag, ta.getDiagnostics().get(0));
+    assertEquals(state, recoveredState);
+  }
+
+  private void verifyEvents(List<Event> events, Class<? extends Event> eventClass,
+      int expectedTimes) {
+    int actualTimes = 0;
+    for (Event event : events) {
+      if (eventClass.isInstance(event)) {
+        actualTimes ++;
+      }
+    }
+    assertEquals(expectedTimes, actualTimes);
+  }
+
+  /**
+   * No any event to restore -> RecoverTransition
+   */
+  @Test
+  public void testTARecovery_NEW() {
+    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+    assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
+
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    assertEquals(2, events.size());
+    verifyEvents(events, TaskEventTAUpdate.class, 1);
+    // one for task killed
+    verifyEvents(events, DAGEventCounterUpdate.class, 1);
+  }
+
+  /**
+   * restoreFromTAStartEvent -> RecoverTransition
+   */
+  @Test
+  public void testTARecovery_START() {
+    restoreFromTAStartEvent();
+
+    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+    assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
+
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(3)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    assertEquals(3, events.size());
+    verifyEvents(events, TaskEventTAUpdate.class, 1);
+    // one for task launch, one for task killed
+    verifyEvents(events, DAGEventCounterUpdate.class, 2);
+  }
+
+  /**
+   * restoreFromTAStartEvent -> restoreFromTAFinished (SUCCEED)
+   * -> RecoverTransition
+   */
+  @Test
+  public void testTARecovery_SUCCEED() {
+    restoreFromTAStartEvent();
+    restoreFromTAFinishedEvent(TaskAttemptState.SUCCEEDED);
+
+    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+    assertEquals(TaskAttemptStateInternal.SUCCEEDED, ta.getInternalState());
+
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    assertEquals(2, events.size());
+    // one for task launch, one for task succeeded
+    verifyEvents(events, DAGEventCounterUpdate.class, 2);
+  }
+
+  /**
+   * restoreFromTAStartEvent -> restoreFromTAFinished (KILLED)
+   * -> RecoverTransition
+   */
+  @Test
+  public void testTARecovery_KIILED() {
+    restoreFromTAStartEvent();
+    restoreFromTAFinishedEvent(TaskAttemptState.KILLED);
+
+    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+    assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
+
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    assertEquals(2, events.size());
+    // one for task launch, one for task killed
+    verifyEvents(events, DAGEventCounterUpdate.class, 2);
+  }
+
+  /**
+   * restoreFromTAStartEvent -> restoreFromTAFinished (FAILED)
+   * -> RecoverTransition
+   */
+  @Test
+  public void testTARecovery_FAILED() {
+    restoreFromTAStartEvent();
+    restoreFromTAFinishedEvent(TaskAttemptState.FAILED);
+
+    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+    assertEquals(TaskAttemptStateInternal.FAILED, ta.getInternalState());
+
+    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
+    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
+    List<Event> events = eventCaptor.getAllValues();
+    assertEquals(2, events.size());
+    // one for task launch, one for task killed
+    verifyEvents(events, DAGEventCounterUpdate.class, 2);
+  }
+
+  /**
+   * restoreFromTAFinishedEvent ( no TAStartEvent before TAFinishedEvent )
+   */
+  @Test
+  public void testRecover_FINISH_BUT_NO_START() {
+    try {
+      restoreFromTAFinishedEvent(TaskAttemptState.SUCCEEDED);
+      fail("Should fail due to no TaskAttemptStartEvent before TaskAttemptFinishedEvent");
+    } catch (Throwable e) {
+      assertEquals("Finished Event seen but"
+          + " no Started Event was encountered earlier", e.getMessage());
+    }
+  }
+}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
index 9da3eb9..bd13ffe 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
@@ -19,29 +19,28 @@
 package org.apache.tez.dag.app.dag.impl;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 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.security.Credentials;
 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.LocalResource;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.client.VertexStatus.State;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
 import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.app.AppContext;
@@ -59,110 +58,532 @@
 import org.apache.tez.dag.app.dag.event.TaskEventRecoverTask;
 import org.apache.tez.dag.app.dag.event.TaskEventType;
 import org.apache.tez.dag.app.dag.event.VertexEventType;
-import org.apache.tez.dag.app.rm.container.AMContainer;
 import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
 import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
+import org.apache.tez.dag.history.events.TaskFinishedEvent;
 import org.apache.tez.dag.history.events.TaskStartedEvent;
 import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.runtime.api.OutputCommitter;
+import org.apache.tez.runtime.api.OutputCommitterContext;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+
 public class TestTaskRecovery {
 
-  private static final Log LOG = LogFactory.getLog(TestTaskImpl.class);
-
-  private int taskCounter = 0;
-  private int taskAttemptCounter = 0;
-
-  private Configuration conf;
-  private TaskAttemptListener taskAttemptListener;
-  private TaskHeartbeatHandler taskHeartbeatHandler;
-  private Credentials credentials;
-  private Clock clock;
-  private ApplicationId appId;
-  private TezDAGID dagId;
-  private TezVertexID vertexId;
-  private Vertex vertex;
-  private AppContext appContext;
-  private Resource taskResource;
-  private Map<String, LocalResource> localResources;
-  private Map<String, String> environment;
-  private String javaOpts;
-  private boolean leafVertex;
-  private ContainerContext containerContext;
-  private ContainerId mockContainerId;
-  private Container mockContainer;
-  private AMContainer mockAMContainer;
-  private NodeId mockNodeId;
-
   private TaskImpl task;
   private DrainDispatcher dispatcher;
 
-  class TaskEventHandler implements EventHandler<TaskEvent> {
+  private int taskAttemptCounter = 0;
+
+  private Configuration conf = new Configuration();
+  private AppContext mockAppContext;
+  private ApplicationId appId = ApplicationId.newInstance(
+      System.currentTimeMillis(), 1);
+  private TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+  private TezVertexID vertexId = TezVertexID.getInstance(dagId, 1);
+  private Vertex vertex;
+  private String vertexName = "v1";
+  private long taskScheduledTime = 100L;
+  private long taskStartTime = taskScheduledTime + 100L;
+  private long taskFinishTime = taskStartTime + 100L;
+  private TaskAttemptEventHandler taEventHandler =
+      new TaskAttemptEventHandler();
+
+  private class TaskEventHandler implements EventHandler<TaskEvent> {
     @Override
     public void handle(TaskEvent event) {
       task.handle(event);
     }
   }
 
-  class TaskAttemptEventHandler implements EventHandler<TaskAttemptEvent> {
+  private class TaskAttemptEventHandler implements
+      EventHandler<TaskAttemptEvent> {
+
+    private List<TaskAttemptEvent> events = Lists.newArrayList();
+
     @Override
     public void handle(TaskAttemptEvent event) {
+      events.add(event);
       ((TaskAttemptImpl) task.getAttempt(event.getTaskAttemptID()))
           .handle(event);
     }
+
+    public List<TaskAttemptEvent> getEvents() {
+      return events;
+    }
+  }
+
+  private class TestOutputCommitter extends OutputCommitter {
+
+    boolean recoverySupported = false;
+    boolean throwExceptionWhenRecovery = false;
+
+    public TestOutputCommitter(OutputCommitterContext committerContext,
+        boolean recoverySupported, boolean throwExceptionWhenRecovery) {
+      super(committerContext);
+      this.recoverySupported = recoverySupported;
+      this.throwExceptionWhenRecovery = throwExceptionWhenRecovery;
+    }
+
+    @Override
+    public void recoverTask(int taskIndex, int previousDAGAttempt)
+        throws Exception {
+      if (throwExceptionWhenRecovery) {
+        throw new Exception("fail recovery Task");
+      }
+    }
+
+    @Override
+    public boolean isTaskRecoverySupported() {
+      return recoverySupported;
+    }
+
+    @Override
+    public void initialize() throws Exception {
+
+    }
+
+    @Override
+    public void setupOutput() throws Exception {
+
+    }
+
+    @Override
+    public void commitOutput() throws Exception {
+
+    }
+
+    @Override
+    public void abortOutput(State finalState) throws Exception {
+
+    }
+
   }
 
   @Before
   public void setUp() {
-    conf = new Configuration();
-    taskAttemptListener = mock(TaskAttemptListener.class);
-    taskHeartbeatHandler = mock(TaskHeartbeatHandler.class);
-    credentials = new Credentials();
-    clock = new SystemClock();
-    appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
-    dagId = TezDAGID.getInstance(appId, 1);
-    vertexId = TezVertexID.getInstance(dagId, 1);
-    vertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
-    when(vertex.getProcessorDescriptor().getClassName()).thenReturn("");
-    appContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
-    mockContainerId = mock(ContainerId.class);
-    mockContainer = mock(Container.class);
-    mockAMContainer = mock(AMContainer.class);
-    mockNodeId = mock(NodeId.class);
-    when(mockContainer.getId()).thenReturn(mockContainerId);
-    when(mockContainer.getNodeId()).thenReturn(mockNodeId);
-    when(mockAMContainer.getContainer()).thenReturn(mockContainer);
-    when(appContext.getAllContainers().get(mockContainerId)).thenReturn(
-        mockAMContainer);
-    when(appContext.getCurrentDAG().getVertex(any(TezVertexID.class)))
-        .thenReturn(vertex);
-    when(vertex.getProcessorDescriptor().getClassName()).thenReturn("");
-
-    taskResource = Resource.newInstance(1024, 1);
-    localResources = new HashMap<String, LocalResource>();
-    environment = new HashMap<String, String>();
-    javaOpts = "";
-    leafVertex = false;
-    containerContext =
-        new ContainerContext(localResources, credentials, environment, javaOpts);
-
     dispatcher = new DrainDispatcher();
     dispatcher.register(DAGEventType.class, mock(EventHandler.class));
     dispatcher.register(VertexEventType.class, mock(EventHandler.class));
     dispatcher.register(TaskEventType.class, new TaskEventHandler());
-    dispatcher.register(TaskAttemptEventType.class,
-        new TaskAttemptEventHandler());
+    dispatcher.register(TaskAttemptEventType.class, taEventHandler);
     dispatcher.init(new Configuration());
     dispatcher.start();
 
+    vertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
+    when(vertex.getProcessorDescriptor().getClassName()).thenReturn("");
+
+    mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+    when(mockAppContext.getCurrentDAG().getVertex(any(TezVertexID.class)))
+        .thenReturn(vertex);
+
     task =
-        new TaskImpl(vertexId, 1, dispatcher.getEventHandler(), conf,
-            taskAttemptListener, clock, taskHeartbeatHandler, appContext,
-            leafVertex, taskResource, containerContext, mock(StateChangeNotifier.class));
+        new TaskImpl(vertexId, 0, dispatcher.getEventHandler(),
+            new Configuration(), mock(TaskAttemptListener.class),
+            new SystemClock(), mock(TaskHeartbeatHandler.class),
+            mockAppContext, false, Resource.newInstance(1, 1),
+            mock(ContainerContext.class), mock(StateChangeNotifier.class));
+
+    Map<String, OutputCommitter> committers =
+        new HashMap<String, OutputCommitter>();
+    committers.put("out1", new TestOutputCommitter(
+        mock(OutputCommitterContext.class), true, false));
+    when(task.getVertex().getOutputCommitters()).thenReturn(committers);
+  }
+
+  private void restoreFromTaskStartEvent() {
+    TaskState recoveredState =
+        task.restoreFromEvent(new TaskStartedEvent(task.getTaskId(),
+            vertexName, taskScheduledTime, taskStartTime));
+    assertEquals(TaskState.SCHEDULED, recoveredState);
+    assertEquals(0, task.finishedAttempts);
+    assertEquals(taskScheduledTime, task.scheduledTime);
+    assertEquals(0, task.getAttempts().size());
+  }
+
+  private void restoreFromFirstTaskAttemptStartEvent(TezTaskAttemptID taId) {
+    long taStartTime = taskStartTime + 100L;
+    TaskState recoveredState =
+        task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName,
+            taStartTime, mock(ContainerId.class), mock(NodeId.class), "", ""));
+    assertEquals(TaskState.RUNNING, recoveredState);
+    assertEquals(0, task.finishedAttempts);
+    assertEquals(taskScheduledTime, task.scheduledTime);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(TaskAttemptStateInternal.NEW,
+        ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
+    assertEquals(1, task.numberUncompletedAttempts);
+  }
+
+  /**
+   * New -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_New() {
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.NEW, task.getInternalState());
+  }
+
+  /**
+   * -> restoreFromTaskFinishEvent ( no TaskStartEvent )
+   */
+  @Test
+  public void testRecovery_NoStartEvent() {
+    try {
+      task.restoreFromEvent(new TaskFinishedEvent(task.getTaskId(), vertexName,
+          taskStartTime, taskFinishTime, null, TaskState.SUCCEEDED, "",
+          new TezCounters()));
+      fail("Should fail due to no TaskStartEvent before TaskFinishEvent");
+    } catch (Throwable e) {
+      assertTrue(e.getMessage().contains(
+          "Finished Event seen but"
+              + " no Started Event was encountered earlier"));
+    }
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_Started() {
+    restoreFromTaskStartEvent();
+
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+    // new task attempt is scheduled
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(0, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * RecoverTranstion
+   */
+  @Test
+  public void testRecovery_OneTAStarted() {
+    restoreFromTaskStartEvent();
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    restoreFromFirstTaskAttemptStartEvent(taId);
+
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(0, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_OneTAStarted_SUCCEEDED() {
+    restoreFromTaskStartEvent();
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    restoreFromFirstTaskAttemptStartEvent(taId);
+
+    long taStartTime = taskStartTime + 100L;
+    long taFinishTime = taStartTime + 100L;
+    TaskState recoveredState =
+        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+            taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, "",
+            new TezCounters()));
+    assertEquals(TaskState.SUCCEEDED, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(taId, task.successfulAttempt);
+
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(taId, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (FAILED) -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_OneTAStarted_FAILED() {
+    restoreFromTaskStartEvent();
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    restoreFromFirstTaskAttemptStartEvent(taId);
+
+    long taStartTime = taskStartTime + 100L;
+    long taFinishTime = taStartTime + 100L;
+    TaskState recoveredState =
+        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+            taStartTime, taFinishTime, TaskAttemptState.FAILED, "",
+            new TezCounters()));
+    assertEquals(TaskState.RUNNING, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(1, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+    // new task attempt is scheduled
+    assertEquals(2, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(1, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (KILLED) -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_OneTAStarted_KILLED() {
+    restoreFromTaskStartEvent();
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    restoreFromFirstTaskAttemptStartEvent(taId);
+
+    long taStartTime = taskStartTime + 100L;
+    long taFinishTime = taStartTime + 100L;
+    TaskState recoveredState =
+        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+            taStartTime, taFinishTime, TaskAttemptState.KILLED, "",
+            new TezCounters()));
+    assertEquals(TaskState.RUNNING, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+    // new task attempt is scheduled
+    assertEquals(2, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) ->
+   * restoreFromTaskFinishedEvent -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_OneTAStarted_SUCCEEDED_Finished() {
+
+    restoreFromTaskStartEvent();
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    restoreFromFirstTaskAttemptStartEvent(taId);
+
+    long taStartTime = taskStartTime + 100L;
+    long taFinishTime = taStartTime + 100L;
+    TaskState recoveredState =
+        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+            taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, "",
+            new TezCounters()));
+    assertEquals(TaskState.SUCCEEDED, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(taId, task.successfulAttempt);
+
+    recoveredState =
+        task.restoreFromEvent(new TaskFinishedEvent(task.getTaskId(),
+            vertexName, taskStartTime, taskFinishTime, taId,
+            TaskState.SUCCEEDED, "", new TezCounters()));
+    assertEquals(TaskState.SUCCEEDED, recoveredState);
+    assertEquals(taId, task.successfulAttempt);
+
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(taId, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_Commit_Failed_Recovery_Not_Supported() {
+    Map<String, OutputCommitter> committers =
+        new HashMap<String, OutputCommitter>();
+    committers.put("out1", new TestOutputCommitter(
+        mock(OutputCommitterContext.class), false, false));
+    when(task.getVertex().getOutputCommitters()).thenReturn(committers);
+
+    restoreFromTaskStartEvent();
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    restoreFromFirstTaskAttemptStartEvent(taId);
+
+    // restoreFromTaskAttemptFinishedEvent (SUCCEEDED)
+    long taStartTime = taskStartTime + 100L;
+    long taFinishTime = taStartTime + 100L;
+    TaskState recoveredState =
+        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+            taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, "",
+            new TezCounters()));
+    assertEquals(TaskState.SUCCEEDED, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(taId, task.successfulAttempt);
+
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+    // new task attempt is scheduled
+    assertEquals(2, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_Commit_Failed_recover_fail() {
+    Map<String, OutputCommitter> committers =
+        new HashMap<String, OutputCommitter>();
+    committers.put("out1", new TestOutputCommitter(
+        mock(OutputCommitterContext.class), true, true));
+    when(task.getVertex().getOutputCommitters()).thenReturn(committers);
+
+    restoreFromTaskStartEvent();
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    restoreFromFirstTaskAttemptStartEvent(taId);
+
+    // restoreFromTaskAttemptFinishedEvent (SUCCEEDED)
+    long taStartTime = taskStartTime + 100L;
+    long taFinishTime = taStartTime + 100L;
+    TaskState recoveredState =
+        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+            taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, "",
+            new TezCounters()));
+    assertEquals(TaskState.SUCCEEDED, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(taId, task.successfulAttempt);
+
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+    // new task attempt is scheduled
+    assertEquals(2, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  @Test
+  public void testRecovery_WithDesired_SUCCEEDED() {
+    restoreFromTaskStartEvent();
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    restoreFromFirstTaskAttemptStartEvent(taId);
+    task.handle(new TaskEventRecoverTask(task.getTaskId(), TaskState.SUCCEEDED,
+        false));
+    assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
+    // no TA_Recovery event sent
+    assertEquals(0, taEventHandler.getEvents().size());
+  }
+
+  @Test
+  public void testRecovery_WithDesired_FAILED() {
+    restoreFromTaskStartEvent();
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    restoreFromFirstTaskAttemptStartEvent(taId);
+    task.handle(new TaskEventRecoverTask(task.getTaskId(), TaskState.FAILED,
+        false));
+    assertEquals(TaskStateInternal.FAILED, task.getInternalState());
+    // no TA_Recovery event sent
+    assertEquals(0, taEventHandler.getEvents().size());
+  }
+
+  @Test
+  public void testRecovery_WithDesired_KILLED() {
+    restoreFromTaskStartEvent();
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    restoreFromFirstTaskAttemptStartEvent(taId);
+    task.handle(new TaskEventRecoverTask(task.getTaskId(), TaskState.KILLED,
+        false));
+    assertEquals(TaskStateInternal.KILLED, task.getInternalState());
+    // no TA_Recovery event sent
+    assertEquals(0, taEventHandler.getEvents().size());
+
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (KILLED) -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_OneTAStarted_Killed() {
+    restoreFromTaskStartEvent();
+
+    long taStartTime = taskStartTime + 100L;
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    TaskState recoveredState =
+        task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName,
+            taStartTime, mock(ContainerId.class), mock(NodeId.class), "", ""));
+    assertEquals(TaskState.RUNNING, recoveredState);
+    assertEquals(TaskAttemptStateInternal.NEW,
+        ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(0, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+
+    long taFinishTime = taStartTime + 100L;
+    recoveredState =
+        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+            taStartTime, taFinishTime, TaskAttemptState.KILLED, "",
+            new TezCounters()));
+    assertEquals(TaskState.RUNNING, recoveredState);
+    assertEquals(TaskAttemptStateInternal.NEW,
+        ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    // wait for Task send TA_RECOVER to TA and TA complete the RecoverTransition
+    dispatcher.await();
+    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+    assertEquals(TaskAttemptStateInternal.KILLED,
+        ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
+    // new task attempt is scheduled
+    assertEquals(2, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
   }
 
   /**
@@ -171,24 +592,23 @@
    * schedule a new task attempt.
    */
   @Test
-  public void testTaskRecovery1() {
-    TezTaskID lastTaskId = getNewTaskID();
-    TezTaskID taskId = getNewTaskID();
+  public void testTaskRecovery_MultipleAttempts1() {
     int maxFailedAttempts =
         conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS,
             TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT);
-    task.restoreFromEvent(new TaskStartedEvent(taskId, "v1", 0, 0));
+    restoreFromTaskStartEvent();
+
     for (int i = 0; i < maxFailedAttempts; ++i) {
-      TezTaskAttemptID attemptId = getNewTaskAttemptID(lastTaskId);
-      task.restoreFromEvent(new TaskAttemptStartedEvent(attemptId, "v1", 0,
-          mockContainerId, mockNodeId, "", ""));
-      task.restoreFromEvent(new TaskAttemptFinishedEvent(attemptId, "v1", 0, 0,
-          TaskAttemptState.KILLED, "", null));
+      TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+      task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L,
+          mock(ContainerId.class), mock(NodeId.class), "", ""));
+      task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0,
+          0, TaskAttemptState.KILLED, "", null));
     }
     assertEquals(maxFailedAttempts, task.getAttempts().size());
     assertEquals(0, task.failedAttempts);
 
-    task.handle(new TaskEventRecoverTask(lastTaskId));
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
     // if the previous task attempt is killed, it should not been take into
     // account when checking whether exceed the max attempts
     assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
@@ -202,24 +622,23 @@
    * failed_attempt is exceeded.
    */
   @Test
-  public void testTaskRecovery2() {
-    TezTaskID lastTaskId = getNewTaskID();
-    TezTaskID taskId = getNewTaskID();
+  public void testTaskRecovery_MultipleAttempts2() {
     int maxFailedAttempts =
         conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS,
             TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT);
-    task.restoreFromEvent(new TaskStartedEvent(taskId, "v1", 0, 0));
+    restoreFromTaskStartEvent();
+
     for (int i = 0; i < maxFailedAttempts; ++i) {
-      TezTaskAttemptID attemptId = getNewTaskAttemptID(lastTaskId);
-      task.restoreFromEvent(new TaskAttemptStartedEvent(attemptId, "v1", 0,
-          mockContainerId, mockNodeId, "", ""));
-      task.restoreFromEvent(new TaskAttemptFinishedEvent(attemptId, "v1", 0, 0,
-          TaskAttemptState.FAILED, "", null));
+      TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+      task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L,
+          mock(ContainerId.class), mock(NodeId.class), "", ""));
+      task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0,
+          0, TaskAttemptState.FAILED, "", null));
     }
     assertEquals(maxFailedAttempts, task.getAttempts().size());
     assertEquals(maxFailedAttempts, task.failedAttempts);
 
-    task.handle(new TaskEventRecoverTask(lastTaskId));
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
     // it should transit to failed because of the failed task attempt in the
     // last application attempt.
     assertEquals(TaskStateInternal.FAILED, task.getInternalState());
@@ -233,34 +652,34 @@
    * state and new task attempt is scheduled.
    */
   @Test
-  public void testTaskRecovery3() throws InterruptedException {
-    TezTaskID lastTaskId = getNewTaskID();
-    TezTaskID taskId = getNewTaskID();
+  public void testTaskRecovery_MultipleAttempts3() throws InterruptedException {
     int maxFailedAttempts =
         conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS,
             TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT);
-    task.restoreFromEvent(new TaskStartedEvent(taskId, "v1", 0, 0));
+    restoreFromTaskStartEvent();
+
     for (int i = 0; i < maxFailedAttempts - 1; ++i) {
-      TezTaskAttemptID attemptId = getNewTaskAttemptID(lastTaskId);
-      task.restoreFromEvent(new TaskAttemptStartedEvent(attemptId, "v1", 0,
-          mockContainerId, mockNodeId, "", ""));
-      task.restoreFromEvent(new TaskAttemptFinishedEvent(attemptId, "v1", 0, 0,
-          TaskAttemptState.FAILED, "", null));
+      TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+      task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L,
+          mock(ContainerId.class), mock(NodeId.class), "", ""));
+      task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0,
+          0, TaskAttemptState.FAILED, "", null));
     }
     assertEquals(maxFailedAttempts - 1, task.getAttempts().size());
     assertEquals(maxFailedAttempts - 1, task.failedAttempts);
 
-    TezTaskAttemptID newTaskAttemptId = getNewTaskAttemptID(lastTaskId);
+    TezTaskAttemptID newTaskAttemptId = getNewTaskAttemptID(task.getTaskId());
     TaskState recoveredState =
         task.restoreFromEvent(new TaskAttemptStartedEvent(newTaskAttemptId,
-            "v1", 0, mockContainerId, mockNodeId, "", ""));
+            vertexName, 0, mock(ContainerId.class), mock(NodeId.class), "", ""));
+
     assertEquals(TaskState.RUNNING, recoveredState);
     assertEquals(TaskAttemptStateInternal.NEW,
         ((TaskAttemptImpl) task.getAttempt(newTaskAttemptId))
             .getInternalState());
     assertEquals(maxFailedAttempts, task.getAttempts().size());
 
-    task.handle(new TaskEventRecoverTask(lastTaskId));
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
     // wait until task attempt receive the Recover event from task
     dispatcher.await();
 
@@ -274,12 +693,8 @@
     assertEquals(maxFailedAttempts + 1, task.getAttempts().size());
   }
 
-  private TezTaskID getNewTaskID() {
-    TezTaskID taskID = TezTaskID.getInstance(vertexId, ++taskCounter);
-    return taskID;
-  }
-
   private TezTaskAttemptID getNewTaskAttemptID(TezTaskID taskId) {
     return TezTaskAttemptID.getInstance(taskId, taskAttemptCounter++);
   }
+
 }
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
index bd60ccb..e71acb6 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
@@ -429,7 +429,7 @@
             SchedulingType.SEQUENTIAL, OutputDescriptor.create("out.class"),
             InputDescriptor.create("out.class"))));
    
-    return dag.createDag(conf);
+    return dag.createDag(conf, null, null, null, true);
   }
 
   private DAGPlan createDAGPlanWithInitializer0Tasks(String initializerClassName) {
@@ -496,6 +496,52 @@
     return dag;
   }
 
+  private DAGPlan createDAGPlanWithMultipleInitializers(String initializerClassName) {
+    LOG.info("Setting up dag plan with multiple input initializer");
+    DAGPlan dag = DAGPlan.newBuilder()
+        .setName("testVertexWithMultipleInitializers")
+        .addVertex(
+            VertexPlan.newBuilder()
+                .setName("vertex1")
+                .setType(PlanVertexType.NORMAL)
+                .addInputs(
+                    RootInputLeafOutputProto.newBuilder()
+                        .setControllerDescriptor(
+                            TezEntityDescriptorProto.newBuilder().setClassName(
+                                initializerClassName))
+                        .setName("input1")
+                        .setIODescriptor(
+                            TezEntityDescriptorProto.newBuilder()
+                                .setClassName("InputClazz")
+                                .build()
+                        ).build()
+                )
+                .addInputs(
+                    RootInputLeafOutputProto.newBuilder()
+                        .setControllerDescriptor(
+                            TezEntityDescriptorProto.newBuilder().setClassName(
+                                initializerClassName))
+                        .setName("input2")
+                        .setIODescriptor(
+                            TezEntityDescriptorProto.newBuilder()
+                                .setClassName("InputClazz")
+                                .build()
+                        ).build()
+                )
+                .setTaskConfig(
+                    PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(-1)
+                        .setVirtualCores(4)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("x1.y1")
+                        .build()
+                )
+                .build()
+        ).build();
+    return dag;
+  }
+
   private DAGPlan createDAGPlanWithInputInitializer(String initializerClassName) {
     LOG.info("Setting up dag plan with input initializer");
     DAGPlan dag = DAGPlan.newBuilder()
@@ -3969,6 +4015,60 @@
     return dag;
   }
 
+  @Test(timeout = 5000)
+  public void testVertexWithMultipleInitializers1() {
+    useCustomInitializer = true;
+    setupPreDagCreation();
+    dagPlan = createDAGPlanWithMultipleInitializers("TestInputInitializer");
+    setupPostDagCreation();
+
+    VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices
+        .get("vertex1");
+
+    dispatcher.getEventHandler().handle(
+        new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT));
+    dispatcher.await();
+    Assert.assertEquals(VertexState.INITIALIZING, v1.getState());
+
+    RootInputInitializerManagerControlled initializerManager1 = v1.getRootInputInitializerManager();
+    List<TaskLocationHint> v1Hints = createTaskLocationHints(5);
+
+    // Complete initializer which sets parallelism first
+    initializerManager1.completeInputInitialization(0, 5, v1Hints);
+    Assert.assertEquals(VertexState.INITIALIZING, v1.getState());
+
+    // Complete second initializer
+    initializerManager1.completeInputInitialization(1);
+    Assert.assertEquals(VertexState.INITED, v1.getState());
+  }
+
+  @Test(timeout = 5000)
+  public void testVertexWithMultipleInitializers2() {
+    useCustomInitializer = true;
+    setupPreDagCreation();
+    dagPlan = createDAGPlanWithMultipleInitializers("TestInputInitializer");
+    setupPostDagCreation();
+
+    VertexImplWithControlledInitializerManager v1 = (VertexImplWithControlledInitializerManager) vertices
+        .get("vertex1");
+
+    dispatcher.getEventHandler().handle(
+        new VertexEvent(v1.getVertexId(), VertexEventType.V_INIT));
+    dispatcher.await();
+    Assert.assertEquals(VertexState.INITIALIZING, v1.getState());
+
+    RootInputInitializerManagerControlled initializerManager1 = v1.getRootInputInitializerManager();
+    List<TaskLocationHint> v1Hints = createTaskLocationHints(5);
+
+    // Complete initializer which does not set parallelism
+    initializerManager1.completeInputInitialization(1);
+    Assert.assertEquals(VertexState.INITIALIZING, v1.getState());
+
+    // Complete second initializer which sets parallelism
+    initializerManager1.completeInputInitialization(0, 5, v1Hints);
+    Assert.assertEquals(VertexState.INITED, v1.getState());
+  }
+
   @SuppressWarnings("unchecked")
   @Test(timeout = 5000)
   public void testVertexWithInitializerSuccess() {
@@ -4363,6 +4463,12 @@
       dispatcher.await();
     }
 
+    public void completeInputInitialization(int initializerIndex) {
+      eventHandler.handle(new VertexEventRootInputInitialized(vertexID, inputs
+          .get(initializerIndex).getName(), null));
+      dispatcher.await();
+    }
+
     public void completeInputInitialization(int initializerIndex, int targetTasks,
         List<TaskLocationHint> locationHints) {
       List<Event> events = Lists.newArrayListWithCapacity(targetTasks + 1);
@@ -4614,7 +4720,7 @@
 
     @Override
     public List<Event> initialize() throws Exception {
-      context.registerForVertexStatusUpdates("vertex1", null);
+      context.registerForVertexStateUpdates("vertex1", null);
       initStarted.set(true);
       lock.lock();
       try {
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java
new file mode 100644
index 0000000..b3e66bc
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java
@@ -0,0 +1,76 @@
+/**
+ * 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.tez.dag.app.dag.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.VertexManagerPluginDescriptor;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.dag.Vertex;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.junit.Test;
+
+public class TestVertexManager {
+
+  @Test
+  public void testOnRootVertexInitialized() {
+    Vertex mockVertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
+    AppContext mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+    doReturn("vertex1").when(mockVertex).getName();
+    when(
+        mockAppContext.getCurrentDAG().getVertex(any(String.class))
+            .getTotalTasks()).thenReturn(1);
+
+    VertexManager vm =
+        new VertexManager(
+            VertexManagerPluginDescriptor.create(RootInputVertexManager.class
+                .getName()), mockVertex, mockAppContext);
+    vm.initialize();
+    InputDescriptor id1 = mock(InputDescriptor.class);
+    List<Event> events1 = new LinkedList<Event>();
+    InputDataInformationEvent diEvent1 =
+        InputDataInformationEvent.createWithSerializedPayload(0, null);
+    events1.add(diEvent1);
+    List<TezEvent> tezEvents1 =
+        vm.onRootVertexInitialized("input1", id1, events1);
+    assertEquals(1, tezEvents1.size());
+    assertEquals(diEvent1, tezEvents1.get(0).getEvent());
+
+    InputDescriptor id2 = mock(InputDescriptor.class);
+    List<Event> events2 = new LinkedList<Event>();
+    InputDataInformationEvent diEvent2 =
+        InputDataInformationEvent.createWithSerializedPayload(0, null);
+    events2.add(diEvent2);
+    List<TezEvent> tezEvents2 =
+        vm.onRootVertexInitialized("input1", id2, events2);
+    assertEquals(tezEvents2.size(), 1);
+    assertEquals(diEvent2, tezEvents2.get(0).getEvent());
+  }
+}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
new file mode 100644
index 0000000..b278d8f
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
@@ -0,0 +1,860 @@
+/**
+ * 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.tez.dag.app.dag.impl;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+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.security.Credentials;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.event.DrainDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.records.DAGProtos;
+import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataMovementType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataSourceType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeSchedulingType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
+import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.VertexState;
+import org.apache.tez.dag.app.dag.event.DAGEvent;
+import org.apache.tez.dag.app.dag.event.DAGEventType;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
+import org.apache.tez.dag.app.dag.event.TaskEvent;
+import org.apache.tez.dag.app.dag.event.TaskEventRecoverTask;
+import org.apache.tez.dag.app.dag.event.TaskEventType;
+import org.apache.tez.dag.app.dag.event.VertexEvent;
+import org.apache.tez.dag.app.dag.event.VertexEventRecoverVertex;
+import org.apache.tez.dag.app.dag.event.VertexEventType;
+import org.apache.tez.dag.app.dag.impl.TestVertexImpl.CountingOutputCommitter;
+import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
+import org.apache.tez.dag.history.events.VertexFinishedEvent;
+import org.apache.tez.dag.history.events.VertexInitializedEvent;
+import org.apache.tez.dag.history.events.VertexStartedEvent;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.runtime.api.OutputCommitter;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestVertexRecovery {
+
+  private static final Log LOG = LogFactory.getLog(TestVertexRecovery.class);
+
+  private DrainDispatcher dispatcher;
+
+  private AppContext mockAppContext;
+  private ApplicationId appId = ApplicationId.newInstance(
+      System.currentTimeMillis(), 1);
+  private DAGImpl dag;
+  private TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+  private String user = "user";
+
+  private long initRequestedTime = 100L;
+  private long initedTime = initRequestedTime + 100L;
+
+  /*
+   * v1 v2 \ / v3
+   */
+  private DAGPlan createDAGPlan() {
+    DAGPlan dag =
+        DAGPlan
+            .newBuilder()
+            .setName("testverteximpl")
+            .addVertex(
+                VertexPlan
+                    .newBuilder()
+                    .setName("vertex1")
+                    .setType(PlanVertexType.NORMAL)
+                    .addTaskLocationHint(
+                        PlanTaskLocationHint.newBuilder().addHost("host1")
+                            .addRack("rack1").build())
+                    .setTaskConfig(
+                        PlanTaskConfiguration.newBuilder().setNumTasks(1)
+                            .setVirtualCores(4).setMemoryMb(1024)
+                            .setJavaOpts("").setTaskModule("x1.y1").build())
+                    .addOutEdgeId("e1")
+                    .addOutputs(
+                        DAGProtos.RootInputLeafOutputProto
+                            .newBuilder()
+                            .setIODescriptor(
+                                TezEntityDescriptorProto.newBuilder()
+                                    .setClassName("output").build())
+                            .setName("outputx")
+                            .setControllerDescriptor(
+                                TezEntityDescriptorProto
+                                    .newBuilder()
+                                    .setClassName(
+                                        CountingOutputCommitter.class.getName())))
+                    .build())
+            .addVertex(
+                VertexPlan
+                    .newBuilder()
+                    .setName("vertex2")
+                    .setType(PlanVertexType.NORMAL)
+                    .addTaskLocationHint(
+                        PlanTaskLocationHint.newBuilder().addHost("host2")
+                            .addRack("rack2").build())
+                    .setTaskConfig(
+                        PlanTaskConfiguration.newBuilder().setNumTasks(2)
+                            .setVirtualCores(4).setMemoryMb(1024)
+                            .setJavaOpts("").setTaskModule("x2.y2").build())
+                    .addOutEdgeId("e2").build())
+            .addVertex(
+                VertexPlan
+                    .newBuilder()
+                    .setName("vertex3")
+                    .setType(PlanVertexType.NORMAL)
+                    .setProcessorDescriptor(
+                        TezEntityDescriptorProto.newBuilder().setClassName(
+                            "x3.y3"))
+                    .addTaskLocationHint(
+                        PlanTaskLocationHint.newBuilder().addHost("host3")
+                            .addRack("rack3").build())
+                    .setTaskConfig(
+                        PlanTaskConfiguration.newBuilder().setNumTasks(2)
+                            .setVirtualCores(4).setMemoryMb(1024)
+                            .setJavaOpts("foo").setTaskModule("x3.y3").build())
+                    .addInEdgeId("e1")
+                    .addInEdgeId("e2")
+                    .addOutputs(
+                        DAGProtos.RootInputLeafOutputProto
+                            .newBuilder()
+                            .setIODescriptor(
+                                TezEntityDescriptorProto.newBuilder()
+                                    .setClassName("output").build())
+                            .setName("outputx")
+                            .setControllerDescriptor(
+                                TezEntityDescriptorProto
+                                    .newBuilder()
+                                    .setClassName(
+                                        CountingOutputCommitter.class.getName())))
+                    .build()
+
+            )
+
+            .addEdge(
+                EdgePlan
+                    .newBuilder()
+                    .setEdgeDestination(
+                        TezEntityDescriptorProto.newBuilder().setClassName(
+                            "i3_v1"))
+                    .setInputVertexName("vertex1")
+                    .setEdgeSource(
+                        TezEntityDescriptorProto.newBuilder()
+                            .setClassName("o1"))
+                    .setOutputVertexName("vertex3")
+                    .setDataMovementType(
+                        PlanEdgeDataMovementType.SCATTER_GATHER).setId("e1")
+                    .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                    .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+                    .build())
+            .addEdge(
+                EdgePlan
+                    .newBuilder()
+                    .setEdgeDestination(
+                        TezEntityDescriptorProto.newBuilder().setClassName(
+                            "i3_v2"))
+                    .setInputVertexName("vertex2")
+                    .setEdgeSource(
+                        TezEntityDescriptorProto.newBuilder()
+                            .setClassName("o2"))
+                    .setOutputVertexName("vertex3")
+                    .setDataMovementType(
+                        PlanEdgeDataMovementType.SCATTER_GATHER).setId("e2")
+                    .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                    .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+                    .build()).build();
+
+    return dag;
+  }
+
+  class VertexEventHanlder implements EventHandler<VertexEvent> {
+
+    private List<VertexEvent> events = new ArrayList<VertexEvent>();
+
+    @Override
+    public void handle(VertexEvent event) {
+      events.add(event);
+      ((VertexImpl) dag.getVertex(event.getVertexId())).handle(event);
+    }
+
+    public List<VertexEvent> getEvents() {
+      return this.events;
+    }
+  }
+
+  class TaskEventHandler implements EventHandler<TaskEvent> {
+
+    private List<TaskEvent> events = new ArrayList<TaskEvent>();
+
+    @Override
+    public void handle(TaskEvent event) {
+      events.add(event);
+      ((TaskImpl) dag.getVertex(event.getTaskID().getVertexID()).getTask(
+          event.getTaskID())).handle(event);
+    }
+
+    public List<TaskEvent> getEvents() {
+      return events;
+    }
+  }
+
+  class TaskAttemptEventHandler implements EventHandler<TaskAttemptEvent> {
+
+    @Override
+    public void handle(TaskAttemptEvent event) {
+      // TezTaskID taskId = event.getTaskAttemptID().getTaskID();
+      // ((TaskAttemptImpl) vertex1.getTask(taskId).getAttempt(
+      // event.getTaskAttemptID())).handle(event);
+    }
+  }
+
+  private VertexEventHanlder vertexEventHandler;
+  private TaskEventHandler taskEventHandler;
+
+  @Before
+  public void setUp() throws IOException {
+
+    dispatcher = new DrainDispatcher();
+    dispatcher.register(DAGEventType.class, mock(EventHandler.class));
+    vertexEventHandler = new VertexEventHanlder();
+    dispatcher.register(VertexEventType.class, vertexEventHandler);
+    taskEventHandler = new TaskEventHandler();
+    dispatcher.register(TaskEventType.class, taskEventHandler);
+    dispatcher.register(TaskAttemptEventType.class,
+        new TaskAttemptEventHandler());
+    dispatcher.init(new Configuration());
+    dispatcher.start();
+
+    mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+
+    DAGPlan dagPlan = createDAGPlan();
+    dag =
+        new DAGImpl(dagId, new Configuration(), dagPlan,
+            dispatcher.getEventHandler(), mock(TaskAttemptListener.class),
+            new Credentials(), new SystemClock(), user,
+            mock(TaskHeartbeatHandler.class), mockAppContext);
+    when(mockAppContext.getCurrentDAG()).thenReturn(dag);
+
+    dag.handle(new DAGEvent(dagId, DAGEventType.DAG_INIT));
+    LOG.info("finish setUp");
+  }
+
+  /**
+   * vertex1(New) -> StartRecoveryTransition(SUCCEEDED)
+   */
+  @Test
+  public void testRecovery_Desired_SUCCEEDED() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
+        "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
+    assertEquals(VertexState.INITED, recoveredState);
+    
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.SUCCEEDED));
+    dispatcher.await();
+    assertEquals(VertexState.SUCCEEDED, vertex1.getState());
+    assertEquals(vertex1.numTasks, vertex1.succeededTaskCount);
+    assertEquals(vertex1.numTasks, vertex1.completedTaskCount);
+    // recover its task
+    assertTaskRecoveredEventSent(vertex1);
+
+    // vertex3 is still in NEW, when the desiredState is
+    // Completed State, each vertex recovery by itself, not depend on its parent
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    assertEquals(VertexState.NEW, vertex3.getState());
+    // no VertexEvent pass to downstream vertex
+    assertEquals(0, vertexEventHandler.getEvents().size());
+
+  }
+
+  /**
+   * vertex1(New) -> StartRecoveryTransition(FAILED)
+   */
+  @Test
+  public void testRecovery_Desired_FAILED() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
+        "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
+    assertEquals(VertexState.INITED, recoveredState);
+    
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.FAILED));
+    dispatcher.await();
+    assertEquals(VertexState.FAILED, vertex1.getState());
+    assertEquals(vertex1.numTasks, vertex1.failedTaskCount);
+    assertEquals(0, vertex1.completedTaskCount);
+    // recover its task
+    assertTaskRecoveredEventSent(vertex1);
+
+    // vertex3 is still in NEW, when the desiredState is
+    // Completed State, each vertex recovery by itself, not depend on its parent
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    assertEquals(VertexState.NEW, vertex3.getState());
+    // no VertexEvent pass to downstream vertex
+    assertEquals(0, vertexEventHandler.getEvents().size());
+  }
+
+  /**
+   * vertex1(New) -> StartRecoveryTransition(KILLED)
+   */
+  @Test
+  public void testRecovery_Desired_KILLED() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
+        "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
+    assertEquals(VertexState.INITED, recoveredState);
+    
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.KILLED));
+    dispatcher.await();
+    assertEquals(VertexState.KILLED, vertex1.getState());
+    assertEquals(vertex1.numTasks, vertex1.killedTaskCount);
+    assertEquals(0, vertex1.completedTaskCount);
+    // recover its task
+    assertTaskRecoveredEventSent(vertex1);
+
+    // vertex3 is still in NEW, when the desiredState is
+    // Completed State, each vertex recovery by itself, not depend on its parent
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    assertEquals(VertexState.NEW, vertex3.getState());
+    // no VertexEvent pass to downstream vertex
+    assertEquals(0, vertexEventHandler.getEvents().size());
+  }
+
+  /**
+   * vertex1(New) -> StartRecoveryTransition(ERROR)
+   */
+  @Test
+  public void testRecovery_Desired_ERROR() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
+        "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
+    assertEquals(VertexState.INITED, recoveredState);
+    
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.ERROR));
+    dispatcher.await();
+    assertEquals(VertexState.ERROR, vertex1.getState());
+    assertEquals(vertex1.numTasks, vertex1.failedTaskCount);
+    assertEquals(0, vertex1.completedTaskCount);
+    // recover its task
+    assertTaskRecoveredEventSent(vertex1);
+
+    // vertex3 is still in NEW, when the desiredState is
+    // Completed State, each vertex recovery by itself, not depend on its parent
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    assertEquals(VertexState.NEW, vertex3.getState());
+    // no VertexEvent pass to downstream vertex
+    assertEquals(0, vertexEventHandler.getEvents().size());
+  }
+
+  private TezEvent createTezEvent() {
+    return new TezEvent(InputDataInformationEvent.createWithSerializedPayload(0, ByteBuffer.allocate(0)),
+        new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", null,
+            null));
+  }
+
+  /**
+   * vertex1(New) -> restoreFromDataMovementEvent -> StartRecoveryTransition
+   */
+  @Test
+  public void testRecovery_New_Desired_RUNNING() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    VertexState recoveredState =
+        vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
+            vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
+    assertEquals(VertexState.NEW, recoveredState);
+    assertEquals(1, vertex1.recoveredEvents.size());
+
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+
+    // InputDataInformationEvent is removed
+    assertEquals(0, vertex1.recoveredEvents.size());
+    // V_INIT and V_START is sent
+    assertEquals(VertexState.RUNNING, vertex1.getState());
+
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex1);
+
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    // wait for recovery of vertex2
+    assertEquals(VertexState.RECOVERING, vertex3.getState());
+    assertEquals(1, vertex3.numRecoveredSourceVertices);
+    assertEquals(1, vertex3.numInitedSourceVertices);
+    assertEquals(1, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+
+  }
+
+  private void assertTaskRecoveredEventSent(VertexImpl vertex) {
+    int sentNum = 0;
+    for (TaskEvent event : taskEventHandler.getEvents()) {
+      if (event.getType() == TaskEventType.T_RECOVER) {
+        TaskEventRecoverTask recoverEvent = (TaskEventRecoverTask)event;
+        if (recoverEvent.getTaskID().getVertexID().equals(vertex.getVertexId())){
+          sentNum++;
+        }
+      }
+    }
+    assertEquals("expect " + vertex.getTotalTasks()
+        + " TaskEventTaskRecover sent for vertex:" + vertex.getVertexId() +
+        "but actuall sent " + sentNum, vertex.getTotalTasks(), sentNum);
+  }
+
+  private void assertOutputCommitters(VertexImpl vertex){
+    assertTrue(vertex.getOutputCommitters() != null);
+    for (OutputCommitter c : vertex.getOutputCommitters().values()) {
+      CountingOutputCommitter committer = (CountingOutputCommitter) c;
+      assertEquals(0, committer.abortCounter);
+      assertEquals(0, committer.commitCounter);
+      assertEquals(1, committer.initCounter);
+      assertEquals(1, committer.setupCounter);
+    }
+  }
+
+  private void restoreFromInitializedEvent(VertexImpl vertex) {
+    long initTimeRequested = 100L;
+    long initedTime = initTimeRequested + 100L;
+    VertexState recoveredState =
+        vertex.restoreFromEvent(new VertexInitializedEvent(vertex
+            .getVertexId(), "vertex1", initTimeRequested, initedTime, vertex.getTotalTasks(),
+            "", null));
+    assertEquals(VertexState.INITED, recoveredState);
+    assertEquals(vertex.getTotalTasks(), vertex.getTasks().size());
+    assertEquals(initTimeRequested, vertex.initTimeRequested);
+    assertEquals(initedTime, vertex.initedTime);
+  }
+
+  /**
+   * restoreFromVertexInitializedEvent -> StartRecoveryTransition
+   */
+  @Test
+  public void testRecovery_Inited_Desired_RUNNING() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    restoreFromInitializedEvent(vertex1);
+
+    VertexState recoveredState =
+        vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
+            vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
+    assertEquals(VertexState.INITED, recoveredState);
+
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+
+    // InputDataInformationEvent is removed
+    assertEquals(0, vertex1.recoveredEvents.size());
+    assertEquals(VertexState.RUNNING, vertex1.getState());
+    // task recovered event is sent
+    assertTaskRecoveredEventSent(vertex1);
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex1);
+
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    // wait for recovery of vertex2
+    assertEquals(VertexState.RECOVERING, vertex3.getState());
+    assertEquals(1, vertex3.numRecoveredSourceVertices);
+    assertEquals(1, vertex3.numInitedSourceVertices);
+    assertEquals(1, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+  }
+
+  /**
+   * restoreFromVertexInitializedEvent -> restoreFromVertexStartedEvent ->
+   * StartRecoveryTransition
+   */
+  @Test
+  public void testRecovery_Started_Desired_RUNNING() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    restoreFromInitializedEvent(vertex1);
+
+    long startTimeRequested = initedTime + 100L;
+    long startedTime = startTimeRequested + 100L;
+    VertexState recoveredState =
+        vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
+            startTimeRequested, startedTime));
+    assertEquals(VertexState.RUNNING, recoveredState);
+    assertEquals(startTimeRequested, vertex1.startTimeRequested);
+    assertEquals(startedTime, vertex1.startedTime);
+
+    recoveredState =
+        vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
+            vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
+    assertEquals(VertexState.RUNNING, recoveredState);
+    assertEquals(1, vertex1.recoveredEvents.size());
+
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+
+    // InputDataInformationEvent is removed
+    assertEquals(0, vertex1.recoveredEvents.size());
+    assertEquals(VertexState.RUNNING, vertex1.getState());
+    // task recovered event is sent
+    assertTaskRecoveredEventSent(vertex1);
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex1);
+
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    // wait for recovery of vertex2
+    assertEquals(VertexState.RECOVERING, vertex3.getState());
+    assertEquals(1, vertex3.numRecoveredSourceVertices);
+    assertEquals(1, vertex3.numInitedSourceVertices);
+    assertEquals(1, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+  }
+
+  /**
+   * restoreFromVertexInitializedEvent -> restoreFromVertexStartedEvent ->
+   * restoreFromVertexFinishedEvent -> StartRecoveryTransition
+   */
+  @Test
+  public void testRecovery_Finished_Desired_RUNNING() {
+    // v1: initFromInitializedEvent
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    restoreFromInitializedEvent(vertex1);
+
+    // v1: initFromStartedEvent
+    long startRequestedTime = initedTime + 100L;
+    long startTime = startRequestedTime + 100L;
+    VertexState recoveredState =
+        vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
+            startRequestedTime, startTime));
+    assertEquals(VertexState.RUNNING, recoveredState);
+
+    // v1: initFromFinishedEvent
+    long finishTime = startTime + 100L;
+    recoveredState =
+        vertex1.restoreFromEvent(new VertexFinishedEvent(vertex1.getVertexId(),
+            "vertex1", initRequestedTime, initedTime, startRequestedTime,
+            startTime, finishTime, VertexState.SUCCEEDED, "",
+            new TezCounters(), new VertexStats(), null));
+    assertEquals(finishTime, vertex1.finishTime);
+    assertEquals(VertexState.SUCCEEDED, recoveredState);
+    assertEquals(false, vertex1.recoveryCommitInProgress);
+
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+
+    // InputDataInformationEvent is removed
+    assertEquals(0, vertex1.recoveredEvents.size());
+    assertEquals(VertexState.RUNNING, vertex1.getState());
+    // task recovered event is sent
+    assertTaskRecoveredEventSent(vertex1);
+
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex1);
+
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    // wait for recovery of vertex2
+    assertEquals(VertexState.RECOVERING, vertex3.getState());
+    assertEquals(1, vertex3.numRecoveredSourceVertices);
+    assertEquals(1, vertex3.numInitedSourceVertices);
+    assertEquals(1, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+  }
+
+  /**
+   * vertex1 (New) -> StartRecoveryTransition <br>
+   * vertex2 (New) -> StartRecoveryTransition vertex3 (New) -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_RecoveringFromNew() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+    assertEquals(VertexState.RUNNING, vertex1.getState());
+    assertEquals(1, vertex1.getTasks().size());
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex1);
+
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    VertexState recoveredState =
+        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
+            vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
+    assertEquals(VertexState.NEW, recoveredState);
+    assertEquals(1, vertex3.recoveredEvents.size());
+
+    // wait for recovery of vertex2
+    assertEquals(VertexState.RECOVERING, vertex3.getState());
+    assertEquals(1, vertex3.numRecoveredSourceVertices);
+    assertEquals(1, vertex3.numInitedSourceVertices);
+    assertEquals(1, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+
+    VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
+    vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+    assertEquals(VertexState.RUNNING, vertex2.getState());
+    // no OutputCommitter for vertex2
+    assertNull(vertex2.getOutputCommitters());
+
+    // v3 go to RUNNING because v1 and v2 both start
+    assertEquals(VertexState.RUNNING, vertex3.getState());
+    assertEquals(2, vertex3.numRecoveredSourceVertices);
+    assertEquals(2, vertex3.numInitedSourceVertices);
+    assertEquals(2, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+    // RootInputDataInformation is removed
+    assertEquals(0, vertex3.recoveredEvents.size());
+
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex3);
+
+  }
+
+  /**
+   * vertex1 (New) -> restoreFromInitialized -> StartRecoveryTransition<br>
+   * vertex2 (New) -> restoreFromInitialized -> StartRecoveryTransition<br>
+   * vertex3 (New) -> restoreFromVertexInitedEvent -> RecoverTransition<br>
+   */
+  @Test
+  public void testRecovery_RecoveringFromInited() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    restoreFromInitializedEvent(vertex1);
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+    assertEquals(VertexState.RUNNING, vertex1.getState());
+    assertEquals(vertex1.getTotalTasks(), vertex1.getTasks().size());
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex1);
+
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    VertexState recoveredState =
+        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
+            vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
+    assertEquals(VertexState.NEW, recoveredState);
+    assertEquals(1, vertex3.recoveredEvents.size());
+    recoveredState =
+        vertex3.restoreFromEvent(new VertexInitializedEvent(vertex3
+            .getVertexId(), "vertex3", initRequestedTime, initedTime, 2, "",
+            null));
+    assertEquals(VertexState.INITED, recoveredState);
+    // wait for recovery of vertex2
+    assertEquals(VertexState.RECOVERING, vertex3.getState());
+    assertEquals(1, vertex3.numRecoveredSourceVertices);
+    assertEquals(1, vertex3.numInitedSourceVertices);
+    assertEquals(1, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+
+    VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
+    restoreFromInitializedEvent(vertex2);
+    vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+    assertEquals(VertexState.RUNNING, vertex2.getState());
+
+    // v3 go to RUNNING because v1 and v2 both start
+    assertEquals(VertexState.RUNNING, vertex3.getState());
+    assertEquals(2, vertex3.numRecoveredSourceVertices);
+    // numInitedSourceVertices is wrong but doesn't matter because v3 has
+    // already initialized
+    assertEquals(2, vertex3.numInitedSourceVertices);
+    assertEquals(2, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+    // RootInputDataInformation is removed
+    assertEquals(0, vertex3.recoveredEvents.size());
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex3);
+    // 1 for vertex1, 2 for vertex2, the second 2 for vertex3
+    assertTaskRecoveredEventSent(vertex1);
+    assertTaskRecoveredEventSent(vertex2);
+    assertTaskRecoveredEventSent(vertex3);
+  }
+
+  /**
+   * vertex1 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
+   * StartRecoveryTransition <br>
+   * vertex2 (New) -> restoreFromInitialized -> restoreFromVertexStarted -> StartRecoveryTransition <br>
+   * vertex3 (New) -> restoreFromInitialized -> restoreFromVertexStarted -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_RecoveringFromRunning() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    restoreFromInitializedEvent(vertex1);
+    VertexState recoveredState = vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
+        initRequestedTime + 100L, initRequestedTime + 200L));
+    assertEquals(VertexState.RUNNING, recoveredState);
+
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+    assertEquals(VertexState.RUNNING, vertex1.getState());
+    assertEquals(1, vertex1.getTasks().size());
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex1);
+
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    recoveredState =
+        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
+            vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
+    assertEquals(VertexState.NEW, recoveredState);
+    assertEquals(1, vertex3.recoveredEvents.size());
+    recoveredState =
+        vertex3.restoreFromEvent(new VertexInitializedEvent(vertex3
+            .getVertexId(), "vertex3", initRequestedTime, initedTime, vertex3.getTotalTasks(), "",
+            null));
+    assertEquals(VertexState.INITED, recoveredState);
+    recoveredState =
+        vertex3.restoreFromEvent(new VertexStartedEvent(vertex3.getVertexId(),
+            initRequestedTime + 100L, initRequestedTime + 200L));
+    assertEquals(VertexState.RUNNING, recoveredState);
+    // wait for recovery of vertex2
+    assertEquals(VertexState.RECOVERING, vertex3.getState());
+    assertEquals(1, vertex3.numRecoveredSourceVertices);
+    assertEquals(1, vertex3.numInitedSourceVertices);
+    assertEquals(1, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+
+    VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
+    recoveredState = vertex2.restoreFromEvent(new VertexInitializedEvent(vertex2.getVertexId(),
+        "vertex2", initRequestedTime, initedTime, vertex2.getTotalTasks(), "", null));
+    assertEquals(VertexState.INITED, recoveredState);
+    recoveredState = vertex2.restoreFromEvent(new VertexStartedEvent(vertex2.getVertexId(),
+        initRequestedTime + 100L, initRequestedTime + 200L));
+    assertEquals(VertexState.RUNNING, recoveredState);
+    
+    vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+    assertEquals(VertexState.RUNNING, vertex2.getState());
+
+    // v3 go to RUNNING because v1 and v2 both start
+    assertEquals(VertexState.RUNNING, vertex3.getState());
+    assertEquals(2, vertex3.numRecoveredSourceVertices);
+    assertEquals(2, vertex3.numInitedSourceVertices);
+    assertEquals(2, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+    // RootInputDataInformation is removed
+    assertEquals(0, vertex3.recoveredEvents.size());
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex3);
+
+    assertTaskRecoveredEventSent(vertex1);
+    assertTaskRecoveredEventSent(vertex2);
+    assertTaskRecoveredEventSent(vertex3);
+  }
+
+  /**
+   * vertex1 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
+   * restoreFromVertexFinished -> StartRecoveryTransition<br>
+   * vertex2 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
+   * restoreFromVertexFinished -> StartRecoveryTransition<br>
+   * vertex3 (New) -> restoreFromInitialized -> restoreFromVertexStarted -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_RecoveringFromSUCCEEDED() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    restoreFromInitializedEvent(vertex1);
+    VertexState recoveredState = vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
+        initRequestedTime + 100L, initRequestedTime + 200L));
+    assertEquals(VertexState.RUNNING, recoveredState);
+
+    recoveredState = vertex1.restoreFromEvent(new VertexFinishedEvent(vertex1.getVertexId(),
+        "vertex1", initRequestedTime, initedTime, initRequestedTime + 300L,
+        initRequestedTime + 400L, initRequestedTime + 500L,
+        VertexState.SUCCEEDED, "", new TezCounters(), new VertexStats(), null));
+    assertEquals(VertexState.SUCCEEDED, recoveredState);
+
+    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+    assertEquals(VertexState.RUNNING, vertex1.getState());
+    assertEquals(1, vertex1.getTasks().size());
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex1);
+
+    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
+    recoveredState =
+        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
+            vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
+    assertEquals(VertexState.NEW, recoveredState);
+    assertEquals(1, vertex3.recoveredEvents.size());
+    restoreFromInitializedEvent(vertex3);
+    recoveredState =
+        vertex3.restoreFromEvent(new VertexStartedEvent(vertex3.getVertexId(),
+            initRequestedTime + 100L, initRequestedTime + 200L));
+    assertEquals(VertexState.RUNNING, recoveredState);
+    // wait for recovery of vertex2
+    assertEquals(VertexState.RECOVERING, vertex3.getState());
+    assertEquals(1, vertex3.numRecoveredSourceVertices);
+    assertEquals(1, vertex3.numInitedSourceVertices);
+    assertEquals(1, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+
+    VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
+    recoveredState = vertex2.restoreFromEvent(new VertexInitializedEvent(vertex2.getVertexId(),
+        "vertex2", initRequestedTime, initedTime, vertex2.getTotalTasks(), "", null));
+    assertEquals(VertexState.INITED, recoveredState);
+    recoveredState = vertex2.restoreFromEvent(new VertexStartedEvent(vertex2.getVertexId(),
+        initRequestedTime + 100L, initRequestedTime + 200L));
+    assertEquals(VertexState.RUNNING, recoveredState);
+    vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
+        VertexState.RUNNING));
+    dispatcher.await();
+    assertEquals(VertexState.RUNNING, vertex2.getState());
+
+    // v3 go to RUNNING because v1 and v2 both start
+    assertEquals(VertexState.RUNNING, vertex3.getState());
+    assertEquals(2, vertex3.numRecoveredSourceVertices);
+    assertEquals(2, vertex3.numInitedSourceVertices);
+    assertEquals(2, vertex3.numStartedSourceVertices);
+    assertEquals(1, vertex3.getDistanceFromRoot());
+    // RootInputDataInformation is removed
+    assertEquals(0, vertex3.recoveredEvents.size());
+    // verify OutputCommitter is initialized
+    assertOutputCommitters(vertex3);
+
+    assertTaskRecoveredEventSent(vertex1);
+    assertTaskRecoveredEventSent(vertex2);
+    assertTaskRecoveredEventSent(vertex3);
+  }
+}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
new file mode 100644
index 0000000..5fc5a7d
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestLocalTaskSchedulerService.java
@@ -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.tez.dag.app.rm;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+
+public class TestLocalTaskSchedulerService {
+
+  LocalTaskSchedulerService ltss ;
+  int core =10;
+
+  @Test
+  public void testCreateResource() {
+    Resource resource;
+    //value in integer
+    long value = 4*1024*1024;
+    resource = ltss.createResource(value,core);
+    Assert.assertEquals((int)(value/(1024*1024)),resource.getMemory());
+  }
+
+  @Test
+  public void testCreateResourceLargerThanIntMax() {
+    //value beyond integer but within Long.MAX_VALUE
+    try {
+      ltss.createResource(Long.MAX_VALUE, core);
+      fail("No exception thrown.");
+    } catch (Exception ex) {
+      assertTrue(ex instanceof IllegalArgumentException);
+      assertTrue(ex.getMessage().contains("Out of range:"));
+    }
+  }
+
+  @Test
+  public void testCreateResourceWithNegativeValue() {
+    //value is Long.MAX_VALUE*1024*1024,
+    // it will be negative after it is passed to createResource
+
+    try {
+      ltss.createResource((Long.MAX_VALUE*1024*1024), core);
+      fail("No exception thrown.");
+    } catch (Exception ex) {
+      assertTrue(ex instanceof IllegalArgumentException);
+      assertTrue(ex.getMessage().contains("Negative Memory or Core provided!"));
+    }
+  }
+}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
index f45c379..f52671c 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
@@ -354,7 +354,6 @@
           deserializedEvent.getFinishTime());
       Assert.assertEquals(event.getState(), deserializedEvent.getState());
       Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());
-      Assert.assertEquals(event.getTezCounters(), deserializedEvent.getTezCounters());
       logEvents(event, deserializedEvent);
     }
     {
@@ -370,7 +369,6 @@
           deserializedEvent.getFinishTime());
       Assert.assertEquals(event.getState(), deserializedEvent.getState());
       Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());
-      Assert.assertEquals(event.getTezCounters(), deserializedEvent.getTezCounters());
       logEvents(event, deserializedEvent);
     }
   }
@@ -403,8 +401,6 @@
           deserializedEvent.getFinishTime());
       Assert.assertEquals(event.getState(),
           deserializedEvent.getState());
-      Assert.assertEquals(event.getTezCounters(),
-          deserializedEvent.getTezCounters());
       Assert.assertEquals(event.getSuccessfulAttemptID(),
           deserializedEvent.getSuccessfulAttemptID());
       Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());
@@ -425,8 +421,6 @@
           deserializedEvent.getFinishTime());
       Assert.assertEquals(event.getState(),
           deserializedEvent.getState());
-      Assert.assertEquals(event.getTezCounters(),
-          deserializedEvent.getTezCounters());
       Assert.assertEquals(event.getSuccessfulAttemptID(),
           deserializedEvent.getSuccessfulAttemptID());
       Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java b/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java
index 081ed10..0be67ad 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java
@@ -87,7 +87,7 @@
     dag.addVertex(v2);
     dag.addVertex(v3);
     dag.addEdge(e1);
-    return dag.createDag(conf);
+    return dag.createDag(conf, null, null, null, true);
   }
 
   @Test
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/ExampleDriver.java b/tez-examples/src/main/java/org/apache/tez/examples/ExampleDriver.java
index 6f689e2..5394cc1 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/ExampleDriver.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/ExampleDriver.java
@@ -50,8 +50,10 @@
           "Word Count with words sorted on frequency");
       pgd.addClass("simplesessionexample", SimpleSessionExample.class,
           "Example to run multiple dags in a session");
-      pgd.addClass("joinexample", JoinExample.class,
-          "Identify all occurences of lines in file1 which also occur in file2");
+      pgd.addClass("hashjoin", HashJoinExample.class,
+          "Identify all occurences of lines in file1 which also occur in file2 using hash join");
+      pgd.addClass("sortmergejoin", SortMergeJoinExample.class,
+          "Identify all occurences of lines in file1 which also occur in file2 using sort merge join");
       pgd.addClass("joindatagen", JoinDataGen.class,
           "Generate data to run the joinexample");
       pgd.addClass("joinvalidate", JoinValidate.class,
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java b/tez-examples/src/main/java/org/apache/tez/examples/HashJoinExample.java
similarity index 61%
rename from tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java
rename to tez-examples/src/main/java/org/apache/tez/examples/HashJoinExample.java
index 3611fd6..76e53e1 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/HashJoinExample.java
@@ -63,65 +63,69 @@
 import com.google.common.base.Preconditions;
 
 /**
- * Simple example of joining 2 data sets.
- * <br>The example shows a vertex with multiple inputs that represent the two
- * data sets that need to be joined.
- * <br>The join can be performed using a broadcast (or replicate-fragment) join in 
- * which the small side of the join is broadcast in total to fragments of the 
+ * Simple example of joining 2 data sets using <a
+ * href="http://en.wikipedia.org/wiki/Hash_join">Hash Join</a>.<br>
+ * The example shows a vertex with multiple inputs that represent the two data
+ * sets that need to be joined. This HashJoinExample assume that keys in the
+ * second dataset (hashSide) is unique.<br>
+ * The join can be performed using a broadcast (or replicate-fragment) join in
+ * which the small side of the join is broadcast in total to fragments of the
  * larger side. Each fragment of the larger side can perform the join operation
- * independently using the full data of the smaller side. This shows the usage 
- * of the broadcast edge property in Tez.
- * <br>The join can be performed using the regular repartition join where both 
- * sides are partitioned according to the same scheme into the same number of 
- * fragments. Then the keys in the same fragment are joined with each other. This 
- * is the default join strategy.
- *
+ * independently using the full data of the smaller side. This shows the usage
+ * of the broadcast edge property in Tez. <br>
+ * The join can be performed using the regular repartition join where both sides
+ * are partitioned according to the same scheme into the same number of
+ * fragments. Then the keys in the same fragment are joined with each other.
+ * This is the default join strategy.
  */
-public class JoinExample extends Configured implements Tool {
+public class HashJoinExample extends Configured implements Tool {
 
-  private static final Log LOG = LogFactory.getLog(JoinExample.class);
-  
+  private static final Log LOG = LogFactory.getLog(HashJoinExample.class);
+
   private static final String broadcastOption = "doBroadcast";
   private static final String streamingSide = "streamingSide";
   private static final String hashSide = "hashSide";
   private static final String inputFile = "inputFile";
   private static final String joiner = "joiner";
   private static final String joinOutput = "joinOutput";
-  
 
   public static void main(String[] args) throws Exception {
-    JoinExample job = new JoinExample();
+    HashJoinExample job = new HashJoinExample();
     int status = ToolRunner.run(new Configuration(), job, args);
     System.exit(status);
   }
 
   private static void printUsage() {
-    System.err.println("Usage: " + "joinexample <file1> <file2> <numPartitions> <outPath> [" 
-                       + broadcastOption + "(default false)]");
+    System.err.println("Usage: "
+        + "hashjoin <file1> <file2> <numPartitions> <outPath> ["
+        + broadcastOption + "(default false)]");
     ToolRunner.printGenericCommandUsage(System.err);
   }
 
   @Override
   public int run(String[] args) throws Exception {
     Configuration conf = getConf();
-    String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
+    String[] otherArgs =
+        new GenericOptionsParser(conf, args).getRemainingArgs();
     int result = validateArgs(otherArgs);
     if (result != 0) {
       return result;
     }
     return execute(otherArgs);
   }
-  
-  public int run(Configuration conf, String[] args, TezClient tezClient) throws Exception {
+
+  public int run(Configuration conf, String[] args, TezClient tezClient)
+      throws Exception {
     setConf(conf);
-    String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
+    String[] otherArgs =
+        new GenericOptionsParser(conf, args).getRemainingArgs();
     int result = validateArgs(otherArgs);
     if (result != 0) {
       return result;
     }
     return execute(otherArgs, tezClient);
   }
-  
+
   private int validateArgs(String[] otherArgs) {
     if (!(otherArgs.length == 4 || otherArgs.length == 5)) {
       printUsage();
@@ -130,7 +134,8 @@
     return 0;
   }
 
-  private int execute(String[] args) throws TezException, IOException, InterruptedException {
+  private int execute(String[] args) throws TezException, IOException,
+      InterruptedException {
     TezConfiguration tezConf = new TezConfiguration(getConf());
     TezClient tezClient = null;
     try {
@@ -142,23 +147,26 @@
       }
     }
   }
-  
-  private int execute(String[] args, TezClient tezClient) throws IOException, TezException,
-      InterruptedException {
+
+  private int execute(String[] args, TezClient tezClient) throws IOException,
+      TezException, InterruptedException {
     TezConfiguration tezConf = new TezConfiguration(getConf());
     return execute(args, tezConf, tezClient);
   }
 
-  private TezClient createTezClient(TezConfiguration tezConf) throws TezException, IOException {
-    TezClient tezClient = TezClient.create("JoinExample", tezConf);
+  private TezClient createTezClient(TezConfiguration tezConf)
+      throws TezException, IOException {
+    TezClient tezClient = TezClient.create("HashJoinExample", tezConf);
     tezClient.start();
     return tezClient;
   }
-  
-  private int execute(String[] args, TezConfiguration tezConf, TezClient tezClient)
-      throws IOException, TezException, InterruptedException {
-    boolean doBroadcast = args.length == 5 && args[4].equals(broadcastOption) ? true : false;
-    LOG.info("Running JoinExample" + (doBroadcast ? "-WithBroadcast" : ""));
+
+  private int execute(String[] args, TezConfiguration tezConf,
+      TezClient tezClient) throws IOException, TezException,
+      InterruptedException {
+    boolean doBroadcast =
+        args.length == 5 && args[4].equals(broadcastOption) ? true : false;
+    LOG.info("Running HashJoinExample" + (doBroadcast ? "-WithBroadcast" : ""));
 
     UserGroupInformation.setConfiguration(tezConf);
 
@@ -182,7 +190,9 @@
       return 4;
     }
 
-    DAG dag = createDag(tezConf, streamInputPath, hashInputPath, outputPath, numPartitions, doBroadcast);
+    DAG dag =
+        createDag(tezConf, streamInputPath, hashInputPath, outputPath,
+            numPartitions, doBroadcast);
 
     tezClient.waitTillReady();
     DAGClient dagClient = tezClient.submitDAG(dag);
@@ -195,104 +205,124 @@
 
   }
 
-  private DAG createDag(TezConfiguration tezConf, Path streamPath, Path hashPath, Path outPath,
-      int numPartitions, boolean doBroadcast) throws IOException {
-    DAG dag = DAG.create("JoinExample" + (doBroadcast ? "-WithBroadcast" : ""));
+  private DAG createDag(TezConfiguration tezConf, Path streamPath,
+      Path hashPath, Path outPath, int numPartitions, boolean doBroadcast)
+      throws IOException {
+    DAG dag = DAG.create("HashJoinExample" + (doBroadcast ? "-WithBroadcast" : ""));
 
     /**
-     * This vertex represents the side of the join that will be accumulated in a hash 
-     * table in order to join it against the other side. It reads text data using the
-     * TextInputFormat. ForwardingProcessor simply forwards the data downstream as is.
-     */
-    Vertex hashFileVertex = Vertex.create(hashSide, ProcessorDescriptor.create(
-        ForwardingProcessor.class.getName())).addDataSource(
-        inputFile,
-        MRInput
-            .createConfigBuilder(new Configuration(tezConf), TextInputFormat.class,
-                hashPath.toUri().toString()).groupSplits(false).build());
-
-    /**
-     * This vertex represents that side of the data that will be streamed and joined 
-     * against the other side that has been accumulated into a hash table. It reads 
-     * text data using the TextInputFormat. ForwardingProcessor simply forwards the data 
+     * This vertex represents the side of the join that will be accumulated in a
+     * hash table in order to join it against the other side. It reads text data
+     * using the TextInputFormat. ForwardingProcessor simply forwards the data
      * downstream as is.
      */
-    Vertex streamFileVertex = Vertex.create(streamingSide, ProcessorDescriptor.create(
-        ForwardingProcessor.class.getName())).addDataSource(
-        inputFile,
-        MRInput
-            .createConfigBuilder(new Configuration(tezConf), TextInputFormat.class,
-                streamPath.toUri().toString()).groupSplits(false).build());
+    Vertex hashFileVertex =
+        Vertex.create(hashSide,
+            ProcessorDescriptor.create(ForwardingProcessor.class.getName()))
+            .addDataSource(
+                inputFile,
+                MRInput
+                    .createConfigBuilder(new Configuration(tezConf),
+                        TextInputFormat.class, hashPath.toUri().toString())
+                    .groupSplits(false).build());
 
     /**
-     * This vertex represents the join operation. It writes the join output as text using
-     * the TextOutputFormat. The JoinProcessor is going to perform the join of the 
-     * streaming side and the hash side. It is load balanced across numPartitions 
+     * This vertex represents that side of the data that will be streamed and
+     * joined against the other side that has been accumulated into a hash
+     * table. It reads text data using the TextInputFormat. ForwardingProcessor
+     * simply forwards the data downstream as is.
      */
-    Vertex joinVertex = Vertex.create(joiner, ProcessorDescriptor.create(
-        JoinProcessor.class.getName()), numPartitions).addDataSink(joinOutput,
-        MROutput.createConfigBuilder(new Configuration(tezConf),
-            TextOutputFormat.class, outPath.toUri().toString()).build());
+    Vertex streamFileVertex =
+        Vertex.create(streamingSide,
+            ProcessorDescriptor.create(ForwardingProcessor.class.getName()))
+            .addDataSource(
+                inputFile,
+                MRInput
+                    .createConfigBuilder(new Configuration(tezConf),
+                        TextInputFormat.class, streamPath.toUri().toString())
+                    .groupSplits(false).build());
 
     /**
-     * The streamed side will be partitioned into fragments with the same keys going to 
-     * the same fragments using hash partitioning. The data to be joined is the key itself
-     * and so the value is null. The number of fragments is initially inferred from the 
-     * number of tasks running in the join vertex because each task will be handling one
-     * fragment.
+     * This vertex represents the join operation. It writes the join output as
+     * text using the TextOutputFormat. The JoinProcessor is going to perform
+     * the join of the streaming side and the hash side. It is load balanced
+     * across numPartitions
+     */
+    Vertex joinVertex =
+        Vertex.create(joiner,
+            ProcessorDescriptor.create(HashJoinProcessor.class.getName()),
+            numPartitions).addDataSink(
+            joinOutput,
+            MROutput.createConfigBuilder(new Configuration(tezConf),
+                TextOutputFormat.class, outPath.toUri().toString()).build());
+
+    /**
+     * The streamed side will be partitioned into fragments with the same keys
+     * going to the same fragments using hash partitioning. The data to be
+     * joined is the key itself and so the value is null. The number of
+     * fragments is initially inferred from the number of tasks running in the
+     * join vertex because each task will be handling one fragment.
      */
     UnorderedPartitionedKVEdgeConfig streamConf =
         UnorderedPartitionedKVEdgeConfig
             .newBuilder(Text.class.getName(), NullWritable.class.getName(),
-                HashPartitioner.class.getName()).setFromConfiguration(tezConf).build();
+                HashPartitioner.class.getName()).setFromConfiguration(tezConf)
+            .build();
 
     /**
      * Connect the join vertex with the stream side
      */
-    Edge e1 = Edge.create(streamFileVertex, joinVertex, streamConf.createDefaultEdgeProperty());
-    
+    Edge e1 =
+        Edge.create(streamFileVertex, joinVertex,
+            streamConf.createDefaultEdgeProperty());
+
     EdgeProperty hashSideEdgeProperty = null;
     if (doBroadcast) {
       /**
-       * This option can be used when the hash side is small. We can broadcast the entire data to 
-       * all fragments of the stream side. This avoids re-partitioning the fragments of the stream 
-       * side to match the partitioning scheme of the hash side and avoids costly network data 
-       * transfer. However, in this example the stream side is being partitioned in both cases for 
-       * brevity of code. The join task can perform the join of its fragment of keys with all the 
-       * keys of the hash side.
-       * Using an unpartitioned edge to transfer the complete output of the hash side to be 
-       * broadcasted to all fragments of the streamed side. Again, since the data is the key, the 
-       * value is null.
+       * This option can be used when the hash side is small. We can broadcast
+       * the entire data to all fragments of the stream side. This avoids
+       * re-partitioning the fragments of the stream side to match the
+       * partitioning scheme of the hash side and avoids costly network data
+       * transfer. However, in this example the stream side is being partitioned
+       * in both cases for brevity of code. The join task can perform the join
+       * of its fragment of keys with all the keys of the hash side. Using an
+       * unpartitioned edge to transfer the complete output of the hash side to
+       * be broadcasted to all fragments of the streamed side. Again, since the
+       * data is the key, the value is null.
        */
-      UnorderedKVEdgeConfig broadcastConf = UnorderedKVEdgeConfig.newBuilder(Text.class.getName(),
-          NullWritable.class.getName()).setFromConfiguration(tezConf).build();
+      UnorderedKVEdgeConfig broadcastConf =
+          UnorderedKVEdgeConfig
+              .newBuilder(Text.class.getName(), NullWritable.class.getName())
+              .setFromConfiguration(tezConf).build();
       hashSideEdgeProperty = broadcastConf.createDefaultBroadcastEdgeProperty();
     } else {
       /**
-       * The hash side is also being partitioned into fragments with the same key going to the same
-       * fragment using hash partitioning. This way all keys with the same hash value will go to the
-       * same fragment from both sides. Thus the join task handling that fragment can join both data
-       * set fragments. 
+       * The hash side is also being partitioned into fragments with the same
+       * key going to the same fragment using hash partitioning. This way all
+       * keys with the same hash value will go to the same fragment from both
+       * sides. Thus the join task handling that fragment can join both data set
+       * fragments.
        */
       hashSideEdgeProperty = streamConf.createDefaultEdgeProperty();
     }
 
     /**
-     * Connect the join vertex to the hash side.
-     * The join vertex is connected with 2 upstream vertices that provide it with inputs
+     * Connect the join vertex to the hash side. The join vertex is connected
+     * with 2 upstream vertices that provide it with inputs
      */
     Edge e2 = Edge.create(hashFileVertex, joinVertex, hashSideEdgeProperty);
 
     /**
      * Connect everything up by adding them to the DAG
      */
-    dag.addVertex(streamFileVertex).addVertex(hashFileVertex).addVertex(joinVertex)
-        .addEdge(e1).addEdge(e2);
+    dag.addVertex(streamFileVertex).addVertex(hashFileVertex)
+        .addVertex(joinVertex).addEdge(e1).addEdge(e2);
     return dag;
   }
 
   /**
-   * Reads key-values from the source and forwards the value as the key for the output
+   * Reads key-values from the source and forwards the value as the key for the
+   * output
    */
   public static class ForwardingProcessor extends SimpleProcessor {
     public ForwardingProcessor(ProcessorContext context) {
@@ -304,7 +334,7 @@
       Preconditions.checkState(getInputs().size() == 1);
       Preconditions.checkState(getOutputs().size() == 1);
       // not looking up inputs and outputs by name because there is just one
-      // instance and this processor is used in many different DAGs with 
+      // instance and this processor is used in many different DAGs with
       // different names for inputs and outputs
       LogicalInput input = getInputs().values().iterator().next();
       Reader rawReader = input.getReader();
@@ -316,16 +346,25 @@
 
       while (reader.next()) {
         Object val = reader.getCurrentValue();
-        // The data value itself is the join key. Simply write it out as the key.
+        // The data value itself is the join key. Simply write it out as the
+        // key.
         // The output value is null.
         writer.write(val, NullWritable.get());
       }
     }
   }
 
-  public static class JoinProcessor extends SimpleMRProcessor {
+  /**
+   * Join 2 inputs using Hash Join algorithm. Check the algorithm here <a
+   * href="http://en.wikipedia.org/wiki/Hash_join">Hash Join</a> <br>
+   * It would output all the occurrences keys in the streamFile which also exist
+   * in the hashFile. This require the keys in hashFile should be unique
+   * <br>Disclaimer: The join code here is written as a tutorial for the APIs and
+   * not for performance.
+   */
+  public static class HashJoinProcessor extends SimpleMRProcessor {
 
-    public JoinProcessor(ProcessorContext context) {
+    public HashJoinProcessor(ProcessorContext context) {
       super(context);
     }
 
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/JoinDataGen.java b/tez-examples/src/main/java/org/apache/tez/examples/JoinDataGen.java
index 8231b6f..ff73247 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/JoinDataGen.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/JoinDataGen.java
@@ -73,7 +73,6 @@
     System.err
         .println("Usage: "
             + "joindatagen <outPath1> <path1Size> <outPath2> <path2Size> <expectedResultPath> <numTasks>");
-    ;
     ToolRunner.printGenericCommandUsage(System.err);
   }
 
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
index e94df09..17e3dc8 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
@@ -43,7 +43,7 @@
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
 import org.apache.tez.dag.api.client.StatusGetOpts;
-import org.apache.tez.examples.JoinExample.ForwardingProcessor;
+import org.apache.tez.examples.HashJoinExample.ForwardingProcessor;
 import org.apache.tez.mapreduce.input.MRInput;
 import org.apache.tez.runtime.api.LogicalInput;
 import org.apache.tez.runtime.api.Reader;
@@ -57,7 +57,7 @@
 import com.google.common.collect.Sets;
 
 public class JoinValidate extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(JoinExample.class);
+  private static final Log LOG = LogFactory.getLog(JoinValidate.class);
 
   private static final String LHS_INPUT_NAME = "lhsfile";
   private static final String RHS_INPUT_NAME = "rhsfile";
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/SortMergeJoinExample.java b/tez-examples/src/main/java/org/apache/tez/examples/SortMergeJoinExample.java
new file mode 100644
index 0000000..b9ba87a
--- /dev/null
+++ b/tez-examples/src/main/java/org/apache/tez/examples/SortMergeJoinExample.java
@@ -0,0 +1,374 @@
+/**
+ * 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.tez.examples;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.Edge;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.examples.HashJoinExample.ForwardingProcessor;
+import org.apache.tez.mapreduce.input.MRInput;
+import org.apache.tez.mapreduce.output.MROutput;
+import org.apache.tez.mapreduce.processor.SimpleMRProcessor;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.ProcessorContext;
+import org.apache.tez.runtime.api.Reader;
+import org.apache.tez.runtime.library.api.KeyValueWriter;
+import org.apache.tez.runtime.library.api.KeyValuesReader;
+import org.apache.tez.runtime.library.conf.OrderedPartitionedKVEdgeConfig;
+import org.apache.tez.runtime.library.partitioner.HashPartitioner;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Simple example of joining 2 data sets using <a
+ * href="http://en.wikipedia.org/wiki/Sort-merge_join">Sort-Merge Join</a><br>
+ * There're 2 differences between {@link SortMergeJoinExample} and
+ * {@link HashJoinExample}. <li>We always load one data set(hashFile) in memory
+ * in {@link HashJoinExample} which require one dataset(hashFile) must be small
+ * enough to fit into memory, while in {@link SortMergeJoinExample}, it does not
+ * load one data set into memory, it just sort the output of the datasets before
+ * feeding to {@link SortMergeJoinProcessor}, just like the sort phase before
+ * reduce in traditional MapReduce. Then we could move forward the iterators of
+ * two inputs in {@link SortMergeJoinProcessor} to find the joined keys since
+ * they are both sorted already. <br> <li>Because of the sort implemention
+ * difference we describe above, the data requirement is also different for
+ * these 2 sort algorithms. For {@link HashJoinExample} It is required that keys
+ * in the hashFile are unique. while for {@link SortMergeJoinExample} it is
+ * required that keys in the both 2 datasets are unique.
+ */
+public class SortMergeJoinExample extends Configured implements Tool {
+
+  private static final Log LOG = LogFactory.getLog(SortMergeJoinExample.class);
+
+  private static final String input1 = "input1";
+  private static final String input2 = "input2";
+  private static final String inputFile = "inputFile";
+  private static final String joiner = "joiner";
+  private static final String joinOutput = "joinOutput";
+
+  public static void main(String[] args) throws Exception {
+    SortMergeJoinExample job = new SortMergeJoinExample();
+    int status = ToolRunner.run(new Configuration(), job, args);
+    System.exit(status);
+  }
+
+  private static void printUsage() {
+    System.err.println("Usage: "
+        + "sortmergejoin <file1> <file2> <numPartitions> <outPath>");
+    ToolRunner.printGenericCommandUsage(System.err);
+  }
+
+  @Override
+  public int run(String[] args) throws Exception {
+    Configuration conf = getConf();
+    String[] otherArgs =
+        new GenericOptionsParser(conf, args).getRemainingArgs();
+    int result = validateArgs(otherArgs);
+    if (result != 0) {
+      return result;
+    }
+    return execute(otherArgs);
+  }
+
+  public int run(Configuration conf, String[] args, TezClient tezClient)
+      throws Exception {
+    setConf(conf);
+    String[] otherArgs =
+        new GenericOptionsParser(conf, args).getRemainingArgs();
+    int result = validateArgs(otherArgs);
+    if (result != 0) {
+      return result;
+    }
+    return execute(otherArgs, tezClient);
+  }
+
+  private int validateArgs(String[] otherArgs) {
+    if (otherArgs.length != 4) {
+      printUsage();
+      return 2;
+    }
+    return 0;
+  }
+
+  private int execute(String[] args) throws TezException, IOException,
+      InterruptedException {
+    TezConfiguration tezConf = new TezConfiguration(getConf());
+    TezClient tezClient = null;
+    try {
+      tezClient = createTezClient(tezConf);
+      return execute(args, tezConf, tezClient);
+    } finally {
+      if (tezClient != null) {
+        tezClient.stop();
+      }
+    }
+  }
+
+  private int execute(String[] args, TezClient tezClient) throws IOException,
+      TezException, InterruptedException {
+    TezConfiguration tezConf = new TezConfiguration(getConf());
+    return execute(args, tezConf, tezClient);
+  }
+
+  private TezClient createTezClient(TezConfiguration tezConf)
+      throws TezException, IOException {
+    TezClient tezClient = TezClient.create("SortMergeJoinExample", tezConf);
+    tezClient.start();
+    return tezClient;
+  }
+
+  private int execute(String[] args, TezConfiguration tezConf,
+      TezClient tezClient) throws IOException, TezException,
+      InterruptedException {
+    LOG.info("Running SortMergeJoinExample");
+
+    UserGroupInformation.setConfiguration(tezConf);
+
+    String inputDir1 = args[0];
+    String inputDir2 = args[1];
+    int numPartitions = Integer.parseInt(args[2]);
+    String outputDir = args[3];
+
+    Path inputPath1 = new Path(inputDir1);
+    Path inputPath2 = new Path(inputDir2);
+    Path outputPath = new Path(outputDir);
+
+    // Verify output path existence
+    FileSystem fs = FileSystem.get(tezConf);
+    if (fs.exists(outputPath)) {
+      System.err.println("Output directory: " + outputDir + " already exists");
+      return 3;
+    }
+    if (numPartitions <= 0) {
+      System.err.println("NumPartitions must be > 0");
+      return 4;
+    }
+
+    DAG dag =
+        createDag(tezConf, inputPath1, inputPath2, outputPath, numPartitions);
+
+    tezClient.waitTillReady();
+    DAGClient dagClient = tezClient.submitDAG(dag);
+    DAGStatus dagStatus = dagClient.waitForCompletionWithStatusUpdates(null);
+    if (dagStatus.getState() != DAGStatus.State.SUCCEEDED) {
+      LOG.info("DAG diagnostics: " + dagStatus.getDiagnostics());
+      return -1;
+    }
+    return 0;
+
+  }
+
+  /**
+   * v1 v2 <br>
+   * &nbsp;\&nbsp;/ <br>
+   * &nbsp;&nbsp;v3 <br>
+   * 
+   * @param tezConf
+   * @param inputPath1
+   * @param inputPath2
+   * @param outPath
+   * @param numPartitions
+   * @return
+   * @throws IOException
+   */
+  private DAG createDag(TezConfiguration tezConf, Path inputPath1,
+      Path inputPath2, Path outPath, int numPartitions) throws IOException {
+    DAG dag = DAG.create("SortMergeJoinExample");
+
+    /**
+     * This vertex represents the one side of the join. It reads text data using
+     * the TextInputFormat. ForwardingProcessor simply forwards the data
+     * downstream as is.
+     */
+    Vertex inputVertex1 =
+        Vertex.create("input1",
+            ProcessorDescriptor.create(ForwardingProcessor.class.getName()))
+            .addDataSource(
+                inputFile,
+                MRInput
+                    .createConfigBuilder(new Configuration(tezConf),
+                        TextInputFormat.class, inputPath1.toUri().toString())
+                    .groupSplits(false).build());
+
+    /**
+     * The other vertex represents the other side of the join. It reads text
+     * data using the TextInputFormat. ForwardingProcessor simply forwards the
+     * data downstream as is.
+     */
+    Vertex inputVertex2 =
+        Vertex.create("input2",
+            ProcessorDescriptor.create(ForwardingProcessor.class.getName()))
+            .addDataSource(
+                inputFile,
+                MRInput
+                    .createConfigBuilder(new Configuration(tezConf),
+                        TextInputFormat.class, inputPath2.toUri().toString())
+                    .groupSplits(false).build());
+
+    /**
+     * This vertex represents the join operation. It writes the join output as
+     * text using the TextOutputFormat. The JoinProcessor is going to perform
+     * the join of the two sorted output from inputVertex1 and inputVerex2. It
+     * is load balanced across numPartitions.
+     */
+    Vertex joinVertex =
+        Vertex.create(joiner,
+            ProcessorDescriptor.create(SortMergeJoinProcessor.class.getName()),
+            numPartitions).addDataSink(
+            joinOutput,
+            MROutput.createConfigBuilder(new Configuration(tezConf),
+                TextOutputFormat.class, outPath.toUri().toString()).build());
+
+    /**
+     * The output of inputVertex1 and inputVertex2 will be partitioned into
+     * fragments with the same keys going to the same fragments using hash
+     * partitioning. The data to be joined is the key itself and so the value is
+     * null. And these outputs will be sorted before feeding them to
+     * JoinProcessor. The number of fragments is initially inferred from the
+     * number of tasks running in the join vertex because each task will be
+     * handling one fragment.
+     */
+    OrderedPartitionedKVEdgeConfig edgeConf =
+        OrderedPartitionedKVEdgeConfig
+            .newBuilder(Text.class.getName(), NullWritable.class.getName(),
+                HashPartitioner.class.getName()).setFromConfiguration(tezConf)
+            .build();
+
+    /**
+     * Connect the join vertex with inputVertex1 with the EdgeProperty created
+     * from {@link OrderedPartitionedKVEdgeConfig} so that the output of
+     * inputVertex1 is sorted before feeding it to JoinProcessor
+     */
+    Edge e1 =
+        Edge.create(inputVertex1, joinVertex,
+            edgeConf.createDefaultEdgeProperty());
+    /**
+     * Connect the join vertex with inputVertex2 with the EdgeProperty created
+     * from {@link OrderedPartitionedKVEdgeConfig} so that the output of
+     * inputVertex1 is sorted before feeding it to JoinProcessor
+     */
+    Edge e2 =
+        Edge.create(inputVertex2, joinVertex,
+            edgeConf.createDefaultEdgeProperty());
+
+    dag.addVertex(inputVertex1).addVertex(inputVertex2).addVertex(joinVertex)
+        .addEdge(e1).addEdge(e2);
+    return dag;
+  }
+
+  /**
+   * Join 2 inputs which has already been sorted. Check the algorithm here <a
+   * href="http://en.wikipedia.org/wiki/Sort-merge_join">Sort-Merge Join</a><br>
+   * It require the keys in both datasets are unique. <br>
+   * Disclaimer: The join code here is written as a tutorial for the APIs and
+   * not for performance.
+   */
+  public static class SortMergeJoinProcessor extends SimpleMRProcessor {
+
+    public SortMergeJoinProcessor(ProcessorContext context) {
+      super(context);
+    }
+
+    @Override
+    public void run() throws Exception {
+      Preconditions.checkState(getInputs().size() == 2);
+      Preconditions.checkState(getOutputs().size() == 1);
+      // Get the input data for the 2 sides of the join from the 2 inputs
+      LogicalInput logicalInput1 = getInputs().get(input1);
+      LogicalInput logicalInput2 = getInputs().get(input2);
+      Reader inputReader1 = logicalInput1.getReader();
+      Reader inputReader2 = logicalInput2.getReader();
+      Preconditions.checkState(inputReader1 instanceof KeyValuesReader);
+      Preconditions.checkState(inputReader2 instanceof KeyValuesReader);
+      LogicalOutput lo = getOutputs().get(joinOutput);
+      Preconditions.checkState(lo.getWriter() instanceof KeyValueWriter);
+      KeyValueWriter writer = (KeyValueWriter) lo.getWriter();
+
+      join((KeyValuesReader) inputReader1, (KeyValuesReader) inputReader2,
+          writer);
+    }
+
+    /**
+     * Join 2 sorted inputs both from {@link KeyValuesReader} and write output
+     * using {@link KeyValueWriter}
+     * 
+     * @param inputReader1
+     * @param inputReader2
+     * @param writer
+     * @throws IOException
+     */
+    private void join(KeyValuesReader inputReader1,
+        KeyValuesReader inputReader2, KeyValueWriter writer) throws IOException {
+
+      while (inputReader1.next() && inputReader2.next()) {
+        Text value1 = (Text) inputReader1.getCurrentKey();
+        Text value2 = (Text) inputReader2.getCurrentKey();
+        boolean reachEnd = false;
+        // move the cursor of 2 inputs forward until find the same values or one
+        // of them reach the end.
+        while (value1.compareTo(value2) != 0) {
+          if (value1.compareTo(value2) > 0) {
+            if (inputReader2.next()) {
+              value2 = (Text) inputReader2.getCurrentKey();
+            } else {
+              reachEnd = true;
+              break;
+            }
+          } else {
+            if (inputReader1.next()) {
+              value1 = (Text) inputReader1.getCurrentKey();
+            } else {
+              reachEnd = true;
+              break;
+            }
+          }
+        }
+
+        if (reachEnd) {
+          break;
+        } else {
+          writer.write(value1, NullWritable.get());
+        }
+      }
+    }
+  }
+}
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
index ddb001c..f38fc9c 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
@@ -21,6 +21,7 @@
 
 import java.io.IOException;
 import java.net.URI;
+import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
@@ -39,7 +40,6 @@
 import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
 import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
 import org.apache.hadoop.security.Credentials;
-import org.apache.tez.client.TezClientUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.dag.api.DataSourceDescriptor;
 import org.apache.tez.dag.api.InputDescriptor;
@@ -267,7 +267,7 @@
 
       MRHelpers.translateMRConfToTez(conf);
 
-      Credentials credentials = maybeGetCredentials();
+      Collection<URI> uris = maybeGetURIsForCredentials();
 
       UserPayload payload = null;
       if (groupSplitsInAM) {
@@ -276,16 +276,20 @@
         payload = MRInputHelpersInternal.createMRInputPayload(conf, null);
       }
 
-      return DataSourceDescriptor
+      DataSourceDescriptor ds = DataSourceDescriptor
           .create(InputDescriptor.create(inputClassName).setUserPayload(payload),
-              customInitializerDescriptor, credentials);
+              customInitializerDescriptor, null);
+      if (uris != null) {
+        ds.addURIsForCredentials(uris);
+      }
+      return ds;
     }
 
     private DataSourceDescriptor createGeneratorDataSource() throws IOException {
       setupBasicConf(conf);
       MRHelpers.translateMRConfToTez(conf);
       
-      Credentials credentials = maybeGetCredentials();
+      Collection<URI> uris = maybeGetURIsForCredentials();
 
       UserPayload payload = null;
       if (groupSplitsInAM) {
@@ -293,9 +297,13 @@
       } else {
         payload = MRInputHelpersInternal.createMRInputPayload(conf, null);
       }
-      return DataSourceDescriptor.create(
+      DataSourceDescriptor ds = DataSourceDescriptor.create(
           InputDescriptor.create(inputClassName).setUserPayload(payload),
-          InputInitializerDescriptor.create(MRInputAMSplitGenerator.class.getName()), credentials);
+          InputInitializerDescriptor.create(MRInputAMSplitGenerator.class.getName()), null);
+      if (uris != null) {
+        ds.addURIsForCredentials(uris);
+      }
+      return ds;
     }
 
     private void setupBasicConf(Configuration inputConf) {
@@ -309,8 +317,7 @@
       }
     }
 
-    private Credentials maybeGetCredentials() {
-      Credentials credentials = null;
+    private Collection<URI> maybeGetURIsForCredentials() {
       if (getCredentialsForSourceFilesystem && inputPaths != null) {
         try {
           List<URI> uris = Lists.newLinkedList();
@@ -321,13 +328,12 @@
             Path qPath = fs.makeQualified(path);
             uris.add(qPath.toUri());
           }
-          credentials = new Credentials();
-          TezClientUtils.addFileSystemCredentialsFromURIs(uris, credentials, conf);
+          return uris;
         } catch (IOException e) {
           throw new TezUncheckedException(e);
         }
       }
-      return credentials;
+      return null;
     }
 
   }
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
index ab9b41d..421fc8c 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
@@ -21,7 +21,9 @@
 import javax.annotation.Nullable;
 
 import java.io.IOException;
+import java.net.URI;
 import java.text.NumberFormat;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -164,25 +166,27 @@
                   FileOutputFormat.class.getName());
         }
       }
-      Credentials credentials = null;
+      Collection<URI> uris = null;
       if (getCredentialsForSinkFilesystem && outputPath != null) {
         try {
           Path path = new Path(outputPath);
           FileSystem fs;
           fs = path.getFileSystem(conf);
           Path qPath = fs.makeQualified(path);
-          credentials = new Credentials();
-          TezClientUtils.addFileSystemCredentialsFromURIs(Collections.singletonList(qPath.toUri()),
-              credentials, conf);
+          uris = Collections.singletonList(qPath.toUri());
         } catch (IOException e) {
           throw new TezUncheckedException(e);
         }
       }
 
-      return DataSinkDescriptor.create(
+      DataSinkDescriptor ds = DataSinkDescriptor.create(
           OutputDescriptor.create(outputClassName).setUserPayload(createUserPayload()),
           (doCommit ? OutputCommitterDescriptor.create(
-              MROutputCommitter.class.getName()) : null), credentials);
+              MROutputCommitter.class.getName()) : null), null);
+      if (uris != null) {
+        ds.addURIsForCredentials(uris);
+      }
+      return ds;
     }
     
     /**
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java
index 55f9b11..7f31741 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java
@@ -212,7 +212,7 @@
     }
 
     @Override
-    public void registerForVertexStatusUpdates(String vertexName, Set<VertexState> stateSet) {
+    public void registerForVertexStateUpdates(String vertexName, Set<VertexState> stateSet) {
       throw new UnsupportedOperationException("getVertexNumTasks not implemented in this mock");
     }
 
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java
index ca82877..eeb3676 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java
@@ -19,7 +19,9 @@
 package org.apache.tez.dag.library.vertexmanager;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.protobuf.ByteString;
+
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Collections;
@@ -166,7 +168,8 @@
       this.numDestinationTasks = config.numDestinationTasks;
       this.basePartitionRange = config.basePartitionRange;
       this.remainderRangeForLastShuffler = config.remainderRangeForLastShuffler;
-      this.numSourceTasks = config.numSourceTasks;
+      this.numSourceTasks = getContext().getSourceVertexNumTasks();
+      Preconditions.checkState(this.numDestinationTasks == getContext().getDestinationVertexNumTasks());
     }
 
     @Override
@@ -266,18 +269,15 @@
     int numDestinationTasks;
     int basePartitionRange;
     int remainderRangeForLastShuffler;
-    int numSourceTasks;
 
     private CustomShuffleEdgeManagerConfig(int numSourceTaskOutputs,
         int numDestinationTasks,
-        int numSourceTasks,
         int basePartitionRange,
         int remainderRangeForLastShuffler) {
       this.numSourceTaskOutputs = numSourceTaskOutputs;
       this.numDestinationTasks = numDestinationTasks;
       this.basePartitionRange = basePartitionRange;
       this.remainderRangeForLastShuffler = remainderRangeForLastShuffler;
-      this.numSourceTasks = numSourceTasks;
     }
 
     public UserPayload toUserPayload() {
@@ -287,7 +287,6 @@
               .setNumDestinationTasks(numDestinationTasks)
               .setBasePartitionRange(basePartitionRange)
               .setRemainderRangeForLastShuffler(remainderRangeForLastShuffler)
-              .setNumSourceTasks(numSourceTasks)
               .build().toByteArray()));
     }
 
@@ -298,7 +297,6 @@
       return new CustomShuffleEdgeManagerConfig(
           proto.getNumSourceTaskOutputs(),
           proto.getNumDestinationTasks(),
-          proto.getNumSourceTasks(),
           proto.getBasePartitionRange(),
           proto.getRemainderRangeForLastShuffler());
 
@@ -464,8 +462,7 @@
         // for the source tasks
         CustomShuffleEdgeManagerConfig edgeManagerConfig =
             new CustomShuffleEdgeManagerConfig(
-                currentParallelism, finalTaskParallelism, 
-                getContext().getVertexNumTasks(vertex), basePartitionRange,
+                currentParallelism, finalTaskParallelism, basePartitionRange,
                 ((remainderRangeForLastShuffler > 0) ?
                     remainderRangeForLastShuffler : basePartitionRange));
         EdgeManagerPluginDescriptor edgeManagerDescriptor =
@@ -667,7 +664,7 @@
     }
 
     public ShuffleVertexManagerConfigBuilder setSlowStartMinSrcCompletionFraction(float minFraction) {
-      conf.setFloat(ShuffleVertexManager.TEZ_SHUFFLE_VERTEX_MANAGER_MAX_SRC_FRACTION, minFraction);
+      conf.setFloat(ShuffleVertexManager.TEZ_SHUFFLE_VERTEX_MANAGER_MIN_SRC_FRACTION, minFraction);
       return this;
     }
 
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
index 077ce8e..cb61109 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
@@ -280,6 +280,17 @@
    */
   public static final boolean TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH_DEFAULT = false;
 
+  /**
+   * Share data fetched between tasks running on the same host if applicable
+   */
+  public static final String TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH = TEZ_RUNTIME_PREFIX
+      + "optimize.shared.fetch";
+
+  /**
+   * shared mode bypassing the http fetch is not enabled by default till we have unit tests in.
+   */
+  public static final boolean TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH_DEFAULT = false;
+
   // TODO TEZ-1233 - allow this property to be set per vertex
   // TODO TEZ-1231 - move these properties out since they are not relevant for Inputs / Outputs
 
@@ -333,6 +344,7 @@
     tezRuntimeKeys.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE);
     tezRuntimeKeys.add(TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS);
     tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH);
+    tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH);
 
     defaultConf.addResource("core-default.xml");
     defaultConf.addResource("core-site.xml");
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
index 7c8a23b..9987d26 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
@@ -29,28 +29,38 @@
 
   private final InputIdentifier inputIdentifier;
   private final int attemptNumber;
-  private String pathComponent;
-  
+  private final String pathComponent;
+  private final boolean shared;
+
   public static final String PATH_PREFIX = "attempt";
-  
+
   public InputAttemptIdentifier(int inputIndex, int attemptNumber) {
     this(new InputIdentifier(inputIndex), attemptNumber, null);
   }
-  
+
   public InputAttemptIdentifier(InputIdentifier inputIdentifier, int attemptNumber, String pathComponent) {
+    this(inputIdentifier, attemptNumber, pathComponent, false);
+  }
+
+  public InputAttemptIdentifier(InputIdentifier inputIdentifier, int attemptNumber, String pathComponent, boolean shared) {
     this.inputIdentifier = inputIdentifier;
     this.attemptNumber = attemptNumber;
     this.pathComponent = pathComponent;
+    this.shared = shared;
     if (pathComponent != null && !pathComponent.startsWith(PATH_PREFIX)) {
       throw new TezUncheckedException(
           "Path component must start with: " + PATH_PREFIX + " " + this);
     }
   }
-  
+
   public InputAttemptIdentifier(int taskIndex, int attemptNumber, String pathComponent) {
     this(new InputIdentifier(taskIndex), attemptNumber, pathComponent);
   }
 
+  public InputAttemptIdentifier(int taskIndex, int attemptNumber, String pathComponent, boolean shared) {
+    this(new InputIdentifier(taskIndex), attemptNumber, pathComponent, shared);
+  }
+
   public InputIdentifier getInputIdentifier() {
     return this.inputIdentifier;
   }
@@ -63,7 +73,11 @@
     return pathComponent;
   }
 
-  // PathComponent does not need to be part of the hashCode and equals computation.
+  public boolean isShared() {
+    return this.shared;
+  }
+
+  // PathComponent & shared does not need to be part of the hashCode and equals computation.
   @Override
   public int hashCode() {
     final int prime = 31;
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/comparator/HashComparator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/comparator/HashComparator.java
deleted file mode 100644
index 56752f9..0000000
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/comparator/HashComparator.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * 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.tez.runtime.library.common.comparator;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-
-@Unstable
-@Private
-public interface HashComparator<KEY> {
-
-  int getHashCode(KEY key);
-
-}
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/comparator/ProxyComparator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/comparator/ProxyComparator.java
new file mode 100644
index 0000000..5297f7c
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/comparator/ProxyComparator.java
@@ -0,0 +1,46 @@
+/**
+ * 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.tez.runtime.library.common.comparator;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.io.RawComparator;
+
+@Unstable
+@Private
+public interface ProxyComparator<KEY> extends RawComparator {
+  /**
+   * This comparator interface provides a fast-path for comparisons between keys.
+   *
+   * The implicit assumption is that integer returned from this function serves
+   * as a transitive comparison proxy for the comparator that this implements.
+   *
+   * But this does not serve as a measure of equality.
+   *
+   * getProxy(k1) < getProxy(k2) implies k1 < k2 (transitive between different keys for sorting requirements)
+   *
+   * getProxy(k1) == getProxy(k2) does not imply ordering, but requires actual key comparisons.
+   *
+   * This serves as a way to short-circuit  the RawComparator speeds.
+   *
+   * @param key
+   * @return proxy
+   */
+  int getProxy(KEY key);
+
+}
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/comparator/TezBytesComparator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/comparator/TezBytesComparator.java
index 7e921de..09b2b03 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/comparator/TezBytesComparator.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/comparator/TezBytesComparator.java
@@ -24,7 +24,8 @@
 
 @Public
 @Unstable
-public class TezBytesComparator extends WritableComparator {
+public class TezBytesComparator extends WritableComparator implements
+    ProxyComparator<BytesWritable> {
 
   public TezBytesComparator() {
     super(BytesWritable.class);
@@ -39,4 +40,24 @@
     return compareBytes(b1, s1, l1, b2, s2, l2);
   }
 
+  @Override
+  public int getProxy(BytesWritable key) {
+    int prefix = 0;
+    final int len = key.getLength();
+    final byte[] content = key.getBytes();
+    int b1 = 0, b2 = 0, b3 = 0;
+    switch (len) {
+    default:
+    case 3:
+      b3 = content[2] & 0xff;
+    case 2:
+      b2 = content[1] & 0xff;
+    case 1:
+      b1 = content[0] & 0xff;
+    case 0:
+    }
+    prefix = (b1 << 16) | (b2 << 8) | (b3);
+    return prefix;
+  }
+
 }
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
index 9b009ea..c29401b 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
@@ -35,6 +35,7 @@
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.tez.common.TezUtilsInternal;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.tez.common.TezRuntimeFrameworkConfigs;
@@ -92,6 +93,8 @@
 
   HttpConnection httpConnection;
   HttpConnectionParams httpConnectionParams;
+
+  final static String localhostName = NetUtils.getHostname();
   
   public Fetcher(HttpConnectionParams httpConnectionParams,
                  ShuffleScheduler scheduler, MergeManager merger,
@@ -132,7 +135,7 @@
     this.localDiskFetchEnabled = localDiskFetchEnabled;
 
     this.logIdentifier = "fetcher [" + TezUtilsInternal
-        .cleanVertexName(inputContext.getSourceVertexName()) + "] #" + id;
+        .cleanVertexName(inputContext.getSourceVertexName()) + "] #" + id + " " + localhostName;
     setName(logIdentifier);
     setDaemon(true);
   }  
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
index 8085f98..0a14946 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
@@ -39,7 +39,7 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.tez.common.TezUtilsInternal;
-import org.apache.tez.runtime.library.common.comparator.HashComparator;
+import org.apache.tez.runtime.library.common.comparator.ProxyComparator;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.util.IndexedSortable;
 import org.apache.hadoop.util.IndexedSorter;
@@ -78,7 +78,7 @@
 
   int numSpills = 0;
   private final int minSpillsForCombine;
-  private final HashComparator hasher;
+  private final ProxyComparator hasher;
   // SortSpans  
   private SortSpan span;
   private ByteBuffer largeBuffer;
@@ -135,8 +135,8 @@
         .build());
 
     // k/v serialization    
-    if(comparator instanceof HashComparator) {
-      hasher = (HashComparator)comparator;
+    if(comparator instanceof ProxyComparator) {
+      hasher = (ProxyComparator)comparator;
       LOG.info("Using the HashComparator");
     } else {
       hasher = null;
@@ -236,7 +236,7 @@
     int prefix = 0;
 
     if(hasher != null) {
-      prefix = hasher.getHashCode(key);
+      prefix = hasher.getProxy(key);
     }
 
     prefix = (partition << (32 - partitionBits)) | (prefix >>> partitionBits);
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValueInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValueInput.java
index e875240..39e0fff 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValueInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValueInput.java
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.tez.dag.api.GroupInputEdge;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.runtime.api.Input;
@@ -34,6 +35,7 @@
  * (e.g. from a {@link GroupInputEdge} and provide a unified view of the 
  * input. It concatenates all the inputs to provide a unified view
  */
+@Public
 public class ConcatenatedMergedKeyValueInput extends MergedLogicalInput {
 
   public ConcatenatedMergedKeyValueInput(MergedInputContext context,
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValuesInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValuesInput.java
index 7a57240..0cc3244 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValuesInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ConcatenatedMergedKeyValuesInput.java
@@ -21,6 +21,7 @@
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.tez.dag.api.GroupInputEdge;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.runtime.api.Input;
@@ -35,6 +36,7 @@
  * input. It concatenates all the inputs to provide a unified view
  */
 
+@Public
 public class ConcatenatedMergedKeyValuesInput extends MergedLogicalInput {
 
   public ConcatenatedMergedKeyValuesInput(MergedInputContext context,
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
index 7e7f2c4..fe85a99 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedKVInput.java
@@ -30,6 +30,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.tez.common.TezUtils;
@@ -66,6 +67,7 @@
  * completion. Attempting to get a reader on a non-complete input will block.
  *
  */
+@Public
 public class OrderedGroupedKVInput extends AbstractLogicalInput {
 
   static final Log LOG = LogFactory.getLog(OrderedGroupedKVInput.class);
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedMergedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedMergedKVInput.java
index 45c68aa..08cf662 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedMergedKVInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/OrderedGroupedMergedKVInput.java
@@ -29,6 +29,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.tez.runtime.api.Input;
 import org.apache.tez.runtime.api.MergedLogicalInput;
@@ -43,6 +44,7 @@
  * Combiners and Secondary Sort are not implemented, so there is no guarantee on
  * the order of values.
  */
+@Public
 public class OrderedGroupedMergedKVInput extends MergedLogicalInput {
 
   private static final Log LOG = LogFactory.getLog(OrderedGroupedMergedKVInput.class);
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
index 87caf4c..75fa64a 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/UnorderedKVInput.java
@@ -27,6 +27,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.hadoop.io.compress.DefaultCodec;
@@ -57,6 +58,7 @@
  * unified view to that data. There are no ordering constraints applied by
  * this input.
  */
+@Public
 public class UnorderedKVInput extends AbstractLogicalInput {
 
   private static final Log LOG = LogFactory.getLog(UnorderedKVInput.class);
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java
index 15121da..40e99ad 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OrderedPartitionedKVOutput.java
@@ -29,6 +29,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
@@ -65,6 +66,7 @@
  * key/value pairs written to it. It also partitions the output based on a
  * {@link Partitioner}
  */
+@Public
 public class OrderedPartitionedKVOutput extends AbstractLogicalOutput {
 
   private static final Log LOG = LogFactory.getLog(OrderedPartitionedKVOutput.class);
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java
index ea02743..76dab45 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedKVOutput.java
@@ -29,6 +29,7 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.tez.common.TezUtils;
@@ -58,6 +59,7 @@
  * value data without applying any ordering or grouping constraints. This can be
  * used to write raw key value data as is.
  */
+@Public
 public class UnorderedKVOutput extends AbstractLogicalOutput {
 
   private static final Log LOG = LogFactory.getLog(UnorderedKVOutput.class);
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java
index 8d4ed3a..9b61df0 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/UnorderedPartitionedKVOutput.java
@@ -29,6 +29,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.TezRuntimeFrameworkConfigs;
@@ -47,6 +48,7 @@
  * write Key-Value pairs. The key-value pairs are written to the correct partition based on the
  * configured Partitioner.
  */
+@Public
 public class UnorderedPartitionedKVOutput extends AbstractLogicalOutput {
 
   private static final Log LOG = LogFactory.getLog(UnorderedPartitionedKVOutput.class);
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
index 1d26c6e..b0b911b 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
@@ -65,6 +65,13 @@
   public InputStream getInputStream() throws IOException {
     return localFS.open(outputPath);
   }
+
+  public final Path getInputPath() {
+    if (state == State.COMMITTED) {
+      return this.outputPath;
+    }
+    return this.tmpOutputPath;
+  }
   
   @Override
   public void commit() throws IOException {
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
index fde19b7..0a83dc9 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
@@ -110,7 +110,8 @@
 
   /**
    * Return an input stream to be used to read the previously fetched data.
-   * Users are expected to close the InputStream when they're done
+   * All calls to getInputStream() produce new reset streams for reading.
+   * Users are expected to close the InputStream when they're done.
    */
   public abstract InputStream getInputStream() throws IOException;
 
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
index 1707ab7..288df6d 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
@@ -21,11 +21,14 @@
 import java.io.IOException;
 
 import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
 
 public interface FetchedInputAllocator {
 
   public FetchedInput allocate(long actualSize, long compresedSize,
       InputAttemptIdentifier inputAttemptIdentifier) throws IOException;
   
+  public FetchedInput allocateType(Type type, long actualSize, long compresedSize,
+      InputAttemptIdentifier inputAttemptIdentifier) throws IOException;
   
 }
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
index 9cb8617..e25124b 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
@@ -19,10 +19,17 @@
 package org.apache.tez.runtime.library.shuffle.common;
 
 import java.io.DataInputStream;
+import java.io.File;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.RandomAccessFile;
 import java.net.URL;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -34,12 +41,19 @@
 import javax.crypto.SecretKey;
 
 import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.tez.common.TezRuntimeFrameworkConfigs;
@@ -54,6 +68,7 @@
 import org.apache.tez.runtime.library.shuffle.common.HttpConnection.HttpConnectionParams;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
 
 /**
  * Responsible for fetching inputs served by the ShuffleHandler for a single
@@ -101,10 +116,22 @@
   private HttpConnectionParams httpConnectionParams;
 
   private final boolean localDiskFetchEnabled;
+  private final boolean sharedFetchEnabled;
+
+  private final LocalDirAllocator localDirAllocator;
+  private final Path lockPath;
+  private final RawLocalFileSystem localFs;
+
+  private final boolean isDebugEnabled = LOG.isDebugEnabled();
 
   private Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params,
-      FetchedInputAllocator inputManager, ApplicationId appId, SecretKey shuffleSecret,
-      String srcNameTrimmed, Configuration conf, boolean localDiskFetchEnabled) {
+      FetchedInputAllocator inputManager, ApplicationId appId,
+      SecretKey shuffleSecret, String srcNameTrimmed, Configuration conf,
+      RawLocalFileSystem localFs,
+      LocalDirAllocator localDirAllocator,
+      Path lockPath,
+      boolean localDiskFetchEnabled,
+      boolean sharedFetchEnabled) {
     this.fetcherCallback = fetcherCallback;
     this.inputManager = inputManager;
     this.shuffleSecret = shuffleSecret;
@@ -114,19 +141,42 @@
     this.conf = conf;
 
     this.localDiskFetchEnabled = localDiskFetchEnabled;
+    this.sharedFetchEnabled = sharedFetchEnabled;
 
     this.fetcherIdentifier = fetcherIdGen.getAndIncrement();
-    this.logIdentifier = "fetcher [" + srcNameTrimmed +"] " + fetcherIdentifier;
+    this.logIdentifier = " fetcher [" + srcNameTrimmed +"] " + fetcherIdentifier;
+
+    this.localFs = localFs;
+    this.localDirAllocator = localDirAllocator;
+    this.lockPath = lockPath;
+
+    try {
+      if (this.sharedFetchEnabled) {
+        this.localFs.mkdirs(this.lockPath);
+      }
+    } catch (Exception e) {
+      LOG.warn("Error initializing local dirs for shared transfer " + e);
+    }
   }
 
   @Override
   public FetchResult call() throws Exception {
+    boolean multiplex = (this.sharedFetchEnabled && this.localDiskFetchEnabled);
+
     if (srcAttempts.size() == 0) {
       return new FetchResult(host, port, partition, srcAttempts);
     }
 
     for (InputAttemptIdentifier in : srcAttempts) {
       pathToAttemptMap.put(in.getPathComponent(), in);
+      // do only if all of them are shared fetches
+      multiplex &= in.isShared();
+    }
+
+    if (multiplex) {
+      Preconditions.checkArgument(partition == 0,
+          "Shared fetches cannot be done for partitioned input"
+              + "- partition is non-zero (%d)", partition);
     }
 
     remaining = new LinkedHashSet<InputAttemptIdentifier>(srcAttempts);
@@ -136,7 +186,9 @@
     if (localDiskFetchEnabled &&
         host.equals(System.getenv(ApplicationConstants.Environment.NM_HOST.toString()))) {
       hostFetchResult = setupLocalDiskFetch();
-    } else {
+    } else if (multiplex) {
+      hostFetchResult = doSharedFetch();
+    } else{
       hostFetchResult = doHttpFetch();
     }
 
@@ -151,15 +203,197 @@
 
     // Sanity check
     if (hostFetchResult.failedInputs == null && !remaining.isEmpty()) {
-      throw new IOException("server didn't return all expected map outputs: "
-          + remaining.size() + " left.");
+      if (!multiplex) {
+        throw new IOException("server didn't return all expected map outputs: "
+            + remaining.size() + " left.");
+      } else {
+        LOG.info("Shared fetch failed to return " + remaining.size() + " inputs on this try");
+      }
     }
 
     return hostFetchResult.fetchResult;
   }
 
+  private final class CachingCallBack {
+    // this is a closure object wrapping this in an inner class
+    public void cache(String host,
+        InputAttemptIdentifier srcAttemptId, FetchedInput fetchedInput,
+        long compressedLength, long decompressedLength) {
+      try {
+        // this breaks badly on partitioned input - please use responsibly
+        Preconditions.checkArgument(partition == 0, "Partition == 0");
+        final String tmpSuffix = "." + System.currentTimeMillis() + ".tmp";
+        final String finalOutput = getMapOutputFile(srcAttemptId.getPathComponent());
+        final Path outputPath = localDirAllocator.getLocalPathForWrite(finalOutput, compressedLength, conf);
+        final TezSpillRecord spillRec = new TezSpillRecord(1);
+        final TezIndexRecord indexRec;
+        Path tmpIndex = outputPath.suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING+tmpSuffix);
+
+        if (localFs.exists(tmpIndex)) {
+          LOG.warn("Found duplicate instance of input index file " + tmpIndex);
+          return;
+        }
+
+        Path tmpPath = null;
+
+        switch (fetchedInput.getType()) {
+        case DISK: {
+          DiskFetchedInput input = (DiskFetchedInput) fetchedInput;
+          indexRec = new TezIndexRecord(0, decompressedLength, compressedLength);
+          localFs.mkdirs(outputPath.getParent());
+          // avoid pit-falls of speculation
+          tmpPath = outputPath.suffix(tmpSuffix);
+          // JDK7 - TODO: use Files implementation to speed up this process
+          localFs.copyFromLocalFile(input.getInputPath(), tmpPath);
+          // rename is atomic
+          boolean renamed = localFs.rename(tmpPath, outputPath);
+          if(!renamed) {
+            LOG.warn("Could not rename to cached file name " + outputPath);
+            localFs.delete(tmpPath, false);
+            return;
+          }
+        }
+        break;
+        default:
+          LOG.warn("Incorrect use of CachingCallback for " + srcAttemptId);
+          return;
+        }
+
+        spillRec.putIndex(indexRec, 0);
+        spillRec.writeToFile(tmpIndex, conf);
+        // everything went well so far - rename it
+        boolean renamed = localFs.rename(tmpIndex, outputPath
+            .suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING));
+        if (!renamed) {
+          localFs.delete(tmpIndex, false);
+          if (outputPath != null) {
+            // invariant: outputPath was renamed from tmpPath
+            localFs.delete(outputPath, false);
+          }
+          LOG.warn("Could not rename the index file to "
+              + outputPath
+                  .suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING));
+          return;
+        }
+      } catch (IOException ioe) {
+        // do mostly nothing
+        LOG.warn("Cache threw an error " + ioe);
+      }
+    }
+  }
+
+  private int findInputs() throws IOException {
+    int k = 0;
+    for (InputAttemptIdentifier src : srcAttempts) {
+      try {
+        if (getShuffleInputFileName(src.getPathComponent(),
+            Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING) != null) {
+          k++;
+        }
+      } catch (DiskErrorException de) {
+        // missing file, ignore
+      }
+    }
+    return k;
+  }
+
+  private FileLock getLock() throws OverlappingFileLockException, InterruptedException, IOException {
+    File lockFile = localFs.pathToFile(new Path(lockPath, host + ".lock"));
+
+    final boolean created = lockFile.createNewFile();
+
+    if (created == false && !lockFile.exists()) {
+      // bail-out cleanly
+      return null;
+    }
+
+    // invariant - file created (winner writes to this file)
+    // caveat: closing lockChannel does close the file (do not double close)
+    // JDK7 - TODO: use AsynchronousFileChannel instead of RandomAccessFile
+    FileChannel lockChannel = new RandomAccessFile(lockFile, "rws")
+        .getChannel();
+    FileLock xlock = null;
+
+    xlock = lockChannel.tryLock(0, Long.MAX_VALUE, false);
+    if (xlock != null) {
+      return xlock;
+    }
+    lockChannel.close();
+    return null;
+  }
+
+  private void releaseLock(FileLock lock) throws IOException {
+    if (lock != null && lock.isValid()) {
+      FileChannel lockChannel = lock.channel();
+      lock.release();
+      lockChannel.close();
+    }
+  }
+
+  protected HostFetchResult doSharedFetch() throws IOException {
+    int inputs = findInputs();
+
+    if (inputs == srcAttempts.size()) {
+      if (isDebugEnabled) {
+        LOG.debug("Using the copies found locally");
+      }
+      return doLocalDiskFetch(true);
+    }
+
+    if (inputs > 0) {
+      if (isDebugEnabled) {
+        LOG.debug("Found " + input
+            + " local fetches right now, using them first");
+      }
+      return doLocalDiskFetch(false);
+    }
+
+    FileLock lock = null;
+    try {
+      lock = getLock();
+      if (lock == null) {
+        // re-queue until we get a lock
+        LOG.info("Requeuing " + host + ":" + port
+            + " downloads because we didn't get a lock");
+        return new HostFetchResult(new FetchResult(host, port, partition,
+            remaining), null, false);
+      } else {
+        if (findInputs() == srcAttempts.size()) {
+          // double checked after lock
+          releaseLock(lock);
+          lock = null;
+          return doLocalDiskFetch(true);
+        }
+        // cache data if possible
+        return doHttpFetch(new CachingCallBack());
+      }
+    } catch (OverlappingFileLockException jvmCrossLock) {
+      // fall back to HTTP fetch below
+      LOG.warn("Double locking detected for " + host);
+    } catch (InterruptedException sleepInterrupted) {
+      // fall back to HTTP fetch below
+      LOG.warn("Lock was interrupted for " + host);
+    } finally {
+      releaseLock(lock);
+    }
+
+    if (isShutDown.get()) {
+      // if any exception was due to shut-down don't bother firing any more
+      // requests
+      return new HostFetchResult(new FetchResult(host, port, partition,
+          remaining), null, false);
+    }
+    // no more caching
+    return doHttpFetch();
+  }
+
   @VisibleForTesting
   protected HostFetchResult doHttpFetch() {
+    return doHttpFetch(null);
+  }
+
+  @VisibleForTesting
+  protected HostFetchResult doHttpFetch(CachingCallBack callback) {
     try {
       StringBuilder baseURI = ShuffleUtils.constructBaseURIForShuffleHandler(host,
           port, partition, appId.toString(), httpConnectionParams.isSSLShuffleEnabled());
@@ -227,7 +461,7 @@
     // yet_to_be_fetched list and marking the failed tasks.
     InputAttemptIdentifier[] failedInputs = null;
     while (!remaining.isEmpty() && failedInputs == null) {
-      failedInputs = fetchInputs(input);
+      failedInputs = fetchInputs(input, callback);
     }
 
     return new HostFetchResult(new FetchResult(host, port, partition, remaining), failedInputs,
@@ -236,6 +470,11 @@
 
   @VisibleForTesting
   protected HostFetchResult setupLocalDiskFetch() {
+    return doLocalDiskFetch(true);
+  }
+
+  @VisibleForTesting
+  private HostFetchResult doLocalDiskFetch(boolean failMissing) {
 
     Iterator<InputAttemptIdentifier> iterator = remaining.iterator();
     while (iterator.hasNext()) {
@@ -246,6 +485,7 @@
       FetchedInput fetchedInput = null;
       try {
         TezIndexRecord idxRecord;
+        // for missing files, this will throw an exception
         idxRecord = getTezIndexRecord(srcAttemptId);
 
         fetchedInput = new LocalDiskFetchedInput(idxRecord.getStartOffset(),
@@ -283,8 +523,10 @@
     }
 
     InputAttemptIdentifier[] failedFetches = null;
-    if (remaining.size() > 0) {
+    if (failMissing && remaining.size() > 0) {
       failedFetches = remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
+    } else {
+      // nothing needs to be done to requeue remaining entries
     }
     return new HostFetchResult(new FetchResult(host, port, partition, remaining),
         failedFetches, false);
@@ -296,19 +538,24 @@
     TezIndexRecord idxRecord;
     Path indexFile = getShuffleInputFileName(srcAttemptId.getPathComponent(),
         Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING);
+
     TezSpillRecord spillRecord = new TezSpillRecord(indexFile, conf);
     idxRecord = spillRecord.getIndex(partition);
     return idxRecord;
   }
 
+  private static final String getMapOutputFile(String pathComponent) {
+    return Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + pathComponent + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING;
+  }
+
   @VisibleForTesting
-  protected Path getShuffleInputFileName(String pathComponent, String suffix) throws IOException {
-    LocalDirAllocator localDirAllocator = new LocalDirAllocator(TezRuntimeFrameworkConfigs.LOCAL_DIRS);
+  protected Path getShuffleInputFileName(String pathComponent, String suffix)
+      throws IOException {
     suffix = suffix != null ? suffix : "";
 
-    String pathFromLocalDir = Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR + pathComponent +
-        Path.SEPARATOR + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING + suffix;
-
+    String pathFromLocalDir = getMapOutputFile(pathComponent) + suffix;
     return localDirAllocator.getLocalPathToRead(pathFromLocalDir, conf);
   }
 
@@ -350,7 +597,7 @@
     }
   }
 
-  private InputAttemptIdentifier[] fetchInputs(DataInputStream input) {
+  private InputAttemptIdentifier[] fetchInputs(DataInputStream input, CachingCallBack callback) {
     FetchedInput fetchedInput = null;
     InputAttemptIdentifier srcAttemptId = null;
     long decompressedLength = -1;
@@ -392,12 +639,16 @@
         LOG.debug("header: " + srcAttemptId + ", len: " + compressedLength
             + ", decomp len: " + decompressedLength);
       }
-
-      // Get the location for the map output - either in-memory or on-disk
       
       // TODO TEZ-957. handle IOException here when Broadcast has better error checking
-      fetchedInput = inputManager.allocate(decompressedLength, compressedLength, srcAttemptId);
-
+      if (srcAttemptId.isShared() && callback != null) {
+        // force disk if input is being shared
+        fetchedInput = inputManager.allocateType(Type.DISK, decompressedLength,
+            compressedLength, srcAttemptId);
+      } else {
+        fetchedInput = inputManager.allocate(decompressedLength,
+            compressedLength, srcAttemptId);
+      }
       // TODO NEWTEZ No concept of WAIT at the moment.
       // // Check if we can shuffle *now* ...
       // if (fetchedInput.getType() == FetchedInput.WAIT) {
@@ -427,6 +678,14 @@
             fetchedInput);
       }
 
+      // offer the fetched input for caching
+      if (srcAttemptId.isShared() && callback != null) {
+        // this has to be before the fetchSucceeded, because that goes across
+        // threads into the reader thread and can potentially shutdown this thread
+        // while it is still caching.
+        callback.cache(host, srcAttemptId, fetchedInput, compressedLength, decompressedLength);
+      }
+
       // Inform the shuffle scheduler
       long endTime = System.currentTimeMillis();
       fetcherCallback.fetchSucceeded(host, srcAttemptId, fetchedInput,
@@ -434,6 +693,7 @@
 
       // Note successful shuffle
       remaining.remove(srcAttemptId);
+
       // metrics.successFetch();
       return null;
     } catch (IOException ioe) {
@@ -521,12 +781,24 @@
     private Fetcher fetcher;
     private boolean workAssigned = false;
 
-    public FetcherBuilder(FetcherCallback fetcherCallback, HttpConnectionParams params,
-                          FetchedInputAllocator inputManager, ApplicationId appId,
-                          SecretKey shuffleSecret, String srcNameTrimmed, Configuration conf,
-                          boolean localDiskFetchEnabled) {
+    public FetcherBuilder(FetcherCallback fetcherCallback,
+        HttpConnectionParams params, FetchedInputAllocator inputManager,
+        ApplicationId appId, SecretKey shuffleSecret, String srcNameTrimmed,
+        Configuration conf, boolean localDiskFetchEnabled) {
       this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
-          shuffleSecret, srcNameTrimmed, conf, localDiskFetchEnabled);
+          shuffleSecret, srcNameTrimmed, conf, null, null, null, localDiskFetchEnabled,
+          false);
+    }
+
+    public FetcherBuilder(FetcherCallback fetcherCallback,
+        HttpConnectionParams params, FetchedInputAllocator inputManager,
+        ApplicationId appId, SecretKey shuffleSecret, String srcNameTrimmed,
+        Configuration conf, RawLocalFileSystem localFs,
+        LocalDirAllocator localDirAllocator, Path lockPath,
+        boolean localDiskFetchEnabled, boolean sharedFetchEnabled) {
+      this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
+          shuffleSecret, srcNameTrimmed, conf, localFs, localDirAllocator,
+          lockPath, localDiskFetchEnabled, sharedFetchEnabled);
     }
 
     public FetcherBuilder setHttpConnectionParameters(HttpConnectionParams httpParams) {
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java
index d664f88..1311a4f 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java
@@ -210,11 +210,13 @@
     if (replyHash == null) {
       throw new IOException("security validation of TT Map output failed");
     }
-    LOG.debug("url=" + msgToEncode + ";encHash=" + encHash + ";replyHash="
-        + replyHash);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("url=" + msgToEncode + ";encHash=" + encHash + ";replyHash="
+          + replyHash);
+    }
     // verify that replyHash is HMac of encHash
     SecureShuffleUtils.verifyReply(replyHash, encHash, jobTokenSecret);
-    LOG.info("for url=" + msgToEncode +
+    LOG.info("for url=" + url +
       " sent hash and receievd reply " + stopWatch.elapsedTime(TimeUnit.MILLISECONDS) + " ms");
   }
 
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java
index 9d621e8..2ac45d4 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java
@@ -24,6 +24,7 @@
 import java.util.List;
 
 import com.google.protobuf.ByteString;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.compress.CompressionCodec;
@@ -53,6 +54,7 @@
   private final CompressionCodec codec;
   private final boolean ifileReadAhead;
   private final int ifileReadAheadLength;
+  private final boolean useSharedInputs;
 
   public ShuffleInputEventHandlerImpl(InputContext inputContext,
                                       ShuffleManager shuffleManager,
@@ -63,6 +65,9 @@
     this.codec = codec;
     this.ifileReadAhead = ifileReadAhead;
     this.ifileReadAheadLength = ifileReadAheadLength;
+    // this currently relies on a user to enable the flag
+    // expand on idea based on vertex parallelism and num inputs
+    this.useSharedInputs = (inputContext.getTaskAttemptNumber() == 0);
   }
 
   @Override
@@ -109,9 +114,12 @@
         shuffleManager.addCompletedInputWithNoData(srcAttemptIdentifier);
         return;
       }
-    } 
-    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dme.getTargetIndex(),
-        dme.getVersion(), shufflePayload.getPathComponent());
+    }
+
+    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(
+        dme.getTargetIndex(), dme.getVersion(),
+        shufflePayload.getPathComponent(), (useSharedInputs && srcIndex == 0));
+
     if (shufflePayload.hasData()) {
       DataProto dataProto = shufflePayload.getData();
       FetchedInput fetchedInput = inputAllocator.allocate(dataProto.getRawLength(),
@@ -119,8 +127,8 @@
       moveDataToFetchedInput(dataProto, fetchedInput, hostIdentifier);
       shuffleManager.addCompletedInputWithData(srcAttemptIdentifier, fetchedInput);
     } else {
-      shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(),
-              srcAttemptIdentifier, srcIndex);
+      shuffleManager.addKnownInput(shufflePayload.getHost(),
+          shufflePayload.getPort(), srcAttemptIdentifier, srcIndex);
     }
 
   }
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java
index 4e1a06c..ddf98b6 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java
@@ -22,6 +22,7 @@
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.text.DecimalFormat;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -43,7 +44,13 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.tez.common.TezRuntimeFrameworkConfigs;
 import org.apache.tez.common.TezUtilsInternal;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
@@ -68,7 +75,9 @@
 import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
 import org.apache.tez.runtime.library.shuffle.common.Fetcher.FetcherBuilder;
 
+import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
@@ -120,6 +129,7 @@
   private final SecretKey shuffleSecret;
   private final CompressionCodec codec;
   private final boolean localDiskFetchEnabled;
+  private final boolean sharedFetchEnabled;
   
   private final int ifileBufferSize;
   private final boolean ifileReadAhead;
@@ -140,6 +150,12 @@
   private volatile Throwable shuffleError;
   private final HttpConnectionParams httpConnectionParams;
   
+
+  private final LocalDirAllocator localDirAllocator;
+  private final RawLocalFileSystem localFs;
+  private final Path[] localDisks;
+  private final static String localhostName = NetUtils.getHostname();
+
   // TODO More counters - FetchErrors, speed?
   
   public ShuffleManager(InputContext inputContext, Configuration conf, int numInputs,
@@ -163,6 +179,8 @@
     this.inputManager = inputAllocator;
     this.localDiskFetchEnabled = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH,
         TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH_DEFAULT);
+    this.sharedFetchEnabled = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH,
+        TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH_DEFAULT);
     
     this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName());
   
@@ -183,7 +201,7 @@
     ExecutorService fetcherRawExecutor = Executors.newFixedThreadPool(
         numFetchers,
         new ThreadFactoryBuilder().setDaemon(true)
-            .setNameFormat("Fetcher [" + srcNameTrimmed + "] #%d").build());
+            .setNameFormat("Fetcher [" + srcNameTrimmed + "] #%d " + localhostName).build());
     this.fetcherExecutor = MoreExecutors.listeningDecorator(fetcherRawExecutor);
     
     ExecutorService schedulerRawExecutor = Executors.newFixedThreadPool(1, new ThreadFactoryBuilder()
@@ -199,10 +217,23 @@
             .getServiceConsumerMetaData(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID));
     httpConnectionParams =
         ShuffleUtils.constructHttpShuffleConnectionParams(conf);
+
+    this.localFs = (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw();
+
+    this.localDirAllocator = new LocalDirAllocator(
+        TezRuntimeFrameworkConfigs.LOCAL_DIRS);
+
+    this.localDisks = Iterables.toArray(
+        localDirAllocator.getAllLocalPathsToRead(".", conf), Path.class);
+
+    Arrays.sort(this.localDisks);
+
     LOG.info(this.getClass().getSimpleName() + " : numInputs=" + numInputs + ", compressionCodec="
         + (codec == null ? "NoCompressionCodec" : codec.getClass().getName()) + ", numFetchers="
         + numFetchers + ", ifileBufferSize=" + ifileBufferSize + ", ifileReadAheadEnabled="
         + ifileReadAhead + ", ifileReadAheadLength=" + ifileReadAheadLength +", "
+        + "localDiskFetchEnabled=" + localDiskFetchEnabled + ", "
+        + "sharedFetchEnabled=" + sharedFetchEnabled + ", "
         + httpConnectionParams.toString());
   }
 
@@ -301,9 +332,20 @@
   }
   
   private Fetcher constructFetcherForHost(InputHost inputHost, Configuration conf) {
+
+    Path lockDisk = null;
+
+    if (sharedFetchEnabled) {
+      // pick a single lock disk from the edge name's hashcode + host hashcode
+      final int h = Math.abs(Objects.hashCode(this.srcNameTrimmed, inputHost.getHost()));
+      lockDisk = new Path(this.localDisks[h % this.localDisks.length], "locks");
+    }
+
     FetcherBuilder fetcherBuilder = new FetcherBuilder(ShuffleManager.this,
       httpConnectionParams, inputManager, inputContext.getApplicationId(),
-      shuffleSecret, srcNameTrimmed, conf, localDiskFetchEnabled);
+        shuffleSecret, srcNameTrimmed, conf, localFs, localDirAllocator,
+        lockDisk, localDiskFetchEnabled, sharedFetchEnabled);
+
     if (codec != null) {
       fetcherBuilder.setCompressionParameters(codec);
     }
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java
index d365aa4..20ee665 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java
@@ -33,6 +33,7 @@
 import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
 import org.apache.tez.runtime.library.shuffle.common.DiskFetchedInput;
 import org.apache.tez.runtime.library.shuffle.common.FetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
 import org.apache.tez.runtime.library.shuffle.common.FetchedInputAllocator;
 import org.apache.tez.runtime.library.shuffle.common.FetchedInputCallback;
 import org.apache.tez.runtime.library.shuffle.common.MemoryFetchedInput;
@@ -140,10 +141,26 @@
   }
 
   @Override
+  public synchronized FetchedInput allocateType(Type type, long actualSize,
+      long compressedSize, InputAttemptIdentifier inputAttemptIdentifier)
+      throws IOException {
+
+    switch (type) {
+    case DISK:
+      return new DiskFetchedInput(actualSize, compressedSize,
+          inputAttemptIdentifier, this, conf, localDirAllocator,
+          fileNameAllocator);
+    default:
+      return allocate(actualSize, compressedSize, inputAttemptIdentifier);
+    }
+  }
+
+  @Override
   public synchronized void fetchComplete(FetchedInput fetchedInput) {
     switch (fetchedInput.getType()) {
     // Not tracking anything here.
     case DISK:
+    case DISK_DIRECT:
     case MEMORY:
       break;
     default:
diff --git a/tez-runtime-library/src/main/proto/ShufflePayloads.proto b/tez-runtime-library/src/main/proto/ShufflePayloads.proto
index 2d658ba..9c711bb 100644
--- a/tez-runtime-library/src/main/proto/ShufflePayloads.proto
+++ b/tez-runtime-library/src/main/proto/ShufflePayloads.proto
@@ -46,7 +46,6 @@
 message ShuffleEdgeManagerConfigPayloadProto {
   optional int32 num_source_task_outputs = 1;
   optional int32 num_destination_tasks = 2;
-  optional int32 num_source_tasks = 3;
-  optional int32 base_partition_range = 4;
-  optional int32 remainder_range_for_last_shuffler = 5;
+  optional int32 base_partition_range = 3;
+  optional int32 remainder_range_for_last_shuffler = 4;
 }
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java
index 9ac8210..6d065fc 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManager.java
@@ -186,12 +186,12 @@
 
               @Override
               public int getSourceVertexNumTasks() {
-                return 0;
+                return 2;
               }
 
               @Override
               public int getDestinationVertexNumTasks() {
-                return 0;
+                return 2;
               }
             };
             EdgeManagerPlugin edgeManager = ReflectionUtils
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/comparator/TestProxyComparator.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/comparator/TestProxyComparator.java
new file mode 100644
index 0000000..441feff
--- /dev/null
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/comparator/TestProxyComparator.java
@@ -0,0 +1,81 @@
+/**
+ * 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.tez.runtime.library.common.comparator;
+
+import static org.junit.Assert.assertTrue;
+
+import java.nio.charset.Charset;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.BytesWritable;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestProxyComparator {
+  private static final Log LOG = LogFactory.getLog(TestProxyComparator.class);
+
+  final static String[] keys = {
+    "",
+    "A", "B",
+    "AA", "BB", "BA", "CB",
+    "AAA", "BBBB", "CCCCC",
+    /* utf-8 comparisons */
+    "\u00E6AAAA", "\u00F7", "A\u00F7", "\u00F7AAAAAAAAA",
+    "\u00F7\u00F7", "\u00F7\u00F7\u00E6\u00E6A",
+    "\u00F7\u00F7\u00E6\u00E6A"
+  };
+
+  private static final void set(BytesWritable bw, String s) {
+    byte[] b = s.getBytes(Charset.forName("utf-8"));
+    bw.set(b, 0, b.length);
+  }
+
+  @Before
+  public void setUp() throws Exception {
+  }
+
+  @After
+  public void cleanup() throws Exception {
+  }
+
+  @Test
+  public void testProxyComparator() {
+    final ProxyComparator<BytesWritable> comparator = new TezBytesComparator();
+    BytesWritable lhs = new BytesWritable();
+    BytesWritable rhs = new BytesWritable();
+    for (String l : keys) {
+      for (String r : keys) {
+        set(lhs, l);
+        set(rhs, r);
+        final int lproxy = comparator.getProxy(lhs);
+        final int rproxy = comparator.getProxy(rhs);
+        if (lproxy < rproxy) {
+          assertTrue(String.format("(%s) %d < (%s) %d", l, lproxy, r, rproxy),
+              comparator.compare(lhs, rhs) < 0);
+        }
+        if (lproxy > rproxy) {
+          assertTrue(String.format("(%s) %d > (%s) %d", l, lproxy, r, rproxy),
+              comparator.compare(lhs, rhs) > 0);
+        }
+      }
+    }
+  }
+}
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
index 977b767..3824607 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
@@ -31,9 +31,6 @@
 import org.apache.tez.dag.api.client.Progress;
 import org.apache.tez.dag.api.client.StatusGetOpts;
 import org.apache.tez.dag.api.client.VertexStatus;
-import org.apache.tez.mapreduce.examples.terasort.TeraGen;
-import org.apache.tez.mapreduce.examples.terasort.TeraSort;
-import org.apache.tez.mapreduce.examples.terasort.TeraValidate;
 
 /**
  * A description of an example program based on its class and a
@@ -65,12 +62,6 @@
       pgd.addClass("join", Join.class,
           "A job that effects a join over sorted, equally partitioned"
           + " datasets");
-      pgd.addClass("teragen", TeraGen.class,
-          "Generate data for the terasort");
-      pgd.addClass("terasort", TeraSort.class,
-          "Run the terasort");
-      pgd.addClass("teravalidate", TeraValidate.class,
-          "Checking results of terasort");
       pgd.addClass("groupbyorderbymrrtest", GroupByOrderByMRRTest.class,
           "A map-reduce-reduce program that does groupby-order by. Takes input"
           + " containing employee_name department name per line of input"
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java
index 939bea0..393faea 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java
@@ -18,34 +18,54 @@
 
 package org.apache.tez.mapreduce.examples;
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 import java.util.StringTokenizer;
+import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.client.MRTezClient;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.DataSourceDescriptor;
+import org.apache.tez.dag.api.Edge;
+import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.Vertex;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.mapreduce.hadoop.MRHelpers;
+import org.apache.tez.mapreduce.hadoop.MRInputHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
+import org.apache.tez.mapreduce.output.MROutputLegacy;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
+import org.apache.tez.runtime.library.conf.OrderedPartitionedKVEdgeConfig;
+import org.apache.tez.runtime.library.partitioner.HashPartitioner;
 
 /**
  * Simple example that does a GROUP BY ORDER BY in an MRR job
@@ -94,7 +114,7 @@
     public void map(Object key, Text value, Context context
         ) throws IOException, InterruptedException {
       StringTokenizer itr = new StringTokenizer(value.toString());
-      String empName = "";
+      String empName;
       String deptName = "";
       if (itr.hasMoreTokens()) {
         empName = itr.nextToken();
@@ -149,25 +169,141 @@
     }
   }
 
+  private static DAG createDAG(Configuration conf, Map<String, LocalResource> commonLocalResources,
+      Path stagingDir, String inputPath, String outputPath, boolean useMRSettings)
+      throws Exception {
+
+    Configuration mapStageConf = new JobConf(conf);
+    mapStageConf.set(MRJobConfig.MAP_CLASS_ATTR,
+        MyMapper.class.getName());
+
+    MRHelpers.translateMRConfToTez(mapStageConf);
+
+    Configuration iReduceStageConf = new JobConf(conf);
+    // TODO replace with auto-reduce parallelism
+    iReduceStageConf.setInt(MRJobConfig.NUM_REDUCES, 2);
+    iReduceStageConf.set(MRJobConfig.REDUCE_CLASS_ATTR,
+        MyGroupByReducer.class.getName());
+    iReduceStageConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, Text.class.getName());
+    iReduceStageConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS,
+        IntWritable.class.getName());
+    iReduceStageConf.setBoolean("mapred.mapper.new-api", true);
+    MRHelpers.translateMRConfToTez(iReduceStageConf);
+
+    Configuration finalReduceConf = new JobConf(conf);
+    finalReduceConf.setInt(MRJobConfig.NUM_REDUCES, 1);
+    finalReduceConf.set(MRJobConfig.REDUCE_CLASS_ATTR,
+        MyOrderByNoOpReducer.class.getName());
+    finalReduceConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, IntWritable.class.getName());
+    finalReduceConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, Text.class.getName());
+    MRHelpers.translateMRConfToTez(finalReduceConf);
+
+    MRHelpers.configureMRApiUsage(mapStageConf);
+    MRHelpers.configureMRApiUsage(iReduceStageConf);
+    MRHelpers.configureMRApiUsage(finalReduceConf);
+
+    List<Vertex> vertices = new ArrayList<Vertex>();
+
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream(4096);
+    mapStageConf.writeXml(outputStream);
+    String mapStageHistoryText = new String(outputStream.toByteArray(), "UTF-8");
+    mapStageConf.set(MRJobConfig.INPUT_FORMAT_CLASS_ATTR,
+        TextInputFormat.class.getName());
+    mapStageConf.set(FileInputFormat.INPUT_DIR, inputPath);
+    mapStageConf.setBoolean("mapred.mapper.new-api", true);
+    DataSourceDescriptor dsd = MRInputHelpers.configureMRInputWithLegacySplitGeneration(
+        mapStageConf, stagingDir, true);
+
+    Vertex mapVertex;
+    ProcessorDescriptor mapProcessorDescriptor =
+        ProcessorDescriptor.create(MapProcessor.class.getName())
+            .setUserPayload(
+                TezUtils.createUserPayloadFromConf(mapStageConf))
+            .setHistoryText(mapStageHistoryText);
+    if (!useMRSettings) {
+      mapVertex = Vertex.create("initialmap", mapProcessorDescriptor);
+    } else {
+      mapVertex = Vertex.create("initialmap", mapProcessorDescriptor, -1,
+          MRHelpers.getResourceForMRMapper(mapStageConf));
+      mapVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRMapper(mapStageConf));
+    }
+    mapVertex.addTaskLocalFiles(commonLocalResources)
+        .addDataSource("MRInput", dsd);
+    vertices.add(mapVertex);
+
+    ByteArrayOutputStream iROutputStream = new ByteArrayOutputStream(4096);
+    iReduceStageConf.writeXml(iROutputStream);
+    String iReduceStageHistoryText = new String(iROutputStream.toByteArray(), "UTF-8");
+
+    ProcessorDescriptor iReduceProcessorDescriptor = ProcessorDescriptor.create(
+        ReduceProcessor.class.getName())
+        .setUserPayload(TezUtils.createUserPayloadFromConf(iReduceStageConf))
+        .setHistoryText(iReduceStageHistoryText);
+
+    Vertex intermediateVertex;
+    if (!useMRSettings) {
+      intermediateVertex = Vertex.create("ireduce1", iReduceProcessorDescriptor, 1);
+    } else {
+      intermediateVertex = Vertex.create("ireduce1", iReduceProcessorDescriptor,
+          1, MRHelpers.getResourceForMRReducer(iReduceStageConf));
+      intermediateVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(iReduceStageConf));
+    }
+    intermediateVertex.addTaskLocalFiles(commonLocalResources);
+    vertices.add(intermediateVertex);
+
+    ByteArrayOutputStream finalReduceOutputStream = new ByteArrayOutputStream(4096);
+    finalReduceConf.writeXml(finalReduceOutputStream);
+    String finalReduceStageHistoryText = new String(finalReduceOutputStream.toByteArray(), "UTF-8");
+    UserPayload finalReducePayload = TezUtils.createUserPayloadFromConf(finalReduceConf);
+    Vertex finalReduceVertex;
+
+    ProcessorDescriptor finalReduceProcessorDescriptor =
+        ProcessorDescriptor.create(
+            ReduceProcessor.class.getName())
+            .setUserPayload(finalReducePayload)
+            .setHistoryText(finalReduceStageHistoryText);
+    if (!useMRSettings) {
+      finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1);
+    } else {
+      finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1,
+          MRHelpers.getResourceForMRReducer(finalReduceConf));
+      finalReduceVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(finalReduceConf));
+    }
+    finalReduceVertex.addTaskLocalFiles(commonLocalResources);
+    finalReduceVertex.addDataSink("MROutput",
+        MROutputLegacy.createConfigBuilder(finalReduceConf, TextOutputFormat.class, outputPath)
+            .build());
+    vertices.add(finalReduceVertex);
+
+    DAG dag = DAG.create("groupbyorderbymrrtest");
+    for (Vertex v : vertices) {
+      dag.addVertex(v);
+    }
+
+    OrderedPartitionedKVEdgeConfig edgeConf1 = OrderedPartitionedKVEdgeConfig
+        .newBuilder(Text.class.getName(), IntWritable.class.getName(),
+            HashPartitioner.class.getName()).setFromConfiguration(conf)
+        .configureInput().useLegacyInput().done().build();
+    dag.addEdge(
+        Edge.create(dag.getVertex("initialmap"), dag.getVertex("ireduce1"),
+            edgeConf1.createDefaultEdgeProperty()));
+
+    OrderedPartitionedKVEdgeConfig edgeConf2 = OrderedPartitionedKVEdgeConfig
+        .newBuilder(IntWritable.class.getName(), Text.class.getName(),
+            HashPartitioner.class.getName()).setFromConfiguration(conf)
+        .configureInput().useLegacyInput().done().build();
+    dag.addEdge(
+        Edge.create(dag.getVertex("ireduce1"), dag.getVertex("finalreduce"),
+            edgeConf2.createDefaultEdgeProperty()));
+
+    return dag;
+  }
+
+
   @Override
   public int run(String[] args) throws Exception {
     Configuration conf = getConf();
 
-    // Configure intermediate reduces
-    conf.setInt(MRJobConfig.MRR_INTERMEDIATE_STAGES, 1);
-
-    // Set reducer class for intermediate reduce
-    conf.setClass(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
-        "mapreduce.job.reduce.class"), MyGroupByReducer.class, Reducer.class);
-    // Set reducer output key class
-    conf.setClass(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
-        "mapreduce.map.output.key.class"), IntWritable.class, Object.class);
-    // Set reducer output value class
-    conf.setClass(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
-        "mapreduce.map.output.value.class"), Text.class, Object.class);
-    conf.setInt(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
-        "mapreduce.job.reduces"), 2);
-
     String[] otherArgs = new GenericOptionsParser(conf, args).
         getRemainingArgs();
     if (otherArgs.length != 2) {
@@ -176,66 +312,55 @@
       return 2;
     }
 
-    @SuppressWarnings("deprecation")
-    Job job = new Job(conf, "groupbyorderbymrrtest");
+    String inputPath = otherArgs[0];
+    String outputPath = otherArgs[1];
 
-    job.setJarByClass(GroupByOrderByMRRTest.class);
+    UserGroupInformation.setConfiguration(conf);
 
-    // Configure map
-    job.setMapperClass(MyMapper.class);
-    job.setMapOutputKeyClass(Text.class);
-    job.setMapOutputValueClass(IntWritable.class);
+    TezConfiguration tezConf = new TezConfiguration(conf);
+    FileSystem fs = FileSystem.get(conf);
 
-    // Configure reduce
-    job.setReducerClass(MyOrderByNoOpReducer.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(IntWritable.class);
-    job.setNumReduceTasks(1);
-
-    FileInputFormat.addInputPath(job, new Path(otherArgs[0]));
-    FileOutputFormat.setOutputPath(job, new Path(otherArgs[1]));
-
-    job.submit();
-    JobID jobId = job.getJobID();
-    ApplicationId appId = TypeConverter.toYarn(jobId).getAppId();
-
-    DAGClient dagClient = MRTezClient.getDAGClient(appId, new TezConfiguration(conf), null);
-    DAGStatus dagStatus;
-    String[] vNames = { "initialmap" , "ireduce1" , "finalreduce" };
-    while (true) {
-      dagStatus = dagClient.getDAGStatus(null);
-      if(dagStatus.getState() == DAGStatus.State.RUNNING ||
-         dagStatus.getState() == DAGStatus.State.SUCCEEDED ||
-         dagStatus.getState() == DAGStatus.State.FAILED ||
-         dagStatus.getState() == DAGStatus.State.KILLED ||
-         dagStatus.getState() == DAGStatus.State.ERROR) {
-        break;
-      }
-      try {
-        Thread.sleep(500);
-      } catch (InterruptedException e) {
-        // continue;
-      }
+    if (fs.exists(new Path(outputPath))) {
+      throw new FileAlreadyExistsException("Output directory "
+          + outputPath + " already exists");
     }
 
-    while (dagStatus.getState() == DAGStatus.State.RUNNING) {
-      try {
-        ExampleDriver.printDAGStatus(dagClient, vNames);
-        try {
-          Thread.sleep(1000);
-        } catch (InterruptedException e) {
-          // continue;
-        }
-        dagStatus = dagClient.getDAGStatus(null);
-      } catch (TezException e) {
-        LOG.fatal("Failed to get application progress. Exiting");
+    Map<String, LocalResource> localResources =
+        new TreeMap<String, LocalResource>();
+
+    String stagingDirStr =  conf.get(TezConfiguration.TEZ_AM_STAGING_DIR,
+        TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT) + Path.SEPARATOR +
+        Long.toString(System.currentTimeMillis());
+    Path stagingDir = new Path(stagingDirStr);
+    FileSystem pathFs = stagingDir.getFileSystem(tezConf);
+    pathFs.mkdirs(new Path(stagingDirStr));
+
+    tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirStr);
+    stagingDir = pathFs.makeQualified(new Path(stagingDirStr));
+
+    TezClient tezClient = TezClient.create("groupbyorderbymrrtest", tezConf);
+    tezClient.start();
+
+    LOG.info("Submitting groupbyorderbymrrtest DAG as a new Tez Application");
+
+    try {
+      DAG dag = createDAG(conf, localResources, stagingDir, inputPath, outputPath, true);
+
+      tezClient.waitTillReady();
+
+      DAGClient dagClient = tezClient.submitDAG(dag);
+
+      DAGStatus dagStatus = dagClient.waitForCompletionWithStatusUpdates(null);
+      if (dagStatus.getState() != DAGStatus.State.SUCCEEDED) {
+        LOG.error("groupbyorderbymrrtest failed, state=" + dagStatus.getState()
+            + ", diagnostics=" + dagStatus.getDiagnostics());
         return -1;
       }
+      LOG.info("Application completed. " + "FinalState=" + dagStatus.getState());
+      return 0;
+    } finally {
+      tezClient.stop();
     }
-
-    ExampleDriver.printDAGStatus(dagClient, vNames);
-    LOG.info("Application completed. " + "FinalState=" + dagStatus.getState());
-    return dagStatus.getState() == DAGStatus.State.SUCCEEDED ? 0 : 1;
   }
 
   public static void main(String[] args) throws Exception {
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
index 3f9d222..bbb4d64 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
@@ -524,22 +524,32 @@
     UserPayload mapUserPayload = TezUtils.createUserPayloadFromConf(mapStageConf);
     int numTasks = generateSplitsInAM ? -1 : numMapper;
 
+    Map<String, String> mapEnv = Maps.newHashMap();
+    MRHelpers.updateEnvBasedOnMRTaskEnv(mapStageConf, mapEnv, true);
+    Map<String, String> reduceEnv = Maps.newHashMap();
+    MRHelpers.updateEnvBasedOnMRTaskEnv(mapStageConf, reduceEnv, false);
+
     Vertex mapVertex = Vertex.create("map", ProcessorDescriptor.create(
-        MapProcessor.class.getName()).setUserPayload(mapUserPayload), numTasks)
-        .addTaskLocalFiles(commonLocalResources);
-    mapVertex.addDataSource("MRInput", dataSource);
+        MapProcessor.class.getName()).setUserPayload(mapUserPayload), numTasks,
+        MRHelpers.getResourceForMRMapper(mapStageConf));
+    mapVertex.addTaskLocalFiles(commonLocalResources)
+        .addDataSource("MRInput", dataSource)
+        .setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRMapper(mapStageConf)).setTaskEnvironment(mapEnv);
     vertices.add(mapVertex);
 
     if (iReduceStagesCount > 0
-        && numIReducer > 0) {
+        && numIReducer > 0) {      
       for (int i = 0; i < iReduceStagesCount; ++i) {
         Configuration iconf =
             intermediateReduceStageConfs[i];
         UserPayload iReduceUserPayload = TezUtils.createUserPayloadFromConf(iconf);
         Vertex ivertex = Vertex.create("ireduce" + (i + 1),
             ProcessorDescriptor.create(ReduceProcessor.class.getName()).
-                setUserPayload(iReduceUserPayload), numIReducer);
-        ivertex.addTaskLocalFiles(commonLocalResources);
+                setUserPayload(iReduceUserPayload), numIReducer,
+            MRHelpers.getResourceForMRReducer(intermediateReduceStageConfs[i]));
+        ivertex.addTaskLocalFiles(commonLocalResources)
+            .setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(
+                intermediateReduceStageConfs[i])).setTaskEnvironment(reduceEnv);
         vertices.add(ivertex);
       }
     }
@@ -548,10 +558,13 @@
     if (numReducer > 0) {
       UserPayload reducePayload = TezUtils.createUserPayloadFromConf(finalReduceConf);
       finalReduceVertex = Vertex.create("reduce", ProcessorDescriptor.create(
-          ReduceProcessor.class.getName()).setUserPayload(reducePayload), numReducer);
-      finalReduceVertex.addTaskLocalFiles(commonLocalResources);
-      finalReduceVertex.addDataSink("MROutput", MROutputLegacy.createConfigBuilder(finalReduceConf,
-          NullOutputFormat.class).build());
+          ReduceProcessor.class.getName()).setUserPayload(reducePayload), numReducer,
+          MRHelpers.getResourceForMRReducer(finalReduceConf));
+      finalReduceVertex.addTaskLocalFiles(commonLocalResources)
+          .addDataSink("MROutput", MROutputLegacy.createConfigBuilder(
+              finalReduceConf, NullOutputFormat.class).build())
+          .setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(finalReduceConf))
+          .setTaskEnvironment(reduceEnv);
       vertices.add(finalReduceVertex);
     } else {
       // Map only job
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java
index 2c5db10..e519085 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java
@@ -79,6 +79,7 @@
 import org.apache.tez.runtime.library.partitioner.HashPartitioner;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
 
 /**
  * An MRR job built on top of word count to return words sorted by
@@ -149,7 +150,9 @@
   public DAG createDAG(FileSystem fs, Configuration conf,
       Map<String, LocalResource> commonLocalResources, Path stagingDir,
       int dagIndex, String inputPath, String outputPath,
-      boolean generateSplitsInClient) throws Exception {
+      boolean generateSplitsInClient,
+      boolean useMRSettings,
+      int intermediateNumReduceTasks) throws Exception {
 
     Configuration mapStageConf = new JobConf(conf);
     mapStageConf.set(MRJobConfig.MAP_CLASS_ATTR,
@@ -196,32 +199,70 @@
       dsd = MRInputLegacy.createConfigBuilder(mapStageConf, TextInputFormat.class, inputPath).build();
     }
 
-    Vertex mapVertex = Vertex.create("initialmap", ProcessorDescriptor.create(
-        MapProcessor.class.getName()).setUserPayload(
-        TezUtils.createUserPayloadFromConf(mapStageConf))
-        .setHistoryText(mapStageHistoryText)).addTaskLocalFiles(commonLocalResources);
-    mapVertex.addDataSource("MRInput", dsd);
+    Map<String, String> mapEnv = Maps.newHashMap();
+    MRHelpers.updateEnvBasedOnMRTaskEnv(mapStageConf, mapEnv, true);
+    Map<String, String> reduceEnv = Maps.newHashMap();
+    MRHelpers.updateEnvBasedOnMRTaskEnv(mapStageConf, reduceEnv, false);
+
+    Vertex mapVertex;
+    ProcessorDescriptor mapProcessorDescriptor =
+        ProcessorDescriptor.create(MapProcessor.class.getName())
+            .setUserPayload(
+                TezUtils.createUserPayloadFromConf(mapStageConf))
+            .setHistoryText(mapStageHistoryText);
+    if (!useMRSettings) {
+      mapVertex = Vertex.create("initialmap", mapProcessorDescriptor);
+    } else {
+      mapVertex = Vertex.create("initialmap", mapProcessorDescriptor, -1,
+          MRHelpers.getResourceForMRMapper(mapStageConf));
+      mapVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRMapper(mapStageConf));
+      mapVertex.setTaskEnvironment(mapEnv);
+    }
+    mapVertex.addTaskLocalFiles(commonLocalResources)
+        .addDataSource("MRInput", dsd);
     vertices.add(mapVertex);
 
     ByteArrayOutputStream iROutputStream = new ByteArrayOutputStream(4096);
     iReduceStageConf.writeXml(iROutputStream);
     String iReduceStageHistoryText = new String(iROutputStream.toByteArray(), "UTF-8");
-    Vertex ivertex = Vertex.create("intermediate_reducer", ProcessorDescriptor.create(
+
+    ProcessorDescriptor iReduceProcessorDescriptor = ProcessorDescriptor.create(
         ReduceProcessor.class.getName())
         .setUserPayload(TezUtils.createUserPayloadFromConf(iReduceStageConf))
-        .setHistoryText(iReduceStageHistoryText), 2);
-    ivertex.addTaskLocalFiles(commonLocalResources);
-    vertices.add(ivertex);
+        .setHistoryText(iReduceStageHistoryText);
+
+    Vertex intermediateVertex;
+    if (!useMRSettings) {
+      intermediateVertex = Vertex.create("intermediate_reducer", iReduceProcessorDescriptor,
+          intermediateNumReduceTasks);
+    } else {
+      intermediateVertex = Vertex.create("intermediate_reducer", iReduceProcessorDescriptor,
+          intermediateNumReduceTasks, MRHelpers.getResourceForMRReducer(iReduceStageConf));
+      intermediateVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(iReduceStageConf));
+      intermediateVertex.setTaskEnvironment(reduceEnv);
+    }
+    intermediateVertex.addTaskLocalFiles(commonLocalResources);
+    vertices.add(intermediateVertex);
 
     ByteArrayOutputStream finalReduceOutputStream = new ByteArrayOutputStream(4096);
     finalReduceConf.writeXml(finalReduceOutputStream);
     String finalReduceStageHistoryText = new String(finalReduceOutputStream.toByteArray(), "UTF-8");
     UserPayload finalReducePayload = TezUtils.createUserPayloadFromConf(finalReduceConf);
-    Vertex finalReduceVertex = Vertex.create("finalreduce",
+    Vertex finalReduceVertex;
+
+    ProcessorDescriptor finalReduceProcessorDescriptor =
         ProcessorDescriptor.create(
             ReduceProcessor.class.getName())
             .setUserPayload(finalReducePayload)
-            .setHistoryText(finalReduceStageHistoryText), 1);
+            .setHistoryText(finalReduceStageHistoryText);
+    if (!useMRSettings) {
+      finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1);
+    } else {
+      finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1,
+          MRHelpers.getResourceForMRReducer(finalReduceConf));
+      finalReduceVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(finalReduceConf));
+      finalReduceVertex.setTaskEnvironment(reduceEnv);
+    }
     finalReduceVertex.addTaskLocalFiles(commonLocalResources);
     finalReduceVertex.addDataSink("MROutput",
         MROutputLegacy.createConfigBuilder(finalReduceConf, TextOutputFormat.class, outputPath)
@@ -283,6 +324,9 @@
         * 1000;
 
     boolean retainStagingDir = conf.getBoolean("RETAIN_STAGING_DIR", false);
+    boolean useMRSettings = conf.getBoolean("USE_MR_CONFIGS", true);
+    // TODO needs to use auto reduce parallelism
+    int intermediateNumReduceTasks = conf.getInt("IREDUCE_NUM_TASKS", 2);
 
     if (((otherArgs.length%2) != 0)
         || (!useTezSession && otherArgs.length != 2)) {
@@ -371,7 +415,7 @@
         
         DAG dag = instance.createDAG(fs, conf, localResources,
             stagingDir, dagIndex, inputPath, outputPath,
-            generateSplitsInClient);
+            generateSplitsInClient, useMRSettings, intermediateNumReduceTasks);
 
         boolean doPreWarm = dagIndex == 1 && useTezSession
             && conf.getBoolean("PRE_WARM_SESSION", true);
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/GenSort.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/GenSort.java
deleted file mode 100644
index c8517e5..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/GenSort.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/**
- * 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.tez.mapreduce.examples.terasort;
-
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.math.BigInteger;
-import java.util.zip.Checksum;
-
-import org.apache.hadoop.util.PureJavaCrc32;
-
-/** 
- * A single process data generator for the terasort data. Based on gensort.c 
- * version 1.1 (3 Mar 2009) from Chris Nyberg <chris.nyberg@ordinal.com>.
- */
-public class GenSort {
-
-  /**
-   * Generate a "binary" record suitable for all sort benchmarks *except* 
-   * PennySort.
-   */
-  static void generateRecord(byte[] recBuf, Unsigned16 rand, 
-                                     Unsigned16 recordNumber) {
-    /* generate the 10-byte key using the high 10 bytes of the 128-bit
-     * random number
-     */
-    for(int i=0; i < 10; ++i) {
-      recBuf[i] = rand.getByte(i);
-    }
-
-    /* add 2 bytes of "break" */
-    recBuf[10] = 0x00;
-    recBuf[11] = 0x11;
-
-    /* convert the 128-bit record number to 32 bits of ascii hexadecimal
-     * as the next 32 bytes of the record.
-     */
-    for (int i = 0; i < 32; i++) {
-      recBuf[12 + i] = (byte) recordNumber.getHexDigit(i);
-    }
-
-    /* add 4 bytes of "break" data */
-    recBuf[44] = (byte) 0x88;
-    recBuf[45] = (byte) 0x99;
-    recBuf[46] = (byte) 0xAA;
-    recBuf[47] = (byte) 0xBB;
-
-    /* add 48 bytes of filler based on low 48 bits of random number */
-    for(int i=0; i < 12; ++i) {
-      recBuf[48+i*4] = recBuf[49+i*4] = recBuf[50+i*4] = recBuf[51+i*4] =
-        (byte) rand.getHexDigit(20 + i);
-    }
-
-    /* add 4 bytes of "break" data */
-    recBuf[96] = (byte) 0xCC;
-    recBuf[97] = (byte) 0xDD;
-    recBuf[98] = (byte) 0xEE;
-    recBuf[99] = (byte) 0xFF;
-  }
-
-
-  private static BigInteger makeBigInteger(long x) {
-    byte[] data = new byte[8];
-    for(int i=0; i < 8; ++i) {
-      data[i] = (byte) (x >>> (56 - 8*i));
-    }
-    return new BigInteger(1, data);
-  }
-
-  private static final BigInteger NINETY_FIVE = new BigInteger("95");
-
-  /**
-   * Generate an ascii record suitable for all sort benchmarks including 
-   * PennySort.
-   */
-  static void generateAsciiRecord(byte[] recBuf, Unsigned16 rand, 
-                                  Unsigned16 recordNumber) {
-
-    /* generate the 10-byte ascii key using mostly the high 64 bits.
-     */
-    long temp = rand.getHigh8();
-    if (temp < 0) {
-      // use biginteger to avoid the negative sign problem
-      BigInteger bigTemp = makeBigInteger(temp);
-      recBuf[0] = (byte) (' ' + (bigTemp.mod(NINETY_FIVE).longValue()));
-      temp = bigTemp.divide(NINETY_FIVE).longValue();
-    } else {
-      recBuf[0] = (byte) (' ' + (temp % 95));
-      temp /= 95;      
-    }
-    for(int i=1; i < 8; ++i) {
-      recBuf[i] = (byte) (' ' + (temp % 95));
-      temp /= 95;      
-    }
-    temp = rand.getLow8();
-    if (temp < 0) {
-      BigInteger bigTemp = makeBigInteger(temp);
-      recBuf[8] = (byte) (' ' + (bigTemp.mod(NINETY_FIVE).longValue()));
-      temp = bigTemp.divide(NINETY_FIVE).longValue();      
-    } else {
-      recBuf[8] = (byte) (' ' + (temp % 95));
-      temp /= 95;
-    }
-    recBuf[9] = (byte)(' ' + (temp % 95));
-
-    /* add 2 bytes of "break" */
-    recBuf[10] = ' ';
-    recBuf[11] = ' ';
-
-    /* convert the 128-bit record number to 32 bits of ascii hexadecimal
-     * as the next 32 bytes of the record.
-     */
-    for (int i = 0; i < 32; i++) {
-      recBuf[12 + i] = (byte) recordNumber.getHexDigit(i);
-    }
-
-    /* add 2 bytes of "break" data */
-    recBuf[44] = ' ';
-    recBuf[45] = ' ';
-
-    /* add 52 bytes of filler based on low 48 bits of random number */
-    for(int i=0; i < 13; ++i) {
-      recBuf[46+i*4] = recBuf[47+i*4] = recBuf[48+i*4] = recBuf[49+i*4] =
-        (byte) rand.getHexDigit(19 + i);
-    }
-
-    /* add 2 bytes of "break" data */
-    recBuf[98] = '\r';	/* nice for Windows */
-    recBuf[99] = '\n';
-}
-
-
-  private static void usage() {
-    PrintStream out = System.out;
-    out.println("usage: gensort [-a] [-c] [-bSTARTING_REC_NUM] NUM_RECS FILE_NAME");
-    out.println("-a        Generate ascii records required for PennySort or JouleSort.");
-    out.println("          These records are also an alternative input for the other");
-    out.println("          sort benchmarks.  Without this flag, binary records will be");
-    out.println("          generated that contain the highest density of randomness in");
-    out.println("          the 10-byte key.");
-    out.println( "-c        Calculate the sum of the crc32 checksums of each of the");
-    out.println("          generated records and send it to standard error.");
-    out.println("-bN       Set the beginning record generated to N. By default the");
-    out.println("          first record generated is record 0.");
-    out.println("NUM_RECS  The number of sequential records to generate.");
-    out.println("FILE_NAME The name of the file to write the records to.\n");
-    out.println("Example 1 - to generate 1000000 ascii records starting at record 0 to");
-    out.println("the file named \"pennyinput\":");
-    out.println("    gensort -a 1000000 pennyinput\n");
-    out.println("Example 2 - to generate 1000 binary records beginning with record 2000");
-    out.println("to the file named \"partition2\":");
-    out.println("    gensort -b2000 1000 partition2");
-    System.exit(1);
-  }
-
-
-  public static void outputRecords(OutputStream out,
-                                   boolean useAscii,
-                                   Unsigned16 firstRecordNumber,
-                                   Unsigned16 recordsToGenerate,
-                                   Unsigned16 checksum
-                                   ) throws IOException {
-    byte[] row = new byte[100];
-    Unsigned16 recordNumber = new Unsigned16(firstRecordNumber);
-    Unsigned16 lastRecordNumber = new Unsigned16(firstRecordNumber);
-    Checksum crc = new PureJavaCrc32();
-    Unsigned16 tmp = new Unsigned16();
-    lastRecordNumber.add(recordsToGenerate);
-    Unsigned16 ONE = new Unsigned16(1);
-    Unsigned16 rand = Random16.skipAhead(firstRecordNumber);
-    while (!recordNumber.equals(lastRecordNumber)) {
-      Random16.nextRand(rand);
-      if (useAscii) {
-        generateAsciiRecord(row, rand, recordNumber);
-      } else {
-        generateRecord(row, rand, recordNumber);
-      }
-      if (checksum != null) {
-        crc.reset();
-        crc.update(row, 0, row.length);
-        tmp.set(crc.getValue());
-        checksum.add(tmp);
-      }
-      recordNumber.add(ONE);
-      out.write(row);
-    }
-  }
-                                   
-  public static void main(String[] args) throws Exception {
-    Unsigned16 startingRecord = new Unsigned16();
-    Unsigned16 numberOfRecords;
-    OutputStream out;
-    boolean useAscii = false;
-    Unsigned16 checksum = null;
-
-    int i;
-    for(i=0; i < args.length; ++i) {
-      String arg = args[i];
-      int argLength = arg.length();
-      if (argLength >= 1 && arg.charAt(0) == '-') {
-        if (argLength < 2) {
-          usage();
-        }
-        switch (arg.charAt(1)) {
-        case 'a':
-          useAscii = true;
-          break;
-        case 'b':
-          startingRecord = Unsigned16.fromDecimal(arg.substring(2));
-          break;
-        case 'c':
-          checksum = new Unsigned16();
-          break;
-        default:
-          usage();
-        }
-      } else {
-        break;
-      }
-    }
-    if (args.length - i != 2) {
-      usage();
-    }
-    numberOfRecords = Unsigned16.fromDecimal(args[i]);
-    out = new FileOutputStream(args[i+1]);
-
-    outputRecords(out, useAscii, startingRecord, numberOfRecords, checksum);
-    out.close();
-    if (checksum != null) {
-      System.out.println(checksum);
-    }
-  }
-
-}
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Random16.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Random16.java
deleted file mode 100644
index 31cbd48..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Random16.java
+++ /dev/null
@@ -1,374 +0,0 @@
-/**
- * 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.tez.mapreduce.examples.terasort;
-
-/**
- * This class implements a 128-bit linear congruential generator.
- * Specifically, if X0 is the most recently issued 128-bit random
- * number (or a seed of 0 if no random number has already been generated,
- * the next number to be generated, X1, is equal to:
- * X1 = (a * X0 + c) mod 2**128
- * where a is 47026247687942121848144207491837523525
- *            or 0x2360ed051fc65da44385df649fccf645
- *   and c is 98910279301475397889117759788405497857
- *            or 0x4a696d47726179524950202020202001
- * The coefficient "a" is suggested by:
- * Pierre L'Ecuyer, "Tables of linear congruential generators of different
- * sizes and good lattice structure", Mathematics of Computation, 68
- * pp. 249 - 260 (1999)
- * http://www.ams.org/mcom/1999-68-225/S0025-5718-99-00996-5/S0025-5718-99-00996-5.pdf
- * The constant "c" meets the simple suggestion by the same reference that
- * it be odd.
- *
- * There is also a facility for quickly advancing the state of the
- * generator by a fixed number of steps - this facilitates parallel
- * generation.
- *
- * This is based on 1.0 of rand16.c from Chris Nyberg 
- * <chris.nyberg@ordinal.com>.
- */
-class Random16 {
-
-  /** 
-   * The "Gen" array contain powers of 2 of the linear congruential generator.
-   * The index 0 struct contain the "a" coefficient and "c" constant for the
-   * generator.  That is, the generator is:
-   *    f(x) = (Gen[0].a * x + Gen[0].c) mod 2**128
-   *
-   * All structs after the first contain an "a" and "c" that
-   * comprise the square of the previous function.
-   *
-   * f**2(x) = (Gen[1].a * x + Gen[1].c) mod 2**128
-   * f**4(x) = (Gen[2].a * x + Gen[2].c) mod 2**128
-   * f**8(x) = (Gen[3].a * x + Gen[3].c) mod 2**128
-   * ...
-
-   */
-  private static class RandomConstant {
-    final Unsigned16 a;
-    final Unsigned16 c;
-    public RandomConstant(String left, String right) {
-      a = new Unsigned16(left);
-      c = new Unsigned16(right);
-    }
-  }
-
-  private static final RandomConstant[] genArray = new RandomConstant[]{
-    /* [  0] */ new RandomConstant("2360ed051fc65da44385df649fccf645", 
-                                   "4a696d47726179524950202020202001"),
-    /* [  1] */ new RandomConstant("17bce35bdf69743c529ed9eb20e0ae99", 
-                                   "95e0e48262b3edfe04479485c755b646"),
-    /* [  2] */ new RandomConstant("f4dd417327db7a9bd194dfbe42d45771", 
-                                   "882a02c315362b60765f100068b33a1c"),
-    /* [  3] */ new RandomConstant("6347af777a7898f6d1a2d6f33505ffe1", 
-                                   "5efc4abfaca23e8ca8edb1f2dfbf6478"),
-    /* [  4] */ new RandomConstant("b6a4239f3b315f84f6ef6d3d288c03c1", 
-                                   "f25bd15439d16af594c1b1bafa6239f0"),
-    /* [  5] */ new RandomConstant("2c82901ad1cb0cd182b631ba6b261781", 
-                                   "89ca67c29c9397d59c612596145db7e0"),
-    /* [  6] */ new RandomConstant("dab03f988288676ee49e66c4d2746f01", 
-                                   "8b6ae036713bd578a8093c8eae5c7fc0"),
-    /* [  7] */ new RandomConstant("602167331d86cf5684fe009a6d09de01", 
-                                   "98a2542fd23d0dbdff3b886cdb1d3f80"),
-    /* [  8] */ new RandomConstant("61ecb5c24d95b058f04c80a23697bc01", 
-                                   "954db923fdb7933e947cd1edcecb7f00"),
-    /* [  9] */ new RandomConstant("4a5c31e0654c28aa60474e83bf3f7801", 
-                                   "00be4a36657c98cd204e8c8af7dafe00"),
-    /* [ 10] */ new RandomConstant("ae4f079d54fbece1478331d3c6bef001", 
-                                   "991965329dccb28d581199ab18c5fc00"),
-    /* [ 11] */ new RandomConstant("101b8cb830c7cb927ff1ed50ae7de001", 
-                                   "e1a8705b63ad5b8cd6c3d268d5cbf800"),
-    /* [ 12] */ new RandomConstant("f54a27fc056b00e7563f3505e0fbc001", 
-                                   "2b657bbfd6ed9d632079e70c3c97f000"),
-    /* [ 13] */ new RandomConstant("df8a6fc1a833d201f98d719dd1f78001",
-                                   "59b60ee4c52fa49e9fe90682bd2fe000"),
-    /* [ 14] */ new RandomConstant("5480a5015f101a4ea7e3f183e3ef0001", 
-                                   "cc099c88030679464fe86aae8a5fc000"),
-    /* [ 15] */ new RandomConstant("a498509e76e5d7925f539c28c7de0001", 
-                                   "06b9abff9f9f33dd30362c0154bf8000"),
-    /* [ 16] */ new RandomConstant("0798a3d8b10dc72e60121cd58fbc0001", 
-                                   "e296707121688d5a0260b293a97f0000"),
-    /* [ 17] */ new RandomConstant("1647d1e78ec02e665fafcbbb1f780001", 
-                                   "189ffc4701ff23cb8f8acf6b52fe0000"),
-    /* [ 18] */ new RandomConstant("a7c982285e72bf8c0c8ddfb63ef00001", 
-                                   "5141110ab208fb9d61fb47e6a5fc0000"),
-    /* [ 19] */ new RandomConstant("3eb78ee8fb8c56dbc5d4e06c7de00001", 
-                                   "3c97caa62540f2948d8d340d4bf80000"),
-    /* [ 20] */ new RandomConstant("72d03b6f4681f2f9fe8e44d8fbc00001", 
-                                   "1b25cb9cfe5a0c963174f91a97f00000"),
-    /* [ 21] */ new RandomConstant("ea85f81e4f502c9bc8ae99b1f7800001", 
-                                   "0c644570b4a487103c5436352fe00000"),
-    /* [ 22] */ new RandomConstant("629c320db08b00c6bfa57363ef000001", 
-                                   "3d0589c28869472bde517c6a5fc00000"),
-    /* [ 23] */ new RandomConstant("c5c4b9ce268d074a386be6c7de000001", 
-                                   "bc95e5ab36477e65534738d4bf800000"),
-    /* [ 24] */ new RandomConstant("f30bbbbed1596187555bcd8fbc000001", 
-                                   "ddb02ff72a031c01011f71a97f000000"),
-    /* [ 25] */ new RandomConstant("4a1000fb26c9eeda3cc79b1f78000001", 
-                                   "2561426086d9acdb6c82e352fe000000"),
-    /* [ 26] */ new RandomConstant("89fb5307f6bf8ce2c1cf363ef0000001", 
-                                   "64a788e3c118ed1c8215c6a5fc000000"),
-    /* [ 27] */ new RandomConstant("830b7b3358a5d67ea49e6c7de0000001", 
-                                   "e65ea321908627cfa86b8d4bf8000000"),
-    /* [ 28] */ new RandomConstant("fd8a51da91a69fe1cd3cd8fbc0000001", 
-                                   "53d27225604d85f9e1d71a97f0000000"),
-    /* [ 29] */ new RandomConstant("901a48b642b90b55aa79b1f780000001", 
-                                   "ca5ec7a3ed1fe55e07ae352fe0000000"),
-    /* [ 30] */ new RandomConstant("118cdefdf32144f394f363ef00000001", 
-                                   "4daebb2e085330651f5c6a5fc0000000"),
-    /* [ 31] */ new RandomConstant("0a88c0a91cff430829e6c7de00000001", 
-                                   "9d6f1a00a8f3f76e7eb8d4bf80000000"),
-    /* [ 32] */ new RandomConstant("433bef4314f16a9453cd8fbc00000001", 
-                                   "158c62f2b31e496dfd71a97f00000000"),
-    /* [ 33] */ new RandomConstant("c294b02995ae6738a79b1f7800000001", 
-                                   "290e84a2eb15fd1ffae352fe00000000"),
-    /* [ 34] */ new RandomConstant("913575e0da8b16b14f363ef000000001", 
-                                   "e3dc1bfbe991a34ff5c6a5fc00000000"),
-    /* [ 35] */ new RandomConstant("2f61b9f871cf4e629e6c7de000000001", 
-                                   "ddf540d020b9eadfeb8d4bf800000000"),
-    /* [ 36] */ new RandomConstant("78d26ccbd68320c53cd8fbc000000001", 
-                                   "8ee4950177ce66bfd71a97f000000000"),
-    /* [ 37] */ new RandomConstant("8b7ebd037898518a79b1f78000000001", 
-                                   "39e0f787c907117fae352fe000000000"),
-    /* [ 38] */ new RandomConstant("0b5507b61f78e314f363ef0000000001", 
-                                   "659d2522f7b732ff5c6a5fc000000000"),
-    /* [ 39] */ new RandomConstant("4f884628f812c629e6c7de0000000001", 
-                                   "9e8722938612a5feb8d4bf8000000000"),
-    /* [ 40] */ new RandomConstant("be896744d4a98c53cd8fbc0000000001", 
-                                   "e941a65d66b64bfd71a97f0000000000"),
-    /* [ 41] */ new RandomConstant("daf63a553b6318a79b1f780000000001", 
-                                   "7b50d19437b097fae352fe0000000000"),
-    /* [ 42] */ new RandomConstant("2d7a23d8bf06314f363ef00000000001", 
-                                   "59d7b68e18712ff5c6a5fc0000000000"),
-    /* [ 43] */ new RandomConstant("392b046a9f0c629e6c7de00000000001", 
-                                   "4087bab2d5225feb8d4bf80000000000"),
-    /* [ 44] */ new RandomConstant("eb30fbb9c218c53cd8fbc00000000001", 
-                                   "b470abc03b44bfd71a97f00000000000"),
-    /* [ 45] */ new RandomConstant("b9cdc30594318a79b1f7800000000001", 
-                                   "366630eaba897fae352fe00000000000"),
-    /* [ 46] */ new RandomConstant("014ab453686314f363ef000000000001", 
-                                   "a2dfc77e8512ff5c6a5fc00000000000"),
-    /* [ 47] */ new RandomConstant("395221c7d0c629e6c7de000000000001", 
-                                   "1e0d25a14a25feb8d4bf800000000000"),
-    /* [ 48] */ new RandomConstant("4d972813a18c53cd8fbc000000000001", 
-                                   "9d50a5d3944bfd71a97f000000000000"),
-    /* [ 49] */ new RandomConstant("06f9e2374318a79b1f78000000000001", 
-                                   "bf7ab5eb2897fae352fe000000000000"),
-    /* [ 50] */ new RandomConstant("bd220cae86314f363ef0000000000001", 
-                                   "925b14e6512ff5c6a5fc000000000000"),
-    /* [ 51] */ new RandomConstant("36fd3a5d0c629e6c7de0000000000001", 
-                                   "724cce0ca25feb8d4bf8000000000000"),
-    /* [ 52] */ new RandomConstant("60def8ba18c53cd8fbc0000000000001", 
-                                   "1af42d1944bfd71a97f0000000000000"),
-    /* [ 53] */ new RandomConstant("8d500174318a79b1f780000000000001", 
-                                   "0f529e32897fae352fe0000000000000"),
-    /* [ 54] */ new RandomConstant("48e842e86314f363ef00000000000001", 
-                                   "844e4c6512ff5c6a5fc0000000000000"),
-    /* [ 55] */ new RandomConstant("4af185d0c629e6c7de00000000000001",
-                                   "9f40d8ca25feb8d4bf80000000000000"),
-    /* [ 56] */ new RandomConstant("7a670ba18c53cd8fbc00000000000001",
-                                   "9912b1944bfd71a97f00000000000000"),
-    /* [ 57] */ new RandomConstant("86de174318a79b1f7800000000000001",
-                                   "9c69632897fae352fe00000000000000"),
-    /* [ 58] */ new RandomConstant("55fc2e86314f363ef000000000000001",
-                                   "e1e2c6512ff5c6a5fc00000000000000"),
-    /* [ 59] */ new RandomConstant("ccf85d0c629e6c7de000000000000001",
-                                   "68058ca25feb8d4bf800000000000000"),
-    /* [ 60] */ new RandomConstant("1df0ba18c53cd8fbc000000000000001",
-                                   "610b1944bfd71a97f000000000000000"),
-    /* [ 61] */ new RandomConstant("4be174318a79b1f78000000000000001",
-                                   "061632897fae352fe000000000000000"),
-    /* [ 62] */ new RandomConstant("d7c2e86314f363ef0000000000000001",
-                                   "1c2c6512ff5c6a5fc000000000000000"),
-    /* [ 63] */ new RandomConstant("af85d0c629e6c7de0000000000000001",
-                                   "7858ca25feb8d4bf8000000000000000"),
-    /* [ 64] */ new RandomConstant("5f0ba18c53cd8fbc0000000000000001",
-                                   "f0b1944bfd71a97f0000000000000000"),
-    /* [ 65] */ new RandomConstant("be174318a79b1f780000000000000001",
-                                   "e1632897fae352fe0000000000000000"),
-    /* [ 66] */ new RandomConstant("7c2e86314f363ef00000000000000001",
-                                   "c2c6512ff5c6a5fc0000000000000000"),
-    /* [ 67] */ new RandomConstant("f85d0c629e6c7de00000000000000001",
-                                   "858ca25feb8d4bf80000000000000000"),
-    /* [ 68] */ new RandomConstant("f0ba18c53cd8fbc00000000000000001",
-                                   "0b1944bfd71a97f00000000000000000"),
-    /* [ 69] */ new RandomConstant("e174318a79b1f7800000000000000001",
-                                   "1632897fae352fe00000000000000000"),
-    /* [ 70] */ new RandomConstant("c2e86314f363ef000000000000000001",
-                                   "2c6512ff5c6a5fc00000000000000000"),
-    /* [ 71] */ new RandomConstant("85d0c629e6c7de000000000000000001",
-                                   "58ca25feb8d4bf800000000000000000"),
-    /* [ 72] */ new RandomConstant("0ba18c53cd8fbc000000000000000001",
-                                   "b1944bfd71a97f000000000000000000"),
-    /* [ 73] */ new RandomConstant("174318a79b1f78000000000000000001",
-                                   "632897fae352fe000000000000000000"),
-    /* [ 74] */ new RandomConstant("2e86314f363ef0000000000000000001",
-                                   "c6512ff5c6a5fc000000000000000000"),
-    /* [ 75] */ new RandomConstant("5d0c629e6c7de0000000000000000001",
-                                   "8ca25feb8d4bf8000000000000000000"),
-    /* [ 76] */ new RandomConstant("ba18c53cd8fbc0000000000000000001",
-                                   "1944bfd71a97f0000000000000000000"),
-    /* [ 77] */ new RandomConstant("74318a79b1f780000000000000000001",
-                                   "32897fae352fe0000000000000000000"),
-    /* [ 78] */ new RandomConstant("e86314f363ef00000000000000000001",
-                                   "6512ff5c6a5fc0000000000000000000"),
-    /* [ 79] */ new RandomConstant("d0c629e6c7de00000000000000000001",
-                                   "ca25feb8d4bf80000000000000000000"),
-    /* [ 80] */ new RandomConstant("a18c53cd8fbc00000000000000000001",
-                                   "944bfd71a97f00000000000000000000"),
-    /* [ 81] */ new RandomConstant("4318a79b1f7800000000000000000001",
-                                   "2897fae352fe00000000000000000000"),
-    /* [ 82] */ new RandomConstant("86314f363ef000000000000000000001",
-                                   "512ff5c6a5fc00000000000000000000"),
-    /* [ 83] */ new RandomConstant("0c629e6c7de000000000000000000001",
-                                   "a25feb8d4bf800000000000000000000"),
-    /* [ 84] */ new RandomConstant("18c53cd8fbc000000000000000000001",
-                                   "44bfd71a97f000000000000000000000"),
-    /* [ 85] */ new RandomConstant("318a79b1f78000000000000000000001",
-                                   "897fae352fe000000000000000000000"),
-    /* [ 86] */ new RandomConstant("6314f363ef0000000000000000000001",
-                                   "12ff5c6a5fc000000000000000000000"),
-    /* [ 87] */ new RandomConstant("c629e6c7de0000000000000000000001",
-                                   "25feb8d4bf8000000000000000000000"),
-    /* [ 88] */ new RandomConstant("8c53cd8fbc0000000000000000000001",
-                                   "4bfd71a97f0000000000000000000000"),
-    /* [ 89] */ new RandomConstant("18a79b1f780000000000000000000001",
-                                   "97fae352fe0000000000000000000000"),
-    /* [ 90] */ new RandomConstant("314f363ef00000000000000000000001",
-                                   "2ff5c6a5fc0000000000000000000000"),
-    /* [ 91] */ new RandomConstant("629e6c7de00000000000000000000001",
-                                   "5feb8d4bf80000000000000000000000"),
-    /* [ 92] */ new RandomConstant("c53cd8fbc00000000000000000000001",
-                                   "bfd71a97f00000000000000000000000"),
-    /* [ 93] */ new RandomConstant("8a79b1f7800000000000000000000001",
-                                   "7fae352fe00000000000000000000000"),
-    /* [ 94] */ new RandomConstant("14f363ef000000000000000000000001",
-                                   "ff5c6a5fc00000000000000000000000"),
-    /* [ 95] */ new RandomConstant("29e6c7de000000000000000000000001",
-                                   "feb8d4bf800000000000000000000000"),
-    /* [ 96] */ new RandomConstant("53cd8fbc000000000000000000000001",
-                                   "fd71a97f000000000000000000000000"),
-    /* [ 97] */ new RandomConstant("a79b1f78000000000000000000000001",
-                                   "fae352fe000000000000000000000000"),
-    /* [ 98] */ new RandomConstant("4f363ef0000000000000000000000001",
-                                   "f5c6a5fc000000000000000000000000"),
-    /* [ 99] */ new RandomConstant("9e6c7de0000000000000000000000001",
-                                   "eb8d4bf8000000000000000000000000"),
-    /* [100] */ new RandomConstant("3cd8fbc0000000000000000000000001",
-                                   "d71a97f0000000000000000000000000"),
-    /* [101] */ new RandomConstant("79b1f780000000000000000000000001",
-                                   "ae352fe0000000000000000000000000"),
-    /* [102] */ new RandomConstant("f363ef00000000000000000000000001",
-                                   "5c6a5fc0000000000000000000000000"),
-    /* [103] */ new RandomConstant("e6c7de00000000000000000000000001",
-                                   "b8d4bf80000000000000000000000000"),
-    /* [104] */ new RandomConstant("cd8fbc00000000000000000000000001",
-                                   "71a97f00000000000000000000000000"),
-    /* [105] */ new RandomConstant("9b1f7800000000000000000000000001",
-                                   "e352fe00000000000000000000000000"),
-    /* [106] */ new RandomConstant("363ef000000000000000000000000001",
-                                   "c6a5fc00000000000000000000000000"),
-    /* [107] */ new RandomConstant("6c7de000000000000000000000000001",
-                                   "8d4bf800000000000000000000000000"),
-    /* [108] */ new RandomConstant("d8fbc000000000000000000000000001",
-                                   "1a97f000000000000000000000000000"),
-    /* [109] */ new RandomConstant("b1f78000000000000000000000000001",
-                                   "352fe000000000000000000000000000"),
-    /* [110] */ new RandomConstant("63ef0000000000000000000000000001",
-                                   "6a5fc000000000000000000000000000"),
-    /* [111] */ new RandomConstant("c7de0000000000000000000000000001",
-                                   "d4bf8000000000000000000000000000"),
-    /* [112] */ new RandomConstant("8fbc0000000000000000000000000001",
-                                   "a97f0000000000000000000000000000"),
-    /* [113] */ new RandomConstant("1f780000000000000000000000000001",
-                                   "52fe0000000000000000000000000000"),
-    /* [114] */ new RandomConstant("3ef00000000000000000000000000001",
-                                   "a5fc0000000000000000000000000000"),
-    /* [115] */ new RandomConstant("7de00000000000000000000000000001",
-                                   "4bf80000000000000000000000000000"),
-    /* [116] */ new RandomConstant("fbc00000000000000000000000000001",
-                                   "97f00000000000000000000000000000"),
-    /* [117] */ new RandomConstant("f7800000000000000000000000000001",
-                                   "2fe00000000000000000000000000000"),
-    /* [118] */ new RandomConstant("ef000000000000000000000000000001",
-                                   "5fc00000000000000000000000000000"),
-    /* [119] */ new RandomConstant("de000000000000000000000000000001",
-                                   "bf800000000000000000000000000000"),
-    /* [120] */ new RandomConstant("bc000000000000000000000000000001",
-                                   "7f000000000000000000000000000000"),
-    /* [121] */ new RandomConstant("78000000000000000000000000000001",
-                                   "fe000000000000000000000000000000"),
-    /* [122] */ new RandomConstant("f0000000000000000000000000000001",
-                                   "fc000000000000000000000000000000"),
-    /* [123] */ new RandomConstant("e0000000000000000000000000000001",
-                                   "f8000000000000000000000000000000"),
-    /* [124] */ new RandomConstant("c0000000000000000000000000000001",
-                                   "f0000000000000000000000000000000"),
-    /* [125] */ new RandomConstant("80000000000000000000000000000001",
-                                   "e0000000000000000000000000000000"),
-    /* [126] */ new RandomConstant("00000000000000000000000000000001",
-                                   "c0000000000000000000000000000000"),
-    /* [127] */ new RandomConstant("00000000000000000000000000000001",
-                                   "80000000000000000000000000000000")};
-
-  /**
-   *  generate the random number that is "advance" steps
-   *  from an initial random number of 0.  This is done by
-   *  starting with 0, and then advancing the by the
-   *  appropriate powers of 2 of the linear congruential
-   *  generator.
-   */
-  public static Unsigned16 skipAhead(Unsigned16 advance) {
-    Unsigned16 result = new Unsigned16();
-    long          bit_map;
-
-    bit_map = advance.getLow8();
-    for (int i = 0; bit_map != 0 && i < 64; i++) {
-      if ((bit_map & (1L << i)) != 0) {
-        /* advance random number by f**(2**i) (x)
-         */
-        result.multiply(genArray[i].a);
-        result.add(genArray[i].c);
-        bit_map &= ~(1L << i);
-      }
-    }
-    bit_map = advance.getHigh8();
-    for (int i = 0; bit_map != 0 && i < 64; i++)
-    {
-      if ((bit_map & (1L << i)) != 0) {
-        /* advance random number by f**(2**(i + 64)) (x)
-         */
-        result.multiply(genArray[i+64].a);
-        result.add(genArray[i+64].c);
-        bit_map &= ~(1L << i);
-      }
-    }
-    return result;
-  }
-
-  /** 
-   * Generate the next 16 byte random number.
-   */
-  public static void nextRand(Unsigned16 rand) {
-    /* advance the random number forward once using the linear congruential
-     * generator, and then return the new random number
-     */
-    rand.multiply(genArray[0].a);
-    rand.add(genArray[0].c);
-  }
-}
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraChecksum.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraChecksum.java
deleted file mode 100644
index a5b408b..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraChecksum.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * 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.tez.mapreduce.examples.terasort;
-
-import java.io.IOException;
-import java.util.zip.Checksum;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-public class TeraChecksum extends Configured implements Tool {
-  static class ChecksumMapper 
-      extends Mapper<Text, Text, NullWritable, Unsigned16> {
-    private Unsigned16 checksum = new Unsigned16();
-    private Unsigned16 sum = new Unsigned16();
-    private Checksum crc32 = new PureJavaCrc32();
-
-    public void map(Text key, Text value, 
-                    Context context) throws IOException {
-      crc32.reset();
-      crc32.update(key.getBytes(), 0, key.getLength());
-      crc32.update(value.getBytes(), 0, value.getLength());
-      checksum.set(crc32.getValue());
-      sum.add(checksum);
-    }
-
-    public void cleanup(Context context) 
-        throws IOException, InterruptedException {
-      context.write(NullWritable.get(), sum);
-    }
-  }
-
-  static class ChecksumReducer 
-      extends Reducer<NullWritable, Unsigned16, NullWritable, Unsigned16> {
-
-    public void reduce(NullWritable key, Iterable<Unsigned16> values,
-        Context context) throws IOException, InterruptedException  {
-      Unsigned16 sum = new Unsigned16();
-      for (Unsigned16 val : values) {
-        sum.add(val);
-      }
-      context.write(key, sum);
-    }
-  }
-
-  private static void usage() throws IOException {
-    System.err.println("terasum <out-dir> <report-dir>");
-  }
-
-  public int run(String[] args) throws Exception {
-    Job job = Job.getInstance(getConf());
-    if (args.length != 2) {
-      usage();
-      return 2;
-    }
-    TeraInputFormat.setInputPaths(job, new Path(args[0]));
-    FileOutputFormat.setOutputPath(job, new Path(args[1]));
-    job.setJobName("TeraSum");
-    job.setJarByClass(TeraChecksum.class);
-    job.setMapperClass(ChecksumMapper.class);
-    job.setReducerClass(ChecksumReducer.class);
-    job.setOutputKeyClass(NullWritable.class);
-    job.setOutputValueClass(Unsigned16.class);
-    // force a single reducer
-    job.setNumReduceTasks(1);
-    job.setInputFormatClass(TeraInputFormat.class);
-    return job.waitForCompletion(true) ? 0 : 1;
-  }
-
-  /**
-   * @param args
-   */
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new TeraChecksum(), args);
-    System.exit(res);
-  }
-
-}
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraGen.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraGen.java
deleted file mode 100644
index 70a4207..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraGen.java
+++ /dev/null
@@ -1,311 +0,0 @@
-/**
- * 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.tez.mapreduce.examples.terasort;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.zip.Checksum;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * Generate the official GraySort input data set.
- * The user specifies the number of rows and the output directory and this
- * class runs a map/reduce program to generate the data.
- * The format of the data is:
- * <ul>
- * <li>(10 bytes key) (constant 2 bytes) (32 bytes rowid) 
- *     (constant 4 bytes) (48 bytes filler) (constant 4 bytes)
- * <li>The rowid is the right justified row id as a hex number.
- * </ul>
- *
- * <p>
- * To run the program: 
- * <b>bin/hadoop jar hadoop-*-examples.jar teragen 10000000000 in-dir</b>
- */
-public class TeraGen extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(TeraSort.class);
-
-  public static enum Counters {CHECKSUM}
-
-  public static final String NUM_ROWS = "mapreduce.terasort.num-rows";
-  /**
-   * An input format that assigns ranges of longs to each mapper.
-   */
-  static class RangeInputFormat 
-      extends InputFormat<LongWritable, NullWritable> {
-    
-    /**
-     * An input split consisting of a range on numbers.
-     */
-    static class RangeInputSplit extends InputSplit implements Writable {
-      long firstRow;
-      long rowCount;
-
-      public RangeInputSplit() { }
-
-      public RangeInputSplit(long offset, long length) {
-        firstRow = offset;
-        rowCount = length;
-      }
-
-      public long getLength() throws IOException {
-        return 0;
-      }
-
-      public String[] getLocations() throws IOException {
-        return new String[]{};
-      }
-
-      public void readFields(DataInput in) throws IOException {
-        firstRow = WritableUtils.readVLong(in);
-        rowCount = WritableUtils.readVLong(in);
-      }
-
-      public void write(DataOutput out) throws IOException {
-        WritableUtils.writeVLong(out, firstRow);
-        WritableUtils.writeVLong(out, rowCount);
-      }
-    }
-    
-    /**
-     * A record reader that will generate a range of numbers.
-     */
-    static class RangeRecordReader 
-        extends RecordReader<LongWritable, NullWritable> {
-      long startRow;
-      long finishedRows;
-      long totalRows;
-      LongWritable key = null;
-
-      public RangeRecordReader() {
-      }
-      
-      public void initialize(InputSplit split, TaskAttemptContext context) 
-          throws IOException, InterruptedException {
-        startRow = ((RangeInputSplit)split).firstRow;
-        finishedRows = 0;
-        totalRows = ((RangeInputSplit)split).rowCount;
-      }
-
-      public void close() throws IOException {
-        // NOTHING
-      }
-
-      public LongWritable getCurrentKey() {
-        return key;
-      }
-
-      public NullWritable getCurrentValue() {
-        return NullWritable.get();
-      }
-
-      public float getProgress() throws IOException {
-        return finishedRows / (float) totalRows;
-      }
-
-      public boolean nextKeyValue() {
-        if (key == null) {
-          key = new LongWritable();
-        }
-        if (finishedRows < totalRows) {
-          key.set(startRow + finishedRows);
-          finishedRows += 1;
-          return true;
-        } else {
-          return false;
-        }
-      }
-      
-    }
-
-    public RecordReader<LongWritable, NullWritable> 
-        createRecordReader(InputSplit split, TaskAttemptContext context) 
-        throws IOException {
-      return new RangeRecordReader();
-    }
-
-    /**
-     * Create the desired number of splits, dividing the number of rows
-     * between the mappers.
-     */
-    public List<InputSplit> getSplits(JobContext job) {
-      long totalRows = getNumberOfRows(job);
-      int numSplits = job.getConfiguration().getInt(MRJobConfig.NUM_MAPS, 1);
-      LOG.info("Generating " + totalRows + " using " + numSplits);
-      List<InputSplit> splits = new ArrayList<InputSplit>();
-      long currentRow = 0;
-      for(int split = 0; split < numSplits; ++split) {
-        long goal = 
-          (long) Math.ceil(totalRows * (double)(split + 1) / numSplits);
-        splits.add(new RangeInputSplit(currentRow, goal - currentRow));
-        currentRow = goal;
-      }
-      return splits;
-    }
-
-  }
-  
-  static long getNumberOfRows(JobContext job) {
-    return job.getConfiguration().getLong(NUM_ROWS, 0);
-  }
-  
-  static void setNumberOfRows(Job job, long numRows) {
-    job.getConfiguration().setLong(NUM_ROWS, numRows);
-  }
-
-  /**
-   * The Mapper class that given a row number, will generate the appropriate 
-   * output line.
-   */
-  public static class SortGenMapper 
-      extends Mapper<LongWritable, NullWritable, Text, Text> {
-
-    private Text key = new Text();
-    private Text value = new Text();
-    private Unsigned16 rand = null;
-    private Unsigned16 rowId = null;
-    private Unsigned16 checksum = new Unsigned16();
-    private Checksum crc32 = new PureJavaCrc32();
-    private Unsigned16 total = new Unsigned16();
-    private static final Unsigned16 ONE = new Unsigned16(1);
-    private byte[] buffer = new byte[TeraInputFormat.KEY_LENGTH +
-                                     TeraInputFormat.VALUE_LENGTH];
-    private Counter checksumCounter;
-
-    public void map(LongWritable row, NullWritable ignored,
-        Context context) throws IOException, InterruptedException {
-      if (rand == null) {
-        rowId = new Unsigned16(row.get());
-        rand = Random16.skipAhead(rowId);
-        checksumCounter = context.getCounter(Counters.CHECKSUM);
-      }
-      Random16.nextRand(rand);
-      GenSort.generateRecord(buffer, rand, rowId);
-      key.set(buffer, 0, TeraInputFormat.KEY_LENGTH);
-      value.set(buffer, TeraInputFormat.KEY_LENGTH, 
-                TeraInputFormat.VALUE_LENGTH);
-      context.write(key, value);
-      crc32.reset();
-      crc32.update(buffer, 0, 
-                   TeraInputFormat.KEY_LENGTH + TeraInputFormat.VALUE_LENGTH);
-      checksum.set(crc32.getValue());
-      total.add(checksum);
-      rowId.add(ONE);
-    }
-
-    @Override
-    public void cleanup(Context context) {
-      if (checksumCounter != null) {
-        checksumCounter.increment(total.getLow8());
-      }
-    }
-  }
-
-  private static void usage() throws IOException {
-    System.err.println("teragen <num rows> <output dir>");
-  }
-
-  /**
-   * Parse a number that optionally has a postfix that denotes a base.
-   * @param str an string integer with an option base {k,m,b,t}.
-   * @return the expanded value
-   */
-  private static long parseHumanLong(String str) {
-    char tail = str.charAt(str.length() - 1);
-    long base = 1;
-    switch (tail) {
-    case 't':
-      base *= 1000 * 1000 * 1000 * 1000;
-      break;
-    case 'b':
-      base *= 1000 * 1000 * 1000;
-      break;
-    case 'm':
-      base *= 1000 * 1000;
-      break;
-    case 'k':
-      base *= 1000;
-      break;
-    default:
-    }
-    if (base != 1) {
-      str = str.substring(0, str.length() - 1);
-    }
-    return Long.parseLong(str) * base;
-  }
-  
-  /**
-   * @param args the cli arguments
-   */
-  public int run(String[] args) 
-      throws IOException, InterruptedException, ClassNotFoundException {
-    Job job = Job.getInstance(getConf());
-    if (args.length != 2) {
-      usage();
-      return 2;
-    }
-    setNumberOfRows(job, parseHumanLong(args[0]));
-    Path outputDir = new Path(args[1]);
-    if (outputDir.getFileSystem(getConf()).exists(outputDir)) {
-      throw new IOException("Output directory " + outputDir + 
-                            " already exists.");
-    }
-    FileOutputFormat.setOutputPath(job, outputDir);
-    job.setJobName("TeraGen");
-    job.setJarByClass(TeraGen.class);
-    job.setMapperClass(SortGenMapper.class);
-    job.setNumReduceTasks(0);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    job.setInputFormatClass(RangeInputFormat.class);
-    job.setOutputFormatClass(TeraOutputFormat.class);
-    return job.waitForCompletion(true) ? 0 : 1;
-  }
-
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new TeraGen(), args);
-    System.exit(res);
-  }
-}
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraInputFormat.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraInputFormat.java
deleted file mode 100644
index 6c9b201..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraInputFormat.java
+++ /dev/null
@@ -1,353 +0,0 @@
-/**
- * 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.tez.mapreduce.examples.terasort;
-
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.hadoop.util.IndexedSortable;
-import org.apache.hadoop.util.QuickSort;
-
-/**
- * An input format that reads the first 10 characters of each line as the key
- * and the rest of the line as the value. Both key and value are represented
- * as Text.
- */
-public class TeraInputFormat extends FileInputFormat<Text,Text> {
-
-  static final String PARTITION_FILENAME = "_partition.lst";
-  private static final String NUM_PARTITIONS = 
-    "mapreduce.terasort.num.partitions";
-  private static final String SAMPLE_SIZE = 
-    "mapreduce.terasort.partitions.sample";
-  static final int KEY_LENGTH = 10;
-  static final int VALUE_LENGTH = 90;
-  static final int RECORD_LENGTH = KEY_LENGTH + VALUE_LENGTH;
-  private static MRJobConfig lastContext = null;
-  private static List<InputSplit> lastResult = null;
-
-  static class TeraFileSplit extends FileSplit {
-    static private String[] ZERO_LOCATIONS = new String[0];
-
-    private String[] locations;
-
-    public TeraFileSplit() {
-      locations = ZERO_LOCATIONS;
-    }
-    public TeraFileSplit(Path file, long start, long length, String[] hosts) {
-      super(file, start, length, hosts);
-      try {
-        locations = super.getLocations();
-      } catch (IOException e) {
-        locations = ZERO_LOCATIONS;
-      }
-    }
-
-    // XXXXXX should this also be null-protected?
-    protected void setLocations(String[] hosts) {
-      locations = hosts;
-    }
-
-    @Override
-    public String[] getLocations() {
-      return locations;
-    }
-
-    public String toString() {
-      StringBuffer result = new StringBuffer();
-      result.append(getPath());
-      result.append(" from ");
-      result.append(getStart());
-      result.append(" length ");
-      result.append(getLength());
-      for(String host: getLocations()) {
-        result.append(" ");
-        result.append(host);
-      }
-      return result.toString();
-    }
-  }
-
-  static class TextSampler implements IndexedSortable {
-    private ArrayList<Text> records = new ArrayList<Text>();
-
-    public int compare(int i, int j) {
-      Text left = records.get(i);
-      Text right = records.get(j);
-      return left.compareTo(right);
-    }
-
-    public void swap(int i, int j) {
-      Text left = records.get(i);
-      Text right = records.get(j);
-      records.set(j, left);
-      records.set(i, right);
-    }
-
-    public void addKey(Text key) {
-      synchronized (this) {
-        records.add(new Text(key));
-      }
-    }
-
-    /**
-     * Find the split points for a given sample. The sample keys are sorted
-     * and down sampled to find even split points for the partitions. The
-     * returned keys should be the start of their respective partitions.
-     * @param numPartitions the desired number of partitions
-     * @return an array of size numPartitions - 1 that holds the split points
-     */
-    Text[] createPartitions(int numPartitions) {
-      int numRecords = records.size();
-      System.out.println("Making " + numPartitions + " from " + numRecords + 
-                         " sampled records");
-      if (numPartitions > numRecords) {
-        throw new IllegalArgumentException
-          ("Requested more partitions than input keys (" + numPartitions +
-           " > " + numRecords + ")");
-      }
-      new QuickSort().sort(this, 0, records.size());
-      float stepSize = numRecords / (float) numPartitions;
-      Text[] result = new Text[numPartitions-1];
-      for(int i=1; i < numPartitions; ++i) {
-        result[i-1] = records.get(Math.round(stepSize * i));
-      }
-      return result;
-    }
-  }
-  
-  /**
-   * Use the input splits to take samples of the input and generate sample
-   * keys. By default reads 100,000 keys from 10 locations in the input, sorts
-   * them and picks N-1 keys to generate N equally sized partitions.
-   * @param job the job to sample
-   * @param partFile where to write the output file to
-   * @throws Throwable if something goes wrong
-   */
-  public static void writePartitionFile(final JobContext job, 
-      Path partFile) throws Throwable  {
-    long t1 = System.currentTimeMillis();
-    Configuration conf = job.getConfiguration();
-    final TeraInputFormat inFormat = new TeraInputFormat();
-    final TextSampler sampler = new TextSampler();
-    int partitions = job.getNumReduceTasks();
-    long sampleSize = conf.getLong(SAMPLE_SIZE, 100000);
-    final List<InputSplit> splits = inFormat.getSplits(job);
-    long t2 = System.currentTimeMillis();
-    System.out.println("Computing input splits took " + (t2 - t1) + "ms");
-    int samples = Math.min(conf.getInt(NUM_PARTITIONS, 10), splits.size());
-    System.out.println("Sampling " + samples + " splits of " + splits.size());
-    final long recordsPerSample = sampleSize / samples;
-    final int sampleStep = splits.size() / samples;
-    Thread[] samplerReader = new Thread[samples];
-    SamplerThreadGroup threadGroup = new SamplerThreadGroup("Sampler Reader Thread Group");
-    // take N samples from different parts of the input
-    for(int i=0; i < samples; ++i) {
-      final int idx = i;
-      samplerReader[i] = 
-        new Thread (threadGroup,"Sampler Reader " + idx) {
-        {
-          setDaemon(true);
-        }
-        public void run() {
-          long records = 0;
-          try {
-            TaskAttemptContext context = new TaskAttemptContextImpl(
-              job.getConfiguration(), new TaskAttemptID());
-            RecordReader<Text, Text> reader = 
-              inFormat.createRecordReader(splits.get(sampleStep * idx),
-              context);
-            reader.initialize(splits.get(sampleStep * idx), context);
-            while (reader.nextKeyValue()) {
-              sampler.addKey(new Text(reader.getCurrentKey()));
-              records += 1;
-              if (recordsPerSample <= records) {
-                break;
-              }
-            }
-          } catch (IOException ie){
-            System.err.println("Got an exception while reading splits " +
-                ExceptionUtils.getStackTrace(ie));
-            throw new RuntimeException(ie);
-          } catch (InterruptedException e) {
-        	  
-          }
-        }
-      };
-      samplerReader[i].start();
-    }
-    FileSystem outFs = partFile.getFileSystem(conf);
-    DataOutputStream writer = outFs.create(partFile, true, 64*1024, (short) 10, 
-                                           outFs.getDefaultBlockSize(partFile));
-    for (int i = 0; i < samples; i++) {
-      try {
-        samplerReader[i].join();
-        if(threadGroup.getThrowable() != null){
-          throw threadGroup.getThrowable();
-        }
-      } catch (InterruptedException e) {
-      }
-    }
-    for(Text split : sampler.createPartitions(partitions)) {
-      split.write(writer);
-    }
-    writer.close();
-    long t3 = System.currentTimeMillis();
-    System.out.println("Computing parititions took " + (t3 - t2) + "ms");
-  }
-  
-  static class SamplerThreadGroup extends ThreadGroup{
-
-    private Throwable throwable;
-
-    public SamplerThreadGroup(String s) {
-      super(s);
-    }
-    
-    @Override
-    public void uncaughtException(Thread thread, Throwable throwable) {
-      this.throwable = throwable;
-    }
-    
-    public Throwable getThrowable() {
-      return this.throwable;
-    }
-    
-  }
-
-  static class TeraRecordReader extends RecordReader<Text,Text> {
-    private FSDataInputStream in;
-    private long offset;
-    private long length;
-    private static final int RECORD_LENGTH = KEY_LENGTH + VALUE_LENGTH;
-    private byte[] buffer = new byte[RECORD_LENGTH];
-    private Text key;
-    private Text value;
-
-    public TeraRecordReader() throws IOException {
-    }
-
-    public void initialize(InputSplit split, TaskAttemptContext context) 
-        throws IOException, InterruptedException {
-      Path p = ((FileSplit)split).getPath();
-      FileSystem fs = p.getFileSystem(context.getConfiguration());
-      in = fs.open(p);
-      long start = ((FileSplit)split).getStart();
-      // find the offset to start at a record boundary
-      offset = (RECORD_LENGTH - (start % RECORD_LENGTH)) % RECORD_LENGTH;
-      in.seek(start + offset);
-      length = ((FileSplit)split).getLength();
-    }
-
-    public void close() throws IOException {
-      in.close();
-    }
-
-    public Text getCurrentKey() {
-      return key;
-    }
-
-    public Text getCurrentValue() {
-      return value;
-    }
-
-    public float getProgress() throws IOException {
-      return (float) offset / length;
-    }
-
-    public boolean nextKeyValue() throws IOException {
-      if (offset >= length) {
-        return false;
-      }
-      int read = 0;
-      while (read < RECORD_LENGTH) {
-        long newRead = in.read(buffer, read, RECORD_LENGTH - read);
-        if (newRead == -1) {
-          if (read == 0) {
-            return false;
-          } else {
-            throw new EOFException("read past eof");
-          }
-        }
-        read += newRead;
-      }
-      if (key == null) {
-        key = new Text();
-      }
-      if (value == null) {
-        value = new Text();
-      }
-      key.set(buffer, 0, KEY_LENGTH);
-      value.set(buffer, KEY_LENGTH, VALUE_LENGTH);
-      offset += RECORD_LENGTH;
-      return true;
-    }
-  }
-
-  @Override
-  public RecordReader<Text, Text> 
-      createRecordReader(InputSplit split, TaskAttemptContext context) 
-      throws IOException {
-    return new TeraRecordReader();
-  }
-
-  protected FileSplit makeSplit(Path file, long start, long length, 
-                                String[] hosts) {
-    return new TeraFileSplit(file, start, length, hosts);
-  }
-
-  @Override
-  public List<InputSplit> getSplits(JobContext job) throws IOException {
-    if (job == lastContext) {
-      return lastResult;
-    }
-    long t1, t2, t3;
-    t1 = System.currentTimeMillis();
-    lastContext = job;
-    lastResult = super.getSplits(job);
-    t2 = System.currentTimeMillis();
-    System.out.println("Spent " + (t2 - t1) + "ms computing base-splits.");
-    if (job.getConfiguration().getBoolean(TeraScheduler.USE, true)) {
-      TeraScheduler scheduler = new TeraScheduler(
-        lastResult.toArray(new TeraFileSplit[0]), job.getConfiguration());
-      lastResult = scheduler.getNewFileSplits();
-      t3 = System.currentTimeMillis(); 
-      System.out.println("Spent " + (t3 - t2) + "ms computing TeraScheduler splits.");
-    }
-    return lastResult;
-  }
-}
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraOutputFormat.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraOutputFormat.java
deleted file mode 100644
index 4ff0f06..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraOutputFormat.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/**
- * 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.tez.mapreduce.examples.terasort;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InvalidJobConfException;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.security.TokenCache;
-
-/**
- * An output format that writes the key and value appended together.
- */
-public class TeraOutputFormat extends FileOutputFormat<Text,Text> {
-  static final String FINAL_SYNC_ATTRIBUTE = "mapreduce.terasort.final.sync";
-  private OutputCommitter committer = null;
-
-  /**
-   * Set the requirement for a final sync before the stream is closed.
-   */
-  static void setFinalSync(JobContext job, boolean newValue) {
-    job.getConfiguration().setBoolean(FINAL_SYNC_ATTRIBUTE, newValue);
-  }
-
-  /**
-   * Does the user want a final sync at close?
-   */
-  public static boolean getFinalSync(JobContext job) {
-    return job.getConfiguration().getBoolean(FINAL_SYNC_ATTRIBUTE, false);
-  }
-
-  static class TeraRecordWriter extends RecordWriter<Text,Text> {
-    private boolean finalSync = false;
-    private FSDataOutputStream out;
-
-    public TeraRecordWriter(FSDataOutputStream out,
-                            JobContext job) {
-      finalSync = getFinalSync(job);
-      this.out = out;
-    }
-
-    public synchronized void write(Text key, 
-                                   Text value) throws IOException {
-      out.write(key.getBytes(), 0, key.getLength());
-      out.write(value.getBytes(), 0, value.getLength());
-    }
-    
-    public void close(TaskAttemptContext context) throws IOException {
-      if (finalSync) {
-        out.hsync();
-      }
-      out.close();
-    }
-  }
-
-  @Override
-  public void checkOutputSpecs(JobContext job
-                              ) throws InvalidJobConfException, IOException {
-    // Ensure that the output directory is set
-    Path outDir = getOutputPath(job);
-    if (outDir == null) {
-      throw new InvalidJobConfException("Output directory not set in JobConf.");
-    }
-
-    // get delegation token for outDir's file system
-    TokenCache.obtainTokensForNamenodes(job.getCredentials(),
-        new Path[] { outDir }, job.getConfiguration());
-  }
-
-  public RecordWriter<Text,Text> getRecordWriter(TaskAttemptContext job
-                                                 ) throws IOException {
-    Path file = getDefaultWorkFile(job, "");
-    FileSystem fs = file.getFileSystem(job.getConfiguration());
-     FSDataOutputStream fileOut = fs.create(file);
-    return new TeraRecordWriter(fileOut, job);
-  }
-  
-  public OutputCommitter getOutputCommitter(TaskAttemptContext context) 
-      throws IOException {
-    if (committer == null) {
-      Path output = getOutputPath(context);
-      committer = new FileOutputCommitter(output, context);
-    }
-    return committer;
-  }
-
-}
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraScheduler.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraScheduler.java
deleted file mode 100644
index 82638d8..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraScheduler.java
+++ /dev/null
@@ -1,253 +0,0 @@
-/**
- * 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.tez.mapreduce.examples.terasort;
-
-import java.io.*;
-import java.util.*;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
-import org.apache.tez.mapreduce.examples.terasort.TeraInputFormat.TeraFileSplit;
-
-import com.google.common.base.Charsets;
-
-class TeraScheduler {
-  static String USE = "mapreduce.terasort.use.terascheduler";
-  private static final Log LOG = LogFactory.getLog(TeraScheduler.class);
-  private Split[] splits;
-  private List<Host> hosts = new ArrayList<Host>();
-  private int slotsPerHost;
-  private int remainingSplits = 0;
-  private FileSplit[] realSplits = null;
-
-  static class Split {
-    String filename;
-    boolean isAssigned = false;
-    List<Host> locations = new ArrayList<Host>();
-    Split(String filename) {
-      this.filename = filename;
-    }
-    public String toString() {
-      StringBuffer result = new StringBuffer();
-      result.append(filename);
-      result.append(" on ");
-      for(Host host: locations) {
-        result.append(host.hostname);
-        result.append(", ");
-      }
-      return result.toString();
-    }
-  }
-  static class Host {
-    String hostname;
-    List<Split> splits = new ArrayList<Split>();
-    Host(String hostname) {
-      this.hostname = hostname;
-    }
-    public String toString() {
-      StringBuffer result = new StringBuffer();
-      result.append(splits.size());
-      result.append(" ");
-      result.append(hostname);
-      return result.toString();
-    }
-  }
-
-  List<String> readFile(String filename) throws IOException {
-    List<String> result = new ArrayList<String>(10000);
-    BufferedReader in = new BufferedReader(
-        new InputStreamReader(new FileInputStream(filename), Charsets.UTF_8));
-    String line = in.readLine();
-    while (line != null) {
-      result.add(line);
-      line = in.readLine();
-    }
-    in.close();
-    return result;
-  }
-
-  public TeraScheduler(String splitFilename, 
-                       String nodeFilename) throws IOException {
-    slotsPerHost = 4;
-    // get the hosts
-    Map<String, Host> hostIds = new HashMap<String,Host>();
-    for(String hostName: readFile(nodeFilename)) {
-      Host host = new Host(hostName);
-      hosts.add(host);
-      hostIds.put(hostName, host);
-    }
-    // read the blocks
-    List<String> splitLines = readFile(splitFilename);
-    splits = new Split[splitLines.size()];
-    remainingSplits = 0;
-    for(String line: splitLines) {
-      StringTokenizer itr = new StringTokenizer(line);
-      Split newSplit = new Split(itr.nextToken());
-      splits[remainingSplits++] = newSplit;
-      while (itr.hasMoreTokens()) {
-        Host host = hostIds.get(itr.nextToken());
-        newSplit.locations.add(host);
-        host.splits.add(newSplit);
-      }
-    }
-  }
-
-  public TeraScheduler(FileSplit[] realSplits,
-                       Configuration conf) throws IOException {
-    this.realSplits = realSplits;
-    this.slotsPerHost = conf.getInt(TTConfig.TT_MAP_SLOTS, 4);
-    Map<String, Host> hostTable = new HashMap<String, Host>();
-    splits = new Split[realSplits.length];
-    for(FileSplit realSplit: realSplits) {
-      Split split = new Split(realSplit.getPath().toString());
-      splits[remainingSplits++] = split;
-      for(String hostname: realSplit.getLocations()) {
-        Host host = hostTable.get(hostname);
-        if (host == null) {
-          host = new Host(hostname);
-          hostTable.put(hostname, host);
-          hosts.add(host);
-        }
-        host.splits.add(split);
-        split.locations.add(host);
-      }
-    }
-  }
-
-  Host pickBestHost() {
-    Host result = null;
-    int splits = Integer.MAX_VALUE;
-    for(Host host: hosts) {
-      if (host.splits.size() < splits) {
-        result = host;
-        splits = host.splits.size();
-      }
-    }
-    if (result != null) {
-      hosts.remove(result);
-      LOG.debug("picking " + result);
-    }
-    return result;
-  }
-
-  void pickBestSplits(Host host) {
-    int tasksToPick = Math.min(slotsPerHost, 
-                               (int) Math.ceil((double) remainingSplits / 
-                                               hosts.size()));
-    Split[] best = new Split[tasksToPick];
-    for(Split cur: host.splits) {
-      LOG.debug("  examine: " + cur.filename + " " + cur.locations.size());
-      int i = 0;
-      while (i < tasksToPick && best[i] != null && 
-             best[i].locations.size() <= cur.locations.size()) {
-        i += 1;
-      }
-      if (i < tasksToPick) {
-        for(int j = tasksToPick - 1; j > i; --j) {
-          best[j] = best[j-1];
-        }
-        best[i] = cur;
-      }
-    }
-    // for the chosen blocks, remove them from the other locations
-    for(int i=0; i < tasksToPick; ++i) {
-      if (best[i] != null) {
-        LOG.debug(" best: " + best[i].filename);
-        for (Host other: best[i].locations) {
-          other.splits.remove(best[i]);
-        }
-        best[i].locations.clear();
-        best[i].locations.add(host);
-        best[i].isAssigned = true;
-        remainingSplits -= 1;
-      }
-    }
-    // for the non-chosen blocks, remove this host
-    for(Split cur: host.splits) {
-      if (!cur.isAssigned) {
-        cur.locations.remove(host);
-      }
-    }
-  }
-  
-  void solve() throws IOException {
-    Host host = pickBestHost();
-    while (host != null) {
-      pickBestSplits(host);
-      host = pickBestHost();
-    }
-  }
-
-  /**
-   * Solve the schedule and modify the FileSplit array to reflect the new
-   * schedule. It will move placed splits to front and unplacable splits
-   * to the end.
-   * @return a new list of FileSplits that are modified to have the
-   *    best host as the only host.
-   * @throws IOException
-   */
-  public List<InputSplit> getNewFileSplits() throws IOException {
-    solve();
-    FileSplit[] result = new FileSplit[realSplits.length];
-    int left = 0;
-    int right = realSplits.length - 1;
-    for(int i=0; i < splits.length; ++i) {
-      if (splits[i].isAssigned) {
-        // copy the split and fix up the locations
-        ((TeraFileSplit) realSplits[i]).setLocations
-           (new String[]{splits[i].locations.get(0).hostname});
-        result[left++] = realSplits[i];
-      } else {
-        result[right--] = realSplits[i];
-      }
-    }
-    List<InputSplit> ret = new ArrayList<InputSplit>();
-    for (FileSplit fs : result) {
-      ret.add(fs);
-    }
-    return ret;
-  }
-
-  public static void main(String[] args) throws IOException {
-    TeraScheduler problem = new TeraScheduler("block-loc.txt", "nodes");
-    for(Host host: problem.hosts) {
-      System.out.println(host);
-    }
-    LOG.info("starting solve");
-    problem.solve();
-    List<Split> leftOvers = new ArrayList<Split>();
-    for(int i=0; i < problem.splits.length; ++i) {
-      if (problem.splits[i].isAssigned) {
-        System.out.println("sched: " + problem.splits[i]);        
-      } else {
-        leftOvers.add(problem.splits[i]);
-      }
-    }
-    for(Split cur: leftOvers) {
-      System.out.println("left: " + cur);
-    }
-    System.out.println("left over: " + leftOvers.size());
-    LOG.info("done");
-  }
-
-}
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraSort.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraSort.java
deleted file mode 100644
index 5a097f2..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraSort.java
+++ /dev/null
@@ -1,335 +0,0 @@
-/**
- * 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.tez.mapreduce.examples.terasort;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.URI;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.Partitioner;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * Generates the sampled split points, launches the job, and waits for it to
- * finish. 
- * <p>
- * To run the program: 
- * <b>bin/hadoop jar hadoop-*-examples.jar terasort in-dir out-dir</b>
- */
-public class TeraSort extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(TeraSort.class);
-  static String SIMPLE_PARTITIONER = "mapreduce.terasort.simplepartitioner";
-  static String OUTPUT_REPLICATION = "mapreduce.terasort.output.replication";
-
-  /**
-   * A partitioner that splits text keys into roughly equal partitions
-   * in a global sorted order.
-   */
-  static class TotalOrderPartitioner extends Partitioner<Text,Text>
-      implements Configurable {
-    private TrieNode trie;
-    private Text[] splitPoints;
-    private Configuration conf;
-
-    /**
-     * A generic trie node
-     */
-    static abstract class TrieNode {
-      private int level;
-      TrieNode(int level) {
-        this.level = level;
-      }
-      abstract int findPartition(Text key);
-      abstract void print(PrintStream strm) throws IOException;
-      int getLevel() {
-        return level;
-      }
-    }
-
-    /**
-     * An inner trie node that contains 256 children based on the next
-     * character.
-     */
-    static class InnerTrieNode extends TrieNode {
-      private TrieNode[] child = new TrieNode[256];
-      
-      InnerTrieNode(int level) {
-        super(level);
-      }
-      int findPartition(Text key) {
-        int level = getLevel();
-        if (key.getLength() <= level) {
-          return child[0].findPartition(key);
-        }
-        return child[key.getBytes()[level] & 0xff].findPartition(key);
-      }
-      void setChild(int idx, TrieNode child) {
-        this.child[idx] = child;
-      }
-      void print(PrintStream strm) throws IOException {
-        for(int ch=0; ch < 256; ++ch) {
-          for(int i = 0; i < 2*getLevel(); ++i) {
-            strm.print(' ');
-          }
-          strm.print(ch);
-          strm.println(" ->");
-          if (child[ch] != null) {
-            child[ch].print(strm);
-          }
-        }
-      }
-    }
-
-    /**
-     * A leaf trie node that does string compares to figure out where the given
-     * key belongs between lower..upper.
-     */
-    static class LeafTrieNode extends TrieNode {
-      int lower;
-      int upper;
-      Text[] splitPoints;
-      LeafTrieNode(int level, Text[] splitPoints, int lower, int upper) {
-        super(level);
-        this.splitPoints = splitPoints;
-        this.lower = lower;
-        this.upper = upper;
-      }
-      int findPartition(Text key) {
-        for(int i=lower; i<upper; ++i) {
-          if (splitPoints[i].compareTo(key) > 0) {
-            return i;
-          }
-        }
-        return upper;
-      }
-      void print(PrintStream strm) throws IOException {
-        for(int i = 0; i < 2*getLevel(); ++i) {
-          strm.print(' ');
-        }
-        strm.print(lower);
-        strm.print(", ");
-        strm.println(upper);
-      }
-    }
-
-
-    /**
-     * Read the cut points from the given sequence file.
-     * @param fs the file system
-     * @param p the path to read
-     * @param job the job config
-     * @return the strings to split the partitions on
-     * @throws IOException
-     */
-    private static Text[] readPartitions(FileSystem fs, Path p,
-        Configuration conf) throws IOException {
-      int reduces = conf.getInt(MRJobConfig.NUM_REDUCES, 1);
-      Text[] result = new Text[reduces - 1];
-      DataInputStream reader = fs.open(p);
-      for(int i=0; i < reduces - 1; ++i) {
-        result[i] = new Text();
-        result[i].readFields(reader);
-      }
-      reader.close();
-      return result;
-    }
-
-    /**
-     * Given a sorted set of cut points, build a trie that will find the correct
-     * partition quickly.
-     * @param splits the list of cut points
-     * @param lower the lower bound of partitions 0..numPartitions-1
-     * @param upper the upper bound of partitions 0..numPartitions-1
-     * @param prefix the prefix that we have already checked against
-     * @param maxDepth the maximum depth we will build a trie for
-     * @return the trie node that will divide the splits correctly
-     */
-    private static TrieNode buildTrie(Text[] splits, int lower, int upper, 
-                                      Text prefix, int maxDepth) {
-      int depth = prefix.getLength();
-      if (depth >= maxDepth || lower == upper) {
-        return new LeafTrieNode(depth, splits, lower, upper);
-      }
-      InnerTrieNode result = new InnerTrieNode(depth);
-      Text trial = new Text(prefix);
-      // append an extra byte on to the prefix
-      trial.append(new byte[1], 0, 1);
-      int currentBound = lower;
-      for(int ch = 0; ch < 255; ++ch) {
-        trial.getBytes()[depth] = (byte) (ch + 1);
-        lower = currentBound;
-        while (currentBound < upper) {
-          if (splits[currentBound].compareTo(trial) >= 0) {
-            break;
-          }
-          currentBound += 1;
-        }
-        trial.getBytes()[depth] = (byte) ch;
-        result.child[ch] = buildTrie(splits, lower, currentBound, trial, 
-                                     maxDepth);
-      }
-      // pick up the rest
-      trial.getBytes()[depth] = (byte) 255;
-      result.child[255] = buildTrie(splits, currentBound, upper, trial,
-                                    maxDepth);
-      return result;
-    }
-
-    public void setConf(Configuration conf) {
-      try {
-        FileSystem fs = FileSystem.getLocal(conf);
-        this.conf = conf;
-        Path partFile = new Path(TeraInputFormat.PARTITION_FILENAME);
-        splitPoints = readPartitions(fs, partFile, conf);
-        trie = buildTrie(splitPoints, 0, splitPoints.length, new Text(), 2);
-      } catch (IOException ie) {
-        throw new IllegalArgumentException("can't read paritions file", ie);
-      }
-    }
-
-    public Configuration getConf() {
-      return conf;
-    }
-    
-    public TotalOrderPartitioner() {
-    }
-
-    public int getPartition(Text key, Text value, int numPartitions) {
-      return trie.findPartition(key);
-    }
-    
-  }
-  
-  /**
-   * A total order partitioner that assigns keys based on their first 
-   * PREFIX_LENGTH bytes, assuming a flat distribution.
-   */
-  public static class SimplePartitioner extends Partitioner<Text, Text>
-      implements Configurable {
-    int prefixesPerReduce;
-    private static final int PREFIX_LENGTH = 3;
-    private Configuration conf = null;
-    public void setConf(Configuration conf) {
-      this.conf = conf;
-      prefixesPerReduce = (int) Math.ceil((1 << (8 * PREFIX_LENGTH)) / 
-        (float) conf.getInt(MRJobConfig.NUM_REDUCES, 1));
-    }
-    
-    public Configuration getConf() {
-      return conf;
-    }
-    
-    @Override
-    public int getPartition(Text key, Text value, int numPartitions) {
-      byte[] bytes = key.getBytes();
-      int len = Math.min(PREFIX_LENGTH, key.getLength());
-      int prefix = 0;
-      for(int i=0; i < len; ++i) {
-        prefix = (prefix << 8) | (0xff & bytes[i]);
-      }
-      return prefix / prefixesPerReduce;
-    }
-  }
-
-  public static boolean getUseSimplePartitioner(JobContext job) {
-    return job.getConfiguration().getBoolean(SIMPLE_PARTITIONER, false);
-  }
-
-  public static void setUseSimplePartitioner(Job job, boolean value) {
-    job.getConfiguration().setBoolean(SIMPLE_PARTITIONER, value);
-  }
-
-  public static int getOutputReplication(JobContext job) {
-    return job.getConfiguration().getInt(OUTPUT_REPLICATION, 1);
-  }
-
-  public static void setOutputReplication(Job job, int value) {
-    job.getConfiguration().setInt(OUTPUT_REPLICATION, value);
-  }
-
-  public int run(String[] args) throws Exception {
-    if (args.length != 2) {
-      System.err.println("Invalid no. of arguments provided");
-      System.err.println("Usage: terasort <input-dir> <output-dir>");
-      return -1;
-    }
-
-    LOG.info("starting");
-    Job job = Job.getInstance(getConf());
-    Path inputDir = new Path(args[0]);
-    Path outputDir = new Path(args[1]);
-    boolean useSimplePartitioner = getUseSimplePartitioner(job);
-    TeraInputFormat.setInputPaths(job, inputDir);
-    FileOutputFormat.setOutputPath(job, outputDir);
-    job.setJobName("TeraSort");
-    job.setJarByClass(TeraSort.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    job.setInputFormatClass(TeraInputFormat.class);
-    job.setOutputFormatClass(TeraOutputFormat.class);
-    if (useSimplePartitioner) {
-      job.setPartitionerClass(SimplePartitioner.class);
-    } else {
-      long start = System.currentTimeMillis();
-      Path partitionFile = new Path(outputDir, 
-                                    TeraInputFormat.PARTITION_FILENAME);
-      URI partitionUri = new URI(partitionFile.toString() +
-                                 "#" + TeraInputFormat.PARTITION_FILENAME);
-      try {
-        TeraInputFormat.writePartitionFile(job, partitionFile);
-      } catch (Throwable e) {
-        LOG.error(e.getMessage());
-        return -1;
-      }
-      job.addCacheFile(partitionUri);  
-      long end = System.currentTimeMillis();
-      System.out.println("Spent " + (end - start) + "ms computing partitions.");
-      job.setPartitionerClass(TotalOrderPartitioner.class);
-    }
-    
-    job.getConfiguration().setInt("dfs.replication", getOutputReplication(job));
-    TeraOutputFormat.setFinalSync(job, true);
-    int ret = job.waitForCompletion(true) ? 0 : 1;
-    LOG.info("done");
-    return ret;
-  }
-
-  /**
-   * @param args
-   */
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new TeraSort(), args);
-    System.exit(res);
-  }
-
-}
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraValidate.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraValidate.java
deleted file mode 100644
index 24c3145..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraValidate.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/**
- * 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.tez.mapreduce.examples.terasort;
-
-import java.io.IOException;
-import java.util.zip.Checksum;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * Generate 1 mapper per a file that checks to make sure the keys
- * are sorted within each file. The mapper also generates 
- * "$file:begin", first key and "$file:end", last key. The reduce verifies that
- * all of the start/end items are in order.
- * Any output from the reduce is problem report.
- * <p>
- * To run the program: 
- * <b>bin/hadoop jar hadoop-*-examples.jar teravalidate out-dir report-dir</b>
- * <p>
- * If there is any output, something is wrong and the output of the reduce
- * will have the problem report.
- */
-public class TeraValidate extends Configured implements Tool {
-  private static final Text ERROR = new Text("error");
-  private static final Text CHECKSUM = new Text("checksum");
-  
-  private static String textifyBytes(Text t) {
-    BytesWritable b = new BytesWritable();
-    b.set(t.getBytes(), 0, t.getLength());
-    return b.toString();
-  }
-
-  static class ValidateMapper extends Mapper<Text,Text,Text,Text> {
-    private Text lastKey;
-    private String filename;
-    private Unsigned16 checksum = new Unsigned16();
-    private Unsigned16 tmp = new Unsigned16();
-    private Checksum crc32 = new PureJavaCrc32();
-
-    /**
-     * Get the final part of the input name
-     * @param split the input split
-     * @return the "part-r-00000" for the input
-     */
-    private String getFilename(FileSplit split) {
-      return split.getPath().getName();
-    }
-
-    public void map(Text key, Text value, Context context) 
-        throws IOException, InterruptedException {
-      if (lastKey == null) {
-        FileSplit fs = (FileSplit) context.getInputSplit();
-        filename = getFilename(fs);
-        context.write(new Text(filename + ":begin"), key);
-        lastKey = new Text();
-      } else {
-        if (key.compareTo(lastKey) < 0) {
-          context.write(ERROR, new Text("misorder in " + filename + 
-                                         " between " + textifyBytes(lastKey) + 
-                                         " and " + textifyBytes(key)));
-        }
-      }
-      // compute the crc of the key and value and add it to the sum
-      crc32.reset();
-      crc32.update(key.getBytes(), 0, key.getLength());
-      crc32.update(value.getBytes(), 0, value.getLength());
-      tmp.set(crc32.getValue());
-      checksum.add(tmp);
-      lastKey.set(key);
-    }
-    
-    public void cleanup(Context context) 
-        throws IOException, InterruptedException  {
-      if (lastKey != null) {
-        context.write(new Text(filename + ":end"), lastKey);
-        context.write(CHECKSUM, new Text(checksum.toString()));
-      }
-    }
-  }
-
-  /**
-   * Check the boundaries between the output files by making sure that the
-   * boundary keys are always increasing.
-   * Also passes any error reports along intact.
-   */
-  static class ValidateReducer extends Reducer<Text,Text,Text,Text> {
-    private boolean firstKey = true;
-    private Text lastKey = new Text();
-    private Text lastValue = new Text();
-    public void reduce(Text key, Iterable<Text> values,
-        Context context) throws IOException, InterruptedException  {
-      if (ERROR.equals(key)) {
-        for (Text val : values) {
-          context.write(key, val);
-        }
-      } else if (CHECKSUM.equals(key)) {
-        Unsigned16 tmp = new Unsigned16();
-        Unsigned16 sum = new Unsigned16();
-        for (Text val : values) {
-          tmp.set(val.toString());
-          sum.add(tmp);
-        }
-        context.write(CHECKSUM, new Text(sum.toString()));
-      } else {
-        Text value = values.iterator().next();
-        if (firstKey) {
-          firstKey = false;
-        } else {
-          if (value.compareTo(lastValue) < 0) {
-            context.write(ERROR, 
-                           new Text("bad key partitioning:\n  file " + 
-                                    lastKey + " key " + 
-                                    textifyBytes(lastValue) +
-                                    "\n  file " + key + " key " + 
-                                    textifyBytes(value)));
-          }
-        }
-        lastKey.set(key);
-        lastValue.set(value);
-      }
-    }
-    
-  }
-
-  private static void usage() throws IOException {
-    System.err.println("teravalidate <out-dir> <report-dir>");
-  }
-
-  public int run(String[] args) throws Exception {
-    Job job = Job.getInstance(getConf());
-    if (args.length != 2) {
-      usage();
-      return 1;
-    }
-    TeraInputFormat.setInputPaths(job, new Path(args[0]));
-    FileOutputFormat.setOutputPath(job, new Path(args[1]));
-    job.setJobName("TeraValidate");
-    job.setJarByClass(TeraValidate.class);
-    job.setMapperClass(ValidateMapper.class);
-    job.setReducerClass(ValidateReducer.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    // force a single reducer
-    job.setNumReduceTasks(1);
-    // force a single split 
-    FileInputFormat.setMinInputSplitSize(job, Long.MAX_VALUE);
-    job.setInputFormatClass(TeraInputFormat.class);
-    return job.waitForCompletion(true) ? 0 : 1;
-  }
-
-  /**
-   * @param args
-   */
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new TeraValidate(), args);
-    System.exit(res);
-  }
-
-}
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Unsigned16.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Unsigned16.java
deleted file mode 100644
index 1335bf4..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Unsigned16.java
+++ /dev/null
@@ -1,297 +0,0 @@
-/**
- * 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.tez.mapreduce.examples.terasort;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-
-/**
- * An unsigned 16 byte integer class that supports addition, multiplication,
- * and left shifts.
- */
-class Unsigned16 implements Writable {
-  private long hi8;
-  private long lo8;
-
-  public Unsigned16() {
-    hi8 = 0;
-    lo8 = 0;
-  }
-
-  public Unsigned16(long l) {
-    hi8 = 0;
-    lo8 = l;
-  }
-
-  public Unsigned16(Unsigned16 other) {
-    hi8 = other.hi8;
-    lo8 = other.lo8;
-  }
-  
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof Unsigned16) {
-      Unsigned16 other = (Unsigned16) o;
-      return other.hi8 == hi8 && other.lo8 == lo8;
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return (int) lo8;
-  }
-
-  /**
-   * Parse a hex string
-   * @param s the hex string
-   */
-  public Unsigned16(String s) throws NumberFormatException {
-    set(s);
-  }
-
-  /**
-   * Set the number from a hex string
-   * @param s the number in hexadecimal
-   * @throws NumberFormatException if the number is invalid
-   */
-  public void set(String s) throws NumberFormatException {
-    hi8 = 0;
-    lo8 = 0;
-    final long lastDigit = 0xfl << 60;
-    for (int i = 0; i < s.length(); ++i) {
-      int digit = getHexDigit(s.charAt(i));
-      if ((lastDigit & hi8) != 0) {
-        throw new NumberFormatException(s + " overflowed 16 bytes");
-      }
-      hi8 <<= 4;
-      hi8 |= (lo8 & lastDigit) >>> 60;
-      lo8 <<= 4;
-      lo8 |= digit;
-    }    
-  }
-
-  /**
-   * Set the number to a given long.
-   * @param l the new value, which is treated as an unsigned number
-   */
-  public void set(long l) {
-    lo8 = l;
-    hi8 = 0;
-  }
-
-  /**
-   * Map a hexadecimal character into a digit.
-   * @param ch the character
-   * @return the digit from 0 to 15
-   * @throws NumberFormatException
-   */
-  private static int getHexDigit(char ch) throws NumberFormatException {
-    if (ch >= '0' && ch <= '9') {
-      return ch - '0';
-    }
-    if (ch >= 'a' && ch <= 'f') {
-      return ch - 'a' + 10;
-    }
-    if (ch >= 'A' && ch <= 'F') {
-      return ch - 'A' + 10;
-    }
-    throw new NumberFormatException(ch + " is not a valid hex digit");
-  }
-
-  private static final Unsigned16 TEN = new Unsigned16(10);
-
-  public static Unsigned16 fromDecimal(String s) throws NumberFormatException {
-    Unsigned16 result = new Unsigned16();
-    Unsigned16 tmp = new Unsigned16();
-    for(int i=0; i < s.length(); i++) {
-      char ch = s.charAt(i);
-      if (ch < '0' || ch > '9') {
-        throw new NumberFormatException(ch + " not a valid decimal digit");
-      }
-      int digit = ch - '0';
-      result.multiply(TEN);
-      tmp.set(digit);
-      result.add(tmp);
-    }
-    return result;
-  }
-
-  /**
-   * Return the number as a hex string.
-   */
-  public String toString() {
-    if (hi8 == 0) {
-      return Long.toHexString(lo8);
-    } else {
-      StringBuilder result = new StringBuilder();
-      result.append(Long.toHexString(hi8));
-      String loString = Long.toHexString(lo8);
-      for(int i=loString.length(); i < 16; ++i) {
-        result.append('0');
-      }
-      result.append(loString);
-      return result.toString();
-    }
-  }
-
-  /**
-   * Get a given byte from the number.
-   * @param b the byte to get with 0 meaning the most significant byte
-   * @return the byte or 0 if b is outside of 0..15
-   */
-  public byte getByte(int b) {
-    if (b >= 0 && b < 16) {
-      if (b < 8) {
-        return (byte) (hi8 >> (56 - 8*b));
-      } else {
-        return (byte) (lo8 >> (120 - 8*b));
-      }
-    }
-    return 0;
-  }
-
-  /**
-   * Get the hexadecimal digit at the given position.
-   * @param p the digit position to get with 0 meaning the most significant
-   * @return the character or '0' if p is outside of 0..31
-   */
-  public char getHexDigit(int p) {
-    byte digit = getByte(p / 2);
-    if (p % 2 == 0) {
-      digit >>>= 4;
-    }
-    digit &= 0xf;
-    if (digit < 10) {
-      return (char) ('0' + digit);
-    } else {
-      return (char) ('A' + digit - 10);
-    }
-  }
-
-  /**
-   * Get the high 8 bytes as a long.
-   */
-  public long getHigh8() {
-    return hi8;
-  }
-  
-  /**
-   * Get the low 8 bytes as a long.
-   */
-  public long getLow8() {
-    return lo8;
-  }
-
-  /**
-   * Multiple the current number by a 16 byte unsigned integer. Overflow is not
-   * detected and the result is the low 16 bytes of the result. The numbers 
-   * are divided into 32 and 31 bit chunks so that the product of two chucks
-   * fits in the unsigned 63 bits of a long.
-   * @param b the other number
-   */
-  void multiply(Unsigned16 b) {
-    // divide the left into 4 32 bit chunks
-    long[] left = new long[4];
-    left[0] = lo8 & 0xffffffffl;
-    left[1] = lo8 >>> 32;
-    left[2] = hi8 & 0xffffffffl;
-    left[3] = hi8 >>> 32;
-    // divide the right into 5 31 bit chunks
-    long[] right = new long[5];
-    right[0] = b.lo8 & 0x7fffffffl;
-    right[1] = (b.lo8 >>> 31) & 0x7fffffffl;
-    right[2] = (b.lo8 >>> 62) + ((b.hi8 & 0x1fffffffl) << 2);
-    right[3] = (b.hi8 >>> 29) & 0x7fffffffl;
-    right[4] = (b.hi8 >>> 60);
-    // clear the cur value
-    set(0);
-    Unsigned16 tmp = new Unsigned16();
-    for(int l=0; l < 4; ++l) {
-      for (int r=0; r < 5; ++r) {
-        long prod = left[l] * right[r];
-        if (prod != 0) {
-          int off = l*32 + r*31;
-          tmp.set(prod);
-          tmp.shiftLeft(off);
-          add(tmp);
-        }
-      }
-    }
-  }
-
-  /**
-   * Add the given number into the current number.
-   * @param b the other number
-   */
-  public void add(Unsigned16 b) {
-    long sumHi;
-    long sumLo;
-    long  reshibit, hibit0, hibit1;
-
-    sumHi = hi8 + b.hi8;
-
-    hibit0 = (lo8 & 0x8000000000000000L);
-    hibit1 = (b.lo8 & 0x8000000000000000L);
-    sumLo = lo8 + b.lo8;
-    reshibit = (sumLo & 0x8000000000000000L);
-    if ((hibit0 & hibit1) != 0 | ((hibit0 ^ hibit1) != 0 && reshibit == 0))
-      sumHi++;  /* add carry bit */
-    hi8 = sumHi;
-    lo8 = sumLo;
-  }
-
-  /**
-   * Shift the number a given number of bit positions. The number is the low
-   * order bits of the result.
-   * @param bits the bit positions to shift by
-   */
-  public void shiftLeft(int bits) {
-    if (bits != 0) {
-      if (bits < 64) {
-        hi8 <<= bits;
-        hi8 |= (lo8 >>> (64 - bits));
-        lo8 <<= bits;
-      } else if (bits < 128) {
-        hi8 = lo8 << (bits - 64);
-        lo8 = 0;
-      } else {
-        hi8 = 0;
-        lo8 = 0;
-      }
-    }
-  }
-  
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    hi8 = in.readLong();
-    lo8 = in.readLong();
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(hi8);
-    out.writeLong(lo8);
-  }
-  
-  
-}
diff --git a/tez-tests/src/test/java/org/apache/tez/mapreduce/examples/terasort/TestTeraSort.java b/tez-tests/src/test/java/org/apache/tez/mapreduce/examples/terasort/TestTeraSort.java
deleted file mode 100644
index 97b6538..0000000
--- a/tez-tests/src/test/java/org/apache/tez/mapreduce/examples/terasort/TestTeraSort.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/**
- * 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.tez.mapreduce.examples.terasort;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.tez.mapreduce.examples.terasort.TeraGen;
-import org.apache.tez.mapreduce.examples.terasort.TeraSort;
-import org.apache.tez.mapreduce.examples.terasort.TeraValidate;
-
-import org.junit.Ignore;
-
-@Ignore
-public class TestTeraSort extends HadoopTestCase {
-  
-  public TestTeraSort()
-      throws IOException {
-    super(CLUSTER_MR, DFS_FS, 1, 1);
-  }
-
-  protected void tearDown() throws Exception {
-    getFileSystem().delete(new Path(TEST_DIR), true);
-    super.tearDown();
-  }
-  
-  // Input/Output paths for sort
-  private static final String TEST_DIR = 
-    new File(System.getProperty("test.build.data", "/tmp"), "org/apache/tez/mapreduce/examples/terasort")
-    .getAbsolutePath();
-  private static final Path SORT_INPUT_PATH = new Path(TEST_DIR, "sortin");
-  private static final Path SORT_OUTPUT_PATH = new Path(TEST_DIR, "sortout");
-  private static final Path TERA_OUTPUT_PATH = new Path(TEST_DIR, "validate");
-  private static final String NUM_ROWS = "100"; 
-
-  private void runTeraGen(Configuration conf, Path sortInput) 
-      throws Exception {
-    String[] genArgs = {NUM_ROWS, sortInput.toString()};
-    
-    // Run TeraGen
-    assertEquals(ToolRunner.run(conf, new TeraGen(), genArgs), 0);
-  }
-  
-  private void runTeraSort(Configuration conf,
-      Path sortInput, Path sortOutput) throws Exception {
-
-    // Setup command-line arguments to 'sort'
-    String[] sortArgs = {sortInput.toString(), sortOutput.toString()};
-    
-    // Run Sort
-    assertEquals(ToolRunner.run(conf, new TeraSort(), sortArgs), 0);
-  }
-  
-  private void runTeraValidator(Configuration job, 
-                                       Path sortOutput, Path valOutput) 
-  throws Exception {
-    String[] svArgs = {sortOutput.toString(), valOutput.toString()};
-
-    // Run Tera-Validator
-    assertEquals(ToolRunner.run(job, new TeraValidate(), svArgs), 0);
-  }
-  
-  public void testTeraSort() throws Exception {
-    // Run TeraGen to generate input for 'terasort'
-    runTeraGen(createJobConf(), SORT_INPUT_PATH);
-
-    // Run terasort
-    runTeraSort(createJobConf(), SORT_INPUT_PATH, SORT_OUTPUT_PATH);
-
-    // Run tera-validator to check if sort worked correctly
-    runTeraValidator(createJobConf(), SORT_OUTPUT_PATH,
-      TERA_OUTPUT_PATH);
-  }
-
-}
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java
new file mode 100644
index 0000000..e79d562
--- /dev/null
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java
@@ -0,0 +1,708 @@
+/**
+ * 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.tez.test;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.client.TezClientUtils;
+import org.apache.tez.common.TezCommonUtils;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.DAGCounter;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.Edge;
+import org.apache.tez.dag.api.EdgeProperty;
+import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
+import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
+import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.VertexManagerPluginContext;
+import org.apache.tez.dag.api.VertexManagerPluginDescriptor;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.dag.api.client.StatusGetOpts;
+import org.apache.tez.dag.app.RecoveryParser;
+import org.apache.tez.dag.app.dag.impl.ImmediateStartVertexManager;
+import org.apache.tez.dag.history.HistoryEvent;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
+import org.apache.tez.dag.history.recovery.RecoveryService;
+import org.apache.tez.dag.library.vertexmanager.InputReadyVertexManager;
+import org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager;
+import org.apache.tez.runtime.api.ProcessorContext;
+import org.apache.tez.runtime.library.processor.SimpleProcessor;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestAMRecovery {
+
+  private static final Log LOG = LogFactory.getLog(TestAMRecovery.class);
+
+  private static Configuration conf = new Configuration();
+  private static TezConfiguration tezConf;
+  private static int MAX_AM_ATTEMPT = 10;
+  private static MiniTezCluster miniTezCluster = null;
+  private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR
+      + TestAMRecovery.class.getName() + "-tmpDir";
+  private static MiniDFSCluster dfsCluster = null;
+  private static TezClient tezSession = null;
+  private static FileSystem remoteFs = null;
+  private static String FAIL_ON_PARTIAL_FINISHED = "FAIL_ON_PARTIAL_COMPLETED";
+  private static String FAIL_ON_ATTEMPT = "FAIL_ON_ATTEMPT";
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    LOG.info("Starting mini clusters");
+    try {
+      conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR);
+      dfsCluster =
+          new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true)
+              .racks(null).build();
+      remoteFs = dfsCluster.getFileSystem();
+    } catch (IOException io) {
+      throw new RuntimeException("problem starting mini dfs cluster", io);
+    }
+    if (miniTezCluster == null) {
+      miniTezCluster =
+          new MiniTezCluster(TestAMRecovery.class.getName(), 1, 1, 1);
+      Configuration miniTezconf = new Configuration(conf);
+      miniTezconf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, MAX_AM_ATTEMPT);
+      miniTezconf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS
+      miniTezCluster.init(miniTezconf);
+      miniTezCluster.start();
+    }
+  }
+
+  @AfterClass
+  public static void afterClass() throws InterruptedException {
+    if (tezSession != null) {
+      try {
+        LOG.info("Stopping Tez Session");
+        tezSession.stop();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+    Thread.sleep(10000);
+    if (miniTezCluster != null) {
+      try {
+        LOG.info("Stopping MiniTezCluster");
+        miniTezCluster.stop();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+    if (dfsCluster != null) {
+      try {
+        LOG.info("Stopping DFSCluster");
+        dfsCluster.shutdown();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+  }
+
+  @Before
+  public void setup() throws Exception {
+    LOG.info("Starting session");
+    Path remoteStagingDir =
+        remoteFs.makeQualified(new Path(TEST_ROOT_DIR, String
+            .valueOf(new Random().nextInt(100000))));
+    TezClientUtils.ensureStagingDirExists(conf, remoteStagingDir);
+
+    tezConf = new TezConfiguration(miniTezCluster.getConfig());
+    tezConf.setInt(TezConfiguration.DAG_RECOVERY_MAX_UNFLUSHED_EVENTS, 0);
+    tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "INFO");
+    tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR,
+        remoteStagingDir.toString());
+    tezConf
+        .setBoolean(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_ENABLED, false);
+    tezConf.setInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, MAX_AM_ATTEMPT);
+    tezConf.setInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB, 500);
+    tezConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, " -Xmx256m");
+    tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
+    tezConf.setBoolean(
+        TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, false);
+    tezConf.setBoolean(
+        RecoveryService.TEZ_AM_RECOVERY_HANDLE_REMAINING_EVENT_WHEN_STOPPED,
+        true);
+    tezSession = TezClient.create("TestDAGRecovery", tezConf);
+    tezSession.start();
+  }
+
+  @After
+  public void teardown() throws InterruptedException {
+    if (tezSession != null) {
+      try {
+        LOG.info("Stopping Tez Session");
+        tezSession.stop();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+    tezSession = null;
+  }
+
+  /**
+   * Fine-grained recovery task-level, In a vertex (v1), task 0 is done task 1
+   * is running. History flush happens. AM dies. Once AM is recovered, task 0 is
+   * not re-run. Task 1 is re-run. (Broadcast)
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 120000)
+  public void testVertexPartiallyFinished_Broadcast() throws Exception {
+    DAG dag =
+        createDAG(ControlledInputReadyVertexManager.class,
+            DataMovementType.BROADCAST, true);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(5, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(1, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
+
+    List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
+    List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
+
+    // task_0 of v1 is finished in attempt 1, task_1 of v1 is not finished in
+    // attempt 1
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents1, 0, 0).size());
+    assertEquals(0, findTaskAttemptFinishedEvent(historyEvents1, 0, 1).size());
+
+    // task_0 of v1 is finished in attempt 1 and not rerun, task_1 of v1 is
+    // finished in attempt 2
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 0).size());
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 1).size());
+  }
+
+  /**
+   * Fine-grained recovery task-level, In a vertex (v1), task 0 is done task 1
+   * is also done. History flush happens. AM dies. Once AM is recovered, task 0
+   * and Task 1 is not re-run. (Broadcast)
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 120000)
+  public void testVertexCompletelyFinished_Broadcast() throws Exception {
+    DAG dag =
+        createDAG(ControlledInputReadyVertexManager.class,
+            DataMovementType.BROADCAST, false);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(4, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(0, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
+
+    List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
+    List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
+
+    // task_0 of v1 is finished in attempt 1, task_1 of v1 is not finished in
+    // attempt 1
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents1, 0, 0).size());
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents1, 0, 1).size());
+
+    // task_0 of v1 is finished in attempt 1 and not rerun, task_1 of v1 is
+    // finished in attempt 2
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 0).size());
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 1).size());
+  }
+
+  /**
+   * Fine-grained recovery task-level, In a vertex (v1), task 0 is done task 1
+   * is running. History flush happens. AM dies. Once AM is recovered, task 0 is
+   * not re-run. Task 1 is re-run. (ONE_TO_ONE)
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 120000)
+  public void testVertexPartialFinished_One2One() throws Exception {
+    DAG dag =
+        createDAG(ControlledInputReadyVertexManager.class,
+            DataMovementType.ONE_TO_ONE, true);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(5, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(1, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
+
+    List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
+    List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
+
+    // task_0 of v1 is finished in attempt 1, task_1 of v1 is not finished in
+    // attempt 1
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents1, 0, 0).size());
+    assertEquals(0, findTaskAttemptFinishedEvent(historyEvents1, 0, 1).size());
+
+    // task_0 of v1 is finished in attempt 1 and not rerun, task_1 of v1 is
+    // finished in attempt 2
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 0).size());
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 1).size());
+
+  }
+
+  /**
+   * Fine-grained recovery task-level, In a vertex (v1), task 0 is done task 1
+   * is also done. History flush happens. AM dies. Once AM is recovered, task 0
+   * and Task 1 is not re-run. (ONE_TO_ONE)
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 120000)
+  public void testVertexCompletelyFinished_One2One() throws Exception {
+    DAG dag =
+        createDAG(ControlledInputReadyVertexManager.class,
+            DataMovementType.ONE_TO_ONE, false);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(4, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(0, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
+
+    List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
+    List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
+
+    // task_0 of v1 is finished in attempt 1, task_1 of v1 is not finished in
+    // attempt 1
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents1, 0, 0).size());
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents1, 0, 1).size());
+
+    // task_0 of v1 is finished in attempt 1 and not rerun, task_1 of v1 is
+    // finished in attempt 2
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 0).size());
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 1).size());
+
+  }
+
+  /**
+   * Fine-grained recovery task-level, In a vertex (v1), task 0 is done task 1
+   * is running. History flush happens. AM dies. Once AM is recovered, task 0 is
+   * not re-run. Task 1 is re-run. (SCATTER_GATHER)
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 120000)
+  public void testVertexPartiallyFinished_ScatterGather() throws Exception {
+    DAG dag =
+        createDAG(ControlledShuffleVertexManager.class,
+            DataMovementType.SCATTER_GATHER, true);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(5, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(1, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
+
+    List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
+    List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
+
+    // task_0 of v1 is finished in attempt 1, task_1 of v1 is not finished in
+    // attempt 1
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents1, 0, 0).size());
+    assertEquals(0, findTaskAttemptFinishedEvent(historyEvents1, 0, 1).size());
+
+    // task_0 of v1 is finished in attempt 1 and not rerun, task_1 of v1 is
+    // finished in attempt 2
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 0).size());
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 1).size());
+
+  }
+
+  /**
+   * Fine-grained recovery task-level, In a vertex (v1), task 0 is done task 1
+   * is also done. History flush happens. AM dies. Once AM is recovered, task 0
+   * and Task 1 is not re-run. (SCATTER_GATHER)
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 120000)
+  public void testVertexCompletelyFinished_ScatterGather() throws Exception {
+    DAG dag =
+        createDAG(ControlledShuffleVertexManager.class,
+            DataMovementType.SCATTER_GATHER, false);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(4, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(0, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
+
+    List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
+    List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
+
+    // task_0 of v1 is finished in attempt 1, task_1 of v1 is not finished in
+    // attempt 1
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents1, 0, 0).size());
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents1, 0, 1).size());
+
+    // task_0 of v1 is finished in attempt 1 and not rerun, task_1 of v1 is
+    // finished in attempt 2
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 0).size());
+    assertEquals(1, findTaskAttemptFinishedEvent(historyEvents2, 0, 1).size());
+  }
+
+  /**
+   * Set AM max attempt to high number. Kill many attempts. Last AM can still be
+   * recovered with latest AM history data.
+   *
+   * @throws Exception
+   */
+  @Test(timeout = 600000)
+  public void testHighMaxAttempt() throws Exception {
+    Random rand = new Random();
+    tezConf.set(FAIL_ON_ATTEMPT, rand.nextInt(MAX_AM_ATTEMPT) + "");
+    LOG.info("Set FAIL_ON_ATTEMPT=" + tezConf.get(FAIL_ON_ATTEMPT));
+    DAG dag =
+        createDAG(FailOnAttemptVertexManager.class,
+            DataMovementType.SCATTER_GATHER, false);
+    runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+
+  }
+
+  TezCounters runDAGAndVerify(DAG dag, DAGStatus.State finalState) throws Exception {
+    tezSession.waitTillReady();
+    DAGClient dagClient = tezSession.submitDAG(dag);
+    DAGStatus dagStatus =
+        dagClient.waitForCompletionWithStatusUpdates(EnumSet
+            .of(StatusGetOpts.GET_COUNTERS));
+    Assert.assertEquals(finalState, dagStatus.getState());
+    return dagStatus.getDAGCounters();
+  }
+
+  /**
+   * v1 --> v2 <br>
+   * v2 has a customized VM which could control when to kill AM
+   *
+   * @param vertexManagerClass
+   * @param dmType
+   * @param failOnParitialCompleted
+   * @return
+   * @throws IOException
+   */
+  private DAG createDAG(Class vertexManagerClass, DataMovementType dmType,
+      boolean failOnParitialCompleted) throws IOException {
+    if (failOnParitialCompleted) {
+      tezConf.set(FAIL_ON_PARTIAL_FINISHED, "true");
+    } else {
+      tezConf.set(FAIL_ON_PARTIAL_FINISHED, "false");
+    }
+    DAG dag = DAG.create("dag");
+    UserPayload payload = UserPayload.create(null);
+    Vertex v1 = Vertex.create("v1", MyProcessor.getProcDesc(), 2);
+    Vertex v2 = Vertex.create("v2", DoNothingProcessor.getProcDesc(), 2);
+    v2.setVertexManagerPlugin(VertexManagerPluginDescriptor.create(
+        vertexManagerClass.getName()).setUserPayload(
+        TezUtils.createUserPayloadFromConf(tezConf)));
+
+    dag.addVertex(v1).addVertex(v2);
+    dag.addEdge(Edge.create(v1, v2, EdgeProperty.create(dmType,
+        DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL,
+        TestOutput.getOutputDesc(payload), TestInput.getInputDesc(payload))));
+    return dag;
+  }
+
+  private List<TaskAttemptFinishedEvent> findTaskAttemptFinishedEvent(
+      List<HistoryEvent> historyEvents, int vertexId, int taskId) {
+    List<TaskAttemptFinishedEvent> resultEvents =
+        new ArrayList<TaskAttemptFinishedEvent>();
+    for (HistoryEvent historyEvent : historyEvents) {
+      if (historyEvent.getEventType() == HistoryEventType.TASK_ATTEMPT_FINISHED) {
+        TaskAttemptFinishedEvent taFinishedEvent =
+            (TaskAttemptFinishedEvent) historyEvent;
+        if (taFinishedEvent.getTaskAttemptID().getTaskID().getVertexID()
+            .getId() == vertexId
+            && taFinishedEvent.getTaskAttemptID().getTaskID().getId() == taskId) {
+          resultEvents.add(taFinishedEvent);
+        }
+      }
+    }
+    return resultEvents;
+  }
+
+  private List<HistoryEvent> readRecoveryLog(int attemptNum) throws IOException {
+    ApplicationId appId = tezSession.getAppMasterApplicationId();
+    Path tezSystemStagingDir =
+        TezCommonUtils.getTezSystemStagingPath(tezConf, appId.toString());
+    Path recoveryDataDir =
+        TezCommonUtils.getRecoveryPath(tezSystemStagingDir, tezConf);
+    FileSystem fs = tezSystemStagingDir.getFileSystem(tezConf);
+    Path currentAttemptRecoveryDataDir =
+        TezCommonUtils.getAttemptRecoveryPath(recoveryDataDir, attemptNum);
+    Path recoveryFilePath =
+        new Path(currentAttemptRecoveryDataDir, appId.toString().replace(
+            "application", "dag")
+            + "_1" + TezConstants.DAG_RECOVERY_RECOVER_FILE_SUFFIX);
+    return RecoveryParser.parseDAGRecoveryFile(fs.open(recoveryFilePath));
+  }
+
+  public static class ControlledInputReadyVertexManager extends
+      InputReadyVertexManager {
+
+    private Configuration conf;
+
+    public ControlledInputReadyVertexManager(VertexManagerPluginContext context) {
+      super(context);
+    }
+
+    @Override
+    public void initialize() {
+      super.initialize();
+      try {
+        conf =
+            TezUtils.createConfFromUserPayload(getContext().getUserPayload());
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+    }
+
+    @Override
+    public void onSourceTaskCompleted(String srcVertexName, Integer taskId) {
+      super.onSourceTaskCompleted(srcVertexName, taskId);
+      if (getContext().getDAGAttemptNumber() == 1) {
+        if (conf.getBoolean(FAIL_ON_PARTIAL_FINISHED, true)) {
+          if (taskId == 0) {
+            System.exit(-1);
+          }
+        } else {
+          if (taskId == 1) {
+            System.exit(-1);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void onVertexStarted(Map<String, List<Integer>> completions) {
+      // sleep for 1 seconds to delay the running of task in v2.
+      // this could keep the case that task of v1 is partial finished or completely
+      // finished, and at the same time the task of v2 is not started
+      try {
+        Thread.sleep(1*1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      super.onVertexStarted(completions);
+    }
+  }
+
+  public static class ControlledShuffleVertexManager extends
+      ShuffleVertexManager {
+
+    private Configuration conf;
+
+    public ControlledShuffleVertexManager(VertexManagerPluginContext context) {
+      super(context);
+    }
+
+    @Override
+    public void initialize() {
+      super.initialize();
+      try {
+        conf =
+            TezUtils.createConfFromUserPayload(getContext().getUserPayload());
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+    }
+
+    @Override
+    public void onSourceTaskCompleted(String srcVertexName, Integer taskId) {
+      super.onSourceTaskCompleted(srcVertexName, taskId);
+      if (getContext().getDAGAttemptNumber() == 1) {
+        if (conf.getBoolean(FAIL_ON_PARTIAL_FINISHED, true)) {
+          if (taskId == 0) {
+            System.exit(-1);
+          }
+        } else {
+          if (taskId == 1) {
+            System.exit(-1);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void onVertexStarted(Map<String, List<Integer>> completions) {
+      // sleep for 1 seconds to delay the running of task in v2.
+      // this could keep the case that task of v1 is partial finished or completely
+      // finished, and at the same time the task of v2 is not started
+      try {
+        Thread.sleep(1*1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      super.onVertexStarted(completions);
+    }
+  }
+
+  public static class ControlledImmediateStartVertexManager extends
+      ImmediateStartVertexManager {
+
+    private Configuration conf;
+
+    public ControlledImmediateStartVertexManager(
+        VertexManagerPluginContext context) {
+      super(context);
+    }
+
+    @Override
+    public void initialize() {
+      super.initialize();
+      try {
+        conf =
+            TezUtils.createConfFromUserPayload(getContext().getUserPayload());
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+    }
+
+    @Override
+    public void onSourceTaskCompleted(String srcVertexName, Integer taskId) {
+      super.onSourceTaskCompleted(srcVertexName, taskId);
+      if (getContext().getDAGAttemptNumber() == 1) {
+        if (conf.getBoolean(FAIL_ON_PARTIAL_FINISHED, true)) {
+          if (taskId == 0) {
+            System.exit(-1);
+          }
+        } else {
+          if (taskId == 1) {
+            System.exit(-1);
+          }
+        }
+      }
+    }
+
+    @Override
+    public void onVertexStarted(Map<String, List<Integer>> completions) {
+      // sleep for 1 seconds to delay the running of task in v2.
+      // this could keep the case that task of v1 is partial finished or completely
+      // finished, and at the same time the task of v2 is not started
+      try {
+        Thread.sleep(1*1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      super.onVertexStarted(completions);
+    }
+  }
+
+  /**
+   * VM which could control fail on attempt less than a specified number
+   *
+   */
+  public static class FailOnAttemptVertexManager extends ShuffleVertexManager {
+
+    private Configuration conf;
+
+    public FailOnAttemptVertexManager(VertexManagerPluginContext context) {
+      super(context);
+    }
+
+    @Override
+    public void initialize() {
+      super.initialize();
+      try {
+        conf =
+            TezUtils.createConfFromUserPayload(getContext().getUserPayload());
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+    }
+
+    @Override
+    public void onSourceTaskCompleted(String srcVertexName, Integer taskId) {
+      super.onSourceTaskCompleted(srcVertexName, taskId);
+      int curAttempt = getContext().getDAGAttemptNumber();
+      int failOnAttempt = conf.getInt(FAIL_ON_ATTEMPT, 1);
+      LOG.info("failOnAttempt:" + failOnAttempt);
+      LOG.info("curAttempt:" + curAttempt);
+      if (curAttempt < failOnAttempt) {
+        System.exit(-1);
+      }
+    }
+  }
+
+  public static enum TestCounter {
+    Counter_1,
+  }
+
+  /**
+   * Do nothing if it is in task 0, sleep 3 seconds for other tasks. This enable
+   * us to kill AM in VM when some tasks are still running.
+   *
+   */
+  public static class MyProcessor extends SimpleProcessor {
+
+    public MyProcessor(ProcessorContext context) {
+      super(context);
+    }
+
+    @Override
+    public void run() throws Exception {
+      getContext().getCounters().findCounter(TestCounter.Counter_1).increment(1);
+      if (getContext().getTaskIndex() == 0) {
+        // keep task_0 running for 1 seconds to wait for task_1 start running
+        Thread.sleep(1 * 1000);;
+      } else {
+        Thread.sleep(3 * 1000);
+      }
+    }
+
+    public static ProcessorDescriptor getProcDesc() {
+      return ProcessorDescriptor.create(MyProcessor.class.getName());
+    }
+  }
+
+  public static class DoNothingProcessor extends SimpleProcessor {
+
+    public DoNothingProcessor(ProcessorContext context) {
+      super(context);
+    }
+
+    @Override
+    public void run() throws Exception {
+    }
+
+    public static ProcessorDescriptor getProcDesc() {
+      return ProcessorDescriptor.create(DoNothingProcessor.class.getName());
+    }
+  }
+
+}
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
index c9acdc2..60c1efc 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
@@ -24,21 +24,30 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.tez.client.TezClientUtils;
 import org.apache.tez.client.TezClient;
+import org.apache.tez.common.TezCommonUtils;
 import org.apache.tez.dag.api.DAG;
 import org.apache.tez.dag.api.DataSourceDescriptor;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.InputInitializerDescriptor;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
 import org.apache.tez.dag.api.client.DAGStatus.State;
+import org.apache.tez.dag.app.RecoveryParser;
+import org.apache.tez.dag.history.HistoryEvent;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.events.VertexInitializedEvent;
+import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
 import org.apache.tez.test.dag.MultiAttemptDAG;
 import org.apache.tez.test.dag.MultiAttemptDAG.FailingInputInitializer;
 import org.apache.tez.test.dag.MultiAttemptDAG.NoOpInput;
+import org.apache.tez.test.dag.MultiAttemptDAG.TestRootInputInitializer;
 import org.apache.tez.test.dag.SimpleVTestDAG;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -48,6 +57,7 @@
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Random;
 
 public class TestDAGRecovery {
@@ -61,6 +71,7 @@
   private static MiniDFSCluster dfsCluster = null;
   private static TezClient tezSession = null;
   private static FileSystem remoteFs = null;
+  private static TezConfiguration tezConf = null;
 
   @BeforeClass
   public static void beforeClass() throws Exception {
@@ -120,7 +131,7 @@
         .valueOf(new Random().nextInt(100000))));
     TezClientUtils.ensureStagingDirExists(conf, remoteStagingDir);
 
-    TezConfiguration tezConf = new TezConfiguration(miniTezCluster.getConfig());
+    tezConf = new TezConfiguration(miniTezCluster.getConfig());
     tezConf.setInt(TezConfiguration.DAG_RECOVERY_MAX_UNFLUSHED_EVENTS, 0);
     tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "DEBUG");
     tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR,
@@ -130,6 +141,7 @@
     tezConf.setInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB, 500);
     tezConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, " -Xmx256m");
     tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
+    tezConf.set(TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, "false");
 
     tezSession = TezClient.create("TestDAGRecovery", tezConf);
     tezSession.start();
@@ -146,7 +158,6 @@
       }
     }
     tezSession = null;
-    Thread.sleep(10000);
   }
 
   void runDAGAndVerify(DAG dag, DAGStatus.State finalState) throws Exception {
@@ -165,13 +176,64 @@
     Assert.assertEquals(finalState, dagStatus.getState());
   }
 
+  private void verifyRecoveryLog() throws IOException{
+    ApplicationId appId = tezSession.getAppMasterApplicationId();
+    Path tezSystemStagingDir = TezCommonUtils.getTezSystemStagingPath(tezConf, appId.toString());
+    Path recoveryDataDir = TezCommonUtils.getRecoveryPath(tezSystemStagingDir, tezConf);
+
+    FileSystem fs = tezSystemStagingDir.getFileSystem(tezConf);
+    for (int i=1; i<=3; ++i) {
+      Path currentAttemptRecoveryDataDir = TezCommonUtils.getAttemptRecoveryPath(recoveryDataDir,i);
+      Path recoveryFilePath = new Path(currentAttemptRecoveryDataDir,
+      appId.toString().replace("application", "dag") + "_1" + TezConstants.DAG_RECOVERY_RECOVER_FILE_SUFFIX);
+      List<HistoryEvent> historyEvents = RecoveryParser.parseDAGRecoveryFile(
+          fs.open(recoveryFilePath));
+
+      int inputInfoEventIndex = -1;
+      int vertexInitedEventIndex = -1;
+      for (int j=0;j<historyEvents.size(); ++j) {
+        HistoryEvent historyEvent = historyEvents.get(j);
+        LOG.info("Parsed event from recovery stream"
+            + ", eventType=" + historyEvent.getEventType()
+            + ", event=" + historyEvent);
+        if (historyEvent.getEventType() ==  HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED) {
+          VertexRecoverableEventsGeneratedEvent dmEvent =
+              (VertexRecoverableEventsGeneratedEvent) historyEvent;
+          // TODO do not need to check whether it is -1 after Tez-1521 is resolved
+          if (dmEvent.getVertexID().getId() == 0 && inputInfoEventIndex == -1) {
+            inputInfoEventIndex = j;
+          }
+        }
+        if (historyEvent.getEventType() == HistoryEventType.VERTEX_INITIALIZED) {
+          VertexInitializedEvent vInitedEvent = (VertexInitializedEvent) historyEvent;
+          if (vInitedEvent.getVertexID().getId() == 0) {
+            vertexInitedEventIndex = j;
+          }
+        }
+      }
+      // v1's init events must be logged before its VertexInitializedEvent (Tez-1345)
+      Assert.assertTrue("can not find VERTEX_DATA_MOVEMENT_EVENTS_GENERATED for v1", inputInfoEventIndex != -1);
+      Assert.assertTrue("can not find VERTEX_INITIALIZED for v1", vertexInitedEventIndex != -1);
+      Assert.assertTrue("VERTEX_DATA_MOVEMENT_EVENTS_GENERATED is logged before VERTEX_INITIALIZED for v1",
+          inputInfoEventIndex < vertexInitedEventIndex);
+    }
+  }
+
   @Test(timeout=120000)
   public void testBasicRecovery() throws Exception {
     DAG dag = MultiAttemptDAG.createDAG("TestBasicRecovery", null);
+    // add input to v1 to make sure that there will be init events for v1 (TEZ-1345)
+    DataSourceDescriptor dataSource =
+        DataSourceDescriptor.create(InputDescriptor.create(NoOpInput.class.getName()),
+           InputInitializerDescriptor.create(TestRootInputInitializer.class.getName()), null);
+    dag.getVertex("v1").addDataSource("Input", dataSource);
+
     runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
 
+    verifyRecoveryLog();
+
     // it should fail if submitting same dags in recovery mode (TEZ-1064)
-    try{
+    try {
       DAGClient dagClient = tezSession.submitDAG(dag);
       Assert.fail("Expected DAG submit to fail on duplicate dag name");
     } catch (TezException e) {
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java b/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java
new file mode 100644
index 0000000..454f603
--- /dev/null
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestExceptionPropagation.java
@@ -0,0 +1,510 @@
+/**
+ * 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.tez.test;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.DataSinkDescriptor;
+import org.apache.tez.dag.api.DataSourceDescriptor;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.InputInitializerDescriptor;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.runtime.api.AbstractLogicalIOProcessor;
+import org.apache.tez.runtime.api.AbstractLogicalInput;
+import org.apache.tez.runtime.api.AbstractLogicalOutput;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.InputContext;
+import org.apache.tez.runtime.api.InputInitializer;
+import org.apache.tez.runtime.api.InputInitializerContext;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.OutputContext;
+import org.apache.tez.runtime.api.ProcessorContext;
+import org.apache.tez.runtime.api.Reader;
+import org.apache.tez.runtime.api.Writer;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.events.InputInitializerEvent;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
+import org.junit.Test;
+
+public class TestExceptionPropagation {
+
+  private static final Log LOG = LogFactory
+      .getLog(TestExceptionPropagation.class);
+
+  private static TezConfiguration tezConf;
+  private static Configuration conf = new Configuration();
+  private static MiniTezCluster miniTezCluster = null;
+  private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR
+      + TestExceptionPropagation.class.getName() + "-tmpDir";
+  private static MiniDFSCluster dfsCluster = null;
+  private static FileSystem remoteFs = null;
+
+  private static TezClient tezSession = null;
+  private static TezClient tezClient = null;
+
+  private void startMiniTezCluster() {
+    LOG.info("Starting mini clusters");
+    try {
+      conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR);
+      dfsCluster =
+          new MiniDFSCluster.Builder(conf).numDataNodes(3).format(true)
+              .racks(null).build();
+      remoteFs = dfsCluster.getFileSystem();
+    } catch (IOException io) {
+      throw new RuntimeException("problem starting mini dfs cluster", io);
+    }
+    miniTezCluster =
+        new MiniTezCluster(TestExceptionPropagation.class.getName(), 1, 1, 1);
+    Configuration miniTezconf = new Configuration(conf);
+    miniTezconf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 4);
+    miniTezconf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS
+    miniTezCluster.init(miniTezconf);
+    miniTezCluster.start();
+  }
+
+  private void stopTezMiniCluster() {
+    if (miniTezCluster != null) {
+      try {
+        LOG.info("Stopping MiniTezCluster");
+        miniTezCluster.stop();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+    if (dfsCluster != null) {
+      try {
+        LOG.info("Stopping DFSCluster");
+        dfsCluster.shutdown();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+  }
+
+  private void startSessionClient() throws Exception {
+    LOG.info("Starting session");
+    tezConf = new TezConfiguration();
+    tezConf.setInt(TezConfiguration.DAG_RECOVERY_MAX_UNFLUSHED_EVENTS, 0);
+    tezConf
+        .setBoolean(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_ENABLED, false);
+    tezConf.setInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, 4);
+    tezConf.setInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB, 500);
+    tezConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, " -Xmx256m");
+    tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
+    // for local mode
+    tezConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true);
+    tezConf.set("fs.defaultFS", "file:///");
+    tezConf.setBoolean(
+        TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH, true);
+
+    tezSession = TezClient.create("TestExceptionPropagation", tezConf);
+    tezSession.start();
+  }
+
+  private void stopSessionClient() {
+    if (tezSession != null) {
+      try {
+        LOG.info("Stopping Tez Session");
+        tezSession.stop();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+    tezSession = null;
+  }
+
+  private void startNonSessionClient() throws Exception {
+    LOG.info("Starting Client");
+    tezConf = new TezConfiguration(miniTezCluster.getConfig());
+    tezConf.setInt(TezConfiguration.DAG_RECOVERY_MAX_UNFLUSHED_EVENTS, 0);
+    tezConf
+        .setBoolean(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_ENABLED, false);
+    tezConf.setInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, 4);
+    tezConf.setInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB, 500);
+    tezConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, " -Xmx256m");
+    tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, false);
+
+    tezClient = TezClient.create("TestExceptionPropagation", tezConf);
+    tezClient.start();
+  }
+
+  private void stopNonSessionClient() {
+    if (tezClient != null) {
+      try {
+        LOG.info("Stopping Tez Client");
+        tezClient.stop();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+    tezClient = null;
+  }
+
+  /**
+   * verify the diagnostics in DAGStatus is correct in session mode, using local
+   * mode for fast speed
+   * 
+   * @throws Exception
+   * 
+   */
+  @Test(timeout = 120000)
+  public void testExceptionPropagationSession() throws Exception {
+    try {
+      startSessionClient();
+      for (ExceptionLocation exLocation : ExceptionLocation.values()) {
+        LOG.info("Session mode, Test for Exception from:" + exLocation.name());
+        DAG dag = createDAG(exLocation);
+        DAGClient dagClient = tezSession.submitDAG(dag);
+        DAGStatus dagStatus = dagClient.waitForCompletion();
+        String diagnostics = StringUtils.join(dagStatus.getDiagnostics(), ",");
+        LOG.info("Diagnostics:" + diagnostics);
+        assertTrue(diagnostics.contains(exLocation.name()));
+      }
+    } finally {
+      stopSessionClient();
+    }
+  }
+
+  /**
+   * verify the diagnostics in {@link DAGStatus} is correct in non-session mode,
+   * and also verify that diagnostics from {@link DAGStatus} should match that
+   * from {@link ApplicationReport}
+   * 
+   * @throws Exception
+   */
+  @Test(timeout = 120000)
+  public void testExceptionPropagationNonSession() throws Exception {
+    try {
+      startMiniTezCluster();
+      startNonSessionClient();
+
+      ExceptionLocation exLocation = ExceptionLocation.INPUT_START;
+      LOG.info("NonSession mode, Test for Exception from:" + exLocation.name());
+      DAG dag = createDAG(exLocation);
+      DAGClient dagClient = tezClient.submitDAG(dag);
+      DAGStatus dagStatus = dagClient.waitForCompletion();
+      String diagnostics = StringUtils.join(dagStatus.getDiagnostics(), ",");
+      LOG.info("Diagnostics:" + diagnostics);
+      assertTrue(diagnostics.contains(exLocation.name()));
+
+      // wait for app complete (unregisterApplicationMaster is done)
+      ApplicationId appId = tezClient.getAppMasterApplicationId();
+      YarnClient yarnClient = YarnClient.createYarnClient();
+      yarnClient.init(tezConf);
+      yarnClient.start();
+      Set<YarnApplicationState> FINAL_APPLICATION_STATES =
+          EnumSet.of(YarnApplicationState.KILLED, YarnApplicationState.FAILED,
+              YarnApplicationState.FINISHED);
+      ApplicationReport appReport = null;
+      while (true) {
+        appReport = yarnClient.getApplicationReport(appId);
+        Thread.sleep(1000);
+        LOG.info("FinalAppStatus:" + appReport.getFinalApplicationStatus());
+        LOG.info("Diagnostics from appReport:" + appReport.getDiagnostics());
+        if (FINAL_APPLICATION_STATES.contains(appReport
+            .getYarnApplicationState())) {
+          break;
+        }
+      }
+      // wait for 1 second and call getApplicationReport again to ensure get the
+      // diagnostics
+      // TODO remove it after YARN-2560
+      Thread.sleep(1000);
+      appReport = yarnClient.getApplicationReport(appId);
+
+      LOG.info("FinalAppStatus:" + appReport.getFinalApplicationStatus());
+      LOG.info("Diagnostics from appReport:" + appReport.getDiagnostics());
+      assertTrue(appReport.getDiagnostics().contains(exLocation.name()));
+      // use "\n" as separator, because we also use it in Tez internally when
+      // assembling the application diagnostics.
+      assertEquals(StringUtils.join(dagStatus.getDiagnostics(), "\n").trim(),
+          appReport.getDiagnostics().trim());
+    } finally {
+      stopNonSessionClient();
+      Thread.sleep(10*1000);
+      stopTezMiniCluster();
+    }
+  }
+
+  public static enum ExceptionLocation {
+    INPUT_START, INPUT_GET_READER, INPUT_HANDLE_EVENTS, INPUT_CLOSE, INPUT_INITIALIZE, OUTPUT_START, OUTPUT_GET_WRITER,
+    // Not Supported yet
+    // OUTPUT_HANDLE_EVENTS,
+    OUTPUT_CLOSE, OUTPUT_INITIALIZE,
+    // Not Supported yet
+    // PROCESSOR_HANDLE_EVENTS
+    PROCESSOR_RUN, PROCESSOR_CLOSE, PROCESSOR_INITIALIZE,
+
+  }
+
+  /**
+   * create a DAG with single vertex, set payload on Input/Output/Processor to
+   * control where throw exception
+   * 
+   * @param exLocation
+   * @return
+   */
+  private DAG createDAG(ExceptionLocation exLocation) {
+    DAG dag = DAG.create("dag_" + exLocation.name());
+    UserPayload payload =
+        UserPayload.create(ByteBuffer.wrap(exLocation.name().getBytes()));
+    Vertex v1 =
+        Vertex.create("v1", ProcessorWithException.getProcDesc(payload), 1);
+    InputDescriptor inputDesc = InputWithException.getInputDesc(payload);
+    InputInitializerDescriptor iiDesc =
+        InputInitializerWithException.getIIDesc(payload);
+    v1.addDataSource("input",
+        DataSourceDescriptor.create(inputDesc, iiDesc, null));
+    OutputDescriptor outputDesc = OutputWithException.getOutputDesc(payload);
+    v1.addDataSink("output", DataSinkDescriptor.create(outputDesc, null, null));
+    dag.addVertex(v1);
+    return dag;
+  }
+
+  public static class InputInitializerWithException extends InputInitializer {
+
+    private ExceptionLocation exLocation;
+
+    public InputInitializerWithException(
+        InputInitializerContext initializerContext) {
+      super(initializerContext);
+      this.exLocation =
+          ExceptionLocation.valueOf(new String(getContext().getUserPayload()
+              .deepCopyAsArray()));
+    }
+
+    @Override
+    public List<Event> initialize() throws Exception {
+      List<Event> events = new ArrayList<Event>();
+      events.add(InputDataInformationEvent.createWithObjectPayload(0, null));
+      return events;
+    }
+
+    @Override
+    public void handleInputInitializerEvent(List<InputInitializerEvent> events)
+        throws Exception {
+    }
+
+    public static InputInitializerDescriptor getIIDesc(UserPayload payload) {
+      return InputInitializerDescriptor.create(
+          InputInitializerWithException.class.getName())
+          .setUserPayload(payload);
+    }
+  }
+
+  public static class InputWithException extends AbstractLogicalInput {
+
+    private ExceptionLocation exLocation;
+    private Object condition = new Object();
+
+    public InputWithException(InputContext inputContext, int numPhysicalInputs) {
+      super(inputContext, numPhysicalInputs);
+      this.exLocation =
+          ExceptionLocation.valueOf(new String(getContext().getUserPayload()
+              .deepCopyAsArray()));
+    }
+
+    @Override
+    public void start() throws Exception {
+      if (this.exLocation == ExceptionLocation.INPUT_START) {
+        throw new Exception(this.exLocation.name());
+      }
+    }
+
+    @Override
+    public Reader getReader() throws Exception {
+      if (this.exLocation == ExceptionLocation.INPUT_HANDLE_EVENTS) {
+        synchronized (condition) {
+          // wait for exception thrown from handleEvents. Otherwise,
+          // processor may exit before the exception from handleEvents is
+          // caught.
+          condition.wait();
+        }
+      }
+      if (this.exLocation == ExceptionLocation.INPUT_GET_READER) {
+        throw new Exception(this.exLocation.name());
+      }
+      return null;
+    }
+
+    @Override
+    public void handleEvents(List<Event> inputEvents) throws Exception {
+      if (this.exLocation == ExceptionLocation.INPUT_HANDLE_EVENTS) {
+        throw new Exception(this.exLocation.name());
+      }
+    }
+
+    @Override
+    public List<Event> close() throws Exception {
+      if (this.exLocation == ExceptionLocation.INPUT_CLOSE) {
+        throw new Exception(this.exLocation.name());
+      }
+      return null;
+    }
+
+    @Override
+    public List<Event> initialize() throws Exception {
+      getContext().requestInitialMemory(0l, null); // mandatory call
+      if (this.exLocation == ExceptionLocation.INPUT_INITIALIZE) {
+        throw new Exception(this.exLocation.name());
+      }
+      return null;
+    }
+
+    public static InputDescriptor getInputDesc(UserPayload payload) {
+      return InputDescriptor.create(InputWithException.class.getName())
+          .setUserPayload(payload);
+    }
+  }
+
+  public static class OutputWithException extends AbstractLogicalOutput {
+
+    private ExceptionLocation exLocation;
+
+    public OutputWithException(OutputContext outputContext,
+        int numPhysicalOutputs) {
+      super(outputContext, numPhysicalOutputs);
+      this.exLocation =
+          ExceptionLocation.valueOf(new String(getContext().getUserPayload()
+              .deepCopyAsArray()));
+    }
+
+    @Override
+    public void start() throws Exception {
+      if (this.exLocation == ExceptionLocation.OUTPUT_START) {
+        throw new Exception(this.exLocation.name());
+      }
+
+    }
+
+    @Override
+    public Writer getWriter() throws Exception {
+      if (this.exLocation == ExceptionLocation.OUTPUT_GET_WRITER) {
+        throw new Exception(this.exLocation.name());
+      }
+      return null;
+    }
+
+    @Override
+    public void handleEvents(List<Event> outputEvents) {
+    }
+
+    @Override
+    public List<Event> close() throws Exception {
+      if (this.exLocation == ExceptionLocation.OUTPUT_CLOSE) {
+        throw new RuntimeException(this.exLocation.name());
+      }
+      return null;
+    }
+
+    @Override
+    public List<Event> initialize() throws Exception {
+      getContext().requestInitialMemory(0l, null); // mandatory call
+      if (this.exLocation == ExceptionLocation.OUTPUT_INITIALIZE) {
+        throw new RuntimeException(this.exLocation.name());
+      }
+      return null;
+    }
+
+    public static OutputDescriptor getOutputDesc(UserPayload payload) {
+      return OutputDescriptor.create(OutputWithException.class.getName())
+          .setUserPayload(payload);
+    }
+  }
+
+  public static class ProcessorWithException extends AbstractLogicalIOProcessor {
+
+    private ExceptionLocation exLocation;
+
+    public ProcessorWithException(ProcessorContext context) {
+      super(context);
+      this.exLocation =
+          ExceptionLocation.valueOf(new String(getContext().getUserPayload()
+              .deepCopyAsArray()));
+    }
+
+    @Override
+    public void run(Map<String, LogicalInput> inputs,
+        Map<String, LogicalOutput> outputs) throws Exception {
+      InputWithException input = (InputWithException) inputs.get("input");
+      input.start();
+      input.getReader();
+
+      OutputWithException output = (OutputWithException) outputs.get("output");
+      output.start();
+      output.getWriter();
+
+      if (this.exLocation == ExceptionLocation.PROCESSOR_RUN) {
+        throw new Exception(this.exLocation.name());
+      }
+    }
+
+    @Override
+    public void handleEvents(List<Event> processorEvents) {
+    }
+
+    @Override
+    public void close() throws Exception {
+      if (this.exLocation == ExceptionLocation.PROCESSOR_CLOSE) {
+        throw new RuntimeException(this.exLocation.name());
+      }
+    }
+
+    @Override
+    public void initialize() throws Exception {
+      if (this.exLocation == ExceptionLocation.PROCESSOR_INITIALIZE) {
+        throw new RuntimeException(this.exLocation.name());
+      }
+    }
+
+    public static ProcessorDescriptor getProcDesc(UserPayload payload) {
+      return ProcessorDescriptor.create(ProcessorWithException.class.getName())
+          .setUserPayload(payload);
+    }
+  }
+}
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
index 1fbacdf..56f62a4 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
@@ -52,8 +52,9 @@
 import org.apache.tez.examples.OrderedWordCount;
 import org.apache.tez.examples.SimpleSessionExample;
 import org.apache.tez.examples.JoinDataGen;
-import org.apache.tez.examples.JoinExample;
+import org.apache.tez.examples.HashJoinExample;
 import org.apache.tez.examples.JoinValidate;
+import org.apache.tez.examples.SortMergeJoinExample;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -117,13 +118,13 @@
   }
 
   @Test(timeout = 60000)
-  public void testIntersectExample() throws Exception {
-    JoinExample intersectExample = new JoinExample();
-    intersectExample.setConf(new Configuration(mrrTezCluster.getConfig()));
+  public void testHashJoinExample() throws Exception {
+    HashJoinExample hashJoinExample = new HashJoinExample();
+    hashJoinExample.setConf(new Configuration(mrrTezCluster.getConfig()));
     Path stagingDirPath = new Path("/tmp/tez-staging-dir");
-    Path inPath1 = new Path("/tmp/inPath1");
-    Path inPath2 = new Path("/tmp/inPath2");
-    Path outPath = new Path("/tmp/outPath");
+    Path inPath1 = new Path("/tmp/hashJoin/inPath1");
+    Path inPath2 = new Path("/tmp/hashJoin/inPath2");
+    Path outPath = new Path("/tmp/hashJoin/outPath");
     remoteFs.mkdirs(inPath1);
     remoteFs.mkdirs(inPath2);
     remoteFs.mkdirs(stagingDirPath);
@@ -152,7 +153,7 @@
     String[] args = new String[] {
         "-D" + TezConfiguration.TEZ_AM_STAGING_DIR + "=" + stagingDirPath.toString(),
         inPath1.toString(), inPath2.toString(), "1", outPath.toString() };
-    assertEquals(0, intersectExample.run(args));
+    assertEquals(0, hashJoinExample.run(args));
 
     FileStatus[] statuses = remoteFs.listStatus(outPath, new PathFilter() {
       public boolean accept(Path p) {
@@ -172,10 +173,71 @@
     assertEquals(0, expectedResult.size());
   }
 
-  @Test(timeout = 120000)
-  public void testIntersect2() throws Exception {
+  @Test(timeout = 60000)
+  public void testSortMergeJoinExample() throws Exception {
+    SortMergeJoinExample sortMergeJoinExample = new SortMergeJoinExample();
+    sortMergeJoinExample.setConf(new Configuration(mrrTezCluster.getConfig()));
+    Path stagingDirPath = new Path("/tmp/tez-staging-dir");
+    Path inPath1 = new Path("/tmp/sortMerge/inPath1");
+    Path inPath2 = new Path("/tmp/sortMerge/inPath2");
+    Path outPath = new Path("/tmp/sortMerge/outPath");
+    remoteFs.mkdirs(inPath1);
+    remoteFs.mkdirs(inPath2);
+    remoteFs.mkdirs(stagingDirPath);
 
-    Path testDir = new Path("/tmp/testIntersect2");
+    Set<String> expectedResult = new HashSet<String>();
+
+    FSDataOutputStream out1 = remoteFs.create(new Path(inPath1, "file"));
+    FSDataOutputStream out2 = remoteFs.create(new Path(inPath2, "file"));
+    BufferedWriter writer1 = new BufferedWriter(new OutputStreamWriter(out1));
+    BufferedWriter writer2 = new BufferedWriter(new OutputStreamWriter(out2));
+    for (int i = 0; i < 20; i++) {
+      String term = "term" + i;
+      writer1.write(term);
+      writer1.newLine();
+      if (i % 2 == 0) {
+        writer2.write(term);
+        writer2.newLine();
+        expectedResult.add(term);
+      }
+    }
+    writer1.close();
+    writer2.close();
+    out1.close();
+    out2.close();
+
+    String[] args = new String[] {
+        "-D" + TezConfiguration.TEZ_AM_STAGING_DIR + "=" + stagingDirPath.toString(),
+        inPath1.toString(), inPath2.toString(), "1", outPath.toString() };
+    assertEquals(0, sortMergeJoinExample.run(args));
+
+    FileStatus[] statuses = remoteFs.listStatus(outPath, new PathFilter() {
+      public boolean accept(Path p) {
+        String name = p.getName();
+        return !name.startsWith("_") && !name.startsWith(".");
+      }
+    });
+    assertEquals(1, statuses.length);
+    FSDataInputStream inStream = remoteFs.open(statuses[0].getPath());
+    BufferedReader reader = new BufferedReader(new InputStreamReader(inStream));
+    String line;
+    while ((line = reader.readLine()) != null) {
+      assertTrue(expectedResult.remove(line));
+    }
+    reader.close();
+    inStream.close();
+    assertEquals(0, expectedResult.size());
+  }
+
+  /**
+   * test whole {@link HashJoinExample} pipeline as following: <br>
+   * {@link JoinDataGen} -> {@link HashJoinExample} -> {@link JoinValidate}
+   * @throws Exception
+   */
+  @Test(timeout = 120000)
+  public void testHashJoinExamplePipeline() throws Exception {
+
+    Path testDir = new Path("/tmp/testHashJoinExample");
     Path stagingDirPath = new Path("/tmp/tez-staging-dir");
     remoteFs.mkdirs(stagingDirPath);
     remoteFs.mkdirs(testDir);
@@ -189,7 +251,7 @@
     tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
     TezClient tezSession = null;
     try {
-      tezSession = TezClient.create("IntersectExampleSession", tezConf);
+      tezSession = TezClient.create("HashJoinExampleSession", tezConf, true);
       tezSession.start();
 
       JoinDataGen dataGen = new JoinDataGen();
@@ -198,15 +260,63 @@
           expectedOutputPath.toString(), "2" };
       assertEquals(0, dataGen.run(tezConf, dataGenArgs, tezSession));
 
-      JoinExample intersect = new JoinExample();
-      String[] intersectArgs = new String[] {
+      HashJoinExample joinExample = new HashJoinExample();
+      String[] args = new String[] {
           dataPath1.toString(), dataPath2.toString(), "2", outPath.toString() };
-      assertEquals(0, intersect.run(tezConf, intersectArgs, tezSession));
+      assertEquals(0, joinExample.run(tezConf, args, tezSession));
 
-      JoinValidate intersectValidate = new JoinValidate();
-      String[] intersectValidateArgs = new String[] {
+      JoinValidate joinValidate = new JoinValidate();
+      String[] validateArgs = new String[] {
           expectedOutputPath.toString(), outPath.toString(), "3" };
-      assertEquals(0, intersectValidate.run(tezConf, intersectValidateArgs, tezSession));
+      assertEquals(0, joinValidate.run(tezConf, validateArgs, tezSession));
+
+    } finally {
+      if (tezSession != null) {
+        tezSession.stop();
+      }
+    }
+  }
+
+  /**
+   * test whole {@link SortMergeJoinExample} pipeline as following: <br>
+   * {@link JoinDataGen} -> {@link SortMergeJoinExample} -> {@link JoinValidate}
+   * @throws Exception
+   */
+  @Test(timeout = 120000)
+  public void testSortMergeJoinExamplePipeline() throws Exception {
+
+    Path testDir = new Path("/tmp/testSortMergeExample");
+    Path stagingDirPath = new Path("/tmp/tez-staging-dir");
+    remoteFs.mkdirs(stagingDirPath);
+    remoteFs.mkdirs(testDir);
+
+    Path dataPath1 = new Path(testDir, "inPath1");
+    Path dataPath2 = new Path(testDir, "inPath2");
+    Path expectedOutputPath = new Path(testDir, "expectedOutputPath");
+    Path outPath = new Path(testDir, "outPath");
+
+    TezConfiguration tezConf = new TezConfiguration(mrrTezCluster.getConfig());
+    tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
+    TezClient tezSession = null;
+    try {
+      tezSession = TezClient.create("SortMergeExampleSession", tezConf, true);
+      tezSession.start();
+
+      JoinDataGen dataGen = new JoinDataGen();
+      String[] dataGenArgs = new String[] {
+          dataPath1.toString(), "1048576", dataPath2.toString(), "524288",
+          expectedOutputPath.toString(), "2" };
+      assertEquals(0, dataGen.run(tezConf, dataGenArgs, tezSession));
+
+      SortMergeJoinExample joinExample = new SortMergeJoinExample();
+      String[] args = new String[] {
+          dataPath1.toString(), dataPath2.toString(), "2", outPath.toString() };
+      assertEquals(0, joinExample.run(tezConf, args, tezSession));
+
+      JoinValidate joinValidate = new JoinValidate();
+      String[] validateArgs = new String[] {
+          expectedOutputPath.toString(), outPath.toString(), "3" };
+      assertEquals(0, joinValidate.run(tezConf, validateArgs, tezSession));
 
     } finally {
       if (tezSession != null) {
diff --git a/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java b/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java
index 7fc9ad7..58b9413 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java
@@ -22,6 +22,7 @@
 import com.google.common.primitives.Ints;
 
 import java.nio.ByteBuffer;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -53,12 +54,15 @@
 import org.apache.tez.runtime.api.InputInitializer;
 import org.apache.tez.runtime.api.InputInitializerContext;
 import org.apache.tez.runtime.api.Writer;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
 import org.apache.tez.runtime.api.events.InputInitializerEvent;
 import org.apache.tez.runtime.api.events.VertexManagerEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
 import org.apache.tez.test.TestInput;
 import org.apache.tez.test.TestOutput;
 import org.apache.tez.test.TestProcessor;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -148,8 +152,15 @@
     }
 
     @Override
-    public void onRootVertexInitialized(String inputName, InputDescriptor inputDescriptor, List<Event> events) {
-      // Do nothing
+    public void onRootVertexInitialized(String inputName,
+        InputDescriptor inputDescriptor, List<Event> events) {
+      List<InputDataInformationEvent> inputInfoEvents = new ArrayList<InputDataInformationEvent>();
+      for (Event event: events) {
+        if (event instanceof InputDataInformationEvent) {
+          inputInfoEvents.add((InputDataInformationEvent)event);
+        }
+      }
+      getContext().addRootInputEvents(inputName, inputInfoEvents);
     }
   }
 
@@ -214,6 +225,26 @@
     }
   }
 
+  public static class TestRootInputInitializer extends InputInitializer {
+
+    public TestRootInputInitializer(InputInitializerContext initializerContext) {
+      super(initializerContext);
+    }
+
+    @Override
+    public List<Event> initialize() throws Exception {
+      List<Event> events = new ArrayList<Event>();
+      events.add(InputDataInformationEvent.createWithSerializedPayload(0, ByteBuffer.allocate(0)));
+      return events;
+    }
+
+    @Override
+    public void handleInputInitializerEvent(List<InputInitializerEvent> events)
+        throws Exception {
+      throw new UnsupportedOperationException("Not supported");
+    }
+  }
+
   public static class FailingInputInitializer extends InputInitializer {
 
     public FailingInputInitializer(InputInitializerContext initializerContext) {