Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2:
  - tez-engine-library for user-visible Input/Output/Processor implementations
  - tez-engine-internals for framework internals
diff --git a/pom.xml b/pom.xml
index 63f17eb..ce10f6d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -100,7 +100,12 @@
       </dependency>
       <dependency>
         <groupId>org.apache.tez</groupId>
-        <artifactId>tez-engine</artifactId>
+        <artifactId>tez-runtime-internals</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tez</groupId>
+        <artifactId>tez-runtime-library</artifactId>
         <version>${project.version}</version>
       </dependency>
       <dependency>
@@ -247,7 +252,8 @@
   <modules>
     <module>tez-api</module>
     <module>tez-common</module>
-    <module>tez-engine</module>
+    <module>tez-runtime-library</module>
+    <module>tez-runtime-internals</module>
     <module>tez-yarn-client</module>
     <module>tez-mapreduce</module>
     <module>tez-mapreduce-examples</module>
diff --git a/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java b/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
index 2c4b911..ace87ca 100644
--- a/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
+++ b/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
@@ -42,16 +42,16 @@
   /**
    * Configuration key to enable/disable IFile readahead.
    */
-  public static final String TEZ_ENGINE_IFILE_READAHEAD =
-      "tez.engine.ifile.readahead";
-  public static final boolean DEFAULT_TEZ_ENGINE_IFILE_READAHEAD = true;
+  public static final String TEZ_RUNTIME_IFILE_READAHEAD =
+      "tez.runtime.ifile.readahead";
+  public static final boolean DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD = true;
 
   /**
    * Configuration key to set the IFile readahead length in bytes.
    */
-  public static final String TEZ_ENGINE_IFILE_READAHEAD_BYTES =
-      "tez.engine.ifile.readahead.bytes";
-  public static final int DEFAULT_TEZ_ENGINE_IFILE_READAHEAD_BYTES =
+  public static final String TEZ_RUNTIME_IFILE_READAHEAD_BYTES =
+      "tez.runtime.ifile.readahead.bytes";
+  public static final int DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD_BYTES =
       4 * 1024 * 1024;
 
   /**
@@ -62,102 +62,103 @@
   public static final long DEFAULT_RECORDS_BEFORE_PROGRESS = 10000; 
 
   /**
-   * List of directories avialble to the engine. 
+   * List of directories avialble to the Runtime. 
    */
   @Private
-  public static final String LOCAL_DIRS = "tez.engine.local.dirs";
+  public static final String LOCAL_DIRS = "tez.runtime.local.dirs";
   public static final String DEFAULT_LOCAL_DIRS = "/tmp";
 
   /**
    * One local dir for the speicfic job.
    */
-  public static final String JOB_LOCAL_DIR = "tez.engine.job.local.dir";
+  public static final String JOB_LOCAL_DIR = "tez.runtime.job.local.dir";
   
   /**
    * The directory which contains the localized files for this task.
    */
   @Private
-  public static final String TASK_LOCAL_RESOURCE_DIR = "tez.engine.task-local-resource.dir";
+  public static final String TASK_LOCAL_RESOURCE_DIR = "tez.runtime.task-local-resource.dir";
   public static final String DEFAULT_TASK_LOCAL_RESOURCE_DIR = "/tmp";
   
-  public static final String TEZ_TASK_WORKING_DIR = "tez.engine.task.working.dir";
+  public static final String TEZ_TASK_WORKING_DIR = "tez.runtime.task.working.dir";
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_IO_SORT_FACTOR = 
-      "tez.engine.io.sort.factor";
-  public static final int DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR = 100;
+  public static final String TEZ_RUNTIME_IO_SORT_FACTOR = 
+      "tez.runtime.io.sort.factor";
+  public static final int DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR = 100;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SORT_SPILL_PERCENT = 
-      "tez.engine.sort.spill.percent";
-  public static float DEFAULT_TEZ_ENGINE_SORT_SPILL_PERCENT = 0.8f; 
+  public static final String TEZ_RUNTIME_SORT_SPILL_PERCENT = 
+      "tez.runtime.sort.spill.percent";
+  public static float DEFAULT_TEZ_RUNTIME_SORT_SPILL_PERCENT = 0.8f; 
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_IO_SORT_MB = "tez.engine.io.sort.mb";
-  public static final int DEFAULT_TEZ_ENGINE_IO_SORT_MB = 100;
+  public static final String TEZ_RUNTIME_IO_SORT_MB = "tez.runtime.io.sort.mb";
+  public static final int DEFAULT_TEZ_RUNTIME_IO_SORT_MB = 100;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
-      "tez.engine.index.cache.memory.limit.bytes";
-  public static final int DEFAULT_TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
+  public static final String TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
+      "tez.runtime.index.cache.memory.limit.bytes";
+  public static final int DEFAULT_TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
       1024 * 1024;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_COMBINE_MIN_SPILLS = 
-      "tez.engine.combine.min.spills";
-  public static final int  DEFAULT_TEZ_ENGINE_COMBINE_MIN_SPILLS = 3;
+  public static final String TEZ_RUNTIME_COMBINE_MIN_SPILLS = 
+      "tez.runtime.combine.min.spills";
+  public static final int  DEFAULT_TEZ_RUNTIME_COMBINE_MIN_SPILLS = 3;
   
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SORT_THREADS = 
-	      "tez.engine.sort.threads";
-  public static final int DEFAULT_TEZ_ENGINE_SORT_THREADS = 1;
+  public static final String TEZ_RUNTIME_SORT_THREADS = 
+	      "tez.runtime.sort.threads";
+  public static final int DEFAULT_TEZ_RUNTIME_SORT_THREADS = 1;
 
   /**
-   * Specifies a partitioner class, which is used in Tez engine components like OnFileSortedOutput
+   * Specifies a partitioner class, which is used in Tez Runtime components
+   * like OnFileSortedOutput
    */
-  public static final String TEZ_ENGINE_PARTITIONER_CLASS = "tez.engine.partitioner.class";
+  public static final String TEZ_RUNTIME_PARTITIONER_CLASS = "tez.runtime.partitioner.class";
   
   /**
    * Specifies a combiner class (primarily for Shuffle)
    */
-  public static final String TEZ_ENGINE_COMBINER_CLASS = "tez.engine.combiner.class";
+  public static final String TEZ_RUNTIME_COMBINER_CLASS = "tez.runtime.combiner.class";
   
-  public static final String TEZ_ENGINE_NUM_EXPECTED_PARTITIONS = "tez.engine.num.expected.partitions";
+  public static final String TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS = "tez.runtime.num.expected.partitions";
   
   /**
    * 
    */
-  public static final String COUNTERS_MAX_KEY = "tez.engine.job.counters.max";
+  public static final String COUNTERS_MAX_KEY = "tez.runtime.job.counters.max";
   public static final int COUNTERS_MAX_DEFAULT = 120;
 
   /**
    * 
    */
-  public static final String COUNTER_GROUP_NAME_MAX_KEY = "tez.engine.job.counters.group.name.max";
+  public static final String COUNTER_GROUP_NAME_MAX_KEY = "tez.runtime.job.counters.group.name.max";
   public static final int COUNTER_GROUP_NAME_MAX_DEFAULT = 128;
 
   /**
    * 
    */
-  public static final String COUNTER_NAME_MAX_KEY = "tez.engine.job.counters.counter.name.max";
+  public static final String COUNTER_NAME_MAX_KEY = "tez.runtime.job.counters.counter.name.max";
   public static final int COUNTER_NAME_MAX_DEFAULT = 64;
 
   /**
    * 
    */
-  public static final String COUNTER_GROUPS_MAX_KEY = "tez.engine.job.counters.groups.max";
+  public static final String COUNTER_GROUPS_MAX_KEY = "tez.runtime.job.counters.groups.max";
   public static final int COUNTER_GROUPS_MAX_DEFAULT = 50;
 
   
@@ -166,145 +167,145 @@
    * in-memory shuffle should be used.
    */
   @Private
-  public static final String TEZ_ENGINE_SHUFFLE_USE_IN_MEMORY =
-      "tez.engine.shuffle.use.in-memory";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_USE_IN_MEMORY = false;
+  public static final String TEZ_RUNTIME_SHUFFLE_USE_IN_MEMORY =
+      "tez.runtime.shuffle.use.in-memory";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_USE_IN_MEMORY = false;
 
   // TODO NEWTEZ Remove these config parameters. Will be part of an event.
   @Private
-  public static final String TEZ_ENGINE_SHUFFLE_PARTITION_RANGE = 
-      "tez.engine.shuffle.partition-range";
-  public static int TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT = 1;
+  public static final String TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE = 
+      "tez.runtime.shuffle.partition-range";
+  public static int TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE_DEFAULT = 1;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES = 
-      "tez.engine.shuffle.parallel.copies";
-  public static final int DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES = 20;
+  public static final String TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES = 
+      "tez.runtime.shuffle.parallel.copies";
+  public static final int DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES = 20;
 
   /**
    * TODO Is this user configurable.
    */
-  public static final String TEZ_ENGINE_METRICS_SESSION_ID = 
-      "tez.engine.metrics.session.id";
-  public static final String DEFAULT_TEZ_ENGINE_METRICS_SESSION_ID = "";
+  public static final String TEZ_RUNTIME_METRICS_SESSION_ID = 
+      "tez.runtime.metrics.session.id";
+  public static final String DEFAULT_TEZ_RUNTIME_METRICS_SESSION_ID = "";
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_FETCH_FAILURES = 
-      "tez.engine.shuffle.fetch.failures.limit";
-  public final static int DEFAULT_TEZ_ENGINE_SHUFFLE_FETCH_FAILURES_LIMIT = 10;
+  public static final String TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES = 
+      "tez.runtime.shuffle.fetch.failures.limit";
+  public final static int DEFAULT_TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT = 10;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR = 
-      "tez.engine.shuffle.notify.readerror";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR = true;
+  public static final String TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR = 
+      "tez.runtime.shuffle.notify.readerror";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR = true;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT = 
-      "tez.engine.shuffle.connect.timeout";
-  public static final int DEFAULT_TEZ_ENGINE_SHUFFLE_STALLED_COPY_TIMEOUT = 
+  public static final String TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT = 
+      "tez.runtime.shuffle.connect.timeout";
+  public static final int DEFAULT_TEZ_RUNTIME_SHUFFLE_STALLED_COPY_TIMEOUT = 
       3 * 60 * 1000;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_READ_TIMEOUT = "tez.engine.shuffle.read.timeout";
-  public final static int DEFAULT_TEZ_ENGINE_SHUFFLE_READ_TIMEOUT = 
+  public static final String TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT = "tez.runtime.shuffle.read.timeout";
+  public final static int DEFAULT_TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT = 
       3 * 60 * 1000;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_ENABLE_SSL = 
-      "tez.engine.shuffle.ssl.enable";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL = false;
+  public static final String TEZ_RUNTIME_SHUFFLE_ENABLE_SSL = 
+      "tez.runtime.shuffle.ssl.enable";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL = false;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT = 
-      "tez.engine.shuffle.input.buffer.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT =
+  public static final String TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT = 
+      "tez.runtime.shuffle.input.buffer.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT =
       0.90f;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT = 
-      "tez.engine.shuffle.memory.limit.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT = 
+  public static final String TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT = 
+      "tez.runtime.shuffle.memory.limit.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT = 
       0.25f;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_MERGE_PERCENT = 
-      "tez.engine.shuffle.merge.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_MERGE_PERCENT = 0.90f;
+  public static final String TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT = 
+      "tez.runtime.shuffle.merge.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT = 0.90f;
   
   /**
    * TODO TEZAM3 default value ?
    */
-  public static final String TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS = 
-      "tez.engine.shuffle.memory-to-memory.segments";
+  public static final String TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS = 
+      "tez.runtime.shuffle.memory-to-memory.segments";
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
-      "tez.engine.shuffle.memory-to-memory.enable";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
+  public static final String TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM = 
+      "tez.runtime.shuffle.memory-to-memory.enable";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM = 
       false;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_INPUT_BUFFER_PERCENT = 
-      "tez.engine.task.input.buffer.percent";
-  public static final float DEFAULT_TEZ_ENGINE_INPUT_BUFFER_PERCENT = 0.0f;
+  public static final String TEZ_RUNTIME_INPUT_BUFFER_PERCENT = 
+      "tez.runtime.task.input.buffer.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_INPUT_BUFFER_PERCENT = 0.0f;
 
   // TODO Rename. 
-  public static final String TEZ_ENGINE_GROUP_COMPARATOR_CLASS = 
-      "tez.engine.group.comparator.class";
+  public static final String TEZ_RUNTIME_GROUP_COMPARATOR_CLASS = 
+      "tez.runtime.group.comparator.class";
   
   // TODO Better name.
-  public static final String TEZ_ENGINE_INTERNAL_SORTER_CLASS = 
-      "tez.engine.internal.sorter.class";
+  public static final String TEZ_RUNTIME_INTERNAL_SORTER_CLASS = 
+      "tez.runtime.internal.sorter.class";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-output.key.comparator.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-input.key.comparator.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS = 
+      "tez.runtime.intermediate-output.key.comparator.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS = 
+      "tez.runtime.intermediate-input.key.comparator.class";
 
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS = 
-      "tez.engine.intermediate-output.key.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_CLASS = 
-      "tez.engine.intermediate-input.key.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS = 
+      "tez.runtime.intermediate-output.key.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_CLASS = 
+      "tez.runtime.intermediate-input.key.class";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS = 
-      "tez.engine.intermediate-output.value.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_VALUE_CLASS = 
-      "tez.engine.intermediate-input.value.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS = 
+      "tez.runtime.intermediate-output.value.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_VALUE_CLASS = 
+      "tez.runtime.intermediate-input.value.class";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS = 
-      "tez.engine.intermediate-output.should-compress";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_IS_COMPRESSED = 
-      "tez.engine.intermdiate-input.is-compressed";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS = 
+      "tez.runtime.intermediate-output.should-compress";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_IS_COMPRESSED = 
+      "tez.runtime.intermdiate-input.is-compressed";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_COMPRESS_CODEC = 
-      "tez.engine.intermediate-output.compress.codec";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_COMPRESS_CODEC = 
-      "tez.engine.intermediate-input.compress.codec";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_COMPRESS_CODEC = 
+      "tez.runtime.intermediate-output.compress.codec";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_COMPRESS_CODEC = 
+      "tez.runtime.intermediate-input.compress.codec";
 
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-input.key.secondary.comparator.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS = 
+      "tez.runtime.intermediate-input.key.secondary.comparator.class";
   
   // TODO This should be in DAGConfiguration
   /* config for tracking the local file where all the credentials for the job
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Event.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Event.java
similarity index 96%
rename from tez-api/src/main/java/org/apache/tez/engine/api/Event.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/Event.java
index 80da655..6f3d667 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Event.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Event.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 /**
  * Base class for all events generated within the Tez execution engine.
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Input.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Input.java
similarity index 98%
rename from tez-api/src/main/java/org/apache/tez/engine/api/Input.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/Input.java
index e333075..a15c106 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Input.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Input.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 import java.util.List;
 
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalIOProcessor.java
similarity index 97%
rename from tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/LogicalIOProcessor.java
index 90be09e..9372c70 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalIOProcessor.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 import java.util.Map;
 
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalInput.java
similarity index 97%
rename from tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/LogicalInput.java
index 4a47ccf..3382d12 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalInput.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 /**
  * An @link {@link Input} which handles all incoming physical connections on an
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalOutput.java
similarity index 97%
rename from tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/LogicalOutput.java
index 4626fbd..475eaef 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalOutput.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 /**
  * An @link {@link Output} which handles all outgoing physical connections on an
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Output.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Output.java
similarity index 98%
rename from tez-api/src/main/java/org/apache/tez/engine/api/Output.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/Output.java
index ec679ed..22f8d66 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Output.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Output.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 import java.util.List;
 
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Processor.java
similarity index 97%
rename from tez-api/src/main/java/org/apache/tez/engine/api/Processor.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/Processor.java
index 05e6b84..17c2d05 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Processor.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 import java.io.IOException;
 import java.util.List;
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Reader.java
similarity index 95%
rename from tez-api/src/main/java/org/apache/tez/engine/api/Reader.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/Reader.java
index 502c5f2..dd006bc 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Reader.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 /**
  * A <code>Reader</code> represents the data being read in an {@link Input}
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezInputContext.java
similarity index 96%
rename from tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/TezInputContext.java
index ddf1ff8..b07e92c 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/TezInputContext.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 /**
  * Context handle for the Input to initialize itself.
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezOutputContext.java
similarity index 96%
rename from tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/TezOutputContext.java
index 791a0f0..fda30ca 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/TezOutputContext.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 /**
  * Context handle for the Output to initialize itself.
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezProcessorContext.java
similarity index 97%
rename from tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/TezProcessorContext.java
index 2bbbe81..001461b 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/TezProcessorContext.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 import java.io.IOException;
 
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezTaskContext.java
similarity index 98%
rename from tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/TezTaskContext.java
index 706e646..ef652b6 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/TezTaskContext.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 import java.nio.ByteBuffer;
 import java.util.List;
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Writer.java
similarity index 95%
rename from tez-api/src/main/java/org/apache/tez/engine/api/Writer.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/Writer.java
index c9503a3..9604e59 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/Writer.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.api;
 
 /**
  * A <code>Writer</code> represents the data being written by an {@link Output}
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/DataMovementEvent.java
similarity index 96%
rename from tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/events/DataMovementEvent.java
index 3f35555..b384676 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/DataMovementEvent.java
@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.events;
+package org.apache.tez.runtime.api.events;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.api.Event;
+import org.apache.tez.runtime.api.Event;
 
 /**
  * Event used by user code to send information between tasks. An output can
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputFailedEvent.java
similarity index 96%
rename from tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/events/InputFailedEvent.java
index 57de09b..dedc8d9 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputFailedEvent.java
@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.events;
+package org.apache.tez.runtime.api.events;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.api.Event;
+import org.apache.tez.runtime.api.Event;
 
 /**
  * Event sent from the AM to an Input to indicate that one of it's sources has
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInformationEvent.java
similarity index 93%
rename from tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInformationEvent.java
index 3656d45..0322b75 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInformationEvent.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.events;
+package org.apache.tez.runtime.api.events;
 
-import org.apache.tez.engine.api.Event;
+import org.apache.tez.runtime.api.Event;
 
 /**
  * Event used to send user specific data from the user 
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java
similarity index 95%
rename from tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java
rename to tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java
index fa49b79..612a1e3 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.events;
+package org.apache.tez.runtime.api.events;
 
-import org.apache.tez.engine.api.Event;
+import org.apache.tez.runtime.api.Event;
 
 /**
  * Event generated by an Input to indicate error when trying to retrieve data.
diff --git a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectLifeCycle.java
similarity index 95%
rename from tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
rename to tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectLifeCycle.java
index 7099299..9dae043 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectLifeCycle.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.objectregistry;
+package org.apache.tez.runtime.common.objectregistry;
 
 /**
  * Defines the valid lifecycle and scope of Objects stored in ObjectRegistry.
diff --git a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistry.java
similarity index 97%
rename from tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
rename to tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistry.java
index a27903d..03f0424 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistry.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.objectregistry;
+package org.apache.tez.runtime.common.objectregistry;
 
 /**
  * Preliminary version of a simple shared object cache to re-use
diff --git a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryFactory.java
similarity index 94%
rename from tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
rename to tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryFactory.java
index 94352b3..3e409cd 100644
--- a/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryFactory.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.objectregistry;
+package org.apache.tez.runtime.common.objectregistry;
 
 import com.google.inject.Inject;
 
diff --git a/tez-api/src/main/proto/Events.proto b/tez-api/src/main/proto/Events.proto
index 21cacf6..b91125d 100644
--- a/tez-api/src/main/proto/Events.proto
+++ b/tez-api/src/main/proto/Events.proto
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-option java_package = "org.apache.tez.engine.api.events";
+option java_package = "org.apache.tez.runtime.api.events";
 option java_outer_classname = "EventProtos";
 option java_generate_equals_and_hash = true;
 
diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml
index bc6aeef..9bf3fe4 100644
--- a/tez-dag/pom.xml
+++ b/tez-dag/pom.xml
@@ -41,7 +41,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-engine</artifactId>
+      <artifactId>tez-runtime-internals</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
diff --git a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
index 1967462..4938d9e 100644
--- a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
+++ b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
@@ -67,26 +67,26 @@
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.api.events.TaskAttemptCompletedEvent;
-import org.apache.tez.engine.api.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.api.impl.EventMetaData;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
-import org.apache.tez.engine.api.impl.TezUmbilical;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.common.objectregistry.ObjectLifeCycle;
-import org.apache.tez.engine.common.objectregistry.ObjectRegistryImpl;
-import org.apache.tez.engine.common.objectregistry.ObjectRegistryModule;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.output.MROutput;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryModule;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.TokenCache;
 
 import com.google.inject.Guice;
 import com.google.inject.Injector;
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 56c89b2..67fe763 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
@@ -108,7 +108,7 @@
 import org.apache.tez.dag.history.avro.HistoryEventType;
 import org.apache.tez.dag.history.events.AMStartedEvent;
 import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
 
 /**
  * The Map-Reduce Application Master.
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
index 7a143a5..582d274 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -51,10 +51,10 @@
 import org.apache.tez.dag.app.security.authorize.MRAMPolicyProvider;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
 
 @SuppressWarnings("unchecked")
 public class TaskAttemptListenerImpTezDag extends AbstractService implements
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
index 2779faf..674d18e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
@@ -20,9 +20,9 @@
 
 import java.util.List;
 
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public abstract class EdgeManager {
   
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
index 088a195..293e4c5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
@@ -26,7 +26,7 @@
 import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 /**
  * Read only view of Task.
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
index 42ff8de..76964a3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
@@ -32,9 +32,9 @@
 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.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 
 /**
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
index 4c79712..4a1a7a6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
@@ -20,7 +20,7 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 // Rename to VertexManager TEZ-364 and move to DAG API. Make abstract class.
 public interface VertexScheduler {
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
index 0b8db76..30aefde 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
@@ -25,7 +25,7 @@
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 
 public class TaskAttemptEventStatusUpdate extends TaskAttemptEvent {
   
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
index 4154bd0..1fbe609 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
@@ -19,7 +19,7 @@
 package org.apache.tez.dag.app.dag.event;
 
 import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 public class TaskEventAddTezEvent extends TaskEvent {
 
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
index 37478cb..a872ae2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
@@ -21,7 +21,7 @@
 import java.util.List;
 
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 public class VertexEventRouteEvent extends VertexEvent {
   
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
index 27d9a02..388beba 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
@@ -19,7 +19,7 @@
 package org.apache.tez.dag.app.dag.event;
 
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 public class VertexEventSourceTaskAttemptCompleted extends VertexEvent {
 
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
index ef6de96..c9d3f7d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
@@ -18,7 +18,7 @@
 
 package org.apache.tez.dag.app.dag.event;
 
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 public class VertexEventTaskAttemptCompleted extends VertexEvent {
 
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
index b05a6f1..55a2c86 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
@@ -22,9 +22,9 @@
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public class BroadcastEdgeManager extends EdgeManager {
 
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 d30d178..ee12221 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
@@ -88,10 +88,10 @@
 import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TezBuilderUtils;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
-import org.apache.tez.engine.common.security.TokenCache;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.TokenCache;
 
 import com.google.common.annotations.VisibleForTesting;
 
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
index 060a112..aaca662 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
@@ -31,14 +31,14 @@
 import org.apache.tez.dag.app.dag.event.TaskEventAddTezEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.api.impl.EventMetaData;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+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 org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
 
 public class Edge {
 
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
index 8ee7e55..a4e5f3b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
@@ -22,7 +22,7 @@
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.VertexScheduler;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 /**
  * Starts all tasks immediately on vertex start
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
index a916ad2..29abfac 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
@@ -22,9 +22,9 @@
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public class OneToOneEdgeManager extends EdgeManager {
 
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
index 1d4df5b..3d1d289 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
@@ -22,9 +22,9 @@
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public class ScatterGatherEdgeManager extends EdgeManager {
 
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
index a0ed329..a4dd555 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
@@ -39,11 +39,11 @@
 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.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 /**
  * Starts scheduling tasks when number of completed source tasks crosses 
@@ -264,14 +264,14 @@
       List<byte[]> taskConfs = new ArrayList<byte[]>(finalTaskParallelism);
       try {
         Configuration taskConf = new Configuration(false);
-        taskConf.setInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE,
+        taskConf.setInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE,
             basePartitionRange);
         // create event user payload to inform the task
         for (int i = 0; i < numShufflersWithBaseRange; ++i) {
           taskConfs.add(MRHelpers.createUserPayloadFromConf(taskConf));
         }
         if(finalTaskParallelism > numShufflersWithBaseRange) {
-          taskConf.setInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE,
+          taskConf.setInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE,
               remainderRangeForLastShuffler);
           taskConfs.add(MRHelpers.createUserPayloadFromConf(taskConf));
         }
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 f2b2776..b79f856 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
@@ -89,8 +89,8 @@
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TezBuilderUtils;
-import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 import com.google.common.annotations.VisibleForTesting;
 
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 13fa915..c1a9415 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
@@ -75,8 +75,8 @@
 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.engine.api.impl.TezEvent;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;
 
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 74005b7..520473d 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
@@ -106,15 +106,15 @@
 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.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.api.impl.EventMetaData;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+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 org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.HashMultiset;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
index 14edd96..7a0b6f7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
@@ -22,7 +22,7 @@
 import org.apache.tez.dag.app.ContainerContext;
 import org.apache.tez.dag.app.dag.TaskAttempt;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
 
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
index 76e80f5..4adc302 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
@@ -19,7 +19,7 @@
 
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class AMContainerEventAssignTA extends AMContainerEvent {
 
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 1855dbd..11e8aeb 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
@@ -44,9 +44,9 @@
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.TaskAttemptListener;
-import org.apache.tez.dag.utils.TezEngineChildJVM;
-import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.dag.utils.TezRuntimeChildJVM;
+import org.apache.tez.runtime.library.common.security.TokenCache;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -163,7 +163,7 @@
     myEnv.putAll(vertexEnv);
 
     // Set up the launch command
-    List<String> commands = TezEngineChildJVM.getVMCommand(
+    List<String> commands = TezRuntimeChildJVM.getVMCommand(
         taskAttemptListener.getAddress(), containerId.toString(),
         appContext.getApplicationID().toString(),
         appContext.getApplicationAttemptId().getAttemptId(),
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 94dd580..4731a24 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
@@ -54,7 +54,7 @@
 import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 //import org.apache.tez.dag.app.dag.event.TaskAttemptEventDiagnosticsUpdate;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 @SuppressWarnings("rawtypes")
 public class AMContainerImpl implements AMContainer {
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
index c0ef524..2015505 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
@@ -18,7 +18,7 @@
 
 package org.apache.tez.dag.app.rm.container;
 
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class AMContainerTask {
   private final boolean shouldDie;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezEngineChildJVM.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java
similarity index 98%
rename from tez-dag/src/main/java/org/apache/tez/dag/utils/TezEngineChildJVM.java
rename to tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java
index 8919698..e1219c1 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezEngineChildJVM.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java
@@ -28,7 +28,7 @@
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
-public class TezEngineChildJVM {
+public class TezRuntimeChildJVM {
 
     // FIXME
   public static enum LogName {
diff --git a/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java
similarity index 99%
rename from tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
rename to tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java
index 7a4dd13..fd4c1ee 100644
--- a/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.records;
+package org.apache.tez.runtime.records;
 
 import java.io.DataInput;
 import java.io.DataOutput;
diff --git a/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java
similarity index 97%
rename from tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
rename to tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java
index 13c9088..ff4f267 100644
--- a/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
+++ b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.records;
+package org.apache.tez.runtime.records;
 
 import java.io.DataInput;
 import java.io.DataOutput;
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 6e42673..31513c5 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
@@ -70,7 +70,7 @@
 import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
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 676e747..f2717be 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
@@ -82,7 +82,7 @@
 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.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
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 3d00cb7..b524f6a 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
@@ -82,8 +82,8 @@
 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.engine.records.TezDependentTaskCompletionEvent;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent.Status;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent.Status;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
index 1a07b5b..81715bd 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
@@ -40,8 +40,8 @@
 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.engine.records.TezDependentTaskCompletionEvent;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
index 3a6e008..68ee532 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -56,9 +56,9 @@
 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.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
index c2457e1..fc89e82 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
@@ -65,9 +65,9 @@
 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.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.TokenCache;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.TokenCache;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 
diff --git a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
index 05675b5..e98b45f 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
@@ -77,17 +77,17 @@
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
-import org.apache.tez.engine.common.objectregistry.ObjectLifeCycle;
-import org.apache.tez.engine.common.objectregistry.ObjectRegistry;
-import org.apache.tez.engine.common.objectregistry.ObjectRegistryFactory;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryFactory;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 import com.google.common.annotations.VisibleForTesting;
 
diff --git a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
index ec419c1..cad79f5 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
@@ -70,14 +70,14 @@
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 /**
  * An MRR job built on top of word count to return words sorted by
diff --git a/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java b/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
index 7e662cb..7280a1f 100644
--- a/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
+++ b/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
@@ -68,8 +68,6 @@
 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.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.examples.MRRSleepJob;
 import org.apache.tez.mapreduce.examples.MRRSleepJob.ISleepReducer;
 import org.apache.tez.mapreduce.examples.MRRSleepJob.MRRSleepJobPartitioner;
@@ -82,6 +80,8 @@
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml
index 2c6b78e..aa3d915 100644
--- a/tez-mapreduce/pom.xml
+++ b/tez-mapreduce/pom.xml
@@ -35,7 +35,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-engine</artifactId>
+      <artifactId>tez-runtime-library</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-internals</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
index f2b0a38..22d4a75 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
@@ -853,7 +853,7 @@
 //    LOG.info(TezJobConfig.LOCAL_DIRS + " for child : " + taskAttemptID +
 //        " is " + childMapredLocalDir);
 //    conf.set(TezJobConfig.LOCAL_DIRS, childMapredLocalDir.toString());
-//    conf.setClass(Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+//    conf.setClass(Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
 //        TezLocalTaskOutputFiles.class, TezTaskOutput.class);
 //  }
 //
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
index 199bbfe..3bc8da2 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
@@ -42,18 +42,18 @@
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.TezTaskContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.ValuesIterator;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezTaskContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.ValuesIterator;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
 
 @SuppressWarnings({"rawtypes", "unchecked"})
 public class MRCombiner implements Combiner {
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java
index 21a3983..dac92ed 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java
@@ -23,9 +23,10 @@
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.runtime.library.common.Constants;
+
 import com.google.common.collect.Maps;
 
 public class DeprecatedKeys {
@@ -50,15 +51,15 @@
   
   
   /**
-   * Keys used by the engine.
+   * Keys used by the Tez Runtime.
    */
-  private static Map<String, String> mrParamToEngineParamMap =
+  private static Map<String, String> mrParamToTezRuntimeParamMap =
       new HashMap<String, String>();
 
   
  
   static {
-    populateMRToEngineParamMap();
+    populateMRToTezRuntimeParamMap();
     populateMRToDagParamMap();
     populateMultiStageParamMap();
     addDeprecatedKeys();
@@ -70,32 +71,32 @@
     multiStageParamMap.put(
         MRJobConfig.KEY_COMPARATOR,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_KEY_CLASS,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_CLASS,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_CLASS,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_VALUE_CLASS,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_VALUE_CLASS,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_VALUE_CLASS,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_COMPRESS,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_IS_COMPRESSED,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_IS_COMPRESSED,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_COMPRESS_CODEC,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_COMPRESS_CODEC));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_COMPRESS_CODEC,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_COMPRESS_CODEC));
   }
   
   private static Map<MultiStageKeys, String> getDeprecationMap(String inputKey, String outputKey) {
@@ -130,23 +131,23 @@
   public static void init() {
   }
   
-  private static void populateMRToEngineParamMap() {
+  private static void populateMRToTezRuntimeParamMap() {
     
-    registerMRToEngineKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD, TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD);
+    registerMRToRuntimeKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD, TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD);
 
-    registerMRToEngineKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD_BYTES, TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD_BYTES);
+    registerMRToRuntimeKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD_BYTES, TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD_BYTES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.RECORDS_BEFORE_PROGRESS, TezJobConfig.RECORDS_BEFORE_PROGRESS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.RECORDS_BEFORE_PROGRESS, TezJobConfig.RECORDS_BEFORE_PROGRESS);
 
-    registerMRToEngineKeyTranslation(MRJobConfig.IO_SORT_FACTOR, TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.IO_SORT_FACTOR, TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.MAP_SORT_SPILL_PERCENT, TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.MAP_SORT_SPILL_PERCENT, TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.IO_SORT_MB, TezJobConfig.TEZ_ENGINE_IO_SORT_MB);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.IO_SORT_MB, TezJobConfig.TEZ_RUNTIME_IO_SORT_MB);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.INDEX_CACHE_MEMORY_LIMIT, TezJobConfig.TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.INDEX_CACHE_MEMORY_LIMIT, TezJobConfig.TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.MAP_COMBINE_MIN_SPILLS, TezJobConfig.TEZ_ENGINE_COMBINE_MIN_SPILLS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.MAP_COMBINE_MIN_SPILLS, TezJobConfig.TEZ_RUNTIME_COMBINE_MIN_SPILLS);
     
     // Counter replacement will work in this manner, as long as TezCounters
     // extends MRCounters and is used directly by the Mapper/Reducer.
@@ -154,56 +155,56 @@
     // may break.
     // Framework counters, like FILESYSTEM will likely be incompatible since
     // they enum key belongs to a different package.
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTERS_MAX_KEY, TezJobConfig.COUNTERS_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTERS_MAX_KEY, TezJobConfig.COUNTERS_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTER_GROUP_NAME_MAX_KEY, TezJobConfig.COUNTER_GROUP_NAME_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTER_GROUP_NAME_MAX_KEY, TezJobConfig.COUNTER_GROUP_NAME_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTER_NAME_MAX_KEY, TezJobConfig.COUNTER_NAME_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTER_NAME_MAX_KEY, TezJobConfig.COUNTER_NAME_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTER_GROUPS_MAX_KEY, TezJobConfig.COUNTER_GROUPS_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTER_GROUPS_MAX_KEY, TezJobConfig.COUNTER_GROUPS_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_MEMORY_TOTAL_BYTES, Constants.TEZ_ENGINE_TASK_MEMORY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_MEMORY_TOTAL_BYTES, Constants.TEZ_RUNTIME_TASK_MEMORY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_PARALLEL_COPIES, TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_PARALLEL_COPIES, TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_FETCH_FAILURES, TezJobConfig.TEZ_ENGINE_SHUFFLE_FETCH_FAILURES);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_FETCH_FAILURES, TezJobConfig.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_NOTIFY_READERROR, TezJobConfig.TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_NOTIFY_READERROR, TezJobConfig.TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_CONNECT_TIMEOUT, TezJobConfig.TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_CONNECT_TIMEOUT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_READ_TIMEOUT, TezJobConfig.TEZ_ENGINE_SHUFFLE_READ_TIMEOUT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_READ_TIMEOUT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT);
     
-    registerMRToEngineKeyTranslation(MRConfig.SHUFFLE_SSL_ENABLED_KEY, TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL);
+    registerMRToRuntimeKeyTranslation(MRConfig.SHUFFLE_SSL_ENABLED_KEY, TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_MEMORY_LIMIT_PERCENT, TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_MEMORY_LIMIT_PERCENT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_MERGE_PERCENT, TezJobConfig.TEZ_ENGINE_SHUFFLE_MERGE_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_MERGE_PERCENT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_THRESHOLD, TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_THRESHOLD, TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_ENABLED, TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_ENABLED, TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT);
 
-    registerMRToEngineKeyTranslation(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY, TezJobConfig.DAG_CREDENTIALS_BINARY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY, TezJobConfig.DAG_CREDENTIALS_BINARY);
     
-    registerMRToEngineKeyTranslation("map.sort.class", TezJobConfig.TEZ_ENGINE_INTERNAL_SORTER_CLASS);
+    registerMRToRuntimeKeyTranslation("map.sort.class", TezJobConfig.TEZ_RUNTIME_INTERNAL_SORTER_CLASS);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_ENGINE_GROUP_COMPARATOR_CLASS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_RUNTIME_GROUP_COMPARATOR_CLASS);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS);
 
   }
   
   private static void addDeprecatedKeys() {
   }
 
-  private static void registerMRToEngineKeyTranslation(String mrKey,
+  private static void registerMRToRuntimeKeyTranslation(String mrKey,
       String tezKey) {
-    mrParamToEngineParamMap.put(mrKey, tezKey);
+    mrParamToTezRuntimeParamMap.put(mrKey, tezKey);
   }
   
   @SuppressWarnings("unused")
@@ -215,8 +216,8 @@
     return Collections.unmodifiableMap(mrParamToDAGParamMap);
   }
 
-  public static Map<String, String> getMRToEngineParamMap() {
-    return Collections.unmodifiableMap(mrParamToEngineParamMap);
+  public static Map<String, String> getMRToTezRuntimeParamMap() {
+    return Collections.unmodifiableMap(mrParamToTezRuntimeParamMap);
   }
 
   // TODO Ideally, multi-stage should not be exposed.
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
index b0ed6ab..c39ca4a 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
@@ -367,16 +367,16 @@
     // the AM anyway.
 
     // TODO eventually ACLs
-    conf.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
+    conf.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS, MRPartitioner.class.getName());
     
     boolean useNewApi = conf.getBoolean("mapred.mapper.new-api", false);
     if (useNewApi) {
       if (conf.get(MRJobConfig.COMBINE_CLASS_ATTR) != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     } else {
       if (conf.get("mapred.combiner.class") != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     }
     
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
index d888c42..b07b04b 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
@@ -221,7 +221,7 @@
     int numStages = numIntermediateStages + (hasFinalReduceStage ? 2 : 1);
 
     // Setup Tez partitioner class
-    conf.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS,
+    conf.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS,
         MRPartitioner.class.getName());
     
     // Setup Tez Combiner class if required.
@@ -229,11 +229,11 @@
     boolean useNewApi = conf.getBoolean("mapred.mapper.new-api", false);
     if (useNewApi) {
       if (conf.get(MRJobConfig.COMBINE_CLASS_ATTR) != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     } else {
       if (conf.get("mapred.combiner.class") != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     }
 
@@ -259,7 +259,7 @@
   }
 
   private static void processDirectConversion(Configuration conf) {
-    for (Entry<String, String> dep : DeprecatedKeys.getMRToEngineParamMap()
+    for (Entry<String, String> dep : DeprecatedKeys.getMRToTezRuntimeParamMap()
         .entrySet()) {
       if (conf.get(dep.getKey()) != null) {
         // TODO Deprecation reason does not seem to reflect in the config ?
@@ -336,7 +336,7 @@
       Configuration baseConf, String stage) {
     JobConf jobConf = new JobConf(baseConf);
     // Don't clobber explicit tez config.
-    if (conf.get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS) == null) {
+    if (conf.get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS) == null) {
       // If this is set, but the comparator is not set, and their types differ -
       // the job will break.
       if (conf.get(MRJobConfig.MAP_OUTPUT_KEY_CLASS) == null) {
@@ -352,7 +352,7 @@
       }
     }
 
-    if (conf.get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS) == null) {
+    if (conf.get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS) == null) {
       if (conf.get(MRJobConfig.MAP_OUTPUT_VALUE_CLASS) == null) {
         conf.set(MRJobConfig.MAP_OUTPUT_VALUE_CLASS, jobConf
             .getMapOutputValueClass().getName());
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java
index 0bcd45e..2378f58 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java
@@ -57,7 +57,7 @@
   public static Configuration getAndRemoveBasicNonIntermediateStageConf(
       Configuration baseConf) {
     Configuration newConf = new Configuration(false);
-    for (String key : DeprecatedKeys.getMRToEngineParamMap().keySet()) {
+    for (String key : DeprecatedKeys.getMRToTezRuntimeParamMap().keySet()) {
       if (baseConf.get(key) != null) {
         newConf.set(key, baseConf.get(key));
         baseConf.unset(key);
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
index 2a926d7..635af90 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
@@ -3,9 +3,9 @@
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.common.Utils;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 public class MRReporter implements Reporter {
 
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
index dcdb3ff..2d27c4b 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
@@ -30,7 +30,7 @@
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.tez.engine.api.TezTaskContext;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 /**
  * The context that is given to the {@link Mapper}.
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
index 4035c71..be65be7 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
@@ -24,8 +24,8 @@
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.common.Utils;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 // NOTE: NEWTEZ: This is a copy of org.apache.tez.mapreduce.hadoop.mapred (not mapreduce). mapred likely does not need it's own copy of this class.
 // Meant for use by the "mapreduce" API
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
index 05ea89c..5b5c8ec 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
@@ -29,7 +29,7 @@
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskInputOutputContext;
-import org.apache.tez.engine.api.TezTaskContext;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 /**
  * A context object that allows input and output from the task. It is only
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 6066d93..b9f2242 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
@@ -46,13 +46,14 @@
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.mapreduce.common.Utils;
 import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
 import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.Input;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.api.KVReader;
 
 import com.google.common.base.Preconditions;
 
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 e6bdbe6..11184e4 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
@@ -26,16 +26,16 @@
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezOutputContext;
 import org.apache.tez.mapreduce.common.Utils;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
 import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.api.KVWriter;
 
 public class MROutput implements LogicalOutput {
 
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
index d061ad5..224900e 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
@@ -24,11 +24,11 @@
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
+import org.apache.tez.runtime.library.common.ConfigUtils;
 
 @SuppressWarnings({ "rawtypes", "unchecked" })
-public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
+public class MRPartitioner implements org.apache.tez.runtime.library.api.Partitioner {
 
   static final Log LOG = LogFactory.getLog(MRPartitioner.class);
 
@@ -40,7 +40,7 @@
 
   public MRPartitioner(Configuration conf) {
     this.useNewApi = ConfigUtils.useNewApi(conf);
-    this.partitions = conf.getInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, 1);
+    this.partitions = conf.getInt(TezJobConfig.TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS, 1);
 
     if (useNewApi) {
       if (partitions > 1) {
@@ -85,4 +85,4 @@
       return oldPartitioner.getPartition(key, value, numPartitions);
     }
   }
-}
\ No newline at end of file
+}
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
index fac1454..5471c55 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
@@ -62,7 +62,6 @@
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezTaskStatus.State;
 import org.apache.tez.common.TezUtils;
@@ -70,11 +69,6 @@
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
@@ -82,6 +76,12 @@
 import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
 import org.apache.tez.mapreduce.hadoop.mapreduce.JobContextImpl;
 import org.apache.tez.mapreduce.output.MROutput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.TokenCache;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
 
 @SuppressWarnings("deprecation")
 public abstract class MRTask {
@@ -155,7 +155,7 @@
     } else {
       this.jobConf = new JobConf(conf);
     }
-    jobConf.set(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID,
+    jobConf.set(Constants.TEZ_RUNTIME_TASK_ATTEMPT_ID,
         taskAttemptId.toString());
     jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
         context.getDAGAttemptNumber());
@@ -687,7 +687,4 @@
     return taskAttemptId;
   }
 
-  public TezProcessorContext getTezEngineTaskContext() {
-    return processorContext;
-  }
 }
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
index 85139ed..74a34af 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
@@ -24,12 +24,12 @@
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
 import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
index e4b990a..b7ecddd 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
@@ -35,20 +35,20 @@
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.dag.api.TezException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalIOProcessor;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.mapreduce.MapContextImpl;
 import org.apache.tez.mapreduce.input.MRInput;
 import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalIOProcessor;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 @SuppressWarnings({ "unchecked", "rawtypes" })
 public class MapProcessor extends MRTask implements LogicalIOProcessor {
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
index 19acb39..1ba76f6 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
@@ -37,20 +37,20 @@
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalIOProcessor;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalIOProcessor;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 
 @SuppressWarnings({ "unchecked", "rawtypes" })
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
index 9de2ed1..08b66eb 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
@@ -24,9 +24,9 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.EventMetaData;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezUmbilical;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
 
 public class TestUmbilical implements TezUmbilical {
 
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java
index b75f01e..5e3d201 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java
@@ -29,7 +29,7 @@
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.tez.engine.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.ConfigUtils;
 import org.junit.Test;
 
 public class TestConfigTranslationMRToTez {
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java
index e5bd108..12a3740 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java
@@ -21,8 +21,8 @@
 import static org.junit.Assert.assertEquals;
 
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.library.common.Constants;
 import org.junit.Test;
 
 public class TestDeprecatedKeys {
@@ -43,22 +43,22 @@
     MultiStageMRConfToTezTranslator.translateVertexConfToTez(jobConf, null);
 
     assertEquals(0.4f, jobConf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT, 0f), 0.01f);
-    assertEquals(20000l, jobConf.getLong(Constants.TEZ_ENGINE_TASK_MEMORY, 0));
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT, 0f), 0.01f);
+    assertEquals(20000l, jobConf.getLong(Constants.TEZ_RUNTIME_TASK_MEMORY, 0));
     assertEquals(2000,
-        jobConf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 0));
+        jobConf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 0));
     assertEquals(0.55f, jobConf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT, 0), 0.01f);
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT, 0), 0.01f);
     assertEquals(0.60f,
-        jobConf.getFloat(TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS, 0),
+        jobConf.getFloat(TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS, 0),
         0.01f);
     assertEquals(0.22f,
-        jobConf.getFloat(TezJobConfig.TEZ_ENGINE_SHUFFLE_MERGE_PERCENT, 0),
+        jobConf.getFloat(TezJobConfig.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT, 0),
         0.01f);
     assertEquals(true, jobConf.getBoolean(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM, false));
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM, false));
     assertEquals(0.33f,
-        jobConf.getFloat(TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT, 0),
+        jobConf.getFloat(TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT, 0),
         0.01f);
   }
 
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
index 4b2c0e8..9590e72 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
@@ -48,15 +48,15 @@
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezUmbilical;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
 
 public class MapUtils {
 
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
index 89292ab..5b8eedf 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
@@ -31,19 +31,9 @@
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.OutputDescriptor;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
-import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TestUmbilical;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
@@ -51,6 +41,16 @@
 import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.partition.MRPartitioner;
 import org.apache.tez.mapreduce.processor.MapUtils;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.task.local.output.TezLocalTaskOutputFiles;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.output.LocalOnFileSorterOutput;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -85,10 +85,10 @@
     job.set(TezJobConfig.LOCAL_DIRS, workDir.toString());
     job.set(MRConfig.LOCAL_DIR, workDir.toString());
     job.setClass(
-        Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+        Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
         TezLocalTaskOutputFiles.class, 
         TezTaskOutput.class);
-    job.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
+    job.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS, MRPartitioner.class.getName());
     job.setNumReduceTasks(1);
   }
 
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
index 274c353..d2c7952 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
@@ -33,21 +33,11 @@
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.security.token.Token;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 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.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.lib.input.LocalMergedInput;
-import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
-import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TestUmbilical;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
@@ -58,6 +48,16 @@
 import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.partition.MRPartitioner;
 import org.apache.tez.mapreduce.processor.MapUtils;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.task.local.output.TezLocalTaskOutputFiles;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.input.LocalMergedInput;
+import org.apache.tez.runtime.library.output.LocalOnFileSorterOutput;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -90,10 +90,10 @@
     job.set(TezJobConfig.LOCAL_DIRS, workDir.toString());
     job.set(MRConfig.LOCAL_DIR, workDir.toString());
     job.setClass(
-        Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+        Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
         TezLocalTaskOutputFiles.class, 
         TezTaskOutput.class);
-    job.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
+    job.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS, MRPartitioner.class.getName());
     job.setNumReduceTasks(1);
   }
 
diff --git a/tez-engine/findbugs-exclude.xml b/tez-runtime-internals/findbugs-exclude.xml
similarity index 100%
rename from tez-engine/findbugs-exclude.xml
rename to tez-runtime-internals/findbugs-exclude.xml
diff --git a/tez-engine/pom.xml b/tez-runtime-internals/pom.xml
similarity index 94%
rename from tez-engine/pom.xml
rename to tez-runtime-internals/pom.xml
index 498f2f2..4f64701 100644
--- a/tez-engine/pom.xml
+++ b/tez-runtime-internals/pom.xml
@@ -22,7 +22,7 @@
     <artifactId>tez</artifactId>
     <version>0.2.0-SNAPSHOT</version>
   </parent>
-  <artifactId>tez-engine</artifactId>
+  <artifactId>tez-runtime-internals</artifactId>
 
   <dependencies>
     <dependency>
@@ -38,6 +38,10 @@
       <artifactId>tez-api</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-library</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-yarn-common</artifactId>
     </dependency>
@@ -79,7 +83,6 @@
                 <directory>${basedir}/src/main/proto</directory>
                 <includes>
                   <include>Events.proto</include>
-                  <include>ShufflePayloads.proto</include>
                 </includes>
               </source>
               <output>${project.build.directory}/generated-sources/java</output>
diff --git a/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerContext.java
similarity index 100%
rename from tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerContext.java
diff --git a/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerTask.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerTask.java
index e90f7fa..c865631 100644
--- a/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerTask.java
@@ -22,7 +22,7 @@
 import java.io.IOException;
 
 import org.apache.hadoop.io.Writable;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class ContainerTask implements Writable {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
similarity index 94%
rename from tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
index 9e4129f..1966790 100644
--- a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
@@ -27,8 +27,8 @@
 import org.apache.tez.common.records.ProceedToCompletionResponse;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
 
 /** Protocol that task child process uses to contact its parent process.  The
  * parent is a daemon which which polls the central master for a new map or
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
similarity index 92%
rename from tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
index 29063f9..8aff6d1 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.newruntime;
+package org.apache.tez.runtime;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -38,28 +38,28 @@
 import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.LogicalIOProcessor;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.api.impl.EventMetaData;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezInputContextImpl;
-import org.apache.tez.engine.api.impl.TezOutputContextImpl;
-import org.apache.tez.engine.api.impl.TezProcessorContextImpl;
-import org.apache.tez.engine.api.impl.TezUmbilical;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.Input;
+import org.apache.tez.runtime.api.LogicalIOProcessor;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.Output;
+import org.apache.tez.runtime.api.Processor;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezInputContextImpl;
+import org.apache.tez.runtime.api.impl.TezOutputContextImpl;
+import org.apache.tez.runtime.api.impl.TezProcessorContextImpl;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
similarity index 93%
rename from tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
index 22cbc7c..f018333 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.newruntime;
+package org.apache.tez.runtime;
 
 import java.util.Collection;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -25,9 +25,9 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezUmbilical;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
 
 public abstract class RuntimeTask {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeUtils.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeUtils.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeUtils.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeUtils.java
index 20a029e..8a1b550 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeUtils.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeUtils.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.newruntime;
+package org.apache.tez.runtime;
 
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptCompletedEvent.java
similarity index 91%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptCompletedEvent.java
index a9f2c98..597718f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptCompletedEvent.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.events;
+package org.apache.tez.runtime.api.events;
 
-import org.apache.tez.engine.api.Event;
+import org.apache.tez.runtime.api.Event;
 
 public class TaskAttemptCompletedEvent extends Event {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptFailedEvent.java
similarity index 92%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptFailedEvent.java
index fc67472..935fdbb 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptFailedEvent.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.events;
+package org.apache.tez.runtime.api.events;
 
-import org.apache.tez.engine.api.Event;
+import org.apache.tez.runtime.api.Event;
 
 public class TaskAttemptFailedEvent extends Event {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskStatusUpdateEvent.java
similarity index 95%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskStatusUpdateEvent.java
index c0d1ee6..47c2998 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskStatusUpdateEvent.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.events;
+package org.apache.tez.runtime.api.events;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -24,7 +24,7 @@
 
 import org.apache.hadoop.io.Writable;
 import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.engine.api.Event;
+import org.apache.tez.runtime.api.Event;
 
 public class TaskStatusUpdateEvent extends Event implements Writable {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java
similarity index 98%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java
index 64df7bb..d650fa3 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.io.DataInput;
 import java.io.DataOutput;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java
similarity index 95%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java
index 52fc10d..81ff5fc 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 public enum EventType {
   TASK_ATTEMPT_COMPLETED_EVENT,
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/InputSpec.java
similarity index 98%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/InputSpec.java
index a9ef333..78ed886 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/InputSpec.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.io.DataInput;
 import java.io.DataOutput;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/OutputSpec.java
similarity index 98%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/OutputSpec.java
index 3a1d5d8..4034cdd 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/OutputSpec.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.io.DataInput;
 import java.io.DataOutput;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java
similarity index 98%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java
index 6527777..6e0995a 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java
@@ -15,7 +15,7 @@
  * the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.io.DataInput;
 import java.io.DataOutput;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
similarity index 88%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
index 6841d72..e195cf9 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.io.DataInput;
 import java.io.DataOutput;
@@ -24,20 +24,20 @@
 
 import org.apache.hadoop.io.Writable;
 import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputInformationEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.api.events.TaskAttemptCompletedEvent;
-import org.apache.tez.engine.api.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.api.events.EventProtos.DataMovementEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputFailedEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputInformationEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputReadErrorEventProto;
-import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptCompletedEventProto;
-import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptFailedEventProto;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.EventProtos.DataMovementEventProto;
+import org.apache.tez.runtime.api.events.EventProtos.InputFailedEventProto;
+import org.apache.tez.runtime.api.events.EventProtos.InputInformationEventProto;
+import org.apache.tez.runtime.api.events.EventProtos.InputReadErrorEventProto;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputInformationEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.internals.api.events.SystemEventProtos.TaskAttemptCompletedEventProto;
+import org.apache.tez.runtime.internals.api.events.SystemEventProtos.TaskAttemptFailedEventProto;
 
 import com.google.protobuf.ByteString;
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java
similarity index 98%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java
index dc1a447..af7cebb 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.io.DataInput;
 import java.io.DataOutput;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatResponse.java
similarity index 98%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatResponse.java
index 22ae7eb..10699ac 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatResponse.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.io.DataInput;
 import java.io.DataOutput;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java
similarity index 91%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java
index 245cd3b..9169895 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -27,10 +27,10 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
+import org.apache.tez.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
 
 public class TezInputContextImpl extends TezTaskContextImpl
     implements TezInputContext {
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java
similarity index 91%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java
index 6b42e13..fd4c3a3 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -27,10 +27,10 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
+import org.apache.tez.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
 
 public class TezOutputContextImpl extends TezTaskContextImpl
     implements TezOutputContext {
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java
similarity index 90%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java
index 7ffcfd6..e73baf4 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.nio.ByteBuffer;
 import java.io.IOException;
@@ -27,10 +27,10 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
+import org.apache.tez.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
 
 public class TezProcessorContextImpl extends TezTaskContextImpl
   implements TezProcessorContext {
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java
similarity index 96%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java
index d5a4037..ee9e96d 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.nio.ByteBuffer;
 import java.util.Arrays;
@@ -30,8 +30,8 @@
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.TezTaskContext;
-import org.apache.tez.engine.newruntime.RuntimeTask;
+import org.apache.tez.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 public abstract class TezTaskContextImpl implements TezTaskContext {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezUmbilical.java
similarity index 96%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezUmbilical.java
index 925d87b..addccda 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezUmbilical.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api.impl;
+package org.apache.tez.runtime.api.impl;
 
 import java.io.IOException;
 import java.util.Collection;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryImpl.java
similarity index 91%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryImpl.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryImpl.java
index 351e01c..a47526b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryImpl.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryImpl.java
@@ -16,13 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.objectregistry;
+package org.apache.tez.runtime.common.objectregistry;
 
 import java.util.AbstractMap;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Map.Entry;
 
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+
 import com.google.inject.Singleton;
 
 @Singleton
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryModule.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryModule.java
similarity index 87%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryModule.java
rename to tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryModule.java
index ab346fd..97ccf7c 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryModule.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryModule.java
@@ -16,7 +16,10 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.objectregistry;
+package org.apache.tez.runtime.common.objectregistry;
+
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.inject.AbstractModule;
diff --git a/tez-engine/src/main/proto/Events.proto b/tez-runtime-internals/src/main/proto/Events.proto
similarity index 93%
rename from tez-engine/src/main/proto/Events.proto
rename to tez-runtime-internals/src/main/proto/Events.proto
index fa9cb2c..558a2b3 100644
--- a/tez-engine/src/main/proto/Events.proto
+++ b/tez-runtime-internals/src/main/proto/Events.proto
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-option java_package = "org.apache.tez.engine.api.events";
+option java_package = "org.apache.tez.runtime.internals.api.events";
 option java_outer_classname = "SystemEventProtos";
 option java_generate_equals_and_hash = true;
 
diff --git a/tez-engine/src/test/java/org/apache/tez/engine/common/objectregistry/TestObjectRegistry.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/common/objectregistry/TestObjectRegistry.java
similarity index 85%
rename from tez-engine/src/test/java/org/apache/tez/engine/common/objectregistry/TestObjectRegistry.java
rename to tez-runtime-internals/src/test/java/org/apache/tez/runtime/common/objectregistry/TestObjectRegistry.java
index 7276782..35192e7 100644
--- a/tez-engine/src/test/java/org/apache/tez/engine/common/objectregistry/TestObjectRegistry.java
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/common/objectregistry/TestObjectRegistry.java
@@ -16,8 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.objectregistry;
+package org.apache.tez.runtime.common.objectregistry;
 
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryFactory;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryModule;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
diff --git a/tez-engine/pom.xml b/tez-runtime-library/pom.xml
similarity index 84%
copy from tez-engine/pom.xml
copy to tez-runtime-library/pom.xml
index 498f2f2..dcdabe1 100644
--- a/tez-engine/pom.xml
+++ b/tez-runtime-library/pom.xml
@@ -22,14 +22,10 @@
     <artifactId>tez</artifactId>
     <version>0.2.0-SNAPSHOT</version>
   </parent>
-  <artifactId>tez-engine</artifactId>
+  <artifactId>tez-runtime-library</artifactId>
 
   <dependencies>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-api</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.apache.tez</groupId>
       <artifactId>tez-common</artifactId>
     </dependency>
@@ -38,14 +34,6 @@
       <artifactId>tez-api</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.google.inject</groupId>
-      <artifactId>guice</artifactId>
-    </dependency>
-    <dependency>
      <groupId>com.google.protobuf</groupId>
      <artifactId>protobuf-java</artifactId>
     </dependency>
@@ -78,7 +66,6 @@
               <source>
                 <directory>${basedir}/src/main/proto</directory>
                 <includes>
-                  <include>Events.proto</include>
                   <include>ShufflePayloads.proto</include>
                 </includes>
               </source>
diff --git a/tez-engine/src/main/java/org/apache/hadoop/io/BufferUtils.java b/tez-runtime-library/src/main/java/org/apache/hadoop/io/BufferUtils.java
similarity index 100%
rename from tez-engine/src/main/java/org/apache/hadoop/io/BufferUtils.java
rename to tez-runtime-library/src/main/java/org/apache/hadoop/io/BufferUtils.java
diff --git a/tez-engine/src/main/java/org/apache/hadoop/io/HashComparator.java b/tez-runtime-library/src/main/java/org/apache/hadoop/io/HashComparator.java
similarity index 100%
rename from tez-engine/src/main/java/org/apache/hadoop/io/HashComparator.java
rename to tez-runtime-library/src/main/java/org/apache/hadoop/io/HashComparator.java
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVReader.java
similarity index 95%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVReader.java
index 0178b3a..9c6b380 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVReader.java
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.library.api;
 
 import java.io.IOException;
 
-import org.apache.tez.engine.api.Reader;
+import org.apache.tez.runtime.api.Reader;
 
 /**
  * A key/value(s) pair based {@link Reader}.
@@ -78,4 +78,4 @@
       return this.values;
     }
   }
-}
\ No newline at end of file
+}
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVWriter.java
similarity index 92%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVWriter.java
index 970831b..ff952ed 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVWriter.java
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.library.api;
 
 import java.io.IOException;
 
-import org.apache.tez.engine.api.Writer;
+import org.apache.tez.runtime.api.Writer;
 
 /**
  * A key/value(s) pair based {@link Writer}
@@ -37,4 +37,4 @@
    *           if an error occurs
    */
   public void write(Object key, Object value) throws IOException;
-}
\ No newline at end of file
+}
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/Partitioner.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/Partitioner.java
similarity index 89%
rename from tez-engine/src/main/java/org/apache/tez/engine/api/Partitioner.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/Partitioner.java
index ccf3cb8..680c9b8 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/Partitioner.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/Partitioner.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.api;
+package org.apache.tez.runtime.library.api;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.TezJobConfig;
@@ -25,7 +25,7 @@
  * key/value pairs.
  * 
  * <b>Partitioner Initialization</b></p> The Partitioner class is picked up
- * using the TEZ_ENGINE_PARTITIONER_CLASS attribute in {@link TezJobConfig}
+ * using the TEZ_RUNTIME_PARTITIONER_CLASS attribute in {@link TezJobConfig}
  * 
  * TODO NEWTEZ Change construction to first check for a Constructor with a bytep[] payload
  * 
@@ -33,7 +33,7 @@
  * constructor or a 0 argument constructor. If both exist, preference is given
  * to the single argument constructor. This is primarily for MR support.
  * 
- * If using the configuration constructor, TEZ_ENGINE_NUM_EXPECTED_PARTITIONS
+ * If using the configuration constructor, TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS
  * will be set in the configuration, to indicate the max number of expected
  * partitions.
  * 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastInputManager.java
similarity index 76%
rename from tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastInputManager.java
index 1211598..cda52da 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastInputManager.java
@@ -16,23 +16,23 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.broadcast.input;
+package org.apache.tez.runtime.library.broadcast.input;
 
 import java.io.IOException;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-import org.apache.tez.engine.shuffle.common.DiskFetchedInput;
-import org.apache.tez.engine.shuffle.common.FetchedInput;
-import org.apache.tez.engine.shuffle.common.FetchedInputAllocator;
-import org.apache.tez.engine.shuffle.common.FetchedInputCallback;
-import org.apache.tez.engine.shuffle.common.MemoryFetchedInput;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+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.FetchedInputAllocator;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInputCallback;
+import org.apache.tez.runtime.library.shuffle.common.MemoryFetchedInput;
 
 public class BroadcastInputManager implements FetchedInputAllocator,
     FetchedInputCallback {
@@ -57,25 +57,25 @@
 
     // Setup configuration
     final float maxInMemCopyUse = conf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT);
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT);
     if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0) {
       throw new IllegalArgumentException("Invalid value for "
-          + TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT + ": "
+          + TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT + ": "
           + maxInMemCopyUse);
     }
 
     // Allow unit tests to fix Runtime memory
-    this.memoryLimit = (long) (conf.getLong(Constants.TEZ_ENGINE_TASK_MEMORY,
+    this.memoryLimit = (long) (conf.getLong(Constants.TEZ_RUNTIME_TASK_MEMORY,
         Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE)) * maxInMemCopyUse);
 
     final float singleShuffleMemoryLimitPercent = conf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT);
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT);
     if (singleShuffleMemoryLimitPercent <= 0.0f
         || singleShuffleMemoryLimitPercent > 1.0f) {
       throw new IllegalArgumentException("Invalid value for "
-          + TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
+          + TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
           + singleShuffleMemoryLimitPercent);
     }
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastKVReader.java
similarity index 92%
rename from tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastKVReader.java
index 2c53e75..16e9645 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastKVReader.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.broadcast.input;
+package org.apache.tez.runtime.library.broadcast.input;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -30,13 +30,13 @@
 import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.shuffle.impl.InMemoryReader;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.shuffle.common.FetchedInput;
-import org.apache.tez.engine.shuffle.common.FetchedInput.Type;
-import org.apache.tez.engine.shuffle.common.MemoryFetchedInput;
+import org.apache.tez.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.InMemoryReader;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.MemoryFetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
 
 public class BroadcastKVReader<K, V> implements KVReader {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleInputEventHandler.java
similarity index 85%
rename from tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleInputEventHandler.java
index e89e892..c64379a 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleInputEventHandler.java
@@ -17,20 +17,20 @@
  */
 
 
-package org.apache.tez.engine.broadcast.input;
+package org.apache.tez.runtime.library.broadcast.input;
 
 import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleInputEventHandler;
-import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleInputEventHandler;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.InvalidProtocolBufferException;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleManager.java
similarity index 91%
rename from tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleManager.java
index 7b205fa..2a5c22f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleManager.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.broadcast.input;
+package org.apache.tez.runtime.library.broadcast.input;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -49,22 +49,22 @@
 import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.InputIdentifier;
-import org.apache.tez.engine.common.TezEngineUtils;
-import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
-import org.apache.tez.engine.shuffle.common.FetchResult;
-import org.apache.tez.engine.shuffle.common.FetchedInput;
-import org.apache.tez.engine.shuffle.common.Fetcher;
-import org.apache.tez.engine.shuffle.common.FetcherCallback;
-import org.apache.tez.engine.shuffle.common.InputHost;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
-import org.apache.tez.engine.shuffle.common.Fetcher.FetcherBuilder;
-import org.apache.tez.engine.shuffle.common.FetchedInputAllocator;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.InputIdentifier;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.runtime.library.shuffle.common.FetchResult;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInputAllocator;
+import org.apache.tez.runtime.library.shuffle.common.Fetcher;
+import org.apache.tez.runtime.library.shuffle.common.FetcherCallback;
+import org.apache.tez.runtime.library.shuffle.common.InputHost;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.library.shuffle.common.Fetcher.FetcherBuilder;
 
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.FutureCallback;
@@ -139,8 +139,8 @@
     
     int maxConfiguredFetchers = 
         conf.getInt(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
     
     this.numFetchers = Math.min(maxConfiguredFetchers, numInputs);
     
@@ -157,11 +157,11 @@
             .getServiceConsumerMetaData(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID));
     
     this.connectionTimeout = conf.getInt(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_STALLED_COPY_TIMEOUT);
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_STALLED_COPY_TIMEOUT);
     this.readTimeout = conf.getInt(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_READ_TIMEOUT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_READ_TIMEOUT);
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT);
     
     if (ConfigUtils.isIntermediateInputCompressed(conf)) {
       Class<? extends CompressionCodec> codecClass = ConfigUtils
@@ -349,7 +349,7 @@
     // For now, reporting immediately.
     InputReadErrorEvent readError = new InputReadErrorEvent(
         "Fetch failure while fetching from "
-            + TezEngineUtils.getTaskAttemptIdentifier(
+            + TezRuntimeUtils.getTaskAttemptIdentifier(
                 inputContext.getSourceVertexName(),
                 srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
                 srcAttemptIdentifier.getAttemptNumber()),
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/output/FileBasedKVWriter.java
similarity index 85%
rename from tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/output/FileBasedKVWriter.java
index 474d1cd..8b19ce0 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/output/FileBasedKVWriter.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.broadcast.output;
+package org.apache.tez.runtime.library.broadcast.output;
 
 import java.io.IOException;
 
@@ -29,14 +29,14 @@
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.TezEngineUtils;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.TezIndexRecord;
-import org.apache.tez.engine.common.sort.impl.TezSpillRecord;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
+import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
 
 public class FileBasedKVWriter implements KVWriter {
 
@@ -81,7 +81,7 @@
       codec = null;
     }
 
-    this.ouputFileManager = TezEngineUtils.instantiateTaskOutputManager(conf,
+    this.ouputFileManager = TezRuntimeUtils.instantiateTaskOutputManager(conf,
         outputContext);
 
     initWriter();
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java
similarity index 84%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java
index f73adfd..d1b7ced 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common;
+package org.apache.tez.runtime.library.common;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.RawComparator;
@@ -34,7 +34,7 @@
       Configuration conf, Class<DefaultCodec> defaultValue) {
     Class<? extends CompressionCodec> codecClass = defaultValue;
     String name = conf
-        .get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_COMPRESS_CODEC);
+        .get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_COMPRESS_CODEC);
     if (name != null) {
       try {
         codecClass = conf.getClassByName(name).asSubclass(
@@ -51,7 +51,7 @@
       Configuration conf, Class<DefaultCodec> defaultValue) {
     Class<? extends CompressionCodec> codecClass = defaultValue;
     String name = conf
-        .get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_COMPRESS_CODEC);
+        .get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_COMPRESS_CODEC);
     if (name != null) {
       try {
         codecClass = conf.getClassByName(name).asSubclass(
@@ -69,45 +69,45 @@
   
   public static boolean shouldCompressIntermediateOutput(Configuration conf) {
     return conf.getBoolean(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS, false);
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS, false);
   }
 
   public static boolean isIntermediateInputCompressed(Configuration conf) {
     return conf.getBoolean(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_IS_COMPRESSED, false);
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_IS_COMPRESSED, false);
   }
 
   public static <V> Class<V> getIntermediateOutputValueClass(Configuration conf) {
     Class<V> retv = (Class<V>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS, null,
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS, null,
         Object.class);
     return retv;
   }
   
   public static <V> Class<V> getIntermediateInputValueClass(Configuration conf) {
     Class<V> retv = (Class<V>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_VALUE_CLASS, null,
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_VALUE_CLASS, null,
         Object.class);
     return retv;
   }
 
   public static <K> Class<K> getIntermediateOutputKeyClass(Configuration conf) {
     Class<K> retv = (Class<K>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS, null,
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS, null,
         Object.class);
     return retv;
   }
 
   public static <K> Class<K> getIntermediateInputKeyClass(Configuration conf) {
     Class<K> retv = (Class<K>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_CLASS, null,
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_CLASS, null,
         Object.class);
     return retv;
   }
 
   public static <K> RawComparator<K> getIntermediateOutputKeyComparator(Configuration conf) {
     Class<? extends RawComparator> theClass = conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS, null,
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS, null,
         RawComparator.class);
     if (theClass != null)
       return ReflectionUtils.newInstance(theClass, conf);
@@ -117,7 +117,7 @@
 
   public static <K> RawComparator<K> getIntermediateInputKeyComparator(Configuration conf) {
     Class<? extends RawComparator> theClass = conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS, null,
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS, null,
         RawComparator.class);
     if (theClass != null)
       return ReflectionUtils.newInstance(theClass, conf);
@@ -132,7 +132,7 @@
       Configuration conf) {
     Class<? extends RawComparator> theClass = conf
         .getClass(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS,
             null, RawComparator.class);
     if (theClass == null) {
       return getIntermediateInputKeyComparator(conf);
diff --git a/tez-engine/src/main/java/org/apache/tez/common/Constants.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java
similarity index 65%
rename from tez-engine/src/main/java/org/apache/tez/common/Constants.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java
index 8ea2909..33cd0f6 100644
--- a/tez-engine/src/main/java/org/apache/tez/common/Constants.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java
@@ -15,7 +15,7 @@
  * the License.
  */
 
-package org.apache.tez.common;
+package org.apache.tez.runtime.library.common;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 
@@ -34,24 +34,28 @@
   public static String MERGED_OUTPUT_PREFIX = ".merged";
   
   // TODO NEWTEZ Remove this constant once the old code is removed.
-  public static final String TEZ_ENGINE_TASK_ATTEMPT_ID = 
-      "tez.engine.task.attempt.id";
+  public static final String TEZ_RUNTIME_TASK_ATTEMPT_ID = 
+      "tez.runtime.task.attempt.id";
 
-  public static final String TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING = "file.out";
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING =
+      "file.out";
 
-  public static final String TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING = ".index";
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING =
+      ".index";
 
-  public static final String TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING = "%s/task_%d.out"; 
+  public static final String TEZ_RUNTIME_TASK_INPUT_FILE_FORMAT_STRING =
+      "%s/task_%d.out"; 
 
-  public static final String TEZ_ENGINE_JOB_CREDENTIALS = 
-      "tez.engine.job.credentials";
+  public static final String TEZ_RUNTIME_JOB_CREDENTIALS =
+      "tez.runtime.job.credentials";
   
   @Private
-  public static final String TEZ_ENGINE_TASK_MEMORY =  "tez.engine.task.memory";
+  public static final String TEZ_RUNTIME_TASK_MEMORY =
+      "tez.runtime.task.memory";
   
-  public static final String TASK_OUTPUT_DIR = "output";
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_DIR = "output";
   
-  public static final String TEZ_ENGINE_TASK_OUTPUT_MANAGER = 
-      "tez.engine.task.local.output.manager";
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_MANAGER = 
+      "tez.runtime.task.local.output.manager";
 
 }
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/InputAttemptIdentifier.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/InputAttemptIdentifier.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
index 076807e..a13f3f1 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/InputAttemptIdentifier.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common;
+package org.apache.tez.runtime.library.common;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/InputIdentifier.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputIdentifier.java
similarity index 96%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/InputIdentifier.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputIdentifier.java
index b694530..f4ce190 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/InputIdentifier.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputIdentifier.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common;
+package org.apache.tez.runtime.library.common;
 
 public class InputIdentifier {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java
similarity index 83%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java
index cc29e94..2381780 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common;
+package org.apache.tez.runtime.library.common;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
@@ -25,20 +25,19 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Partitioner;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.TezTaskContext;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezTaskContext;
+import org.apache.tez.runtime.library.api.Partitioner;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
 
-public class TezEngineUtils {
+public class TezRuntimeUtils {
 
   private static final Log LOG = LogFactory
-      .getLog(TezEngineUtils.class);
+      .getLog(TezRuntimeUtils.class);
   
   public static String getTaskIdentifier(String vertexName, int taskIndex) {
     return String.format("%s_%06d", vertexName, taskIndex);
@@ -59,9 +58,9 @@
   @SuppressWarnings("unchecked")
   public static Combiner instantiateCombiner(Configuration conf, TezTaskContext taskContext) throws IOException {
     Class<? extends Combiner> clazz;
-    String className = conf.get(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS);
+    String className = conf.get(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS);
     if (className == null) {
-      LOG.info("No combiner specified via " + TezJobConfig.TEZ_ENGINE_COMBINER_CLASS + ". Combiner will not be used");
+      LOG.info("No combiner specified via " + TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS + ". Combiner will not be used");
       return null;
     }
     LOG.info("Using Combiner class: " + className);
@@ -99,7 +98,7 @@
     Class<? extends Partitioner> clazz;
     try {
       clazz = (Class<? extends Partitioner>) conf
-          .getClassByName(conf.get(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS));
+          .getClassByName(conf.get(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS));
     } catch (ClassNotFoundException e) {
       throw new IOException("Unable to find Partitioner class in config", e);
     }
@@ -136,7 +135,7 @@
   }
   
   public static TezTaskOutput instantiateTaskOutputManager(Configuration conf, TezOutputContext outputContext) {
-    Class<?> clazz = conf.getClass(Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+    Class<?> clazz = conf.getClass(Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
         TezTaskOutputFiles.class);
     try {
       Constructor<?> ctor = clazz.getConstructor(Configuration.class, String.class);
@@ -146,7 +145,7 @@
     } catch (Exception e) {
       throw new TezUncheckedException(
           "Unable to instantiate configured TezOutputFileManager: "
-              + conf.get(Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+              + conf.get(Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
                   TezTaskOutputFiles.class.getName()), e);
     }
   }
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java
index b7867aa..fef3356 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common;
+package org.apache.tez.runtime.library.common;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -28,7 +28,7 @@
 import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
 
 import com.google.common.base.Preconditions;
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/YARNMaster.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/YARNMaster.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/YARNMaster.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/YARNMaster.java
index 48ad639..8709e05 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/YARNMaster.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/YARNMaster.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common;
+package org.apache.tez.runtime.library.common;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/combine/Combiner.java
similarity index 83%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/combine/Combiner.java
index b387b36..5b10590 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/combine/Combiner.java
@@ -16,19 +16,19 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.combine;
+package org.apache.tez.runtime.library.common.combine;
 
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
 
 /**
  *<b>Combiner Initialization</b></p> The Combiner class is picked up
- * using the TEZ_ENGINE_COMBINER_CLASS attribute in {@link TezJobConfig}
+ * using the TEZ_RUNTIME_COMBINER_CLASS attribute in {@link TezJobConfig}
  * 
  * 
  * Partitioners need to provide a single argument ({@link TezTaskContext})
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/localshuffle/LocalShuffle.java
similarity index 85%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/localshuffle/LocalShuffle.java
index 546151f..b40df6f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/localshuffle/LocalShuffle.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common.localshuffle;
+package org.apache.tez.runtime.library.common.localshuffle;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -32,12 +32,12 @@
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.impl.TezMerger;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.task.local.output.TezLocalTaskOutputFiles;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
 
 @SuppressWarnings({"rawtypes"})
 public class LocalShuffle {
@@ -70,8 +70,8 @@
     
     this.sortFactor =
         conf.getInt(
-            TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
+            TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
     
     this.rfs = FileSystem.getLocal(conf).getRaw();
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenIdentifier.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenIdentifier.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenIdentifier.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenIdentifier.java
index 827001b..4b916fa 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenIdentifier.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenIdentifier.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.security;
+package org.apache.tez.runtime.library.common.security;
 
 import java.io.DataInput;
 import java.io.DataOutput;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSecretManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSecretManager.java
similarity index 98%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSecretManager.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSecretManager.java
index d957b8b..a03ee94 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSecretManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSecretManager.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.security;
+package org.apache.tez.runtime.library.common.security;
 
 import java.util.Map;
 import java.util.TreeMap;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSelector.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSelector.java
similarity index 96%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSelector.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSelector.java
index 080b9e2..b8227ab 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSelector.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSelector.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.security;
+package org.apache.tez.runtime.library.common.security;
 
 import java.util.Collection;
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/Master.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/Master.java
similarity index 96%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/security/Master.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/Master.java
index 20cad0a..5b91e0f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/Master.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/Master.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.security;
+package org.apache.tez.runtime.library.common.security;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/SecureShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java
similarity index 98%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/security/SecureShuffleUtils.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java
index 71d990a..b2d382c 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/SecureShuffleUtils.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.security;
+package org.apache.tez.runtime.library.common.security;
 
 
 import java.io.ByteArrayOutputStream;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/TokenCache.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/TokenCache.java
similarity index 98%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/security/TokenCache.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/TokenCache.java
index 5c42d22..33373b7 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/TokenCache.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/TokenCache.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.security;
+package org.apache.tez.runtime.library.common.security;
 
 import java.io.IOException;
 import java.util.HashSet;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ExceptionReporter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ExceptionReporter.java
similarity index 93%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ExceptionReporter.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ExceptionReporter.java
index 89a5888..a872ba1 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ExceptionReporter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ExceptionReporter.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 /**
  * An interface for reporting exceptions to other threads
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
similarity index 95%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
index b48bb0b..f5d1802 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import java.io.DataInputStream;
 import java.io.IOException;
@@ -46,12 +46,12 @@
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.security.SecureShuffleUtils;
-import org.apache.tez.engine.common.shuffle.impl.MapOutput.Type;
-import org.apache.tez.engine.common.sort.impl.IFileInputStream;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.MapOutput.Type;
+import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -129,18 +129,18 @@
     }
 
     this.connectionTimeout = 
-        job.getInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT,
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_STALLED_COPY_TIMEOUT);
+        job.getInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT,
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_STALLED_COPY_TIMEOUT);
     this.readTimeout = 
-        job.getInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_READ_TIMEOUT, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_READ_TIMEOUT);
+        job.getInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT);
 
     setName("fetcher#" + id);
     setDaemon(true);
 
     synchronized (Fetcher.class) {
-      sslShuffle = job.getBoolean(TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL,
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL);
+      sslShuffle = job.getBoolean(TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL,
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL);
       if (sslShuffle && sslFactory == null) {
         sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, job);
         try {
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryReader.java
similarity index 94%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryReader.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryReader.java
index 1beed44..ae95268 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryReader.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryReader.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import java.io.File;
 import java.io.FileOutputStream;
@@ -25,9 +25,9 @@
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.IFile.Reader;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Reader;
 
 /**
  * <code>IFile.InMemoryReader</code> to read map-outputs present in-memory.
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryWriter.java
similarity index 92%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryWriter.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryWriter.java
index b3ebb8b..f81b28e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryWriter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryWriter.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -27,9 +27,9 @@
 import org.apache.hadoop.io.BoundedByteArrayOutputStream;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.WritableUtils;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.IFileOutputStream;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.IFileOutputStream;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapHost.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapHost.java
similarity index 95%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapHost.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapHost.java
index 20ec472..b8be657 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapHost.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapHost.java
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 
 @Private
 class MapHost {
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapOutput.java
similarity index 96%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapOutput.java
index b2a0b54..9f673a0 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapOutput.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import java.io.IOException;
 import java.io.OutputStream;
@@ -29,8 +29,8 @@
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BoundedByteArrayOutputStream;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
 
 
 class MapOutput {
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeManager.java
similarity index 93%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeManager.java
index b8792fb..0abe530 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeManager.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -43,20 +43,20 @@
 import org.apache.hadoop.io.compress.DefaultCodec;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.TezMerger;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-import org.apache.tez.engine.hadoop.compat.NullProgressable;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger.Segment;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
+import org.apache.tez.runtime.library.hadoop.compat.NullProgressable;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
@@ -143,33 +143,33 @@
 
     final float maxInMemCopyUse =
       conf.getFloat(
-          TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT, 
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT);
+          TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT, 
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT);
     if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0) {
       throw new IllegalArgumentException("Invalid value for " +
-          TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT + ": " +
+          TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT + ": " +
           maxInMemCopyUse);
     }
 
     // Allow unit tests to fix Runtime memory
     this.memoryLimit = 
-      (long)(conf.getLong(Constants.TEZ_ENGINE_TASK_MEMORY,
+      (long)(conf.getLong(Constants.TEZ_RUNTIME_TASK_MEMORY,
           Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE))
         * maxInMemCopyUse);
  
     this.ioSortFactor = 
         conf.getInt(
-            TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
+            TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
 
     final float singleShuffleMemoryLimitPercent =
         conf.getFloat(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT,
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT);
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT,
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT);
     if (singleShuffleMemoryLimitPercent <= 0.0f
         || singleShuffleMemoryLimitPercent > 1.0f) {
       throw new IllegalArgumentException("Invalid value for "
-          + TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
+          + TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
           + singleShuffleMemoryLimitPercent);
     }
 
@@ -177,13 +177,13 @@
       (long)(memoryLimit * singleShuffleMemoryLimitPercent);
     this.memToMemMergeOutputsThreshold = 
             conf.getInt(
-                TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS, 
+                TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS, 
                 ioSortFactor);
     this.mergeThreshold = 
         (long)(this.memoryLimit * 
                conf.getFloat(
-                   TezJobConfig.TEZ_ENGINE_SHUFFLE_MERGE_PERCENT, 
-                   TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_MERGE_PERCENT));
+                   TezJobConfig.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT, 
+                   TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT));
     LOG.info("MergerManager: memoryLimit=" + memoryLimit + ", " +
              "maxSingleShuffleLimit=" + maxSingleShuffleLimit + ", " +
              "mergeThreshold=" + mergeThreshold + ", " + 
@@ -199,8 +199,8 @@
 
     boolean allowMemToMemMerge = 
       conf.getBoolean(
-          TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM, 
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM);
+          TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM, 
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM);
     if (allowMemToMemMerge) {
       this.memToMemMerger = 
         new IntermediateMemoryToMemoryMerger(this,
@@ -649,10 +649,10 @@
     
     final float maxRedPer =
       job.getFloat(
-          TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT,
-          TezJobConfig.DEFAULT_TEZ_ENGINE_INPUT_BUFFER_PERCENT);
+          TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT,
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_INPUT_BUFFER_PERCENT);
     if (maxRedPer > 1.0 || maxRedPer < 0.0) {
-      throw new IOException(TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT +
+      throw new IOException(TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT +
                             maxRedPer);
     }
     int maxInMemReduce = (int)Math.min(
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeThread.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeThread.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeThread.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeThread.java
index bab882e..d8a7722 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeThread.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeThread.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import java.io.IOException;
 import java.util.ArrayList;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Shuffle.java
similarity index 91%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Shuffle.java
index 15332a1..8689d11 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Shuffle.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import java.io.IOException;
 import java.util.List;
@@ -38,13 +38,13 @@
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.TezEngineUtils;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
 
 import com.google.common.base.Preconditions;
 
@@ -68,7 +68,7 @@
   private final AtomicInteger reduceStartId;
   private final SecretKey jobTokenSecret;
   private AtomicInteger reduceRange = new AtomicInteger(
-      TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT);
+      TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE_DEFAULT);
 
   private FutureTask<TezRawKeyValueIterator> runShuffleFuture;
 
@@ -85,7 +85,7 @@
         .getJobTokenSecretFromTokenBytes(inputContext
             .getServiceConsumerMetaData(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID));
     
-    Combiner combiner = TezEngineUtils.instantiateCombiner(conf, inputContext);
+    Combiner combiner = TezRuntimeUtils.instantiateCombiner(conf, inputContext);
     
     FileSystem localFS = FileSystem.getLocal(this.conf);
     LocalDirAllocator localDirAllocator = 
@@ -187,8 +187,8 @@
       // TODO NEWTEZ Limit # fetchers to number of inputs
       final int numFetchers = 
           conf.getInt(
-              TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES, 
-              TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
+              TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES, 
+              TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
       Fetcher[] fetchers = new Fetcher[numFetchers];
       for (int i = 0; i < numFetchers; ++i) {
         fetchers[i] = new Fetcher(conf, scheduler, merger, metrics, Shuffle.this, jobTokenSecret, inputContext);
@@ -266,7 +266,7 @@
       return;
     }
     if (reduceRange.compareAndSet(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT, range)) {
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE_DEFAULT, range)) {
       LOG.info("Reduce range set to: " + range);
     } else {
       TezUncheckedException e = 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleClientMetrics.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleClientMetrics.java
similarity index 84%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleClientMetrics.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleClientMetrics.java
index 850dbeb..70de31f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleClientMetrics.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleClientMetrics.java
@@ -15,16 +15,16 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsRecord;
 import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.metrics.Updater;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.TezEngineUtils;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
 
 class ShuffleClientMetrics implements Updater {
 
@@ -39,19 +39,19 @@
       String user) {
     this.numCopiers = 
         conf.getInt(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
 
     MetricsContext metricsContext = MetricsUtil.getContext(Constants.TEZ);
     this.shuffleMetrics = 
       MetricsUtil.createRecord(metricsContext, "shuffleInput");
     this.shuffleMetrics.setTag("user", user);
     this.shuffleMetrics.setTag("dagName", dagName);
-    this.shuffleMetrics.setTag("taskId", TezEngineUtils.getTaskIdentifier(vertexName, taskIndex));
+    this.shuffleMetrics.setTag("taskId", TezRuntimeUtils.getTaskIdentifier(vertexName, taskIndex));
     this.shuffleMetrics.setTag("sessionId", 
         conf.get(
-            TezJobConfig.TEZ_ENGINE_METRICS_SESSION_ID, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_METRICS_SESSION_ID));
+            TezJobConfig.TEZ_RUNTIME_METRICS_SESSION_ID, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_METRICS_SESSION_ID));
     metricsContext.registerUpdater(this);
   }
   public synchronized void inputBytes(long numBytes) {
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleHeader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleHeader.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleHeader.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleHeader.java
index a918ef1..327473e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleHeader.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleHeader.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import java.io.DataInput;
 import java.io.DataOutput;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandler.java
similarity index 88%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandler.java
index a8e5fe4..8b323b5 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandler.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import java.net.URI;
 import java.util.List;
@@ -24,14 +24,14 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputInformationEvent;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
-import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.InputInformationEventPayloadProto;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputInformationEvent;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.InputInformationEventPayloadProto;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.InvalidProtocolBufferException;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleScheduler.java
similarity index 94%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleScheduler.java
index be75668..a682a09 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleScheduler.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.shuffle.impl;
+package org.apache.tez.runtime.library.common.shuffle.impl;
 
 import java.io.IOException;
 import java.text.DecimalFormat;
@@ -40,11 +40,11 @@
 import org.apache.hadoop.io.IntWritable;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.TezEngineUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
 
 import com.google.common.collect.Lists;
 
@@ -120,12 +120,12 @@
         this.maxFailedUniqueFetches);
     this.maxFetchFailuresBeforeReporting = 
         conf.getInt(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_FETCH_FAILURES, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_FETCH_FAILURES_LIMIT);
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT);
     this.reportReadErrorImmediately = 
         conf.getBoolean(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR);
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR);
   }
 
   public synchronized void copySucceeded(InputAttemptIdentifier srcAttemptIdentifier, 
@@ -134,7 +134,7 @@
                                          long milis,
                                          MapOutput output
                                          ) throws IOException {
-    String taskIdentifier = TezEngineUtils.getTaskAttemptIdentifier(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(), srcAttemptIdentifier.getAttemptNumber());
+    String taskIdentifier = TezRuntimeUtils.getTaskAttemptIdentifier(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(), srcAttemptIdentifier.getAttemptNumber());
     failureCounts.remove(taskIdentifier);
     hostFailures.remove(host.getHostName());
     
@@ -154,7 +154,7 @@
       reduceShuffleBytes.increment(bytes);
       if (LOG.isDebugEnabled()) {
         LOG.debug("src task: "
-            + TezEngineUtils.getTaskAttemptIdentifier(
+            + TezRuntimeUtils.getTaskAttemptIdentifier(
                 inputContext.getSourceVertexName(), srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
                 srcAttemptIdentifier.getAttemptNumber()) + " done");
       }
@@ -195,7 +195,7 @@
       try {
         throw new IOException(failures
             + " failures downloading "
-            + TezEngineUtils.getTaskAttemptIdentifier(
+            + TezRuntimeUtils.getTaskAttemptIdentifier(
                 inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
                 srcAttempt.getAttemptNumber()));
       } catch (IOException ie) {
@@ -223,13 +223,13 @@
     if ((reportReadErrorImmediately && readError)
         || ((failures % maxFetchFailuresBeforeReporting) == 0)) {
       LOG.info("Reporting fetch failure for "
-          + TezEngineUtils.getTaskAttemptIdentifier(
+          + TezRuntimeUtils.getTaskAttemptIdentifier(
               inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
               srcAttempt.getAttemptNumber()) + " to jobtracker.");
 
       List<Event> failedEvents = Lists.newArrayListWithCapacity(1);
       failedEvents.add(new InputReadErrorEvent("Fetch failure for "
-          + TezEngineUtils.getTaskAttemptIdentifier(
+          + TezRuntimeUtils.getTaskAttemptIdentifier(
               inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
               srcAttempt.getAttemptNumber()) + " to jobtracker.", srcAttempt.getInputIdentifier()
           .getSrcTaskIndex(), srcAttempt.getAttemptNumber()));
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/server/ShuffleHandler.java
similarity index 96%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/server/ShuffleHandler.java
index a6d1c5b..9a206c6 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/server/ShuffleHandler.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common.shuffle.server;
+package org.apache.tez.runtime.library.common.shuffle.server;
 
 import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
 import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
@@ -64,13 +64,13 @@
 import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext;
 import org.apache.hadoop.yarn.server.api.AuxiliaryService;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
-import org.apache.tez.engine.common.security.SecureShuffleUtils;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
-import org.apache.tez.engine.common.sort.impl.ExternalSorter;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.ExternalSorter;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
 import org.jboss.netty.bootstrap.ServerBootstrap;
 import org.jboss.netty.buffer.ChannelBuffers;
 import org.jboss.netty.channel.Channel;
@@ -314,8 +314,8 @@
 
     public HttpPipelineFactory(Configuration conf) throws Exception {
       SHUFFLE = new Shuffle(conf);
-      if (conf.getBoolean(TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL,
-              TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL)) {
+      if (conf.getBoolean(TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL,
+              TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL)) {
         sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
         sslFactory.init();
       }
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
similarity index 85%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
index 5aa0ddf..c362d98 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common.sort.impl;
+package org.apache.tez.runtime.library.common.sort.impl;
 
 import java.io.File;
 import java.io.IOException;
@@ -42,15 +42,15 @@
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Partitioner;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.TezEngineUtils;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.hadoop.compat.NullProgressable;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.api.Partitioner;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.hadoop.compat.NullProgressable;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public abstract class ExternalSorter {
@@ -99,7 +99,7 @@
 
     // sorter
     sorter = ReflectionUtils.newInstance(this.conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERNAL_SORTER_CLASS, QuickSort.class,
+        TezJobConfig.TEZ_RUNTIME_INTERNAL_SORTER_CLASS, QuickSort.class,
         IndexedSorter.class), this.conf);
 
     comparator = ConfigUtils.getIntermediateOutputKeyComparator(this.conf);
@@ -130,12 +130,12 @@
     }
 
     // Task outputs
-    mapOutputFile = TezEngineUtils.instantiateTaskOutputManager(conf, outputContext);
+    mapOutputFile = TezRuntimeUtils.instantiateTaskOutputManager(conf, outputContext);
     
-    LOG.info("Instantiating Partitioner: [" + conf.get(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS) + "]");
-    this.conf.setInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, this.partitions);
-    this.partitioner = TezEngineUtils.instantiatePartitioner(this.conf);
-    this.combiner = TezEngineUtils.instantiateCombiner(this.conf, outputContext);
+    LOG.info("Instantiating Partitioner: [" + conf.get(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS) + "]");
+    this.conf.setInt(TezJobConfig.TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS, this.partitions);
+    this.partitioner = TezRuntimeUtils.instantiatePartitioner(this.conf);
+    this.combiner = TezRuntimeUtils.instantiateCombiner(this.conf, outputContext);
   }
 
   /**
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
similarity index 99%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFile.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
index 7d10606..4ce82d5 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFile.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.sort.impl;
+package org.apache.tez.runtime.library.common.sort.impl;
 
 import java.io.DataInput;
 import java.io.DataInputStream;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileInputStream.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileInputStream.java
similarity index 95%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileInputStream.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileInputStream.java
index dfb69f1..e828c0b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileInputStream.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileInputStream.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.sort.impl;
+package org.apache.tez.runtime.library.common.sort.impl;
 
 import java.io.EOFException;
 import java.io.FileDescriptor;
@@ -82,10 +82,10 @@
     dataLength = length - checksumSize;
 
     conf = (conf != null) ? conf : new Configuration();
-    readahead = conf.getBoolean(TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_IFILE_READAHEAD);
-    readaheadLength = conf.getInt(TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD_BYTES,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_IFILE_READAHEAD_BYTES);
+    readahead = conf.getBoolean(TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD);
+    readaheadLength = conf.getInt(TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD_BYTES,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD_BYTES);
 
     doReadahead();
   }
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileOutputStream.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileOutputStream.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java
index 3b39900..3198446 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileOutputStream.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.sort.impl;
+package org.apache.tez.runtime.library.common.sort.impl;
 
 import java.io.FilterOutputStream;
 import java.io.IOException;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
similarity index 95%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
index 1bf17a3..1b153ca 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
@@ -15,7 +15,7 @@
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */
-package org.apache.tez.engine.common.sort.impl;
+package org.apache.tez.runtime.library.common.sort.impl;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -44,10 +44,10 @@
 import org.apache.hadoop.util.IndexedSorter;
 import org.apache.hadoop.util.Progress;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger.Segment;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class PipelinedSorter extends ExternalSorter {
@@ -96,35 +96,35 @@
     //sanity checks
     final float spillper =
       this.conf.getFloat(
-          TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT, 
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SORT_SPILL_PERCENT);
+          TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT, 
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SORT_SPILL_PERCENT);
     final int sortmb = 
         this.conf.getInt(
-            TezJobConfig.TEZ_ENGINE_IO_SORT_MB, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_MB);
-    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES,
-                                       TezJobConfig.DEFAULT_TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES);
+            TezJobConfig.TEZ_RUNTIME_IO_SORT_MB, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_MB);
+    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES,
+                                       TezJobConfig.DEFAULT_TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES);
     if (spillper > (float)1.0 || spillper <= (float)0.0) {
-      throw new IOException("Invalid \"" + TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT +
+      throw new IOException("Invalid \"" + TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT +
           "\": " + spillper);
     }
     if ((sortmb & 0x7FF) != sortmb) {
       throw new IOException(
-          "Invalid \"" + TezJobConfig.TEZ_ENGINE_IO_SORT_MB + "\": " + sortmb);
+          "Invalid \"" + TezJobConfig.TEZ_RUNTIME_IO_SORT_MB + "\": " + sortmb);
     }
     
     // buffers and accounting
     int maxMemUsage = sortmb << 20;
     maxMemUsage -= maxMemUsage % METASIZE;
     largeBuffer = ByteBuffer.allocate(maxMemUsage);
-    LOG.info(TezJobConfig.TEZ_ENGINE_IO_SORT_MB + " = " + sortmb);
+    LOG.info(TezJobConfig.TEZ_RUNTIME_IO_SORT_MB + " = " + sortmb);
     // TODO: configurable setting?
     span = new SortSpan(largeBuffer, 1024*1024, 16);
     merger = new SpanMerger(comparator);
     final int sortThreads = 
             this.conf.getInt(
-                TezJobConfig.TEZ_ENGINE_SORT_THREADS, 
-                TezJobConfig.DEFAULT_TEZ_ENGINE_SORT_THREADS);
+                TezJobConfig.TEZ_RUNTIME_SORT_THREADS, 
+                TezJobConfig.DEFAULT_TEZ_RUNTIME_SORT_THREADS);
     sortmaster = Executors.newFixedThreadPool(sortThreads);
 
     // k/v serialization    
@@ -136,7 +136,7 @@
     }    
     valSerializer.open(span.out);
     keySerializer.open(span.out);
-    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_ENGINE_COMBINE_MIN_SPILLS, 3);
+    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_COMBINE_MIN_SPILLS, 3);
   }
 
   private int bitcount(int n) {
@@ -359,8 +359,8 @@
       }
 
       int mergeFactor = 
-              this.conf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 
-                  TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
+              this.conf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 
+                  TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
       // sort the segments only if there are intermediate merges
       boolean sortSegments = segmentList.size() > mergeFactor;
       //merge
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezIndexRecord.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezIndexRecord.java
similarity index 95%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezIndexRecord.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezIndexRecord.java
index ac0267c..95ae8eb 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezIndexRecord.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezIndexRecord.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common.sort.impl;
+package org.apache.tez.runtime.library.common.sort.impl;
 
 public class TezIndexRecord {
   private long startOffset;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezMerger.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
similarity index 98%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezMerger.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
index 7815569..bb4b4a2 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezMerger.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.sort.impl;
+package org.apache.tez.runtime.library.common.sort.impl;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -39,11 +39,11 @@
 import org.apache.hadoop.util.PriorityQueue;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.Progressable;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.common.sort.impl.IFile.Reader;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Reader;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
 
 /**
  * Merger is an utility class used by the Map and Reduce tasks for merging
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezRawKeyValueIterator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezRawKeyValueIterator.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java
index 39cffcb..3a2c2bf 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezRawKeyValueIterator.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.sort.impl;
+package org.apache.tez.runtime.library.common.sort.impl;
 
 import java.io.IOException;
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezSpillRecord.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezSpillRecord.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java
index 19fbd7f..ab4142b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezSpillRecord.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.common.sort.impl;
+package org.apache.tez.runtime.library.common.sort.impl;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -32,7 +32,7 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.tez.common.Constants;
+import org.apache.tez.runtime.library.common.Constants;
 
 public class TezSpillRecord {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
similarity index 95%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
index 6b48270..1ff486f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common.sort.impl.dflt;
+package org.apache.tez.runtime.library.common.sort.impl.dflt;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
@@ -40,16 +40,16 @@
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.impl.ExternalSorter;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.TezIndexRecord;
-import org.apache.tez.engine.common.sort.impl.TezMerger;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.common.sort.impl.TezSpillRecord;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.ExternalSorter;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger.Segment;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class DefaultSorter extends ExternalSorter implements IndexedSortable {
@@ -116,21 +116,21 @@
 
     // sanity checks
     final float spillper = this.conf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SORT_SPILL_PERCENT);
-    final int sortmb = this.conf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_MB,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_MB);
+        TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SORT_SPILL_PERCENT);
+    final int sortmb = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_MB,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_MB);
     if (spillper > (float) 1.0 || spillper <= (float) 0.0) {
       throw new IOException("Invalid \""
-          + TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT + "\": " + spillper);
+          + TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT + "\": " + spillper);
     }
     if ((sortmb & 0x7FF) != sortmb) {
-      throw new IOException("Invalid \"" + TezJobConfig.TEZ_ENGINE_IO_SORT_MB
+      throw new IOException("Invalid \"" + TezJobConfig.TEZ_RUNTIME_IO_SORT_MB
           + "\": " + sortmb);
     }
 
-    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES,
-                                       TezJobConfig.DEFAULT_TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES);
+    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES,
+                                       TezJobConfig.DEFAULT_TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES);
 
     // buffers and accounting
     int maxMemUsage = sortmb << 20;
@@ -148,7 +148,7 @@
     softLimit = (int)(kvbuffer.length * spillper);
     bufferRemaining = softLimit;
     if (LOG.isInfoEnabled()) {
-      LOG.info(TezJobConfig.TEZ_ENGINE_IO_SORT_MB + ": " + sortmb);
+      LOG.info(TezJobConfig.TEZ_RUNTIME_IO_SORT_MB + ": " + sortmb);
       LOG.info("soft limit at " + softLimit);
       LOG.info("bufstart = " + bufstart + "; bufvoid = " + bufvoid);
       LOG.info("kvstart = " + kvstart + "; length = " + maxRec);
@@ -159,7 +159,7 @@
     keySerializer.open(bb);
 
     spillInProgress = false;
-    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_ENGINE_COMBINE_MIN_SPILLS, 3);
+    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_COMBINE_MIN_SPILLS, 3);
     spillThread.setDaemon(true);
     spillThread.setName("SpillThread");
     spillLock.lock();
@@ -1063,8 +1063,8 @@
         }
 
         int mergeFactor =
-            this.conf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR,
-                TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
+            this.conf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR,
+                TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
         // sort the segments only if there are intermediate merges
         boolean sortSegments = segmentList.size() > mergeFactor;
         //merge
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/InMemoryShuffleSorter.java
similarity index 91%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/InMemoryShuffleSorter.java
index e2b3315..92ae916 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/InMemoryShuffleSorter.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common.sort.impl.dflt;
+package org.apache.tez.runtime.library.common.sort.impl.dflt;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -29,10 +29,10 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
-import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
-import org.apache.tez.engine.common.sort.impl.IFile;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
 
 public class InMemoryShuffleSorter extends DefaultSorter {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/SortBufferInputStream.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/SortBufferInputStream.java
similarity index 96%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/SortBufferInputStream.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/SortBufferInputStream.java
index d74e159..b7b1e82 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/SortBufferInputStream.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/SortBufferInputStream.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common.sort.impl.dflt;
+package org.apache.tez.runtime.library.common.sort.impl.dflt;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -27,8 +27,8 @@
 import org.apache.hadoop.io.BoundedByteArrayOutputStream;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.WritableUtils;
-import org.apache.tez.engine.common.shuffle.impl.InMemoryWriter;
-import org.apache.tez.engine.common.sort.impl.dflt.DefaultSorter.InMemValBytes;
+import org.apache.tez.runtime.library.common.shuffle.impl.InMemoryWriter;
+import org.apache.tez.runtime.library.common.sort.impl.dflt.DefaultSorter.InMemValBytes;
 
   public class SortBufferInputStream extends InputStream {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/impl/ValuesIterator.java
similarity index 96%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/impl/ValuesIterator.java
index 841e54d..88cb750 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/impl/ValuesIterator.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.common.task.impl;
+package org.apache.tez.runtime.library.common.task.impl;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -28,7 +28,7 @@
 import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.util.Progressable;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
 
 
 /**
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezLocalTaskOutputFiles.java
similarity index 73%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezLocalTaskOutputFiles.java
index 40e6b1a..30d28f0 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezLocalTaskOutputFiles.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.task.local.output;
+package org.apache.tez.runtime.library.common.task.local.output;
 
 import java.io.IOException;
 
@@ -26,9 +26,9 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 
 /**
  * Manipulate the working area for the transient store for maps and reduces.
@@ -58,8 +58,8 @@
   @Override
   public Path getOutputFile()
       throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, conf);
+    return lDirAlloc.getLocalPathToRead(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING, conf);
   }
 
   /**
@@ -72,8 +72,8 @@
   @Override
   public Path getOutputFileForWrite(long size)
       throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, size, conf);
+    return lDirAlloc.getLocalPathForWrite(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING, size, conf);
   }
   
   /**
@@ -86,8 +86,8 @@
    */
   @Override
   public Path getOutputFileForWrite() throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR
-        + Path.SEPARATOR + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING,
+    return lDirAlloc.getLocalPathForWrite(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR
+        + Path.SEPARATOR + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING,
         conf);
   }
 
@@ -96,7 +96,7 @@
    */
   @Override
   public Path getOutputFileForWriteInVolume(Path existing) {
-    return new Path(existing.getParent(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
+    return new Path(existing.getParent(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING);
   }
 
   /**
@@ -108,8 +108,8 @@
   @Override
   public Path getOutputIndexFile()
       throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING,
+    return lDirAlloc.getLocalPathToRead(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING,
         conf);
   }
 
@@ -123,8 +123,8 @@
   @Override
   public Path getOutputIndexFileForWrite(long size)
       throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING,
+    return lDirAlloc.getLocalPathForWrite(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING,
         size, conf);
   }
 
@@ -134,7 +134,7 @@
   @Override
   public Path getOutputIndexFileForWriteInVolume(Path existing) {
     return new Path(existing.getParent(),
-        Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
+        Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING);
   }
 
   /**
@@ -147,7 +147,7 @@
   @Override
   public Path getSpillFile(int spillNumber)
       throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
+    return lDirAlloc.getLocalPathToRead(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/spill"
         + spillNumber + ".out", conf);
   }
 
@@ -162,7 +162,7 @@
   @Override
   public Path getSpillFileForWrite(int spillNumber, long size)
       throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
+    return lDirAlloc.getLocalPathForWrite(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/spill"
         + spillNumber + ".out", size, conf);
   }
 
@@ -176,7 +176,7 @@
   @Override
   public Path getSpillIndexFile(int spillNumber)
       throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
+    return lDirAlloc.getLocalPathToRead(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/spill"
         + spillNumber + ".out.index", conf);
   }
 
@@ -191,7 +191,7 @@
   @Override
   public Path getSpillIndexFileForWrite(int spillNumber, long size)
       throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
+    return lDirAlloc.getLocalPathForWrite(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/spill"
         + spillNumber + ".out.index", size, conf);
   }
 
@@ -206,8 +206,8 @@
   public Path getInputFile(InputAttemptIdentifier mapId)
       throws IOException {
     return lDirAlloc.getLocalPathToRead(String.format(
-        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, 
-        Constants.TASK_OUTPUT_DIR, Integer.valueOf(mapId.getInputIdentifier().getSrcTaskIndex())), conf);
+        Constants.TEZ_RUNTIME_TASK_INPUT_FILE_FORMAT_STRING, 
+        Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR, Integer.valueOf(mapId.getInputIdentifier().getSrcTaskIndex())), conf);
   }
 
   /**
@@ -223,7 +223,7 @@
                                    long size)
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(String.format(
-        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, Constants.TASK_OUTPUT_DIR, taskId),
+        Constants.TEZ_RUNTIME_TASK_INPUT_FILE_FORMAT_STRING, Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR, taskId),
         size, conf);
   }
 
@@ -231,7 +231,7 @@
   @Override
   public void removeAll()
       throws IOException {
-    deleteLocalFiles(Constants.TASK_OUTPUT_DIR);
+    deleteLocalFiles(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR);
   }
 
   private String[] getLocalDirs() throws IOException {
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutput.java
similarity index 96%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutput.java
index e1d83ad..d3e7d27 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutput.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.task.local.output;
+package org.apache.tez.runtime.library.common.task.local.output;
 
 import java.io.IOException;
 
@@ -24,7 +24,7 @@
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 
 /**
  * Manipulate the working area for the transient store for maps and reduces.
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java
similarity index 83%
rename from tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java
index b8f051b..2c18b4e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.common.task.local.output;
+package org.apache.tez.runtime.library.common.task.local.output;
 
 import java.io.IOException;
 
@@ -27,9 +27,9 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 
 /**
  * Manipulate the working area for the transient store for maps and reduces.
@@ -65,10 +65,10 @@
   private Path getAttemptOutputDir() {
     if (LOG.isDebugEnabled()) {
       LOG.debug("getAttemptOutputDir: "
-          + Constants.TASK_OUTPUT_DIR + "/"
+          + Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/"
           + uniqueId);
     }
-    return new Path(Constants.TASK_OUTPUT_DIR, uniqueId);
+    return new Path(Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR, uniqueId);
   }
 
   /**
@@ -79,7 +79,7 @@
    */
   public Path getOutputFile() throws IOException {
     Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
+      new Path(getAttemptOutputDir(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING);
     return lDirAlloc.getLocalPathToRead(attemptOutput.toString(), conf);
   }
 
@@ -92,7 +92,7 @@
    */
   public Path getOutputFileForWrite(long size) throws IOException {
     Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
+      new Path(getAttemptOutputDir(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING);
     return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), size, conf);
   }
 
@@ -106,7 +106,7 @@
    */
   public Path getOutputFileForWrite() throws IOException {
     Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
+      new Path(getAttemptOutputDir(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING);
     return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), conf);
   }
 
@@ -114,9 +114,9 @@
    * Create a local map output file name on the same volume.
    */
   public Path getOutputFileForWriteInVolume(Path existing) {
-    Path outputDir = new Path(existing.getParent(), Constants.TASK_OUTPUT_DIR);
+    Path outputDir = new Path(existing.getParent(), Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR);
     Path attemptOutputDir = new Path(outputDir, uniqueId);
-    return new Path(attemptOutputDir, Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
+    return new Path(attemptOutputDir, Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING);
   }
 
   /**
@@ -127,8 +127,8 @@
    */
   public Path getOutputIndexFile() throws IOException {
     Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
-                                      Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
+      new Path(getAttemptOutputDir(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING +
+                                      Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING);
     return lDirAlloc.getLocalPathToRead(attemptIndexOutput.toString(), conf);
   }
 
@@ -141,8 +141,8 @@
    */
   public Path getOutputIndexFileForWrite(long size) throws IOException {
     Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
-                                      Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
+      new Path(getAttemptOutputDir(), Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING +
+                                      Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING);
     return lDirAlloc.getLocalPathForWrite(attemptIndexOutput.toString(),
         size, conf);
   }
@@ -151,10 +151,10 @@
    * Create a local map output index file name on the same volume.
    */
   public Path getOutputIndexFileForWriteInVolume(Path existing) {
-    Path outputDir = new Path(existing.getParent(), Constants.TASK_OUTPUT_DIR);
+    Path outputDir = new Path(existing.getParent(), Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR);
     Path attemptOutputDir = new Path(outputDir, uniqueId);
-    return new Path(attemptOutputDir, Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
-                                      Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
+    return new Path(attemptOutputDir, Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING +
+                                      Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING);
   }
 
   /**
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/hadoop/compat/NullProgressable.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/hadoop/compat/NullProgressable.java
similarity index 94%
rename from tez-engine/src/main/java/org/apache/tez/engine/hadoop/compat/NullProgressable.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/hadoop/compat/NullProgressable.java
index 5071dd2..eb8d176 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/hadoop/compat/NullProgressable.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/hadoop/compat/NullProgressable.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.hadoop.compat;
+package org.apache.tez.runtime.library.hadoop.compat;
 
 import org.apache.hadoop.util.Progressable;
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/LocalMergedInput.java
similarity index 86%
rename from tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/LocalMergedInput.java
index 6371787..3aec247 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/LocalMergedInput.java
@@ -15,17 +15,17 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.lib.input;
+package org.apache.tez.runtime.library.input;
 
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 
 import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.localshuffle.LocalShuffle;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.localshuffle.LocalShuffle;
 
 /**
  * <code>LocalMergedInput</code> in an {@link LogicalInput} which shuffles intermediate
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInput.java
similarity index 91%
rename from tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInput.java
index a984b0f..771ac1b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInput.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.lib.input;
+package org.apache.tez.runtime.library.input;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -29,14 +29,14 @@
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVReader;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.ValuesIterator;
-import org.apache.tez.engine.common.shuffle.impl.Shuffle;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.ValuesIterator;
+import org.apache.tez.runtime.library.common.shuffle.impl.Shuffle;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
 
 /**
  * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInputLegacy.java
similarity index 88%
rename from tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInputLegacy.java
index f2da031..97e19d8 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInputLegacy.java
@@ -10,13 +10,13 @@
  * 
  */
 
-package org.apache.tez.engine.lib.input;
+package org.apache.tez.runtime.library.input;
 
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
 
 @LimitedPrivate("mapreduce")
 public class ShuffledMergedInputLegacy extends ShuffledMergedInput {
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledUnorderedKVInput.java
similarity index 87%
rename from tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledUnorderedKVInput.java
index 44238fd..42b2e00 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledUnorderedKVInput.java
@@ -16,18 +16,18 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.lib.input;
+package org.apache.tez.runtime.library.input;
 
 import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.Reader;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.broadcast.input.BroadcastShuffleManager;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.Reader;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.broadcast.input.BroadcastShuffleManager;
 
 import com.google.common.base.Preconditions;
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/InMemorySortedOutput.java
similarity index 84%
rename from tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/InMemorySortedOutput.java
index 26a01c8..2ec6b2a 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/InMemorySortedOutput.java
@@ -15,20 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.lib.output;
+package org.apache.tez.runtime.library.output;
 
 import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 
 import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.Writer;
-import org.apache.tez.engine.common.sort.impl.dflt.InMemoryShuffleSorter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.Output;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.Writer;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.sort.impl.dflt.InMemoryShuffleSorter;
 
 /**
  * {@link InMemorySortedOutput} is an {@link Output} which sorts key/value pairs 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/LocalOnFileSorterOutput.java
similarity index 91%
rename from tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/LocalOnFileSorterOutput.java
index 7fd26d7..a19d5e1 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/LocalOnFileSorterOutput.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.lib.output;
+package org.apache.tez.runtime.library.output;
 
 import java.io.IOException;
 import java.util.List;
@@ -25,8 +25,8 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
 
 public class LocalOnFileSorterOutput extends OnFileSortedOutput {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileSortedOutput.java
similarity index 86%
rename from tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileSortedOutput.java
index 9c9eba0..42e1eeb 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileSortedOutput.java
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.tez.engine.lib.output;
+package org.apache.tez.runtime.library.output;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -26,15 +26,15 @@
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
-import org.apache.tez.engine.common.sort.impl.ExternalSorter;
-import org.apache.tez.engine.common.sort.impl.dflt.DefaultSorter;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.sort.impl.ExternalSorter;
+import org.apache.tez.runtime.library.common.sort.impl.dflt.DefaultSorter;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
 
 import com.google.common.collect.Lists;
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileUnorderedKVOutput.java
similarity index 84%
rename from tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileUnorderedKVOutput.java
index 3ff603f..dd18149 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileUnorderedKVOutput.java
@@ -16,7 +16,7 @@
 * limitations under the License.
 */
 
-package org.apache.tez.engine.lib.output;
+package org.apache.tez.runtime.library.output;
 
 import java.nio.ByteBuffer;
 import java.util.Collections;
@@ -24,14 +24,14 @@
 
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.broadcast.output.FileBasedKVWriter;
-import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.broadcast.output.FileBasedKVWriter;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
similarity index 94%
rename from tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
index 531e460..a98ce63 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.shuffle.common;
+package org.apache.tez.runtime.library.shuffle.common;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -28,8 +28,8 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
 
 import com.google.common.base.Preconditions;
 
@@ -108,4 +108,4 @@
         + ", inputAttemptIdentifier=" + inputAttemptIdentifier + ", size="
         + size + ", type=" + type + ", id=" + id + ", state=" + state + "]";
   }
-}
\ No newline at end of file
+}
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchResult.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchResult.java
similarity index 94%
rename from tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchResult.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchResult.java
index fb0b324..df38b07 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchResult.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchResult.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.shuffle.common;
+package org.apache.tez.runtime.library.shuffle.common;
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -35,7 +35,7 @@
  * limitations under the License.
  */
 
-import org.apache.tez.engine.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 
 public class FetchResult {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
similarity index 96%
rename from tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
index f5339d3..8f3c407 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.shuffle.common;
+package org.apache.tez.runtime.library.shuffle.common;
 
 import java.io.IOException;
 import java.io.InputStream;
@@ -24,7 +24,7 @@
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 
 @Private
 public abstract class FetchedInput {
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputAllocator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
similarity index 88%
rename from tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputAllocator.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
index 7e573f0..1d60b68 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputAllocator.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.shuffle.common;
+package org.apache.tez.runtime.library.shuffle.common;
 
 import java.io.IOException;
 
-import org.apache.tez.engine.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 
 public interface FetchedInputAllocator {
 
@@ -28,4 +28,4 @@
       InputAttemptIdentifier inputAttemptIdentifier) throws IOException;
   
   
-}
\ No newline at end of file
+}
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputCallback.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputCallback.java
similarity index 94%
rename from tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputCallback.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputCallback.java
index 2d2d73b..a22ce45 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputCallback.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputCallback.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.shuffle.common;
+package org.apache.tez.runtime.library.shuffle.common;
 
 public interface FetchedInputCallback {
   
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
similarity index 97%
rename from tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/Fetcher.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
index a353416..ab7e5ba 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/Fetcher.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.shuffle.common;
+package org.apache.tez.runtime.library.shuffle.common;
 
 import java.io.DataInputStream;
 import java.io.IOException;
@@ -48,11 +48,11 @@
 import org.apache.hadoop.security.ssl.SSLFactory;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.security.SecureShuffleUtils;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
-import org.apache.tez.engine.common.sort.impl.IFileInputStream;
-import org.apache.tez.engine.shuffle.common.FetchedInput.Type;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
 
 import com.google.common.base.Preconditions;
 
@@ -116,8 +116,8 @@
       if (!sslFactoryInited) {
         sslFactoryInited = true;
         sslShuffle = conf.getBoolean(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL,
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL);
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL,
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL);
         if (sslShuffle) {
           sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
           try {
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetcherCallback.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetcherCallback.java
similarity index 90%
rename from tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetcherCallback.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetcherCallback.java
index f0b7cd2..18504b1 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetcherCallback.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetcherCallback.java
@@ -16,11 +16,11 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.shuffle.common;
+package org.apache.tez.runtime.library.shuffle.common;
 
 import java.io.IOException;
 
-import org.apache.tez.engine.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 
 public interface FetcherCallback {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/InputHost.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/InputHost.java
similarity index 94%
rename from tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/InputHost.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/InputHost.java
index 4862b76..66605dd 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/InputHost.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/InputHost.java
@@ -16,14 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.shuffle.common;
+package org.apache.tez.runtime.library.shuffle.common;
 
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 
 public class InputHost {
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/MemoryFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/MemoryFetchedInput.java
similarity index 94%
rename from tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/MemoryFetchedInput.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/MemoryFetchedInput.java
index 59d288e..f56877f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/MemoryFetchedInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/MemoryFetchedInput.java
@@ -16,14 +16,14 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.shuffle.common;
+package org.apache.tez.runtime.library.shuffle.common;
 
 import java.io.ByteArrayInputStream;
 import java.io.InputStream;
 import java.io.OutputStream;
 
 import org.apache.hadoop.io.BoundedByteArrayOutputStream;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 
 import com.google.common.base.Preconditions;
 
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/ShuffleUtils.java
similarity index 92%
rename from tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/ShuffleUtils.java
rename to tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/ShuffleUtils.java
index 84d270d..7479e7e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/ShuffleUtils.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/ShuffleUtils.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.tez.engine.shuffle.common;
+package org.apache.tez.runtime.library.shuffle.common;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
@@ -27,8 +27,8 @@
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
 
 public class ShuffleUtils {
 
diff --git a/tez-engine/src/main/proto/ShufflePayloads.proto b/tez-runtime-library/src/main/proto/ShufflePayloads.proto
similarity index 94%
rename from tez-engine/src/main/proto/ShufflePayloads.proto
rename to tez-runtime-library/src/main/proto/ShufflePayloads.proto
index f831de2..1fb000f 100644
--- a/tez-engine/src/main/proto/ShufflePayloads.proto
+++ b/tez-runtime-library/src/main/proto/ShufflePayloads.proto
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-option java_package = "org.apache.tez.engine.common.shuffle.newimpl";
+option java_package = "org.apache.tez.runtime.library.shuffle.impl";
 option java_outer_classname = "ShuffleUserPayloads";
 option java_generate_equals_and_hash = true;
 
diff --git a/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java b/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
index 6496b55..a6d250f 100644
--- a/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
+++ b/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
@@ -95,8 +95,6 @@
 import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
@@ -104,6 +102,8 @@
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 import com.google.common.annotations.VisibleForTesting;