You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by ss...@apache.org on 2013/09/23 19:45:15 UTC

[01/20] TEZ-444. Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module (part of TEZ-398). (sseth)

Updated Branches:
  refs/heads/TEZ-398 5eb0c86f2 -> d316f7235


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/MRRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/MRRuntimeTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/MRRuntimeTask.java
deleted file mode 100644
index 8a4c6c1..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/task/MRRuntimeTask.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.task;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map.Entry;
-
-import javax.crypto.SecretKey;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSError;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.MapOutputFile;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.hadoop.mapreduce.filecache.DistributedCache;
-import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.task.RuntimeTask;
-import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
-import org.apache.tez.mapreduce.hadoop.MRHelpers;
-import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.processor.MRTask;
-import org.apache.tez.mapreduce.task.impl.YarnOutputFiles;
-
-@SuppressWarnings("deprecation")
-public class MRRuntimeTask extends RuntimeTask {
-
-  private static final Log LOG = LogFactory.getLog(MRRuntimeTask.class);
-
-  private MRTask mrTask;
-
-  public MRRuntimeTask(TezEngineTaskContext taskContext, Processor processor,
-      Input[] inputs, Output[] outputs) {
-    super(taskContext, processor, inputs, outputs);
-  }
-
-  @Override
-  public void initialize(Configuration conf, byte[] userPayload,
-      Master master) throws IOException, InterruptedException {
-
-    DeprecatedKeys.init();
-
-    Configuration taskConf;
-    if (userPayload != null) {
-      taskConf = MRHelpers.createConfFromUserPayload(userPayload);
-    } else {
-      taskConf = new Configuration(false);
-    }
-
-    copyTezConfigParameters(taskConf, conf);
-
-    // TODO Avoid all this extra config manipulation.
-    // FIXME we need I/O/p level configs to be used in init below
-
-    // TODO Post MRR
-    // A single file per vertex will likely be a better solution. Does not
-    // require translation - client can take care of this. Will work independent
-    // of whether the configuration is for intermediate tasks or not. Has the
-    // overhead of localizing multiple files per job - i.e. the client would
-    // need to write these files to hdfs, add them as local resources per
-    // vertex. A solution like this may be more practical once it's possible to
-    // submit configuration parameters to the AM and effectively tasks via RPC.
-
-    final JobConf job = new JobConf(taskConf);
-    job.set(MRJobConfig.VERTEX_NAME, taskContext.getVertexName());
-
-    MRTask mrTask = (MRTask) getProcessor();
-    this.mrTask = mrTask;
-
-    if (LOG.isDebugEnabled() && userPayload != null) {
-      Iterator<Entry<String, String>> iter = taskConf.iterator();
-      String taskIdStr = mrTask.getTaskAttemptId().getTaskID().toString();
-      while (iter.hasNext()) {
-        Entry<String, String> confEntry = iter.next();
-        LOG.debug("TaskConf Entry"
-            + ", taskId=" + taskIdStr
-            + ", key=" + confEntry.getKey()
-            + ", value=" + confEntry.getValue());
-      }
-    }
-
-    configureMRTask(job, mrTask);
-
-    this.conf = job;
-    this.master = master;
-
-    // NOTE: Allow processor to initialize input/output
-    processor.initialize(this.conf, this.master);
-  }
-  
-  /*
-   * Used when creating a conf from the userPayload. Need to copy all the tez
-   * config parameters which are set by YarnTezDagChild
-   */
-  public static void copyTezConfigParameters(Configuration conf,
-      Configuration tezTaskConf) {
-    Iterator<Entry<String, String>> iter = tezTaskConf.iterator();
-    while (iter.hasNext()) {
-      Entry<String, String> entry = iter.next();
-      if (conf.get(entry.getKey()) == null) {
-        conf.set(entry.getKey(), tezTaskConf.get(entry.getKey()));
-      }
-    }
-  }
-
-  @Override
-  public void run() throws IOException, InterruptedException {
-    TezTaskUmbilicalProtocol umbilical = (TezTaskUmbilicalProtocol) master;
-    try {
-      super.run();
-    } catch (FSError e) {
-      throw e;
-    } catch (Exception exception) {
-      LOG.warn("Exception running child : "
-          + StringUtils.stringifyException(exception));
-      try {
-        if (mrTask != null) {
-          mrTask.taskCleanup(umbilical);
-        }
-      } catch (Exception e) {
-        LOG.info("Exception cleanup up: " + StringUtils.stringifyException(e));
-      }
-      ByteArrayOutputStream baos = new ByteArrayOutputStream();
-      exception.printStackTrace(new PrintStream(baos));
-      /* broken code due to engine re-factor
-      if (taskContext.getTaskAttemptId() != null) {
-        umbilical.fatalError(taskContext.getTaskAttemptId(), baos.toString());
-      }
-      */
-    }
-  }
-
-  @Override
-  public void close() throws IOException, InterruptedException {
-    // NOTE: Allow processor to close input/output
-    processor.close();
-  }
-
-  private static void configureMRTask(JobConf job, MRTask task)
-      throws IOException, InterruptedException {
-
-    Credentials credentials = UserGroupInformation.getCurrentUser()
-        .getCredentials();
-    job.setCredentials(credentials);
-    // TODO Can this be avoided all together. Have the MRTezOutputCommitter use
-    // the Tez parameter.
-    // TODO This could be fetched from the env if YARN is setting it for all
-    // Containers.
-    // Set it in conf, so as to be able to be used the the OutputCommitter.
-    job.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
-        job.getInt(MRJobConfig.APPLICATION_ATTEMPT_ID, -1));
-
-    job.setClass(MRConfig.TASK_LOCAL_OUTPUT_CLASS, YarnOutputFiles.class,
-        MapOutputFile.class); // MR
-
-    Token<JobTokenIdentifier> jobToken = TokenCache.getJobToken(credentials);
-    if (jobToken != null) {
-      // Will MR ever run without a job token.
-      SecretKey sk = JobTokenSecretManager.createSecretKey(jobToken
-          .getPassword());
-      task.setJobTokenSecret(sk);
-    } else {
-      LOG.warn("No job token set");
-    }
-
-    job.set(MRJobConfig.JOB_LOCAL_DIR, job.get(TezJobConfig.JOB_LOCAL_DIR));
-    job.set(MRConfig.LOCAL_DIR, job.get(TezJobConfig.LOCAL_DIRS));
-    if (job.get(TezJobConfig.DAG_CREDENTIALS_BINARY) != null) {
-      job.set(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY,
-          job.get(TezJobConfig.DAG_CREDENTIALS_BINARY));
-    }
-
-    // setup the child's attempt directories
-    // Do the task-type specific localization
-    task.localizeConfiguration(job);
-
-    // Set up the DistributedCache related configs
-    setupDistributedCacheConfig(job);
-
-    task.setConf(job);
-  }
-
-  /**
-   * Set up the DistributedCache related configs to make
-   * {@link DistributedCache#getLocalCacheFiles(Configuration)} and
-   * {@link DistributedCache#getLocalCacheArchives(Configuration)} working.
-   * 
-   * @param job
-   * @throws IOException
-   */
-  private static void setupDistributedCacheConfig(final JobConf job)
-      throws IOException {
-
-    String localWorkDir = (job.get(TezJobConfig.TASK_LOCAL_RESOURCE_DIR));
-    // ^ ^ all symlinks are created in the current work-dir
-
-    // Update the configuration object with localized archives.
-    URI[] cacheArchives = DistributedCache.getCacheArchives(job);
-    if (cacheArchives != null) {
-      List<String> localArchives = new ArrayList<String>();
-      for (int i = 0; i < cacheArchives.length; ++i) {
-        URI u = cacheArchives[i];
-        Path p = new Path(u);
-        Path name = new Path((null == u.getFragment()) ? p.getName()
-            : u.getFragment());
-        String linkName = name.toUri().getPath();
-        localArchives.add(new Path(localWorkDir, linkName).toUri().getPath());
-      }
-      if (!localArchives.isEmpty()) {
-        job.set(MRJobConfig.CACHE_LOCALARCHIVES, StringUtils
-            .arrayToString(localArchives.toArray(new String[localArchives
-                .size()])));
-      }
-    }
-
-    // Update the configuration object with localized files.
-    URI[] cacheFiles = DistributedCache.getCacheFiles(job);
-    if (cacheFiles != null) {
-      List<String> localFiles = new ArrayList<String>();
-      for (int i = 0; i < cacheFiles.length; ++i) {
-        URI u = cacheFiles[i];
-        Path p = new Path(u);
-        Path name = new Path((null == u.getFragment()) ? p.getName()
-            : u.getFragment());
-        String linkName = name.toUri().getPath();
-        localFiles.add(new Path(localWorkDir, linkName).toUri().getPath());
-      }
-      if (!localFiles.isEmpty()) {
-        job.set(MRJobConfig.CACHE_LOCALFILES, StringUtils
-            .arrayToString(localFiles.toArray(new String[localFiles.size()])));
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
----------------------------------------------------------------------
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 c5f4e84..2bc327c 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
@@ -54,7 +54,7 @@ import org.apache.tez.engine.api.Task;
 import org.apache.tez.engine.runtime.RuntimeUtils;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
 
 public class MapUtils {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
----------------------------------------------------------------------
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 f5d0b02..84f1f81 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
@@ -44,7 +44,7 @@ import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.hadoop.mapreduce.TezNullOutputCommitter;
-import org.apache.tez.mapreduce.newinput.SimpleInputLegacy;
+import org.apache.tez.mapreduce.input.SimpleInputLegacy;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MapUtils;
 import org.junit.After;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
----------------------------------------------------------------------
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 7ed18d6..8bcd353 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
@@ -49,10 +49,11 @@ import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
-import org.apache.tez.mapreduce.newinput.SimpleInputLegacy;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
+import org.apache.tez.mapreduce.input.SimpleInputLegacy;
+import org.apache.tez.mapreduce.output.SimpleOutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MapUtils;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -150,7 +151,8 @@ public class TestReduceProcessor {
     t.initialize(reduceConf, null, new TestUmbilicalProtocol());
     t.run();
     MRTask mrTask = (MRTask)t.getProcessor();
-    Assert.assertNull(mrTask.getPartitioner());
+//    TODO NEWTEZ Verify the partitioner has been created
+//    Assert.assertNull(mrTask.getPartitioner());
     t.close();
     
     // Can this be done via some utility class ? MapOutputFile derivative, or

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
----------------------------------------------------------------------
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 0dd3c60..56f9035 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
@@ -101,9 +101,9 @@ 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.hadoop.MultiStageMRConfigUtil;
-import org.apache.tez.mapreduce.newinput.ShuffledMergedInputLegacy;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
-import org.apache.tez.mapreduce.newprocessor.reduce.ReduceProcessor;
+import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 
 import com.google.common.annotations.VisibleForTesting;
 


[07/20] Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module - tez-engine module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/task/RuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/task/RuntimeTask.java b/tez-engine/src/main/java/org/apache/tez/engine/task/RuntimeTask.java
deleted file mode 100644
index 5631c78..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/task/RuntimeTask.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.task;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
-import org.apache.tez.engine.api.Task;
-
-public class RuntimeTask implements Task {
-
-  protected final Input[] inputs;
-  protected final Output[] outputs;
-  protected final Processor processor;
-  
-  protected TezEngineTaskContext taskContext;
-  protected byte[] userPayload;
-  protected Configuration conf;
-  protected Master master;
-  
-  public RuntimeTask(TezEngineTaskContext taskContext,
-      Processor processor,
-      Input[] inputs,
-      Output[] outputs) {
-    this.taskContext = taskContext;
-    this.inputs = inputs;
-    this.processor = processor;
-    this.outputs = outputs;
-  }
-
-  @Override
-  public void initialize(Configuration conf, byte[] userPayload,
-      Master master) throws IOException, InterruptedException {
-    this.conf = conf;
-    this.userPayload = userPayload;
-    this.master = master;
-
-    // NOTE: Allow processor to initialize input/output
-    processor.initialize(this.conf, this.master);
-  }
-
-  @Override
-  public Input[] getInputs() {
-    return inputs;
-  }
-
-  @Override
-  public Processor getProcessor() {
-    return processor;
-  }
-
-  @Override
-  public Output[] getOutputs() {
-    return outputs;
-  }
-
-  public void run() throws IOException, InterruptedException {
-    processor.process(inputs, outputs);
-  }
-
-  public void close() throws IOException, InterruptedException {
-    // NOTE: Allow processor to close input/output
-    // This can be changed to close input/output since MRRuntimeTask is used for
-    // MR jobs, which changes the order.
-    processor.close();
-  }
-
-  @Override
-  public Configuration getConfiguration() {
-    return this.conf;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalClientProtocolProviderTez.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalClientProtocolProviderTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalClientProtocolProviderTez.java
index f51b1da..fc2395f 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalClientProtocolProviderTez.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalClientProtocolProviderTez.java
@@ -1,54 +1,54 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.mapred;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
-import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider;
-import org.apache.tez.dag.api.TezConfiguration;
-
-@InterfaceAudience.Private
-public class LocalClientProtocolProviderTez extends ClientProtocolProvider {
-
-  @Override
-  public ClientProtocol create(Configuration conf) throws IOException {
-    String framework =
-        conf.get(MRConfig.FRAMEWORK_NAME);
-    if (!TezConfiguration.LOCAL_FRAMEWORK_NAME.equals(framework)) {
-      return null;
-    }
-    return new LocalJobRunnerTez(conf);
-  }
-
-  @Override
-  public ClientProtocol create(InetSocketAddress addr, Configuration conf) {
-    return null; // LocalJobRunner doesn't use a socket
-  }
-
-  @Override
-  public void close(ClientProtocol clientProtocol) {
-    // no clean up required
-  }
-
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//package org.apache.hadoop.mapred;
+//
+//import java.io.IOException;
+//import java.net.InetSocketAddress;
+//
+//import org.apache.hadoop.classification.InterfaceAudience;
+//import org.apache.hadoop.conf.Configuration;
+//import org.apache.hadoop.mapreduce.MRConfig;
+//import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+//import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider;
+//import org.apache.tez.dag.api.TezConfiguration;
+//
+//@InterfaceAudience.Private
+//public class LocalClientProtocolProviderTez extends ClientProtocolProvider {
+//
+//  @Override
+//  public ClientProtocol create(Configuration conf) throws IOException {
+//    String framework =
+//        conf.get(MRConfig.FRAMEWORK_NAME);
+//    if (!TezConfiguration.LOCAL_FRAMEWORK_NAME.equals(framework)) {
+//      return null;
+//    }
+//    return new LocalJobRunnerTez(conf);
+//  }
+//
+//  @Override
+//  public ClientProtocol create(InetSocketAddress addr, Configuration conf) {
+//    return null; // LocalJobRunner doesn't use a socket
+//  }
+//
+//  @Override
+//  public void close(ClientProtocol clientProtocol) {
+//    // no clean up required
+//  }
+//
+//}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetricsTez.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetricsTez.java b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetricsTez.java
index 10c404e..28dcdfb 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetricsTez.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerMetricsTez.java
@@ -1,98 +1,98 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.mapred;
-
-import 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.hadoop.metrics.jvm.JvmMetrics;
-
-@SuppressWarnings("deprecation")
-class LocalJobRunnerMetricsTez implements Updater {
-  private final MetricsRecord metricsRecord;
-
-  private int numMapTasksLaunched = 0;
-  private int numMapTasksCompleted = 0;
-  private int numReduceTasksLaunched = 0;
-  private int numReduceTasksCompleted = 0;
-  private int numWaitingMaps = 0;
-  private int numWaitingReduces = 0;
-  
-  public LocalJobRunnerMetricsTez(JobConf conf) {
-    String sessionId = conf.getSessionId();
-    // Initiate JVM Metrics
-    JvmMetrics.init("JobTracker", sessionId);
-    // Create a record for map-reduce metrics
-    MetricsContext context = MetricsUtil.getContext("mapred");
-    // record name is jobtracker for compatibility 
-    metricsRecord = MetricsUtil.createRecord(context, "jobtracker");
-    metricsRecord.setTag("sessionId", sessionId);
-    context.registerUpdater(this);
-  }
-    
-  /**
-   * Since this object is a registered updater, this method will be called
-   * periodically, e.g. every 5 seconds.
-   */
-  public void doUpdates(MetricsContext unused) {
-    synchronized (this) {
-      metricsRecord.incrMetric("maps_launched", numMapTasksLaunched);
-      metricsRecord.incrMetric("maps_completed", numMapTasksCompleted);
-      metricsRecord.incrMetric("reduces_launched", numReduceTasksLaunched);
-      metricsRecord.incrMetric("reduces_completed", numReduceTasksCompleted);
-      metricsRecord.incrMetric("waiting_maps", numWaitingMaps);
-      metricsRecord.incrMetric("waiting_reduces", numWaitingReduces);
-
-      numMapTasksLaunched = 0;
-      numMapTasksCompleted = 0;
-      numReduceTasksLaunched = 0;
-      numReduceTasksCompleted = 0;
-      numWaitingMaps = 0;
-      numWaitingReduces = 0;
-    }
-    metricsRecord.update();
-  }
-
-  public synchronized void launchMap(TaskAttemptID taskAttemptID) {
-    ++numMapTasksLaunched;
-    decWaitingMaps(taskAttemptID.getJobID(), 1);
-  }
-
-  public synchronized void completeMap(TaskAttemptID taskAttemptID) {
-    ++numMapTasksCompleted;
-  }
-
-  public synchronized void launchReduce(TaskAttemptID taskAttemptID) {
-    ++numReduceTasksLaunched;
-    decWaitingReduces(taskAttemptID.getJobID(), 1);
-  }
-
-  public synchronized void completeReduce(TaskAttemptID taskAttemptID) {
-    ++numReduceTasksCompleted;
-  }
-
-  private synchronized void decWaitingMaps(JobID id, int task) {
-    numWaitingMaps -= task;
-  }
-  
-  private synchronized void decWaitingReduces(JobID id, int task){
-    numWaitingReduces -= task;
-  }
-
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//package org.apache.hadoop.mapred;
+//
+//import 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.hadoop.metrics.jvm.JvmMetrics;
+//
+//@SuppressWarnings("deprecation")
+//class LocalJobRunnerMetricsTez implements Updater {
+//  private final MetricsRecord metricsRecord;
+//
+//  private int numMapTasksLaunched = 0;
+//  private int numMapTasksCompleted = 0;
+//  private int numReduceTasksLaunched = 0;
+//  private int numReduceTasksCompleted = 0;
+//  private int numWaitingMaps = 0;
+//  private int numWaitingReduces = 0;
+//  
+//  public LocalJobRunnerMetricsTez(JobConf conf) {
+//    String sessionId = conf.getSessionId();
+//    // Initiate JVM Metrics
+//    JvmMetrics.init("JobTracker", sessionId);
+//    // Create a record for map-reduce metrics
+//    MetricsContext context = MetricsUtil.getContext("mapred");
+//    // record name is jobtracker for compatibility 
+//    metricsRecord = MetricsUtil.createRecord(context, "jobtracker");
+//    metricsRecord.setTag("sessionId", sessionId);
+//    context.registerUpdater(this);
+//  }
+//    
+//  /**
+//   * Since this object is a registered updater, this method will be called
+//   * periodically, e.g. every 5 seconds.
+//   */
+//  public void doUpdates(MetricsContext unused) {
+//    synchronized (this) {
+//      metricsRecord.incrMetric("maps_launched", numMapTasksLaunched);
+//      metricsRecord.incrMetric("maps_completed", numMapTasksCompleted);
+//      metricsRecord.incrMetric("reduces_launched", numReduceTasksLaunched);
+//      metricsRecord.incrMetric("reduces_completed", numReduceTasksCompleted);
+//      metricsRecord.incrMetric("waiting_maps", numWaitingMaps);
+//      metricsRecord.incrMetric("waiting_reduces", numWaitingReduces);
+//
+//      numMapTasksLaunched = 0;
+//      numMapTasksCompleted = 0;
+//      numReduceTasksLaunched = 0;
+//      numReduceTasksCompleted = 0;
+//      numWaitingMaps = 0;
+//      numWaitingReduces = 0;
+//    }
+//    metricsRecord.update();
+//  }
+//
+//  public synchronized void launchMap(TaskAttemptID taskAttemptID) {
+//    ++numMapTasksLaunched;
+//    decWaitingMaps(taskAttemptID.getJobID(), 1);
+//  }
+//
+//  public synchronized void completeMap(TaskAttemptID taskAttemptID) {
+//    ++numMapTasksCompleted;
+//  }
+//
+//  public synchronized void launchReduce(TaskAttemptID taskAttemptID) {
+//    ++numReduceTasksLaunched;
+//    decWaitingReduces(taskAttemptID.getJobID(), 1);
+//  }
+//
+//  public synchronized void completeReduce(TaskAttemptID taskAttemptID) {
+//    ++numReduceTasksCompleted;
+//  }
+//
+//  private synchronized void decWaitingMaps(JobID id, int task) {
+//    numWaitingMaps -= task;
+//  }
+//  
+//  private synchronized void decWaitingReduces(JobID id, int task){
+//    numWaitingReduces -= task;
+//  }
+//
+//}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
----------------------------------------------------------------------
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 ac6d5dd..f59e836 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
@@ -1,878 +1,875 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.mapred;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
-import org.apache.hadoop.mapreduce.ClusterMetrics;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.QueueInfo;
-import org.apache.hadoop.mapreduce.TaskCompletionEvent;
-import org.apache.hadoop.mapreduce.TaskTrackerInfo;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
-import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
-import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.hadoop.mapreduce.v2.LogParams;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authorize.AccessControlList;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.ContainerContext;
-import org.apache.tez.common.ContainerTask;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.OutputSpec;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.common.records.ProceedToCompletionResponse;
-import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Task;
-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.oldinput.LocalMergedInput;
-import org.apache.tez.engine.lib.oldoutput.OldLocalOnFileSorterOutput;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
-import org.apache.tez.engine.records.OutputContext;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
-import org.apache.tez.engine.runtime.RuntimeUtils;
-import org.apache.tez.mapreduce.hadoop.IDConverter;
-import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
-import org.apache.tez.mapreduce.input.SimpleInput;
-import org.apache.tez.mapreduce.output.SimpleOutput;
-import org.apache.tez.mapreduce.processor.map.MapProcessor;
-import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/** Implements MapReduce locally, in-process, for debugging. */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class LocalJobRunnerTez implements ClientProtocol {
-  public static final Log LOG =
-    LogFactory.getLog(LocalJobRunnerTez.class);
-
-  /** The maximum number of map tasks to run in parallel in LocalJobRunner */
-  public static final String LOCAL_MAX_MAPS =
-    "mapreduce.local.map.tasks.maximum";
-
-  private FileSystem fs;
-  private HashMap<JobID, Job> jobs = new HashMap<JobID, Job>();
-  private JobConf conf;
-  private AtomicInteger map_tasks = new AtomicInteger(0);
-  private int reduce_tasks = 0;
-  final Random rand = new Random();
-
-  private LocalJobRunnerMetricsTez myMetrics = null;
-
-  private static final String jobDir =  "localRunner/";
-
-  private static final TezCounters EMPTY_COUNTERS = new TezCounters();
-
-  public long getProtocolVersion(String protocol, long clientVersion) {
-    return ClientProtocol.versionID;
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSignature(
-        this, protocol, clientVersion, clientMethodsHash);
-  }
-
-  private class Job extends Thread implements TezTaskUmbilicalProtocol {
-    // The job directory on the system: JobClient places job configurations here.
-    // This is analogous to JobTracker's system directory.
-    private Path systemJobDir;
-    private Path systemJobFile;
-
-    // The job directory for the task.  Analagous to a task's job directory.
-    private Path localJobDir;
-    private Path localJobFile;
-
-    private JobID id;
-    private JobConf job;
-
-    private int numMapTasks;
-    private float [] partialMapProgress;
-    private TezCounters [] mapCounters;
-    private TezCounters reduceCounters;
-
-    private JobStatus status;
-    private List<TaskAttemptID> mapIds = Collections.synchronizedList(
-        new ArrayList<TaskAttemptID>());
-
-    private JobProfile profile;
-    private FileSystem localFs;
-    boolean killed = false;
-
-    private LocalDistributedCacheManager localDistributedCacheManager;
-
-    public long getProtocolVersion(String protocol, long clientVersion) {
-      return TaskUmbilicalProtocol.versionID;
-    }
-
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      return ProtocolSignature.getProtocolSignature(
-          this, protocol, clientVersion, clientMethodsHash);
-    }
-
-    public Job(JobID jobid, String jobSubmitDir) throws IOException {
-      this.systemJobDir = new Path(jobSubmitDir);
-      this.systemJobFile = new Path(systemJobDir, "job.xml");
-      this.id = jobid;
-      JobConf conf = new JobConf(systemJobFile);
-      this.localFs = FileSystem.getLocal(conf);
-      this.localJobDir = localFs.makeQualified(conf.getLocalPath(jobDir));
-      this.localJobFile = new Path(this.localJobDir, id + ".xml");
-
-      // Manage the distributed cache.  If there are files to be copied,
-      // this will trigger localFile to be re-written again.
-      localDistributedCacheManager = new LocalDistributedCacheManager();
-      localDistributedCacheManager.setup(conf);
-
-      // Write out configuration file.  Instead of copying it from
-      // systemJobFile, we re-write it, since setup(), above, may have
-      // updated it.
-      OutputStream out = localFs.create(localJobFile);
-      try {
-        conf.writeXml(out);
-      } finally {
-        out.close();
-      }
-      this.job = new JobConf(localJobFile);
-
-      // Job (the current object) is a Thread, so we wrap its class loader.
-      if (localDistributedCacheManager.hasLocalClasspaths()) {
-        setContextClassLoader(localDistributedCacheManager.makeClassLoader(
-                getContextClassLoader()));
-      }
-
-      profile = new JobProfile(job.getUser(), id, systemJobFile.toString(),
-                               "http://localhost:8080/", job.getJobName());
-      status = new JobStatus(id, 0.0f, 0.0f, JobStatus.RUNNING,
-          profile.getUser(), profile.getJobName(), profile.getJobFile(),
-          profile.getURL().toString());
-
-      jobs.put(id, this);
-
-      this.start();
-    }
-
-    /**
-     * A Runnable instance that handles a map task to be run by an executor.
-     */
-    protected class MapTaskRunnable implements Runnable {
-      private final int taskId;
-      private final TaskSplitMetaInfo info;
-      private final JobID jobId;
-      private final JobConf localConf;
-
-      // This is a reference to a shared object passed in by the
-      // external context; this delivers state to the reducers regarding
-      // where to fetch mapper outputs.
-      private final Map<TaskAttemptID, TezTaskOutput> mapOutputFiles;
-
-      public volatile Throwable storedException;
-
-      public MapTaskRunnable(TaskSplitMetaInfo info, int taskId, JobID jobId,
-          Map<TaskAttemptID, TezTaskOutput> mapOutputFiles) {
-        this.info = info;
-        this.taskId = taskId;
-        this.mapOutputFiles = mapOutputFiles;
-        this.jobId = jobId;
-        this.localConf = new JobConf(job);
-      }
-
-      public void run() {
-        try {
-          TaskAttemptID mapId = new TaskAttemptID(new TaskID(
-              jobId, TaskType.MAP, taskId), 0);
-          LOG.info("Starting task: " + mapId);
-          final String user =
-              UserGroupInformation.getCurrentUser().getShortUserName();
-          setupChildMapredLocalDirs(mapId, user, localConf);
-          localConf.setUser(user);
-
-          TezTaskAttemptID tezMapId =
-              IDConverter.fromMRTaskAttemptId(mapId);
-          mapIds.add(mapId);
-          // FIXME invalid task context
-          ProcessorDescriptor mapProcessorDesc = new ProcessorDescriptor(
-                      MapProcessor.class.getName());
-          TezEngineTaskContext taskContext =
-              new TezEngineTaskContext(
-                  tezMapId, user, localConf.getJobName(), "TODO_vertexName",
-                  mapProcessorDesc,
-                  Collections.singletonList(new InputSpec("srcVertex", 0,
-                      SimpleInput.class.getName())),
-                  Collections.singletonList(new OutputSpec("tgtVertex", 0,
-                      OldLocalOnFileSorterOutput.class.getName())));
-
-          TezTaskOutput mapOutput = new TezLocalTaskOutputFiles();
-          mapOutput.setConf(localConf);
-          mapOutputFiles.put(mapId, mapOutput);
-
-          try {
-            map_tasks.getAndIncrement();
-            myMetrics.launchMap(mapId);
-            Task t = RuntimeUtils.createRuntimeTask(taskContext);
-            t.initialize(localConf, null, Job.this);
-            t.run();
-            myMetrics.completeMap(mapId);
-          } finally {
-            map_tasks.getAndDecrement();
-          }
-
-          LOG.info("Finishing task: " + mapId);
-        } catch (Throwable e) {
-          this.storedException = e;
-        }
-      }
-    }
-
-    /**
-     * Create Runnables to encapsulate map tasks for use by the executor
-     * service.
-     * @param taskInfo Info about the map task splits
-     * @param jobId the job id
-     * @param mapOutputFiles a mapping from task attempts to output files
-     * @return a List of Runnables, one per map task.
-     */
-    protected List<MapTaskRunnable> getMapTaskRunnables(
-        TaskSplitMetaInfo [] taskInfo, JobID jobId,
-        Map<TaskAttemptID, TezTaskOutput> mapOutputFiles) {
-
-      int numTasks = 0;
-      ArrayList<MapTaskRunnable> list = new ArrayList<MapTaskRunnable>();
-      for (TaskSplitMetaInfo task : taskInfo) {
-        list.add(new MapTaskRunnable(task, numTasks++, jobId,
-            mapOutputFiles));
-      }
-
-      return list;
-    }
-
-    /**
-     * Initialize the counters that will hold partial-progress from
-     * the various task attempts.
-     * @param numMaps the number of map tasks in this job.
-     */
-    private synchronized void initCounters(int numMaps) {
-      // Initialize state trackers for all map tasks.
-      this.partialMapProgress = new float[numMaps];
-      this.mapCounters = new TezCounters[numMaps];
-      for (int i = 0; i < numMaps; i++) {
-        this.mapCounters[i] = EMPTY_COUNTERS;
-      }
-
-      this.reduceCounters = EMPTY_COUNTERS;
-    }
-
-    /**
-     * Creates the executor service used to run map tasks.
-     *
-     * @param numMapTasks the total number of map tasks to be run
-     * @return an ExecutorService instance that handles map tasks
-     */
-    protected ExecutorService createMapExecutor(int numMapTasks) {
-
-      // Determine the size of the thread pool to use
-      int maxMapThreads = job.getInt(LOCAL_MAX_MAPS, 1);
-      if (maxMapThreads < 1) {
-        throw new IllegalArgumentException(
-            "Configured " + LOCAL_MAX_MAPS + " must be >= 1");
-      }
-      this.numMapTasks = numMapTasks;
-      maxMapThreads = Math.min(maxMapThreads, this.numMapTasks);
-      maxMapThreads = Math.max(maxMapThreads, 1); // In case of no tasks.
-
-      initCounters(this.numMapTasks);
-
-      LOG.debug("Starting thread pool executor.");
-      LOG.debug("Max local threads: " + maxMapThreads);
-      LOG.debug("Map tasks to process: " + this.numMapTasks);
-
-      // Create a new executor service to drain the work queue.
-      ThreadFactory tf = new ThreadFactoryBuilder()
-        .setNameFormat("LocalJobRunner Map Task Executor #%d")
-        .build();
-      ExecutorService executor = Executors.newFixedThreadPool(maxMapThreads, tf);
-
-      return executor;
-    }
-
-    private org.apache.hadoop.mapreduce.OutputCommitter
-    createOutputCommitter(boolean newApiCommitter, JobID jobId, Configuration conf) throws Exception {
-      org.apache.hadoop.mapreduce.OutputCommitter committer = null;
-
-      LOG.info("OutputCommitter set in config "
-          + conf.get("mapred.output.committer.class"));
-
-      if (newApiCommitter) {
-        org.apache.hadoop.mapreduce.TaskID taskId =
-            new org.apache.hadoop.mapreduce.TaskID(jobId, TaskType.MAP, 0);
-        org.apache.hadoop.mapreduce.TaskAttemptID taskAttemptID =
-            new org.apache.hadoop.mapreduce.TaskAttemptID(taskId, 0);
-        org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
-            new TaskAttemptContextImpl(conf, taskAttemptID);
-        @SuppressWarnings("rawtypes")
-        OutputFormat outputFormat =
-          ReflectionUtils.newInstance(taskContext.getOutputFormatClass(), conf);
-        committer = outputFormat.getOutputCommitter(taskContext);
-      } else {
-        committer = ReflectionUtils.newInstance(conf.getClass(
-            "mapred.output.committer.class", FileOutputCommitter.class,
-            org.apache.hadoop.mapred.OutputCommitter.class), conf);
-      }
-      LOG.info("OutputCommitter is " + committer.getClass().getName());
-      return committer;
-    }
-
-    @Override
-    public void run() {
-      JobID jobId = profile.getJobID();
-      JobContext jContext = new JobContextImpl(job, jobId);
-
-      org.apache.hadoop.mapreduce.OutputCommitter outputCommitter = null;
-      try {
-        outputCommitter = createOutputCommitter(conf.getUseNewMapper(), jobId, conf);
-      } catch (Exception e) {
-        LOG.info("Failed to createOutputCommitter", e);
-        return;
-      }
-
-      try {
-        TaskSplitMetaInfo[] taskSplitMetaInfos =
-          SplitMetaInfoReader.readSplitMetaInfo(jobId, localFs, conf, systemJobDir);
-
-        int numReduceTasks = job.getNumReduceTasks();
-        if (numReduceTasks > 1 || numReduceTasks < 0) {
-          // we only allow 0 or 1 reducer in local mode
-          numReduceTasks = 1;
-          job.setNumReduceTasks(1);
-        }
-        outputCommitter.setupJob(jContext);
-        status.setSetupProgress(1.0f);
-
-        Map<TaskAttemptID, TezTaskOutput> mapOutputFiles =
-            Collections.synchronizedMap(new HashMap<TaskAttemptID, TezTaskOutput>());
-
-        List<MapTaskRunnable> taskRunnables = getMapTaskRunnables(taskSplitMetaInfos,
-            jobId, mapOutputFiles);
-        ExecutorService mapService = createMapExecutor(taskRunnables.size());
-
-        // Start populating the executor with work units.
-        // They may begin running immediately (in other threads).
-        for (Runnable r : taskRunnables) {
-          mapService.submit(r);
-        }
-
-        try {
-          mapService.shutdown(); // Instructs queue to drain.
-
-          // Wait for tasks to finish; do not use a time-based timeout.
-          // (See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6179024)
-          LOG.info("Waiting for map tasks");
-          mapService.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS);
-        } catch (InterruptedException ie) {
-          // Cancel all threads.
-          mapService.shutdownNow();
-          throw ie;
-        }
-
-        LOG.info("Map task executor complete.");
-
-        // After waiting for the map tasks to complete, if any of these
-        // have thrown an exception, rethrow it now in the main thread context.
-        for (MapTaskRunnable r : taskRunnables) {
-          if (r.storedException != null) {
-            throw new Exception(r.storedException);
-          }
-        }
-
-        TaskAttemptID reduceId = new TaskAttemptID(new TaskID(
-            jobId, TaskType.REDUCE, 0), 0);
-        LOG.info("Starting task: " + reduceId);
-        try {
-          if (numReduceTasks > 0) {
-            String user =
-                UserGroupInformation.getCurrentUser().getShortUserName();
-            JobConf localConf = new JobConf(job);
-            localConf.setUser(user);
-            localConf.set("mapreduce.jobtracker.address", "local");
-            setupChildMapredLocalDirs(reduceId, user, localConf);
-            // FIXME invalid task context
-            ProcessorDescriptor reduceProcessorDesc = new ProcessorDescriptor(
-                ReduceProcessor.class.getName());
-            TezEngineTaskContext taskContext = new TezEngineTaskContext(
-                IDConverter.fromMRTaskAttemptId(reduceId), user,
-                localConf.getJobName(), "TODO_vertexName",
-                reduceProcessorDesc,
-                Collections.singletonList(new InputSpec("TODO_srcVertexName",
-                    mapIds.size(), LocalMergedInput.class.getName())),
-                Collections.singletonList(new OutputSpec("TODO_targetVertex",
-                    0, SimpleOutput.class.getName())));
-
-            // move map output to reduce input
-            for (int i = 0; i < mapIds.size(); i++) {
-              if (!this.isInterrupted()) {
-                TaskAttemptID mapId = mapIds.get(i);
-                if (LOG.isDebugEnabled()) {
-                  LOG.debug("XXX mapId: " + i +
-                      " LOCAL_DIR = " +
-                      mapOutputFiles.get(mapId).getConf().get(
-                          TezJobConfig.LOCAL_DIRS));
-                }
-                Path mapOut = mapOutputFiles.get(mapId).getOutputFile();
-                TezTaskOutput localOutputFile = new TezLocalTaskOutputFiles();
-                localOutputFile.setConf(localConf);
-                Path reduceIn =
-                  localOutputFile.getInputFileForWrite(
-                      IDConverter.fromMRTaskId(mapId.getTaskID()),
-                        localFs.getFileStatus(mapOut).getLen());
-                if (!localFs.mkdirs(reduceIn.getParent())) {
-                  throw new IOException("Mkdirs failed to create "
-                      + reduceIn.getParent().toString());
-                }
-                if (!localFs.rename(mapOut, reduceIn))
-                  throw new IOException("Couldn't rename " + mapOut);
-              } else {
-                throw new InterruptedException();
-              }
-            }
-            if (!this.isInterrupted()) {
-              reduce_tasks += 1;
-              myMetrics.launchReduce(reduceId);
-              Task t = RuntimeUtils.createRuntimeTask(taskContext);
-              t.initialize(localConf, null, Job.this);
-              t.run();
-              myMetrics.completeReduce(reduceId);
-              reduce_tasks -= 1;
-            } else {
-              throw new InterruptedException();
-            }
-          }
-        } finally {
-          for (TezTaskOutput output : mapOutputFiles.values()) {
-            output.removeAll();
-          }
-        }
-        // delete the temporary directory in output directory
-        // FIXME
-        //outputCommitter.commitJob(jContext);
-        status.setCleanupProgress(1.0f);
-
-        if (killed) {
-          this.status.setRunState(JobStatus.KILLED);
-        } else {
-          this.status.setRunState(JobStatus.SUCCEEDED);
-        }
-
-        JobEndNotifier.localRunnerNotification(job, status);
-
-      } catch (Throwable t) {
-        try {
-          outputCommitter.abortJob(jContext,
-            org.apache.hadoop.mapreduce.JobStatus.State.FAILED);
-        } catch (IOException ioe) {
-          LOG.info("Error cleaning up job:" + id);
-        }
-        status.setCleanupProgress(1.0f);
-        if (killed) {
-          this.status.setRunState(JobStatus.KILLED);
-        } else {
-          this.status.setRunState(JobStatus.FAILED);
-        }
-        LOG.warn(id, t);
-
-        JobEndNotifier.localRunnerNotification(job, status);
-
-      } finally {
-        try {
-          fs.delete(systemJobFile.getParent(), true);  // delete submit dir
-          localFs.delete(localJobFile, true);              // delete local copy
-          // Cleanup distributed cache
-          localDistributedCacheManager.close();
-        } catch (IOException e) {
-          LOG.warn("Error cleaning up "+id+": "+e);
-        }
-      }
-    }
-
-    // TaskUmbilicalProtocol methods
-    @Override
-    public ContainerTask getTask(ContainerContext containerContext)
-        throws IOException {
-      return null;
-    }
-
-    /** Return the current values of the counters for this job,
-     * including tasks that are in progress.
-     */
-    public synchronized TezCounters getCurrentCounters() {
-      if (null == mapCounters) {
-        // Counters not yet initialized for job.
-        return EMPTY_COUNTERS;
-      }
-
-      TezCounters current = EMPTY_COUNTERS;
-      for (TezCounters c : mapCounters) {
-        current.incrAllCounters(c);
-      }
-      current.incrAllCounters(reduceCounters);
-      return current;
-    }
-
-    @Override
-    public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
-      return true;
-    }
-
-    @Override
-    public TezTaskDependencyCompletionEventsUpdate
-    getDependentTasksCompletionEvents(
-        int fromEventIdx, int maxEventsToFetch,
-        TezTaskAttemptID reduce) {
-      throw new UnsupportedOperationException(
-          "getDependentTasksCompletionEvents not supported in LocalJobRunner");
-    }
-
-    @Override
-    public void outputReady(TezTaskAttemptID taskAttemptId,
-        OutputContext outputContext) throws IOException {
-      // Ignore for now.
-    }
-
-    @Override
-    public ProceedToCompletionResponse proceedToCompletion(
-        TezTaskAttemptID taskAttemptId) throws IOException {
-      // TODO TEZAM5 Really depends on the module - inmem shuffle or not.
-      return new ProceedToCompletionResponse(true, true);
-    }
-
-    @Override
-    public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) {
-      // TODO Auto-generated method stub
-      // TODO TODONEWTEZ
-      return null;
-    }
-
-  }
-
-  public LocalJobRunnerTez(Configuration conf) throws IOException {
-    this(new JobConf(conf));
-  }
-
-  @Deprecated
-  public LocalJobRunnerTez(JobConf conf) throws IOException {
-    this.fs = FileSystem.getLocal(conf);
-    this.conf = conf;
-    myMetrics = new LocalJobRunnerMetricsTez(new JobConf(conf));
-  }
-
-  // JobSubmissionProtocol methods
-
-  private static int jobid = 0;
-  public synchronized org.apache.hadoop.mapreduce.JobID getNewJobID() {
-    return new org.apache.hadoop.mapreduce.JobID("local", ++jobid);
-  }
-
-  public org.apache.hadoop.mapreduce.JobStatus submitJob(
-      org.apache.hadoop.mapreduce.JobID jobid, String jobSubmitDir,
-      Credentials credentials) throws IOException {
-    Job job = new Job(JobID.downgrade(jobid), jobSubmitDir);
-    job.job.setCredentials(credentials);
-    return job.status;
-
-  }
-
-  public void killJob(org.apache.hadoop.mapreduce.JobID id) {
-    jobs.get(JobID.downgrade(id)).killed = true;
-    jobs.get(JobID.downgrade(id)).interrupt();
-  }
-
-  public void setJobPriority(org.apache.hadoop.mapreduce.JobID id,
-      String jp) throws IOException {
-    throw new UnsupportedOperationException("Changing job priority " +
-                      "in LocalJobRunner is not supported.");
-  }
-
-  /** Throws {@link UnsupportedOperationException} */
-  public boolean killTask(org.apache.hadoop.mapreduce.TaskAttemptID taskId,
-      boolean shouldFail) throws IOException {
-    throw new UnsupportedOperationException("Killing tasks in " +
-    "LocalJobRunner is not supported");
-  }
-
-  public org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(
-      org.apache.hadoop.mapreduce.JobID id, TaskType type) {
-    return new org.apache.hadoop.mapreduce.TaskReport[0];
-  }
-
-  public org.apache.hadoop.mapreduce.JobStatus getJobStatus(
-      org.apache.hadoop.mapreduce.JobID id) {
-    Job job = jobs.get(JobID.downgrade(id));
-    if(job != null)
-      return job.status;
-    else
-      return null;
-  }
-
-  public org.apache.hadoop.mapreduce.Counters getJobCounters(
-      org.apache.hadoop.mapreduce.JobID id) {
-    Job job = jobs.get(JobID.downgrade(id));
-
-    return new org.apache.hadoop.mapreduce.Counters(
-        new MRCounters(job.getCurrentCounters()));
-  }
-
-  public String getFilesystemName() throws IOException {
-    return fs.getUri().toString();
-  }
-
-  public ClusterMetrics getClusterMetrics() {
-    int numMapTasks = map_tasks.get();
-    return new ClusterMetrics(numMapTasks, reduce_tasks, numMapTasks,
-        reduce_tasks, 0, 0, 1, 1, jobs.size(), 1, 0, 0);
-  }
-
-  public JobTrackerStatus getJobTrackerStatus() {
-    return JobTrackerStatus.RUNNING;
-  }
-
-  public long getTaskTrackerExpiryInterval()
-      throws IOException, InterruptedException {
-    return 0;
-  }
-
-  /**
-   * Get all active trackers in cluster.
-   * @return array of TaskTrackerInfo
-   */
-  public TaskTrackerInfo[] getActiveTrackers()
-      throws IOException, InterruptedException {
-    return null;
-  }
-
-  /**
-   * Get all blacklisted trackers in cluster.
-   * @return array of TaskTrackerInfo
-   */
-  public TaskTrackerInfo[] getBlacklistedTrackers()
-      throws IOException, InterruptedException {
-    return null;
-  }
-
-  public TaskCompletionEvent[] getTaskCompletionEvents(
-      org.apache.hadoop.mapreduce.JobID jobid
-      , int fromEventId, int maxEvents) throws IOException {
-    return TaskCompletionEvent.EMPTY_ARRAY;
-  }
-
-  public org.apache.hadoop.mapreduce.JobStatus[] getAllJobs() {return null;}
-
-
-  /**
-   * Returns the diagnostic information for a particular task in the given job.
-   * To be implemented
-   */
-  public String[] getTaskDiagnostics(
-      org.apache.hadoop.mapreduce.TaskAttemptID taskid) throws IOException{
-	  return new String [0];
-  }
-
-  /**
-   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getSystemDir()
-   */
-  public String getSystemDir() {
-    Path sysDir = new Path(
-      conf.get(JTConfig.JT_SYSTEM_DIR, "/tmp/hadoop/mapred/system"));
-    return fs.makeQualified(sysDir).toString();
-  }
-
-  /**
-   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getQueueAdmins(String)
-   */
-  public AccessControlList getQueueAdmins(String queueName) throws IOException {
-	  return new AccessControlList(" ");// no queue admins for local job runner
-  }
-
-  /**
-   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getStagingAreaDir()
-   */
-  public String getStagingAreaDir() throws IOException {
-    Path stagingRootDir = new Path(conf.get(JTConfig.JT_STAGING_AREA_ROOT,
-        "/tmp/hadoop/mapred/staging"));
-    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-    String user;
-    if (ugi != null) {
-      user = ugi.getShortUserName() + rand.nextInt();
-    } else {
-      user = "dummy" + rand.nextInt();
-    }
-    return fs.makeQualified(new Path(stagingRootDir, user+"/.staging")).toString();
-  }
-
-  public String getJobHistoryDir() {
-    return null;
-  }
-
-  @Override
-  public QueueInfo[] getChildQueues(String queueName) throws IOException {
-    return null;
-  }
-
-  @Override
-  public QueueInfo[] getRootQueues() throws IOException {
-    return null;
-  }
-
-  @Override
-  public QueueInfo[] getQueues() throws IOException {
-    return null;
-  }
-
-
-  @Override
-  public QueueInfo getQueue(String queue) throws IOException {
-    return null;
-  }
-
-  @Override
-  public org.apache.hadoop.mapreduce.QueueAclsInfo[]
-      getQueueAclsForCurrentUser() throws IOException{
-    return null;
-  }
-
-  /**
-   * Set the max number of map tasks to run concurrently in the LocalJobRunner.
-   * @param job the job to configure
-   * @param maxMaps the maximum number of map tasks to allow.
-   */
-  public static void setLocalMaxRunningMaps(
-      org.apache.hadoop.mapreduce.JobContext job,
-      int maxMaps) {
-    job.getConfiguration().setInt(LOCAL_MAX_MAPS, maxMaps);
-  }
-
-  /**
-   * @return the max number of map tasks to run concurrently in the
-   * LocalJobRunner.
-   */
-  public static int getLocalMaxRunningMaps(
-      org.apache.hadoop.mapreduce.JobContext job) {
-    return job.getConfiguration().getInt(LOCAL_MAX_MAPS, 1);
-  }
-
-  @Override
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token
-                                       ) throws IOException,
-                                                InterruptedException {
-  }
-
-  @Override
-  public Token<DelegationTokenIdentifier>
-     getDelegationToken(Text renewer) throws IOException, InterruptedException {
-    return null;
-  }
-
-  @Override
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token
-                                      ) throws IOException,InterruptedException{
-    return 0;
-  }
-
-  @Override
-  public LogParams getLogFileParams(org.apache.hadoop.mapreduce.JobID jobID,
-      org.apache.hadoop.mapreduce.TaskAttemptID taskAttemptID)
-      throws IOException, InterruptedException {
-    throw new UnsupportedOperationException("Not supported");
-  }
-
-  static void setupChildMapredLocalDirs(
-      TaskAttemptID taskAttemptID, String user, JobConf conf) {
-    String[] localDirs =
-        conf.getTrimmedStrings(
-            TezJobConfig.LOCAL_DIRS, TezJobConfig.DEFAULT_LOCAL_DIRS);
-    String jobId = taskAttemptID.getJobID().toString();
-    String taskId = taskAttemptID.getTaskID().toString();
-    boolean isCleanup = false;
-    StringBuffer childMapredLocalDir =
-        new StringBuffer(localDirs[0] + Path.SEPARATOR
-            + getLocalTaskDir(user, jobId, taskId, isCleanup));
-    for (int i = 1; i < localDirs.length; i++) {
-      childMapredLocalDir.append("," + localDirs[i] + Path.SEPARATOR
-          + getLocalTaskDir(user, jobId, taskId, isCleanup));
-    }
-    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,
-        TezLocalTaskOutputFiles.class, TezTaskOutput.class);
-  }
-
-  static final String TASK_CLEANUP_SUFFIX = ".cleanup";
-  static final String SUBDIR = jobDir;
-  static final String JOBCACHE = "jobcache";
-
-  static String getLocalTaskDir(String user, String jobid, String taskid,
-      boolean isCleanupAttempt) {
-    String taskDir = SUBDIR + Path.SEPARATOR + user + Path.SEPARATOR + JOBCACHE
-      + Path.SEPARATOR + jobid + Path.SEPARATOR + taskid;
-    if (isCleanupAttempt) {
-      taskDir = taskDir + TASK_CLEANUP_SUFFIX;
-    }
-    return taskDir;
-  }
-
-
-}
+///**
+// * Licensed to the Apache Software Foundation (ASF) under one
+// * or more contributor license agreements.  See the NOTICE file
+// * distributed with this work for additional information
+// * regarding copyright ownership.  The ASF licenses this file
+// * to you under the Apache License, Version 2.0 (the
+// * "License"); you may not use this file except in compliance
+// * with the License.  You may obtain a copy of the License at
+// *
+// *     http://www.apache.org/licenses/LICENSE-2.0
+// *
+// * Unless required by applicable law or agreed to in writing, software
+// * distributed under the License is distributed on an "AS IS" BASIS,
+// * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// * See the License for the specific language governing permissions and
+// * limitations under the License.
+// */
+//
+//package org.apache.hadoop.mapred;
+//
+//import java.io.IOException;
+//import java.io.OutputStream;
+//import java.util.ArrayList;
+//import java.util.Collections;
+//import java.util.HashMap;
+//import java.util.List;
+//import java.util.Map;
+//import java.util.Random;
+//import java.util.concurrent.ExecutorService;
+//import java.util.concurrent.Executors;
+//import java.util.concurrent.ThreadFactory;
+//import java.util.concurrent.TimeUnit;
+//import java.util.concurrent.atomic.AtomicInteger;
+//
+//import org.apache.commons.logging.Log;
+//import org.apache.commons.logging.LogFactory;
+//import org.apache.hadoop.classification.InterfaceAudience;
+//import org.apache.hadoop.classification.InterfaceStability;
+//import org.apache.hadoop.conf.Configuration;
+//import org.apache.hadoop.fs.FileSystem;
+//import org.apache.hadoop.fs.Path;
+//import org.apache.hadoop.io.Text;
+//import org.apache.hadoop.ipc.ProtocolSignature;
+//import org.apache.hadoop.mapreduce.Cluster.JobTrackerStatus;
+//import org.apache.hadoop.mapreduce.ClusterMetrics;
+//import org.apache.hadoop.mapreduce.OutputFormat;
+//import org.apache.hadoop.mapreduce.QueueInfo;
+//import org.apache.hadoop.mapreduce.TaskCompletionEvent;
+//import org.apache.hadoop.mapreduce.TaskTrackerInfo;
+//import org.apache.hadoop.mapreduce.TaskType;
+//import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
+//import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
+//import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
+//import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
+//import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
+//import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+//import org.apache.hadoop.mapreduce.v2.LogParams;
+//import org.apache.hadoop.security.Credentials;
+//import org.apache.hadoop.security.UserGroupInformation;
+//import org.apache.hadoop.security.authorize.AccessControlList;
+//import org.apache.hadoop.security.token.Token;
+//import org.apache.hadoop.util.ReflectionUtils;
+//import org.apache.tez.common.Constants;
+//import org.apache.tez.common.ContainerContext;
+//import org.apache.tez.common.ContainerTask;
+//import org.apache.tez.common.InputSpec;
+//import org.apache.tez.common.OutputSpec;
+//import org.apache.tez.common.TezEngineTaskContext;
+//import org.apache.tez.common.TezJobConfig;
+//import org.apache.tez.common.TezTaskUmbilicalProtocol;
+//import org.apache.tez.common.counters.TezCounters;
+//import org.apache.tez.common.records.ProceedToCompletionResponse;
+//import org.apache.tez.dag.api.ProcessorDescriptor;
+//import org.apache.tez.dag.records.TezTaskAttemptID;
+//import org.apache.tez.engine.api.Task;
+//import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
+//import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
+//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.records.OutputContext;
+//import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
+//import org.apache.tez.mapreduce.hadoop.IDConverter;
+//import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
+//import org.apache.tez.mapreduce.input.SimpleInput;
+//import org.apache.tez.mapreduce.output.SimpleOutput;
+//import org.apache.tez.mapreduce.processor.map.MapProcessor;
+//import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+//
+//import com.google.common.util.concurrent.ThreadFactoryBuilder;
+//
+///** Implements MapReduce locally, in-process, for debugging. */
+//@InterfaceAudience.Private
+//@InterfaceStability.Unstable
+//public class LocalJobRunnerTez implements ClientProtocol {
+//  public static final Log LOG =
+//    LogFactory.getLog(LocalJobRunnerTez.class);
+//
+//  /** The maximum number of map tasks to run in parallel in LocalJobRunner */
+//  public static final String LOCAL_MAX_MAPS =
+//    "mapreduce.local.map.tasks.maximum";
+//
+//  private FileSystem fs;
+//  private HashMap<JobID, Job> jobs = new HashMap<JobID, Job>();
+//  private JobConf conf;
+//  private AtomicInteger map_tasks = new AtomicInteger(0);
+//  private int reduce_tasks = 0;
+//  final Random rand = new Random();
+//
+//  private LocalJobRunnerMetricsTez myMetrics = null;
+//
+//  private static final String jobDir =  "localRunner/";
+//
+//  private static final TezCounters EMPTY_COUNTERS = new TezCounters();
+//
+//  public long getProtocolVersion(String protocol, long clientVersion) {
+//    return ClientProtocol.versionID;
+//  }
+//
+//  @Override
+//  public ProtocolSignature getProtocolSignature(String protocol,
+//      long clientVersion, int clientMethodsHash) throws IOException {
+//    return ProtocolSignature.getProtocolSignature(
+//        this, protocol, clientVersion, clientMethodsHash);
+//  }
+//
+//  private class Job extends Thread implements TezTaskUmbilicalProtocol {
+//    // The job directory on the system: JobClient places job configurations here.
+//    // This is analogous to JobTracker's system directory.
+//    private Path systemJobDir;
+//    private Path systemJobFile;
+//
+//    // The job directory for the task.  Analagous to a task's job directory.
+//    private Path localJobDir;
+//    private Path localJobFile;
+//
+//    private JobID id;
+//    private JobConf job;
+//
+//    private int numMapTasks;
+//    private float [] partialMapProgress;
+//    private TezCounters [] mapCounters;
+//    private TezCounters reduceCounters;
+//
+//    private JobStatus status;
+//    private List<TaskAttemptID> mapIds = Collections.synchronizedList(
+//        new ArrayList<TaskAttemptID>());
+//
+//    private JobProfile profile;
+//    private FileSystem localFs;
+//    boolean killed = false;
+//
+//    private LocalDistributedCacheManager localDistributedCacheManager;
+//
+//    public long getProtocolVersion(String protocol, long clientVersion) {
+//      return TaskUmbilicalProtocol.versionID;
+//    }
+//
+//    @Override
+//    public ProtocolSignature getProtocolSignature(String protocol,
+//        long clientVersion, int clientMethodsHash) throws IOException {
+//      return ProtocolSignature.getProtocolSignature(
+//          this, protocol, clientVersion, clientMethodsHash);
+//    }
+//
+//    public Job(JobID jobid, String jobSubmitDir) throws IOException {
+//      this.systemJobDir = new Path(jobSubmitDir);
+//      this.systemJobFile = new Path(systemJobDir, "job.xml");
+//      this.id = jobid;
+//      JobConf conf = new JobConf(systemJobFile);
+//      this.localFs = FileSystem.getLocal(conf);
+//      this.localJobDir = localFs.makeQualified(conf.getLocalPath(jobDir));
+//      this.localJobFile = new Path(this.localJobDir, id + ".xml");
+//
+//      // Manage the distributed cache.  If there are files to be copied,
+//      // this will trigger localFile to be re-written again.
+//      localDistributedCacheManager = new LocalDistributedCacheManager();
+//      localDistributedCacheManager.setup(conf);
+//
+//      // Write out configuration file.  Instead of copying it from
+//      // systemJobFile, we re-write it, since setup(), above, may have
+//      // updated it.
+//      OutputStream out = localFs.create(localJobFile);
+//      try {
+//        conf.writeXml(out);
+//      } finally {
+//        out.close();
+//      }
+//      this.job = new JobConf(localJobFile);
+//
+//      // Job (the current object) is a Thread, so we wrap its class loader.
+//      if (localDistributedCacheManager.hasLocalClasspaths()) {
+//        setContextClassLoader(localDistributedCacheManager.makeClassLoader(
+//                getContextClassLoader()));
+//      }
+//
+//      profile = new JobProfile(job.getUser(), id, systemJobFile.toString(),
+//                               "http://localhost:8080/", job.getJobName());
+//      status = new JobStatus(id, 0.0f, 0.0f, JobStatus.RUNNING,
+//          profile.getUser(), profile.getJobName(), profile.getJobFile(),
+//          profile.getURL().toString());
+//
+//      jobs.put(id, this);
+//
+//      this.start();
+//    }
+//
+//    /**
+//     * A Runnable instance that handles a map task to be run by an executor.
+//     */
+//    protected class MapTaskRunnable implements Runnable {
+//      private final int taskId;
+//      private final TaskSplitMetaInfo info;
+//      private final JobID jobId;
+//      private final JobConf localConf;
+//
+//      // This is a reference to a shared object passed in by the
+//      // external context; this delivers state to the reducers regarding
+//      // where to fetch mapper outputs.
+//      private final Map<TaskAttemptID, TezTaskOutput> mapOutputFiles;
+//
+//      public volatile Throwable storedException;
+//
+//      public MapTaskRunnable(TaskSplitMetaInfo info, int taskId, JobID jobId,
+//          Map<TaskAttemptID, TezTaskOutput> mapOutputFiles) {
+//        this.info = info;
+//        this.taskId = taskId;
+//        this.mapOutputFiles = mapOutputFiles;
+//        this.jobId = jobId;
+//        this.localConf = new JobConf(job);
+//      }
+//
+//      public void run() {
+//        try {
+//          TaskAttemptID mapId = new TaskAttemptID(new TaskID(
+//              jobId, TaskType.MAP, taskId), 0);
+//          LOG.info("Starting task: " + mapId);
+//          final String user =
+//              UserGroupInformation.getCurrentUser().getShortUserName();
+//          setupChildMapredLocalDirs(mapId, user, localConf);
+//          localConf.setUser(user);
+//
+//          TezTaskAttemptID tezMapId =
+//              IDConverter.fromMRTaskAttemptId(mapId);
+//          mapIds.add(mapId);
+//          // FIXME invalid task context
+//          ProcessorDescriptor mapProcessorDesc = new ProcessorDescriptor(
+//                      MapProcessor.class.getName());
+//          TezEngineTaskContext taskContext =
+//              new TezEngineTaskContext(
+//                  tezMapId, user, localConf.getJobName(), "TODO_vertexName",
+//                  mapProcessorDesc,
+//                  Collections.singletonList(new InputSpec("srcVertex", 0,
+//                      SimpleInput.class.getName())),
+//                  Collections.singletonList(new OutputSpec("tgtVertex", 0,
+//                      LocalOnFileSorterOutput.class.getName())));
+//
+//          TezTaskOutput mapOutput = new TezLocalTaskOutputFiles(localConf, "TODO_uniqueId");
+//          mapOutputFiles.put(mapId, mapOutput);
+//
+//          try {
+//            map_tasks.getAndIncrement();
+//            myMetrics.launchMap(mapId);
+//            Task t = RuntimeUtils.createRuntimeTask(taskContext);
+//            t.initialize(localConf, null, Job.this);
+//            t.run();
+//            myMetrics.completeMap(mapId);
+//          } finally {
+//            map_tasks.getAndDecrement();
+//          }
+//
+//          LOG.info("Finishing task: " + mapId);
+//        } catch (Throwable e) {
+//          this.storedException = e;
+//        }
+//      }
+//    }
+//
+//    /**
+//     * Create Runnables to encapsulate map tasks for use by the executor
+//     * service.
+//     * @param taskInfo Info about the map task splits
+//     * @param jobId the job id
+//     * @param mapOutputFiles a mapping from task attempts to output files
+//     * @return a List of Runnables, one per map task.
+//     */
+//    protected List<MapTaskRunnable> getMapTaskRunnables(
+//        TaskSplitMetaInfo [] taskInfo, JobID jobId,
+//        Map<TaskAttemptID, TezTaskOutput> mapOutputFiles) {
+//
+//      int numTasks = 0;
+//      ArrayList<MapTaskRunnable> list = new ArrayList<MapTaskRunnable>();
+//      for (TaskSplitMetaInfo task : taskInfo) {
+//        list.add(new MapTaskRunnable(task, numTasks++, jobId,
+//            mapOutputFiles));
+//      }
+//
+//      return list;
+//    }
+//
+//    /**
+//     * Initialize the counters that will hold partial-progress from
+//     * the various task attempts.
+//     * @param numMaps the number of map tasks in this job.
+//     */
+//    private synchronized void initCounters(int numMaps) {
+//      // Initialize state trackers for all map tasks.
+//      this.partialMapProgress = new float[numMaps];
+//      this.mapCounters = new TezCounters[numMaps];
+//      for (int i = 0; i < numMaps; i++) {
+//        this.mapCounters[i] = EMPTY_COUNTERS;
+//      }
+//
+//      this.reduceCounters = EMPTY_COUNTERS;
+//    }
+//
+//    /**
+//     * Creates the executor service used to run map tasks.
+//     *
+//     * @param numMapTasks the total number of map tasks to be run
+//     * @return an ExecutorService instance that handles map tasks
+//     */
+//    protected ExecutorService createMapExecutor(int numMapTasks) {
+//
+//      // Determine the size of the thread pool to use
+//      int maxMapThreads = job.getInt(LOCAL_MAX_MAPS, 1);
+//      if (maxMapThreads < 1) {
+//        throw new IllegalArgumentException(
+//            "Configured " + LOCAL_MAX_MAPS + " must be >= 1");
+//      }
+//      this.numMapTasks = numMapTasks;
+//      maxMapThreads = Math.min(maxMapThreads, this.numMapTasks);
+//      maxMapThreads = Math.max(maxMapThreads, 1); // In case of no tasks.
+//
+//      initCounters(this.numMapTasks);
+//
+//      LOG.debug("Starting thread pool executor.");
+//      LOG.debug("Max local threads: " + maxMapThreads);
+//      LOG.debug("Map tasks to process: " + this.numMapTasks);
+//
+//      // Create a new executor service to drain the work queue.
+//      ThreadFactory tf = new ThreadFactoryBuilder()
+//        .setNameFormat("LocalJobRunner Map Task Executor #%d")
+//        .build();
+//      ExecutorService executor = Executors.newFixedThreadPool(maxMapThreads, tf);
+//
+//      return executor;
+//    }
+//
+//    private org.apache.hadoop.mapreduce.OutputCommitter
+//    createOutputCommitter(boolean newApiCommitter, JobID jobId, Configuration conf) throws Exception {
+//      org.apache.hadoop.mapreduce.OutputCommitter committer = null;
+//
+//      LOG.info("OutputCommitter set in config "
+//          + conf.get("mapred.output.committer.class"));
+//
+//      if (newApiCommitter) {
+//        org.apache.hadoop.mapreduce.TaskID taskId =
+//            new org.apache.hadoop.mapreduce.TaskID(jobId, TaskType.MAP, 0);
+//        org.apache.hadoop.mapreduce.TaskAttemptID taskAttemptID =
+//            new org.apache.hadoop.mapreduce.TaskAttemptID(taskId, 0);
+//        org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
+//            new TaskAttemptContextImpl(conf, taskAttemptID);
+//        @SuppressWarnings("rawtypes")
+//        OutputFormat outputFormat =
+//          ReflectionUtils.newInstance(taskContext.getOutputFormatClass(), conf);
+//        committer = outputFormat.getOutputCommitter(taskContext);
+//      } else {
+//        committer = ReflectionUtils.newInstance(conf.getClass(
+//            "mapred.output.committer.class", FileOutputCommitter.class,
+//            org.apache.hadoop.mapred.OutputCommitter.class), conf);
+//      }
+//      LOG.info("OutputCommitter is " + committer.getClass().getName());
+//      return committer;
+//    }
+//
+//    @Override
+//    public void run() {
+//      JobID jobId = profile.getJobID();
+//      JobContext jContext = new JobContextImpl(job, jobId);
+//
+//      org.apache.hadoop.mapreduce.OutputCommitter outputCommitter = null;
+//      try {
+//        outputCommitter = createOutputCommitter(conf.getUseNewMapper(), jobId, conf);
+//      } catch (Exception e) {
+//        LOG.info("Failed to createOutputCommitter", e);
+//        return;
+//      }
+//
+//      try {
+//        TaskSplitMetaInfo[] taskSplitMetaInfos =
+//          SplitMetaInfoReader.readSplitMetaInfo(jobId, localFs, conf, systemJobDir);
+//
+//        int numReduceTasks = job.getNumReduceTasks();
+//        if (numReduceTasks > 1 || numReduceTasks < 0) {
+//          // we only allow 0 or 1 reducer in local mode
+//          numReduceTasks = 1;
+//          job.setNumReduceTasks(1);
+//        }
+//        outputCommitter.setupJob(jContext);
+//        status.setSetupProgress(1.0f);
+//
+//        Map<TaskAttemptID, TezTaskOutput> mapOutputFiles =
+//            Collections.synchronizedMap(new HashMap<TaskAttemptID, TezTaskOutput>());
+//
+//        List<MapTaskRunnable> taskRunnables = getMapTaskRunnables(taskSplitMetaInfos,
+//            jobId, mapOutputFiles);
+//        ExecutorService mapService = createMapExecutor(taskRunnables.size());
+//
+//        // Start populating the executor with work units.
+//        // They may begin running immediately (in other threads).
+//        for (Runnable r : taskRunnables) {
+//          mapService.submit(r);
+//        }
+//
+//        try {
+//          mapService.shutdown(); // Instructs queue to drain.
+//
+//          // Wait for tasks to finish; do not use a time-based timeout.
+//          // (See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6179024)
+//          LOG.info("Waiting for map tasks");
+//          mapService.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS);
+//        } catch (InterruptedException ie) {
+//          // Cancel all threads.
+//          mapService.shutdownNow();
+//          throw ie;
+//        }
+//
+//        LOG.info("Map task executor complete.");
+//
+//        // After waiting for the map tasks to complete, if any of these
+//        // have thrown an exception, rethrow it now in the main thread context.
+//        for (MapTaskRunnable r : taskRunnables) {
+//          if (r.storedException != null) {
+//            throw new Exception(r.storedException);
+//          }
+//        }
+//
+//        TaskAttemptID reduceId = new TaskAttemptID(new TaskID(
+//            jobId, TaskType.REDUCE, 0), 0);
+//        LOG.info("Starting task: " + reduceId);
+//        try {
+//          if (numReduceTasks > 0) {
+//            String user =
+//                UserGroupInformation.getCurrentUser().getShortUserName();
+//            JobConf localConf = new JobConf(job);
+//            localConf.setUser(user);
+//            localConf.set("mapreduce.jobtracker.address", "local");
+//            setupChildMapredLocalDirs(reduceId, user, localConf);
+//            // FIXME invalid task context
+//            ProcessorDescriptor reduceProcessorDesc = new ProcessorDescriptor(
+//                ReduceProcessor.class.getName());
+//            TezEngineTaskContext taskContext = new TezEngineTaskContext(
+//                IDConverter.fromMRTaskAttemptId(reduceId), user,
+//                localConf.getJobName(), "TODO_vertexName",
+//                reduceProcessorDesc,
+//                Collections.singletonList(new InputSpec("TODO_srcVertexName",
+//                    mapIds.size(), LocalMergedInput.class.getName())),
+//                Collections.singletonList(new OutputSpec("TODO_targetVertex",
+//                    0, SimpleOutput.class.getName())));
+//
+//            // move map output to reduce input
+//            for (int i = 0; i < mapIds.size(); i++) {
+//              if (!this.isInterrupted()) {
+//                TaskAttemptID mapId = mapIds.get(i);
+//                if (LOG.isDebugEnabled()) {
+//                  // TODO NEWTEZ Fix this logging.
+////                  LOG.debug("XXX mapId: " + i +
+////                      " LOCAL_DIR = " +
+////                      mapOutputFiles.get(mapId).getConf().get(
+////                          TezJobConfig.LOCAL_DIRS));
+//                }
+//                Path mapOut = mapOutputFiles.get(mapId).getOutputFile();
+//                TezTaskOutput localOutputFile = new TezLocalTaskOutputFiles(localConf, "TODO_uniqueId");
+//                Path reduceIn =
+//                  localOutputFile.getInputFileForWrite(
+//                      mapId.getTaskID().getId(), localFs.getFileStatus(mapOut).getLen());
+//                if (!localFs.mkdirs(reduceIn.getParent())) {
+//                  throw new IOException("Mkdirs failed to create "
+//                      + reduceIn.getParent().toString());
+//                }
+//                if (!localFs.rename(mapOut, reduceIn))
+//                  throw new IOException("Couldn't rename " + mapOut);
+//              } else {
+//                throw new InterruptedException();
+//              }
+//            }
+//            if (!this.isInterrupted()) {
+//              reduce_tasks += 1;
+//              myMetrics.launchReduce(reduceId);
+//              Task t = RuntimeUtils.createRuntimeTask(taskContext);
+//              t.initialize(localConf, null, Job.this);
+//              t.run();
+//              myMetrics.completeReduce(reduceId);
+//              reduce_tasks -= 1;
+//            } else {
+//              throw new InterruptedException();
+//            }
+//          }
+//        } finally {
+//          for (TezTaskOutput output : mapOutputFiles.values()) {
+//            output.removeAll();
+//          }
+//        }
+//        // delete the temporary directory in output directory
+//        // FIXME
+//        //outputCommitter.commitJob(jContext);
+//        status.setCleanupProgress(1.0f);
+//
+//        if (killed) {
+//          this.status.setRunState(JobStatus.KILLED);
+//        } else {
+//          this.status.setRunState(JobStatus.SUCCEEDED);
+//        }
+//
+//        JobEndNotifier.localRunnerNotification(job, status);
+//
+//      } catch (Throwable t) {
+//        try {
+//          outputCommitter.abortJob(jContext,
+//            org.apache.hadoop.mapreduce.JobStatus.State.FAILED);
+//        } catch (IOException ioe) {
+//          LOG.info("Error cleaning up job:" + id);
+//        }
+//        status.setCleanupProgress(1.0f);
+//        if (killed) {
+//          this.status.setRunState(JobStatus.KILLED);
+//        } else {
+//          this.status.setRunState(JobStatus.FAILED);
+//        }
+//        LOG.warn(id, t);
+//
+//        JobEndNotifier.localRunnerNotification(job, status);
+//
+//      } finally {
+//        try {
+//          fs.delete(systemJobFile.getParent(), true);  // delete submit dir
+//          localFs.delete(localJobFile, true);              // delete local copy
+//          // Cleanup distributed cache
+//          localDistributedCacheManager.close();
+//        } catch (IOException e) {
+//          LOG.warn("Error cleaning up "+id+": "+e);
+//        }
+//      }
+//    }
+//
+//    // TaskUmbilicalProtocol methods
+//    @Override
+//    public ContainerTask getTask(ContainerContext containerContext)
+//        throws IOException {
+//      return null;
+//    }
+//
+//    /** Return the current values of the counters for this job,
+//     * including tasks that are in progress.
+//     */
+//    public synchronized TezCounters getCurrentCounters() {
+//      if (null == mapCounters) {
+//        // Counters not yet initialized for job.
+//        return EMPTY_COUNTERS;
+//      }
+//
+//      TezCounters current = EMPTY_COUNTERS;
+//      for (TezCounters c : mapCounters) {
+//        current.incrAllCounters(c);
+//      }
+//      current.incrAllCounters(reduceCounters);
+//      return current;
+//    }
+//
+//    @Override
+//    public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
+//      return true;
+//    }
+//
+//    @Override
+//    public TezTaskDependencyCompletionEventsUpdate
+//    getDependentTasksCompletionEvents(
+//        int fromEventIdx, int maxEventsToFetch,
+//        TezTaskAttemptID reduce) {
+//      throw new UnsupportedOperationException(
+//          "getDependentTasksCompletionEvents not supported in LocalJobRunner");
+//    }
+//
+//    @Override
+//    public void outputReady(TezTaskAttemptID taskAttemptId,
+//        OutputContext outputContext) throws IOException {
+//      // Ignore for now.
+//    }
+//
+//    @Override
+//    public ProceedToCompletionResponse proceedToCompletion(
+//        TezTaskAttemptID taskAttemptId) throws IOException {
+//      // TODO TEZAM5 Really depends on the module - inmem shuffle or not.
+//      return new ProceedToCompletionResponse(true, true);
+//    }
+//
+//    @Override
+//    public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) {
+//      // TODO Auto-generated method stub
+//      // TODO TODONEWTEZ
+//      return null;
+//    }
+//
+//  }
+//
+//  public LocalJobRunnerTez(Configuration conf) throws IOException {
+//    this(new JobConf(conf));
+//  }
+//
+//  @Deprecated
+//  public LocalJobRunnerTez(JobConf conf) throws IOException {
+//    this.fs = FileSystem.getLocal(conf);
+//    this.conf = conf;
+//    myMetrics = new LocalJobRunnerMetricsTez(new JobConf(conf));
+//  }
+//
+//  // JobSubmissionProtocol methods
+//
+//  private static int jobid = 0;
+//  public synchronized org.apache.hadoop.mapreduce.JobID getNewJobID() {
+//    return new org.apache.hadoop.mapreduce.JobID("local", ++jobid);
+//  }
+//
+//  public org.apache.hadoop.mapreduce.JobStatus submitJob(
+//      org.apache.hadoop.mapreduce.JobID jobid, String jobSubmitDir,
+//      Credentials credentials) throws IOException {
+//    Job job = new Job(JobID.downgrade(jobid), jobSubmitDir);
+//    job.job.setCredentials(credentials);
+//    return job.status;
+//
+//  }
+//
+//  public void killJob(org.apache.hadoop.mapreduce.JobID id) {
+//    jobs.get(JobID.downgrade(id)).killed = true;
+//    jobs.get(JobID.downgrade(id)).interrupt();
+//  }
+//
+//  public void setJobPriority(org.apache.hadoop.mapreduce.JobID id,
+//      String jp) throws IOException {
+//    throw new UnsupportedOperationException("Changing job priority " +
+//                      "in LocalJobRunner is not supported.");
+//  }
+//
+//  /** Throws {@link UnsupportedOperationException} */
+//  public boolean killTask(org.apache.hadoop.mapreduce.TaskAttemptID taskId,
+//      boolean shouldFail) throws IOException {
+//    throw new UnsupportedOperationException("Killing tasks in " +
+//    "LocalJobRunner is not supported");
+//  }
+//
+//  public org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(
+//      org.apache.hadoop.mapreduce.JobID id, TaskType type) {
+//    return new org.apache.hadoop.mapreduce.TaskReport[0];
+//  }
+//
+//  public org.apache.hadoop.mapreduce.JobStatus getJobStatus(
+//      org.apache.hadoop.mapreduce.JobID id) {
+//    Job job = jobs.get(JobID.downgrade(id));
+//    if(job != null)
+//      return job.status;
+//    else
+//      return null;
+//  }
+//
+//  public org.apache.hadoop.mapreduce.Counters getJobCounters(
+//      org.apache.hadoop.mapreduce.JobID id) {
+//    Job job = jobs.get(JobID.downgrade(id));
+//
+//    return new org.apache.hadoop.mapreduce.Counters(
+//        new MRCounters(job.getCurrentCounters()));
+//  }
+//
+//  public String getFilesystemName() throws IOException {
+//    return fs.getUri().toString();
+//  }
+//
+//  public ClusterMetrics getClusterMetrics() {
+//    int numMapTasks = map_tasks.get();
+//    return new ClusterMetrics(numMapTasks, reduce_tasks, numMapTasks,
+//        reduce_tasks, 0, 0, 1, 1, jobs.size(), 1, 0, 0);
+//  }
+//
+//  public JobTrackerStatus getJobTrackerStatus() {
+//    return JobTrackerStatus.RUNNING;
+//  }
+//
+//  public long getTaskTrackerExpiryInterval()
+//      throws IOException, InterruptedException {
+//    return 0;
+//  }
+//
+//  /**
+//   * Get all active trackers in cluster.
+//   * @return array of TaskTrackerInfo
+//   */
+//  public TaskTrackerInfo[] getActiveTrackers()
+//      throws IOException, InterruptedException {
+//    return null;
+//  }
+//
+//  /**
+//   * Get all blacklisted trackers in cluster.
+//   * @return array of TaskTrackerInfo
+//   */
+//  public TaskTrackerInfo[] getBlacklistedTrackers()
+//      throws IOException, InterruptedException {
+//    return null;
+//  }
+//
+//  public TaskCompletionEvent[] getTaskCompletionEvents(
+//      org.apache.hadoop.mapreduce.JobID jobid
+//      , int fromEventId, int maxEvents) throws IOException {
+//    return TaskCompletionEvent.EMPTY_ARRAY;
+//  }
+//
+//  public org.apache.hadoop.mapreduce.JobStatus[] getAllJobs() {return null;}
+//
+//
+//  /**
+//   * Returns the diagnostic information for a particular task in the given job.
+//   * To be implemented
+//   */
+//  public String[] getTaskDiagnostics(
+//      org.apache.hadoop.mapreduce.TaskAttemptID taskid) throws IOException{
+//	  return new String [0];
+//  }
+//
+//  /**
+//   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getSystemDir()
+//   */
+//  public String getSystemDir() {
+//    Path sysDir = new Path(
+//      conf.get(JTConfig.JT_SYSTEM_DIR, "/tmp/hadoop/mapred/system"));
+//    return fs.makeQualified(sysDir).toString();
+//  }
+//
+//  /**
+//   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getQueueAdmins(String)
+//   */
+//  public AccessControlList getQueueAdmins(String queueName) throws IOException {
+//	  return new AccessControlList(" ");// no queue admins for local job runner
+//  }
+//
+//  /**
+//   * @see org.apache.hadoop.mapreduce.protocol.ClientProtocol#getStagingAreaDir()
+//   */
+//  public String getStagingAreaDir() throws IOException {
+//    Path stagingRootDir = new Path(conf.get(JTConfig.JT_STAGING_AREA_ROOT,
+//        "/tmp/hadoop/mapred/staging"));
+//    UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+//    String user;
+//    if (ugi != null) {
+//      user = ugi.getShortUserName() + rand.nextInt();
+//    } else {
+//      user = "dummy" + rand.nextInt();
+//    }
+//    return fs.makeQualified(new Path(stagingRootDir, user+"/.staging")).toString();
+//  }
+//
+//  public String getJobHistoryDir() {
+//    return null;
+//  }
+//
+//  @Override
+//  public QueueInfo[] getChildQueues(String queueName) throws IOException {
+//    return null;
+//  }
+//
+//  @Override
+//  public QueueInfo[] getRootQueues() throws IOException {
+//    return null;
+//  }
+//
+//  @Override
+//  public QueueInfo[] getQueues() throws IOException {
+//    return null;
+//  }
+//
+//
+//  @Override
+//  public QueueInfo getQueue(String queue) throws IOException {
+//    return null;
+//  }
+//
+//  @Override
+//  public org.apache.hadoop.mapreduce.QueueAclsInfo[]
+//      getQueueAclsForCurrentUser() throws IOException{
+//    return null;
+//  }
+//
+//  /**
+//   * Set the max number of map tasks to run concurrently in the LocalJobRunner.
+//   * @param job the job to configure
+//   * @param maxMaps the maximum number of map tasks to allow.
+//   */
+//  public static void setLocalMaxRunningMaps(
+//      org.apache.hadoop.mapreduce.JobContext job,
+//      int maxMaps) {
+//    job.getConfiguration().setInt(LOCAL_MAX_MAPS, maxMaps);
+//  }
+//
+//  /**
+//   * @return the max number of map tasks to run concurrently in the
+//   * LocalJobRunner.
+//   */
+//  public static int getLocalMaxRunningMaps(
+//      org.apache.hadoop.mapreduce.JobContext job) {
+//    return job.getConfiguration().getInt(LOCAL_MAX_MAPS, 1);
+//  }
+//
+//  @Override
+//  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token
+//                                       ) throws IOException,
+//                                                InterruptedException {
+//  }
+//
+//  @Override
+//  public Token<DelegationTokenIdentifier>
+//     getDelegationToken(Text renewer) throws IOException, InterruptedException {
+//    return null;
+//  }
+//
+//  @Override
+//  public long renewDelegationToken(Token<DelegationTokenIdentifier> token
+//                                      ) throws IOException,InterruptedException{
+//    return 0;
+//  }
+//
+//  @Override
+//  public LogParams getLogFileParams(org.apache.hadoop.mapreduce.JobID jobID,
+//      org.apache.hadoop.mapreduce.TaskAttemptID taskAttemptID)
+//      throws IOException, InterruptedException {
+//    throw new UnsupportedOperationException("Not supported");
+//  }
+//
+//  static void setupChildMapredLocalDirs(
+//      TaskAttemptID taskAttemptID, String user, JobConf conf) {
+//    String[] localDirs =
+//        conf.getTrimmedStrings(
+//            TezJobConfig.LOCAL_DIRS, TezJobConfig.DEFAULT_LOCAL_DIRS);
+//    String jobId = taskAttemptID.getJobID().toString();
+//    String taskId = taskAttemptID.getTaskID().toString();
+//    boolean isCleanup = false;
+//    StringBuffer childMapredLocalDir =
+//        new StringBuffer(localDirs[0] + Path.SEPARATOR
+//            + getLocalTaskDir(user, jobId, taskId, isCleanup));
+//    for (int i = 1; i < localDirs.length; i++) {
+//      childMapredLocalDir.append("," + localDirs[i] + Path.SEPARATOR
+//          + getLocalTaskDir(user, jobId, taskId, isCleanup));
+//    }
+//    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,
+//        TezLocalTaskOutputFiles.class, TezTaskOutput.class);
+//  }
+//
+//  static final String TASK_CLEANUP_SUFFIX = ".cleanup";
+//  static final String SUBDIR = jobDir;
+//  static final String JOBCACHE = "jobcache";
+//
+//  static String getLocalTaskDir(String user, String jobid, String taskid,
+//      boolean isCleanupAttempt) {
+//    String taskDir = SUBDIR + Path.SEPARATOR + user + Path.SEPARATOR + JOBCACHE
+//      + Path.SEPARATOR + jobid + Path.SEPARATOR + taskid;
+//    if (isCleanupAttempt) {
+//      taskDir = taskDir + TASK_CLEANUP_SUFFIX;
+//    }
+//    return taskDir;
+//  }
+//
+//
+//}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
index 1e0b146..f03fdc7 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
@@ -46,8 +46,8 @@ import org.apache.hadoop.util.ReflectionUtils;
 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.KVReader;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
 import org.apache.tez.engine.newapi.LogicalInput;
 import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.mapreduce.common.Utils;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
index 91fb8cc..dd28ab8 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
@@ -26,8 +26,8 @@ import org.apache.hadoop.util.ReflectionUtils;
 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.KVWriter;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalOutput;
 import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.mapreduce.common.Utils;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
----------------------------------------------------------------------
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 80a2337..e1ed168 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,10 +35,10 @@ import org.apache.hadoop.util.ReflectionUtils;
 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.KVReader;
+import org.apache.tez.engine.api.KVWriter;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalIOProcessor;
 import org.apache.tez.engine.newapi.LogicalInput;
 import org.apache.tez.engine.newapi.LogicalOutput;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
----------------------------------------------------------------------
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 88ab754..b7288e8 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,12 +37,12 @@ import org.apache.hadoop.util.Progressable;
 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.KVReader;
+import org.apache.tez.engine.api.KVWriter;
 import org.apache.tez.engine.common.ConfigUtils;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalIOProcessor;
 import org.apache.tez.engine.newapi.LogicalInput;
 import org.apache.tez.engine.newapi.LogicalOutput;


[20/20] git commit: TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
TEZ-443.  Merge tez-dag-api and tez-engine-api into a single module -
tez-api (part of TEZ-398). (sseth)


Project: http://git-wip-us.apache.org/repos/asf/incubator-tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tez/commit/d316f723
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tez/tree/d316f723
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tez/diff/d316f723

Branch: refs/heads/TEZ-398
Commit: d316f723508c77eb90936a9477812195714b59a2
Parents: b4950f9
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Sep 23 10:44:42 2013 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Sep 23 10:44:42 2013 -0700

----------------------------------------------------------------------
 pom.xml                                         |  10 +-
 tez-api/findbugs-exclude.xml                    |  16 +
 tez-api/pom.xml                                 |  90 +++
 .../org/apache/tez/client/AMConfiguration.java  | 100 ++++
 .../java/org/apache/tez/client/TezClient.java   | 144 +++++
 .../org/apache/tez/client/TezClientUtils.java   | 560 +++++++++++++++++++
 .../java/org/apache/tez/client/TezSession.java  | 184 ++++++
 .../tez/client/TezSessionConfiguration.java     |  57 ++
 .../org/apache/tez/common/TezJobConfig.java     | 314 +++++++++++
 .../tez/common/counters/AbstractCounter.java    |  52 ++
 .../common/counters/AbstractCounterGroup.java   | 208 +++++++
 .../tez/common/counters/AbstractCounters.java   | 385 +++++++++++++
 .../tez/common/counters/CounterGroup.java       |  32 ++
 .../tez/common/counters/CounterGroupBase.java   | 108 ++++
 .../common/counters/CounterGroupFactory.java    | 180 ++++++
 .../apache/tez/common/counters/DAGCounter.java  |  39 ++
 .../tez/common/counters/FileSystemCounter.java  |  30 +
 .../common/counters/FileSystemCounterGroup.java | 327 +++++++++++
 .../common/counters/FrameworkCounterGroup.java  | 275 +++++++++
 .../tez/common/counters/GenericCounter.java     | 109 ++++
 .../apache/tez/common/counters/JobCounter.java  |  45 ++
 .../common/counters/LimitExceededException.java |  36 ++
 .../org/apache/tez/common/counters/Limits.java  | 112 ++++
 .../tez/common/counters/ResourceBundles.java    |  94 ++++
 .../apache/tez/common/counters/TaskCounter.java |  66 +++
 .../apache/tez/common/counters/TezCounter.java  |  83 +++
 .../apache/tez/common/counters/TezCounters.java | 144 +++++
 .../main/java/org/apache/tez/dag/api/DAG.java   | 377 +++++++++++++
 .../apache/tez/dag/api/DagTypeConverters.java   | 278 +++++++++
 .../main/java/org/apache/tez/dag/api/Edge.java  |  59 ++
 .../org/apache/tez/dag/api/EdgeProperty.java    | 147 +++++
 .../org/apache/tez/dag/api/InputDescriptor.java |  32 ++
 .../apache/tez/dag/api/OutputDescriptor.java    |  32 ++
 .../apache/tez/dag/api/ProcessorDescriptor.java |  31 +
 .../apache/tez/dag/api/TezConfiguration.java    | 223 ++++++++
 .../org/apache/tez/dag/api/TezConstants.java    |  29 +
 .../apache/tez/dag/api/TezEntityDescriptor.java |  42 ++
 .../org/apache/tez/dag/api/TezException.java    |  31 +
 .../tez/dag/api/TezUncheckedException.java      |  33 ++
 .../java/org/apache/tez/dag/api/Vertex.java     | 153 +++++
 .../apache/tez/dag/api/VertexLocationHint.java  | 154 +++++
 .../apache/tez/dag/api/client/DAGClient.java    |  67 +++
 .../apache/tez/dag/api/client/DAGStatus.java    | 130 +++++
 .../org/apache/tez/dag/api/client/Progress.java |  67 +++
 .../apache/tez/dag/api/client/VertexStatus.java |  78 +++
 .../rpc/DAGClientAMProtocolBlockingPB.java      |  30 +
 .../dag/api/client/rpc/DAGClientRPCImpl.java    | 291 ++++++++++
 .../java/org/apache/tez/engine/api/Event.java   |  28 +
 .../java/org/apache/tez/engine/api/Input.java   |  71 +++
 .../tez/engine/api/LogicalIOProcessor.java      |  43 ++
 .../org/apache/tez/engine/api/LogicalInput.java |  37 ++
 .../apache/tez/engine/api/LogicalOutput.java    |  36 ++
 .../java/org/apache/tez/engine/api/Output.java  |  71 +++
 .../org/apache/tez/engine/api/Processor.java    |  55 ++
 .../java/org/apache/tez/engine/api/Reader.java  |  26 +
 .../apache/tez/engine/api/TezInputContext.java  |  32 ++
 .../apache/tez/engine/api/TezOutputContext.java |  33 ++
 .../tez/engine/api/TezProcessorContext.java     |  41 ++
 .../apache/tez/engine/api/TezTaskContext.java   | 130 +++++
 .../java/org/apache/tez/engine/api/Writer.java  |  26 +
 .../engine/api/events/DataMovementEvent.java    | 109 ++++
 .../tez/engine/api/events/InputFailedEvent.java |  89 +++
 .../api/events/InputInformationEvent.java       |  41 ++
 .../engine/api/events/InputReadErrorEvent.java  |  65 +++
 .../common/objectregistry/ObjectLifeCycle.java  |  37 ++
 .../common/objectregistry/ObjectRegistry.java   |  56 ++
 .../objectregistry/ObjectRegistryFactory.java   |  32 ++
 tez-api/src/main/proto/DAGApiRecords.proto      | 183 ++++++
 .../src/main/proto/DAGClientAMProtocol.proto    |  81 +++
 tez-api/src/main/proto/Events.proto             |  44 ++
 .../org/apache/tez/dag/api/TestDAGPlan.java     | 155 +++++
 .../org/apache/tez/dag/api/TestDAGVerify.java   | 417 ++++++++++++++
 tez-common/pom.xml                              |   2 +-
 .../java/org/apache/tez/common/Constants.java   |  57 --
 .../org/apache/tez/common/ContainerContext.java |  64 ---
 .../java/org/apache/tez/common/InputSpec.java   |  85 ---
 .../java/org/apache/tez/common/OutputSpec.java  |  84 ---
 .../org/apache/tez/common/TezJobConfig.java     | 314 -----------
 .../org/apache/tez/common/TezTaskContext.java   |  88 ---
 .../org/apache/tez/common/TezTaskStatus.java    | 104 ----
 .../tez/common/counters/AbstractCounter.java    |  52 --
 .../common/counters/AbstractCounterGroup.java   | 208 -------
 .../tez/common/counters/AbstractCounters.java   | 385 -------------
 .../tez/common/counters/CounterGroup.java       |  32 --
 .../tez/common/counters/CounterGroupBase.java   | 108 ----
 .../common/counters/CounterGroupFactory.java    | 180 ------
 .../apache/tez/common/counters/DAGCounter.java  |  39 --
 .../tez/common/counters/FileSystemCounter.java  |  30 -
 .../common/counters/FileSystemCounterGroup.java | 327 -----------
 .../common/counters/FrameworkCounterGroup.java  | 275 ---------
 .../tez/common/counters/GenericCounter.java     | 109 ----
 .../apache/tez/common/counters/JobCounter.java  |  45 --
 .../common/counters/LimitExceededException.java |  36 --
 .../org/apache/tez/common/counters/Limits.java  | 112 ----
 .../tez/common/counters/ResourceBundles.java    |  94 ----
 .../apache/tez/common/counters/TaskCounter.java |  66 ---
 .../apache/tez/common/counters/TezCounter.java  |  83 ---
 .../apache/tez/common/counters/TezCounters.java | 144 -----
 .../org/apache/tez/records/TezContainerId.java  |  78 ---
 tez-dag-api/findbugs-exclude.xml                |  16 -
 tez-dag-api/pom.xml                             |  88 ---
 .../org/apache/tez/client/AMConfiguration.java  | 100 ----
 .../java/org/apache/tez/client/TezClient.java   | 144 -----
 .../org/apache/tez/client/TezClientUtils.java   | 560 -------------------
 .../java/org/apache/tez/client/TezSession.java  | 184 ------
 .../tez/client/TezSessionConfiguration.java     |  57 --
 .../main/java/org/apache/tez/dag/api/DAG.java   | 377 -------------
 .../apache/tez/dag/api/DagTypeConverters.java   | 278 ---------
 .../main/java/org/apache/tez/dag/api/Edge.java  |  59 --
 .../org/apache/tez/dag/api/EdgeProperty.java    | 147 -----
 .../org/apache/tez/dag/api/InputDescriptor.java |  32 --
 .../apache/tez/dag/api/OutputDescriptor.java    |  32 --
 .../apache/tez/dag/api/ProcessorDescriptor.java |  31 -
 .../apache/tez/dag/api/TezConfiguration.java    | 223 --------
 .../org/apache/tez/dag/api/TezConstants.java    |  29 -
 .../apache/tez/dag/api/TezEntityDescriptor.java |  42 --
 .../org/apache/tez/dag/api/TezException.java    |  31 -
 .../tez/dag/api/TezUncheckedException.java      |  33 --
 .../java/org/apache/tez/dag/api/Vertex.java     | 153 -----
 .../apache/tez/dag/api/VertexLocationHint.java  | 154 -----
 .../apache/tez/dag/api/client/DAGClient.java    |  67 ---
 .../apache/tez/dag/api/client/DAGStatus.java    | 130 -----
 .../org/apache/tez/dag/api/client/Progress.java |  67 ---
 .../apache/tez/dag/api/client/VertexStatus.java |  78 ---
 .../rpc/DAGClientAMProtocolBlockingPB.java      |  30 -
 .../dag/api/client/rpc/DAGClientRPCImpl.java    | 291 ----------
 tez-dag-api/src/main/proto/DAGApiRecords.proto  | 183 ------
 .../src/main/proto/DAGClientAMProtocol.proto    |  81 ---
 .../org/apache/tez/dag/api/TestDAGPlan.java     | 155 -----
 .../org/apache/tez/dag/api/TestDAGVerify.java   | 417 --------------
 tez-dag/pom.xml                                 |  12 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |  38 --
 .../org/apache/tez/dag/app/dag/EdgeManager.java |   6 +-
 .../event/TaskAttemptEventOutputConsumable.java |  36 --
 .../dag/app/dag/impl/BroadcastEdgeManager.java  |   6 +-
 .../org/apache/tez/dag/app/dag/impl/Edge.java   |   6 +-
 .../dag/app/dag/impl/OneToOneEdgeManager.java   |   6 +-
 .../app/dag/impl/ScatterGatherEdgeManager.java  |   6 +-
 .../dag/app/dag/impl/ShuffleVertexManager.java  |   6 +-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |   4 +-
 .../TezDependentTaskCompletionEvent.java        | 228 ++++++++
 ...TezTaskDependencyCompletionEventsUpdate.java |  64 +++
 .../dag/app/rm/container/TestAMContainer.java   |  14 +-
 tez-dist/src/main/assembly/tez-dist-full.xml    |   6 +-
 tez-dist/src/main/assembly/tez-dist.xml         |   6 +-
 tez-engine-api/findbugs-exclude.xml             |  16 -
 tez-engine-api/pom.xml                          |  91 ---
 .../java/org/apache/tez/engine/api/Input.java   |  83 ---
 .../java/org/apache/tez/engine/api/Master.java  |  39 --
 .../java/org/apache/tez/engine/api/Output.java  |  65 ---
 .../org/apache/tez/engine/api/Processor.java    |  62 --
 .../java/org/apache/tez/engine/api/Task.java    |  79 ---
 .../common/objectregistry/ObjectLifeCycle.java  |  37 --
 .../common/objectregistry/ObjectRegistry.java   |  56 --
 .../objectregistry/ObjectRegistryFactory.java   |  32 --
 .../org/apache/tez/engine/newapi/Event.java     |  28 -
 .../org/apache/tez/engine/newapi/Input.java     |  71 ---
 .../tez/engine/newapi/LogicalIOProcessor.java   |  43 --
 .../apache/tez/engine/newapi/LogicalInput.java  |  37 --
 .../apache/tez/engine/newapi/LogicalOutput.java |  36 --
 .../org/apache/tez/engine/newapi/Output.java    |  71 ---
 .../org/apache/tez/engine/newapi/Processor.java |  58 --
 .../org/apache/tez/engine/newapi/Reader.java    |  26 -
 .../tez/engine/newapi/TezInputContext.java      |  32 --
 .../tez/engine/newapi/TezOutputContext.java     |  33 --
 .../tez/engine/newapi/TezProcessorContext.java  |  41 --
 .../tez/engine/newapi/TezTaskContext.java       | 130 -----
 .../org/apache/tez/engine/newapi/Writer.java    |  26 -
 .../engine/newapi/events/DataMovementEvent.java | 109 ----
 .../engine/newapi/events/InputFailedEvent.java  |  89 ---
 .../newapi/events/InputInformationEvent.java    |  41 --
 .../newapi/events/InputReadErrorEvent.java      |  65 ---
 .../tez/engine/records/OutputContext.java       |  61 --
 .../TezDependentTaskCompletionEvent.java        | 228 --------
 ...TezTaskDependencyCompletionEventsUpdate.java |  64 ---
 tez-engine-api/src/main/proto/Events.proto      |  44 --
 tez-engine/pom.xml                              |   6 +-
 .../java/org/apache/tez/common/Constants.java   |  57 ++
 .../org/apache/tez/common/ContainerContext.java |  64 +++
 .../tez/common/TezTaskUmbilicalProtocol.java    |  20 -
 .../org/apache/tez/engine/api/KVReader.java     |   2 +-
 .../org/apache/tez/engine/api/KVWriter.java     |   2 +-
 .../api/events/TaskAttemptCompletedEvent.java   |   2 +-
 .../api/events/TaskAttemptFailedEvent.java      |   2 +-
 .../api/events/TaskStatusUpdateEvent.java       |   2 +-
 .../apache/tez/engine/api/impl/TezEvent.java    |  10 +-
 .../engine/api/impl/TezInputContextImpl.java    |   4 +-
 .../engine/api/impl/TezOutputContextImpl.java   |   4 +-
 .../api/impl/TezProcessorContextImpl.java       |   4 +-
 .../tez/engine/api/impl/TezTaskContextImpl.java |   2 +-
 .../broadcast/input/BroadcastInputManager.java  |   2 +-
 .../BroadcastShuffleInputEventHandler.java      |   8 +-
 .../input/BroadcastShuffleManager.java          |   6 +-
 .../broadcast/output/FileBasedKVWriter.java     |   2 +-
 .../tez/engine/common/TezEngineUtils.java       |   4 +-
 .../tez/engine/common/combine/Combiner.java     |   1 -
 .../common/localshuffle/LocalShuffle.java       |   2 +-
 .../tez/engine/common/shuffle/impl/Fetcher.java |   2 +-
 .../common/shuffle/impl/MergeManager.java       |   2 +-
 .../tez/engine/common/shuffle/impl/Shuffle.java |   4 +-
 .../shuffle/impl/ShuffleInputEventHandler.java  |  10 +-
 .../common/shuffle/impl/ShuffleScheduler.java   |   6 +-
 .../common/shuffle/server/ShuffleHandler.java   |   2 +-
 .../engine/common/sort/impl/ExternalSorter.java |   2 +-
 .../common/sort/impl/PipelinedSorter.java       |   2 +-
 .../common/sort/impl/dflt/DefaultSorter.java    |   2 +-
 .../sort/impl/dflt/InMemoryShuffleSorter.java   |   2 +-
 .../tez/engine/lib/input/LocalMergedInput.java  |   6 +-
 .../engine/lib/input/ShuffledMergedInput.java   |   6 +-
 .../lib/input/ShuffledUnorderedKVInput.java     |   8 +-
 .../engine/lib/output/InMemorySortedOutput.java |  10 +-
 .../lib/output/LocalOnFileSorterOutput.java     |   2 +-
 .../engine/lib/output/OnFileSortedOutput.java   |   8 +-
 .../lib/output/OnFileUnorderedKVOutput.java     |   8 +-
 .../LogicalIOProcessorRuntimeTask.java          |  20 +-
 tez-mapreduce/pom.xml                           |   2 +-
 .../org/apache/tez/common/TezTaskStatus.java    | 105 ++++
 .../tez/mapreduce/combine/MRCombiner.java       |   6 +-
 .../tez/mapreduce/hadoop/TezTypeConverters.java |   9 -
 .../tez/mapreduce/hadoop/mapred/MRReporter.java |   4 +-
 .../hadoop/mapreduce/MapContextImpl.java        |   2 +-
 .../mapreduce/TaskAttemptContextImpl.java       |   2 +-
 .../mapreduce/TaskInputOutputContextImpl.java   |   2 +-
 .../apache/tez/mapreduce/input/SimpleInput.java |   6 +-
 .../tez/mapreduce/output/SimpleOutput.java      |   6 +-
 .../apache/tez/mapreduce/processor/MRTask.java  |  13 +-
 .../tez/mapreduce/processor/MRTaskReporter.java |   8 +-
 .../mapreduce/processor/map/MapProcessor.java   |  10 +-
 .../processor/reduce/ReduceProcessor.java       |  10 +-
 .../tez/mapreduce/TestUmbilicalProtocol.java    |  17 -
 tez-yarn-client/pom.xml                         |   2 +-
 231 files changed, 8681 insertions(+), 9660 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index fe41471..63f17eb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -90,12 +90,7 @@
     <dependencies>
       <dependency>
         <groupId>org.apache.tez</groupId>
-        <artifactId>tez-dag-api</artifactId>
-        <version>${project.version}</version>
-      </dependency>
-      <dependency>
-        <groupId>org.apache.tez</groupId>
-        <artifactId>tez-engine-api</artifactId>
+        <artifactId>tez-api</artifactId>
         <version>${project.version}</version>
       </dependency>
       <dependency>
@@ -250,8 +245,7 @@
   </dependencyManagement>
 
   <modules>
-    <module>tez-dag-api</module>
-    <module>tez-engine-api</module>
+    <module>tez-api</module>
     <module>tez-common</module>
     <module>tez-engine</module>
     <module>tez-yarn-client</module>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-api/findbugs-exclude.xml b/tez-api/findbugs-exclude.xml
new file mode 100644
index 0000000..5b11308
--- /dev/null
+++ b/tez-api/findbugs-exclude.xml
@@ -0,0 +1,16 @@
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+<FindBugsFilter>
+
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/pom.xml
----------------------------------------------------------------------
diff --git a/tez-api/pom.xml b/tez-api/pom.xml
new file mode 100644
index 0000000..069b0d4
--- /dev/null
+++ b/tez-api/pom.xml
@@ -0,0 +1,90 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.tez</groupId>
+    <artifactId>tez</artifactId>
+    <version>0.2.0-SNAPSHOT</version>
+  </parent>
+  <artifactId>tez-api</artifactId>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-client</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+    </dependency>
+    <dependency>
+     <groupId>com.google.protobuf</groupId>
+     <artifactId>protobuf-java</artifactId>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-maven-plugins</artifactId>
+        <executions>
+          <execution>
+            <id>compile-protoc</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>protoc</goal>
+            </goals>
+            <configuration>
+              <protocVersion>${protobuf.version}</protocVersion>
+              <protocCommand>${protoc.path}</protocCommand>
+              <imports>
+                <param>${basedir}/src/main/proto</param>
+              </imports>
+              <source>
+                <directory>${basedir}/src/main/proto</directory>
+                <includes>
+                  <include>DAGApiRecords.proto</include>
+                  <include>DAGClientAMProtocol.proto</include>
+                  <include>Events.proto</include>
+                </includes>
+              </source>
+              <output>${project.build.directory}/generated-sources/java</output>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java b/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java
new file mode 100644
index 0000000..f452c74
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/client/AMConfiguration.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.client;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezUncheckedException;
+
+public class AMConfiguration {
+
+  private final Path stagingDir;
+  private final String queueName;
+  private final Map<String, String> env;
+  private final Map<String, LocalResource> localResources;
+  private final TezConfiguration amConf;
+  private final Credentials credentials;
+
+  public AMConfiguration(String queueName, Map<String, String> env,
+      Map<String, LocalResource> localResources,
+      TezConfiguration conf, Credentials credentials) {
+    this.queueName = queueName;
+    if (conf != null) {
+      this.amConf = conf;
+    } else {
+      this.amConf = new TezConfiguration();
+    }
+
+    if (env != null) {
+      this.env = env;
+    } else {
+      this.env = new HashMap<String, String>(0);
+    }
+    this.localResources = localResources;
+    String stagingDirStr = amConf.get(TezConfiguration.TEZ_AM_STAGING_DIR);
+    if (stagingDirStr == null || stagingDirStr.isEmpty()) {
+      throw new TezUncheckedException("Staging directory for AM resources"
+          + " not specified in config"
+          + ", property=" + TezConfiguration.TEZ_AM_STAGING_DIR);
+    }
+    try {
+      FileSystem fs = FileSystem.get(amConf);
+      this.stagingDir = fs.resolvePath(new Path(stagingDirStr));
+    } catch (IOException e) {
+      throw new TezUncheckedException(e);
+    }
+    this.credentials = credentials;
+  }
+
+  public Path getStagingDir() {
+    return stagingDir;
+  }
+
+  public String getQueueName() {
+    return queueName;
+  }
+
+  public Map<String, String> getEnv() {
+    return env;
+  }
+
+  public Map<String, LocalResource> getLocalResources() {
+    return localResources;
+  }
+
+  public TezConfiguration getAMConf() {
+    return amConf;
+  }
+
+  public Credentials getCredentials() {
+    return credentials;
+  }
+
+  public void isCompatible(AMConfiguration other) {
+    // TODO implement
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/client/TezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
new file mode 100644
index 0000000..df260ec
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -0,0 +1,144 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.client;
+
+import java.io.IOException;
+import java.text.NumberFormat;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.rpc.DAGClientRPCImpl;
+
+public class TezClient {
+  private static final Log LOG = LogFactory.getLog(TezClient.class);
+
+  private final TezConfiguration conf;
+  private final YarnConfiguration yarnConf;
+  private YarnClient yarnClient;
+  Map<String, LocalResource> tezJarResources = null;
+
+  /**
+   * <p>
+   * Create an instance of the TezClient which will be used to communicate with
+   * a specific instance of YARN, or TezService when that exists.
+   * </p>
+   * <p>
+   * Separate instances of TezClient should be created to communicate with
+   * different instances of YARN
+   * </p>
+   *
+   * @param conf
+   *          the configuration which will be used to establish which YARN or
+   *          Tez service instance this client is associated with.
+   */
+  public TezClient(TezConfiguration conf) {
+    this.conf = conf;
+    this.yarnConf = new YarnConfiguration(conf);
+    yarnClient = new YarnClientImpl();
+    yarnClient.init(yarnConf);
+    yarnClient.start();
+  }
+
+
+  public DAGClient submitDAGApplication(DAG dag, AMConfiguration amConfig)
+      throws TezException, IOException {
+    ApplicationId appId = createApplication();
+    return submitDAGApplication(appId, dag, amConfig);
+  }
+
+  @Private
+  // To be used only by YarnRunner
+  public DAGClient submitDAGApplication(ApplicationId appId,
+      DAG dag, AMConfiguration amConfig)
+          throws TezException, IOException {
+    try {
+      ApplicationSubmissionContext appContext =
+          TezClientUtils.createApplicationSubmissionContext(conf, appId, dag,
+              dag.getName(), amConfig, getTezJarResources());
+      LOG.info("Submitting DAG to YARN"
+          + ", applicationId=" + appId);
+      yarnClient.submitApplication(appContext);
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+    return getDAGClient(appId);
+  }
+
+  /**
+   * Create a new YARN application
+   * @return <code>ApplicationId</code> for the new YARN application
+   * @throws YarnException
+   * @throws IOException
+   */
+  public ApplicationId createApplication() throws TezException, IOException {
+    try {
+      return yarnClient.createApplication().
+          getNewApplicationResponse().getApplicationId();
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+  }
+
+  private synchronized Map<String, LocalResource> getTezJarResources()
+      throws IOException {
+    if (tezJarResources == null) {
+      tezJarResources = TezClientUtils.setupTezJarsLocalResources(conf);
+    }
+    return tezJarResources;
+  }
+
+  @Private
+  public DAGClient getDAGClient(ApplicationId appId)
+      throws IOException, TezException {
+      return new DAGClientRPCImpl(appId, getDefaultTezDAGID(appId),
+                                   conf);
+  }
+
+  // DO NOT CHANGE THIS. This code is replicated from TezDAGID.java
+  private static final char SEPARATOR = '_';
+  private static final String DAG = "dag";
+  private static final NumberFormat idFormat = NumberFormat.getInstance();
+  static {
+    idFormat.setGroupingUsed(false);
+    idFormat.setMinimumIntegerDigits(6);
+  }
+
+  String getDefaultTezDAGID(ApplicationId appId) {
+     return (new StringBuilder(DAG)).append(SEPARATOR).
+                   append(appId.getClusterTimestamp()).
+                   append(SEPARATOR).
+                   append(appId.getId()).
+                   append(SEPARATOR).
+                   append(idFormat.format(1)).toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
new file mode 100644
index 0000000..7c6a5ed
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -0,0 +1,560 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.client;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.Vector;
+import java.util.Map.Entry;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.Apps;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Records;
+import org.apache.log4j.Level;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB;
+import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
+import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class TezClientUtils {
+
+  private static Log LOG = LogFactory.getLog(TezClientUtils.class);
+
+  public static final FsPermission TEZ_AM_DIR_PERMISSION =
+      FsPermission.createImmutable((short) 0700); // rwx--------
+  public static final FsPermission TEZ_AM_FILE_PERMISSION =
+      FsPermission.createImmutable((short) 0644); // rw-r--r--
+
+  private static final int UTF8_CHUNK_SIZE = 16 * 1024;
+
+  /**
+   * Setup LocalResource map for Tez jars based on provided Configuration
+   * @param conf Configuration to use to access Tez jars' locations
+   * @return Map of LocalResources to use when launching Tez AM
+   * @throws IOException
+   */
+  static Map<String, LocalResource> setupTezJarsLocalResources(
+      TezConfiguration conf)
+      throws IOException {
+    Map<String, LocalResource> tezJarResources =
+        new TreeMap<String, LocalResource>();
+    if (conf.getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false)) {
+      return tezJarResources;
+    }
+
+    // Add tez jars to local resource
+    String[] tezJarUris = conf.getStrings(
+        TezConfiguration.TEZ_LIB_URIS);
+    if (tezJarUris == null
+        || tezJarUris.length == 0) {
+      throw new TezUncheckedException("Invalid configuration of tez jars"
+          + ", " + TezConfiguration.TEZ_LIB_URIS
+          + " is not defined in the configurartion");
+    }
+
+    for (String tezJarUri : tezJarUris) {
+      URI uri;
+      try {
+        uri = new URI(tezJarUri.trim());
+      } catch (URISyntaxException e) {
+        String message = "Invalid URI defined in configuration for"
+            + " location of TEZ jars. providedURI=" + tezJarUri;
+        LOG.error(message);
+        throw new TezUncheckedException(message, e);
+      }
+      if (!uri.isAbsolute()) {
+        String message = "Non-absolute URI defined in configuration for"
+            + " location of TEZ jars. providedURI=" + tezJarUri;
+        LOG.error(message);
+        throw new TezUncheckedException(message);
+      }
+      Path p = new Path(uri);
+      FileSystem pathfs = p.getFileSystem(conf);
+      RemoteIterator<LocatedFileStatus> iter = pathfs.listFiles(p, false);
+      while (iter.hasNext()) {
+        LocatedFileStatus fStatus = iter.next();
+        String rsrcName = fStatus.getPath().getName();
+        // FIXME currently not checking for duplicates due to quirks
+        // in assembly generation
+        if (tezJarResources.containsKey(rsrcName)) {
+          String message = "Duplicate resource found"
+              + ", resourceName=" + rsrcName
+              + ", existingPath=" +
+              tezJarResources.get(rsrcName).getResource().toString()
+              + ", newPath=" + fStatus.getPath();
+          LOG.warn(message);
+          // throw new TezUncheckedException(message);
+        }
+        tezJarResources.put(rsrcName,
+            LocalResource.newInstance(
+                ConverterUtils.getYarnUrlFromPath(fStatus.getPath()),
+                LocalResourceType.FILE,
+                LocalResourceVisibility.PUBLIC,
+                fStatus.getLen(),
+                fStatus.getModificationTime()));
+      }
+    }
+    if (tezJarResources.isEmpty()) {
+      LOG.warn("No tez jars found in configured locations"
+          + ". Ignoring for now. Errors may occur");
+    }
+    return tezJarResources;
+  }
+
+  /**
+   * Verify or create the Staging area directory on the configured Filesystem
+   * @param stagingArea Staging area directory path
+   * @return
+   * @throws IOException
+   */
+  public static FileSystem ensureStagingDirExists(Configuration conf,
+      Path stagingArea)
+      throws IOException {
+    FileSystem fs = stagingArea.getFileSystem(conf);
+    String realUser;
+    String currentUser;
+    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+    realUser = ugi.getShortUserName();
+    currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
+    if (fs.exists(stagingArea)) {
+      FileStatus fsStatus = fs.getFileStatus(stagingArea);
+      String owner = fsStatus.getOwner();
+      if (!(owner.equals(currentUser) || owner.equals(realUser))) {
+        throw new IOException("The ownership on the staging directory "
+            + stagingArea + " is not as expected. " + "It is owned by " + owner
+            + ". The directory must " + "be owned by the submitter "
+            + currentUser + " or " + "by " + realUser);
+      }
+      if (!fsStatus.getPermission().equals(TEZ_AM_DIR_PERMISSION)) {
+        LOG.info("Permissions on staging directory " + stagingArea + " are "
+            + "incorrect: " + fsStatus.getPermission()
+            + ". Fixing permissions " + "to correct value "
+            + TEZ_AM_DIR_PERMISSION);
+        fs.setPermission(stagingArea, TEZ_AM_DIR_PERMISSION);
+      }
+    } else {
+      fs.mkdirs(stagingArea, new FsPermission(TEZ_AM_DIR_PERMISSION));
+    }
+    return fs;
+  }
+
+  /**
+   * Create an ApplicationSubmissionContext to launch a Tez AM
+   * @param conf
+   * @param appId
+   * @param dag
+   * @param appStagingDir
+   * @param ts
+   * @param amQueueName
+   * @param amName
+   * @param amArgs
+   * @param amEnv
+   * @param amLocalResources
+   * @param appConf
+   * @return
+   * @throws IOException
+   * @throws YarnException
+   */
+  static ApplicationSubmissionContext createApplicationSubmissionContext(
+      Configuration conf, ApplicationId appId, DAG dag, String amName,
+      AMConfiguration amConfig,
+      Map<String, LocalResource> tezJarResources)
+          throws IOException, YarnException{
+
+    FileSystem fs = TezClientUtils.ensureStagingDirExists(conf,
+        amConfig.getStagingDir());
+
+    // Setup resource requirements
+    Resource capability = Records.newRecord(Resource.class);
+    capability.setMemory(
+        amConfig.getAMConf().getInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB,
+            TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB_DEFAULT));
+    capability.setVirtualCores(
+        amConfig.getAMConf().getInt(TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES,
+            TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES_DEFAULT));
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("AppMaster capability = " + capability);
+    }
+
+    ByteBuffer securityTokens = null;
+    // Setup security tokens
+    if (amConfig.getCredentials() != null) {
+      DataOutputBuffer dob = new DataOutputBuffer();
+      amConfig.getCredentials().writeTokenStorageToStream(dob);
+      securityTokens = ByteBuffer.wrap(dob.getData(), 0,
+          dob.getLength());
+    }
+
+    // Setup the command to run the AM
+    List<String> vargs = new ArrayList<String>(8);
+    vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
+
+    String amLogLevel = amConfig.getAMConf().get(
+        TezConfiguration.TEZ_AM_LOG_LEVEL,
+        TezConfiguration.TEZ_AM_LOG_LEVEL_DEFAULT);
+    addLog4jSystemProperties(amLogLevel, vargs);
+
+    vargs.add(amConfig.getAMConf().get(TezConfiguration.TEZ_AM_JAVA_OPTS,
+        TezConfiguration.DEFAULT_TEZ_AM_JAVA_OPTS));
+
+    vargs.add(TezConfiguration.TEZ_APPLICATION_MASTER_CLASS);
+    vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
+        File.separator + ApplicationConstants.STDOUT);
+    vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
+        File.separator + ApplicationConstants.STDERR);
+
+
+    Vector<String> vargsFinal = new Vector<String>(8);
+    // Final command
+    StringBuilder mergedCommand = new StringBuilder();
+    for (CharSequence str : vargs) {
+      mergedCommand.append(str).append(" ");
+    }
+    vargsFinal.add(mergedCommand.toString());
+
+    LOG.debug("Command to launch container for ApplicationMaster is : "
+        + mergedCommand);
+
+    // Setup the CLASSPATH in environment
+    // i.e. add { Hadoop jars, job jar, CWD } to classpath.
+    Map<String, String> environment = new HashMap<String, String>();
+
+    boolean isMiniCluster =
+        conf.getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false);
+    if (isMiniCluster) {
+      Apps.addToEnvironment(environment, Environment.CLASSPATH.name(),
+          System.getProperty("java.class.path"));
+    }
+
+    Apps.addToEnvironment(environment,
+        Environment.CLASSPATH.name(),
+        Environment.PWD.$());
+
+    Apps.addToEnvironment(environment,
+        Environment.CLASSPATH.name(),
+        Environment.PWD.$() + File.separator + "*");
+
+    // Add YARN/COMMON/HDFS jars to path
+    if (!isMiniCluster) {
+      for (String c : conf.getStrings(
+          YarnConfiguration.YARN_APPLICATION_CLASSPATH,
+          YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH)) {
+        Apps.addToEnvironment(environment, Environment.CLASSPATH.name(),
+            c.trim());
+      }
+    }
+
+    if (amConfig.getEnv() != null) {
+      for (Map.Entry<String, String> entry : amConfig.getEnv().entrySet()) {
+        Apps.addToEnvironment(environment, entry.getKey(), entry.getValue());
+      }
+    }
+
+    Map<String, LocalResource> localResources =
+        new TreeMap<String, LocalResource>();
+
+    if (amConfig.getLocalResources() != null) {
+      localResources.putAll(amConfig.getLocalResources());
+    }
+    localResources.putAll(tezJarResources);
+
+    // emit conf as PB file
+    Configuration finalTezConf = createFinalTezConfForApp(amConfig.getAMConf());
+    Path binaryConfPath =  new Path(amConfig.getStagingDir(),
+        TezConfiguration.TEZ_PB_BINARY_CONF_NAME + "." + appId.toString());
+    FSDataOutputStream amConfPBOutBinaryStream = null;
+    try {
+      ConfigurationProto.Builder confProtoBuilder =
+          ConfigurationProto.newBuilder();
+      Iterator<Entry<String, String>> iter = finalTezConf.iterator();
+      while (iter.hasNext()) {
+        Entry<String, String> entry = iter.next();
+        PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder();
+        kvp.setKey(entry.getKey());
+        kvp.setValue(entry.getValue());
+        confProtoBuilder.addConfKeyValues(kvp);
+      }
+      //binary output
+      amConfPBOutBinaryStream = FileSystem.create(fs, binaryConfPath,
+          new FsPermission(TEZ_AM_FILE_PERMISSION));
+      confProtoBuilder.build().writeTo(amConfPBOutBinaryStream);
+    } finally {
+      if(amConfPBOutBinaryStream != null){
+        amConfPBOutBinaryStream.close();
+      }
+    }
+
+    LocalResource binaryConfLRsrc =
+        TezClientUtils.createLocalResource(fs,
+            binaryConfPath, LocalResourceType.FILE,
+            LocalResourceVisibility.APPLICATION);
+    localResources.put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
+        binaryConfLRsrc);
+
+    if(dag != null) {
+      // Add tez jars to vertices too
+      for (Vertex v : dag.getVertices()) {
+        v.getTaskLocalResources().putAll(tezJarResources);
+        v.getTaskLocalResources().put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
+            binaryConfLRsrc);
+      }
+
+      // emit protobuf DAG file style
+      Path binaryPath =  new Path(amConfig.getStagingDir(),
+          TezConfiguration.TEZ_PB_PLAN_BINARY_NAME + "." + appId.toString());
+      amConfig.getAMConf().set(TezConfiguration.TEZ_AM_PLAN_REMOTE_PATH,
+          binaryPath.toUri().toString());
+
+      DAGPlan dagPB = dag.createDag(null);
+
+      FSDataOutputStream dagPBOutBinaryStream = null;
+
+      try {
+        //binary output
+        dagPBOutBinaryStream = FileSystem.create(fs, binaryPath,
+            new FsPermission(TEZ_AM_FILE_PERMISSION));
+        dagPB.writeTo(dagPBOutBinaryStream);
+      } finally {
+        if(dagPBOutBinaryStream != null){
+          dagPBOutBinaryStream.close();
+        }
+      }
+
+      localResources.put(TezConfiguration.TEZ_PB_PLAN_BINARY_NAME,
+          TezClientUtils.createLocalResource(fs,
+              binaryPath, LocalResourceType.FILE,
+              LocalResourceVisibility.APPLICATION));
+
+      if (Level.DEBUG.isGreaterOrEqual(Level.toLevel(amLogLevel))) {
+        Path textPath = localizeDagPlanAsText(dagPB, fs,
+            amConfig.getStagingDir(), appId);
+        localResources.put(TezConfiguration.TEZ_PB_PLAN_TEXT_NAME,
+            TezClientUtils.createLocalResource(fs,
+                textPath, LocalResourceType.FILE,
+                LocalResourceVisibility.APPLICATION));
+      }
+    } else {
+      Apps.addToEnvironment(environment,
+          TezConstants.TEZ_AM_IS_SESSION_ENV, "set");
+    }
+
+    Map<ApplicationAccessType, String> acls
+        = new HashMap<ApplicationAccessType, String>();
+
+    // Setup ContainerLaunchContext for AM container
+    ContainerLaunchContext amContainer =
+        ContainerLaunchContext.newInstance(localResources, environment,
+            vargsFinal, null, securityTokens, acls);
+
+    // Set up the ApplicationSubmissionContext
+    ApplicationSubmissionContext appContext = Records
+        .newRecord(ApplicationSubmissionContext.class);
+
+    appContext.setApplicationType(TezConfiguration.TEZ_APPLICATION_TYPE);
+    appContext.setApplicationId(appId);
+    appContext.setResource(capability);
+    appContext.setQueue(amConfig.getQueueName());
+    appContext.setApplicationName(amName);
+    appContext.setCancelTokensWhenComplete(amConfig.getAMConf().getBoolean(
+        TezConfiguration.TEZ_AM_CANCEL_DELEGATION_TOKEN,
+        TezConfiguration.TEZ_AM_CANCEL_DELEGATION_TOKEN_DEFAULT));
+    appContext.setAMContainerSpec(amContainer);
+
+    return appContext;
+
+  }
+
+  @VisibleForTesting
+  static void addLog4jSystemProperties(String logLevel,
+      List<String> vargs) {
+    vargs.add("-Dlog4j.configuration="
+        + TezConfiguration.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE);
+    vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR + "="
+        + ApplicationConstants.LOG_DIR_EXPANSION_VAR);
+    vargs.add("-D" + TezConfiguration.TEZ_ROOT_LOGGER_NAME + "=" + logLevel
+        + "," + TezConfiguration.TEZ_CONTAINER_LOGGER_NAME);
+  }
+
+  static Configuration createFinalTezConfForApp(TezConfiguration amConf) {
+    Configuration conf = new Configuration(false);
+    conf.setQuietMode(true);
+
+    assert amConf != null;
+    Iterator<Entry<String, String>> iter = amConf.iterator();
+    while (iter.hasNext()) {
+      Entry<String, String> entry = iter.next();
+      // Copy all tez config parameters.
+      if (entry.getKey().startsWith(TezConfiguration.TEZ_PREFIX)) {
+        conf.set(entry.getKey(), entry.getValue());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Adding tez dag am parameter: " + entry.getKey()
+              + ", with value: " + entry.getValue());
+        }
+      }
+    }
+    return conf;
+  }
+
+  /**
+   * Helper function to create a YARN LocalResource
+   * @param fs FileSystem object
+   * @param p Path of resource to localize
+   * @param type LocalResource Type
+   * @return
+   * @throws IOException
+   */
+  static LocalResource createLocalResource(FileSystem fs, Path p,
+      LocalResourceType type,
+      LocalResourceVisibility visibility) throws IOException {
+    LocalResource rsrc = Records.newRecord(LocalResource.class);
+    FileStatus rsrcStat = fs.getFileStatus(p);
+    rsrc.setResource(ConverterUtils.getYarnUrlFromPath(fs.resolvePath(rsrcStat
+        .getPath())));
+    rsrc.setSize(rsrcStat.getLen());
+    rsrc.setTimestamp(rsrcStat.getModificationTime());
+    rsrc.setType(type);
+    rsrc.setVisibility(visibility);
+    return rsrc;
+  }
+
+  private static Path localizeDagPlanAsText(DAGPlan dagPB, FileSystem fs,
+      Path appStagingDir, ApplicationId appId) throws IOException {
+    Path textPath = new Path(appStagingDir,
+        TezConfiguration.TEZ_PB_PLAN_TEXT_NAME + "." + appId.toString());
+    FSDataOutputStream dagPBOutTextStream = null;
+    try {
+      dagPBOutTextStream = FileSystem.create(fs, textPath, new FsPermission(
+          TEZ_AM_FILE_PERMISSION));
+      String dagPBStr = dagPB.toString();
+      int dagPBStrLen = dagPBStr.length();
+      if (dagPBStrLen <= UTF8_CHUNK_SIZE) {
+        dagPBOutTextStream.writeUTF(dagPBStr);
+      } else {
+        int startIndex = 0;
+        while (startIndex < dagPBStrLen) {
+          int endIndex = startIndex + UTF8_CHUNK_SIZE;
+          if (endIndex > dagPBStrLen) {
+            endIndex = dagPBStrLen;
+          }
+          dagPBOutTextStream.writeUTF(dagPBStr.substring(startIndex, endIndex));
+          startIndex += UTF8_CHUNK_SIZE;
+        }
+      }
+    } finally {
+      if (dagPBOutTextStream != null) {
+        dagPBOutTextStream.close();
+      }
+    }
+    return textPath;
+  }
+
+  static DAGClientAMProtocolBlockingPB getAMProxy(YarnClient yarnClient,
+      Configuration conf,
+      ApplicationId applicationId) throws TezException, IOException {
+    ApplicationReport appReport;
+    try {
+      appReport = yarnClient.getApplicationReport(
+          applicationId);
+
+      if(appReport == null) {
+        throw new TezUncheckedException("Could not retrieve application report"
+            + " from YARN, applicationId=" + applicationId);
+      }
+      YarnApplicationState appState = appReport.getYarnApplicationState();
+      if(appState != YarnApplicationState.RUNNING) {
+        if (appState == YarnApplicationState.FINISHED
+            || appState == YarnApplicationState.KILLED
+            || appState == YarnApplicationState.FAILED) {
+          throw new TezUncheckedException("Application not running"
+              + ", applicationId=" + applicationId
+              + ", yarnApplicationState=" + appReport.getYarnApplicationState()
+              + ", finalApplicationStatus="
+              + appReport.getFinalApplicationStatus()
+              + ", trackingUrl=" + appReport.getTrackingUrl());
+        }
+        return null;
+      }
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+    return getAMProxy(conf, appReport.getHost(), appReport.getRpcPort());
+  }
+
+  static DAGClientAMProtocolBlockingPB getAMProxy(Configuration conf,
+      String amHost, int amRpcPort) throws IOException {
+    InetSocketAddress addr = new InetSocketAddress(amHost,
+        amRpcPort);
+
+    RPC.setProtocolEngine(conf, DAGClientAMProtocolBlockingPB.class,
+        ProtobufRpcEngine.class);
+    DAGClientAMProtocolBlockingPB proxy =
+        (DAGClientAMProtocolBlockingPB) RPC.getProxy(
+            DAGClientAMProtocolBlockingPB.class, 0, addr, conf);
+    return proxy;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/client/TezSession.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezSession.java b/tez-api/src/main/java/org/apache/tez/client/TezSession.java
new file mode 100644
index 0000000..acf523d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/client/TezSession.java
@@ -0,0 +1,184 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.client;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB;
+import org.apache.tez.dag.api.client.rpc.DAGClientRPCImpl;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto;
+import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+
+public class TezSession {
+
+  private static final Log LOG = LogFactory.getLog(TezSession.class);
+
+  private final String sessionName;
+  private ApplicationId applicationId;
+  private LocalResource tezConfPBLRsrc = null;
+  private final TezSessionConfiguration sessionConfig;
+  private YarnClient yarnClient;
+  private Map<String, LocalResource> tezJarResources;
+  private boolean sessionStarted = false;
+
+  public TezSession(String sessionName,
+      ApplicationId applicationId,
+      TezSessionConfiguration sessionConfig) {
+    this.sessionName = sessionName;
+    this.sessionConfig = sessionConfig;
+    this.applicationId = applicationId;
+  }
+
+  public TezSession(String sessionName,
+      TezSessionConfiguration sessionConfig) {
+    this(sessionName, null, sessionConfig);
+  }
+
+  public synchronized void start() throws TezException, IOException {
+    yarnClient = YarnClient.createYarnClient();
+    yarnClient.init(sessionConfig.getYarnConfiguration());
+    yarnClient.start();
+
+    tezJarResources = TezClientUtils.setupTezJarsLocalResources(
+        sessionConfig.getTezConfiguration());
+
+    try {
+      if (applicationId == null) {
+        applicationId = yarnClient.createApplication().
+            getNewApplicationResponse().getApplicationId();
+      }
+
+      ApplicationSubmissionContext appContext =
+          TezClientUtils.createApplicationSubmissionContext(
+              sessionConfig.getTezConfiguration(), applicationId,
+              null, sessionName, sessionConfig.getAMConfiguration(),
+              tezJarResources);
+      tezConfPBLRsrc = appContext.getAMContainerSpec().getLocalResources().get(
+          TezConfiguration.TEZ_PB_BINARY_CONF_NAME);
+      yarnClient.submitApplication(appContext);
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+    sessionStarted = true;
+  }
+
+  public synchronized DAGClient submitDAG(DAG dag)
+      throws TezException, IOException {
+    if (!sessionStarted) {
+      throw new TezUncheckedException("Session not started");
+    }
+
+    String dagId = null;
+    LOG.info("Submitting dag to TezSession"
+        + ", sessionName=" + sessionName
+        + ", applicationId=" + applicationId);
+    // Add tez jars to vertices too
+    for (Vertex v : dag.getVertices()) {
+      v.getTaskLocalResources().putAll(tezJarResources);
+      if (null != tezConfPBLRsrc) {
+        v.getTaskLocalResources().put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
+            tezConfPBLRsrc);
+      }
+    }
+    DAGPlan dagPlan = dag.createDag(sessionConfig.getTezConfiguration());
+    SubmitDAGRequestProto requestProto =
+        SubmitDAGRequestProto.newBuilder().setDAGPlan(dagPlan).build();
+
+    DAGClientAMProtocolBlockingPB proxy;
+    while (true) {
+      proxy = TezClientUtils.getAMProxy(yarnClient,
+          sessionConfig.getYarnConfiguration(), applicationId);
+      if (proxy != null) {
+        break;
+      }
+      try {
+        Thread.sleep(100l);
+      } catch (InterruptedException e) {
+        // Ignore
+      }
+    }
+
+    try {
+      dagId = proxy.submitDAG(null, requestProto).getDagId();
+    } catch (ServiceException e) {
+      throw new TezException(e);
+    }
+    LOG.info("Submitted dag to TezSession"
+        + ", sessionName=" + sessionName
+        + ", applicationId=" + applicationId
+        + ", dagId=" + dagId);
+    return new DAGClientRPCImpl(applicationId, dagId,
+        sessionConfig.getTezConfiguration());
+  }
+
+  public synchronized void stop() throws TezException, IOException {
+    LOG.info("Shutting down Tez Session"
+        + ", sessionName=" + sessionName
+        + ", applicationId=" + applicationId);
+    DAGClientAMProtocolBlockingPB proxy = TezClientUtils.getAMProxy(yarnClient,
+        sessionConfig.getYarnConfiguration(), applicationId);
+    if (proxy != null) {
+      try {
+        ShutdownSessionRequestProto request =
+            ShutdownSessionRequestProto.newBuilder().build();
+        proxy.shutdownSession(null, request);
+        return;
+      } catch (ServiceException e) {
+        LOG.info("Failed to shutdown Tez Session via proxy", e);
+      }
+    }
+    LOG.info("Could not connect to AM, killing session via YARN"
+        + ", sessionName=" + sessionName
+        + ", applicationId=" + applicationId);
+    try {
+      yarnClient.killApplication(applicationId);
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+  }
+
+  public String getSessionName() {
+    return sessionName;
+  }
+
+  @Private
+  @VisibleForTesting
+  public synchronized ApplicationId getApplicationId() {
+    return applicationId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java b/tez-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
new file mode 100644
index 0000000..61ca60b
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.client;
+
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.tez.dag.api.TezConfiguration;
+
+public class TezSessionConfiguration {
+
+  private final AMConfiguration amConfiguration;
+  private final YarnConfiguration yarnConfig;
+  private final TezConfiguration tezConfig;
+
+  public TezSessionConfiguration(AMConfiguration amConfiguration,
+      TezConfiguration tezConfig) {
+    this.amConfiguration = amConfiguration;
+    this.tezConfig = tezConfig;
+    this.yarnConfig = new YarnConfiguration(tezConfig);
+  }
+
+  TezSessionConfiguration(AMConfiguration amConfiguration,
+      TezConfiguration tezConfig,
+      YarnConfiguration yarnConf) {
+    this.amConfiguration = amConfiguration;
+    this.tezConfig = tezConfig;
+    this.yarnConfig = yarnConf;
+  }
+
+  public AMConfiguration getAMConfiguration() {
+    return amConfiguration;
+  }
+
+  public YarnConfiguration getYarnConfiguration() {
+    return yarnConfig;
+  }
+
+  public TezConfiguration getTezConfiguration() {
+    return tezConfig;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7c4540c
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
@@ -0,0 +1,314 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.common;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+
+/**
+ * Meant for user configurable job properties. For others look at {@link Constants}
+ *
+ */
+
+// TODO EVENTUALLY A description for each property.
+@Private
+@Evolving
+public class TezJobConfig {
+
+
+
+
+  /** The number of milliseconds between progress reports. */
+  public static final int PROGRESS_INTERVAL = 3000;
+
+  public static final long DEFAULT_COMBINE_RECORDS_BEFORE_PROGRESS = 10000;
+
+  /**
+   * 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;
+
+  /**
+   * 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 =
+      4 * 1024 * 1024;
+
+  /**
+   * 
+   */
+  public static final String RECORDS_BEFORE_PROGRESS = 
+      "tez.task.merge.progress.records";
+  public static final long DEFAULT_RECORDS_BEFORE_PROGRESS = 10000; 
+
+  /**
+   * List of directories avialble to the engine. 
+   */
+  public static final String LOCAL_DIRS = "tez.engine.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";
+  
+  /**
+   * 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 DEFAULT_TASK_LOCAL_RESOURCE_DIR = "/tmp";
+  
+  public static final String TEZ_TASK_WORKING_DIR = "tez.engine.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_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_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_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 = 
+      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_ENGINE_SORT_THREADS = 
+	      "tez.engine.sort.threads";
+  public static final int DEFAULT_TEZ_ENGINE_SORT_THREADS = 1;
+
+  /**
+   * Specifies a partitioner class, which is used in Tez engine components like OnFileSortedOutput
+   */
+  public static final String TEZ_ENGINE_PARTITIONER_CLASS = "tez.engine.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_ENGINE_NUM_EXPECTED_PARTITIONS = "tez.engine.num.expected.partitions";
+  
+  /**
+   * 
+   */
+  public static final String COUNTERS_MAX_KEY = "tez.engine.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 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 int COUNTER_NAME_MAX_DEFAULT = 64;
+
+  /**
+   * 
+   */
+  public static final String COUNTER_GROUPS_MAX_KEY = "tez.engine.job.counters.groups.max";
+  public static final int COUNTER_GROUPS_MAX_DEFAULT = 50;
+
+  
+  /**
+   * Temporary interface for MR only (not chained Tez) to indicate whether
+   * 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;
+
+  // 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_ENGINE_SHUFFLE_PARALLEL_COPIES = 
+      "tez.engine.shuffle.parallel.copies";
+  public static final int DEFAULT_TEZ_ENGINE_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_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_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_ENGINE_SHUFFLE_CONNECT_TIMEOUT = 
+      "tez.engine.shuffle.connect.timeout";
+  public static final int DEFAULT_TEZ_ENGINE_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 = 
+      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_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT = 
+      "tez.engine.shuffle.input.buffer.percent";
+  public static final float DEFAULT_TEZ_ENGINE_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 = 
+      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;
+  
+  /**
+   * 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_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
+      "tez.engine.shuffle.memory-to-memory.enable";
+  public static final boolean DEFAULT_TEZ_ENGINE_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;
+
+  // TODO Rename. 
+  public static final String TEZ_ENGINE_GROUP_COMPARATOR_CLASS = 
+      "tez.engine.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_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_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_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_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_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_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS = 
+      "tez.engine.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
+   * credentials.
+   */
+  public static final String DAG_CREDENTIALS_BINARY =  "tez.dag.credentials.binary";
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
new file mode 100644
index 0000000..e64a26c
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import com.google.common.base.Objects;
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * An abstract counter class to provide common implementation of
+ * the counter interface in both mapred and mapreduce packages.
+ */
+@InterfaceAudience.Private
+public abstract class AbstractCounter implements TezCounter {
+
+  @Deprecated
+  @Override
+  public void setDisplayName(String name) {}
+
+  @Override
+  public synchronized boolean equals(Object genericRight) {
+    if (genericRight instanceof TezCounter) {
+      synchronized (genericRight) {
+        TezCounter right = (TezCounter) genericRight;
+        return getName().equals(right.getName()) &&
+               getDisplayName().equals(right.getDisplayName()) &&
+               getValue() == right.getValue();
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public synchronized int hashCode() {
+    return Objects.hashCode(getName(), getDisplayName(), getValue());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
new file mode 100644
index 0000000..d8896ed
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.Iterators;
+
+/**
+ * An abstract class to provide common implementation of the
+ * generic counter group in both mapred and mapreduce package.
+ *
+ * @param <T> type of the counter for the group
+ */
+@InterfaceAudience.Private
+public abstract class AbstractCounterGroup<T extends TezCounter>
+    implements CounterGroupBase<T> {
+
+  private final String name;
+  private String displayName;
+  private final ConcurrentMap<String, T> counters =
+      new ConcurrentSkipListMap<String, T>();
+  private final Limits limits;
+
+  public AbstractCounterGroup(String name, String displayName,
+                              Limits limits) {
+    this.name = name;
+    this.displayName = displayName;
+    this.limits = limits;
+  }
+
+  @Override
+  public String getName() {
+    return name;
+  }
+
+  @Override
+  public synchronized String getDisplayName() {
+    return displayName;
+  }
+
+  @Override
+  public synchronized void setDisplayName(String displayName) {
+    this.displayName = displayName;
+  }
+
+  @Override
+  public synchronized void addCounter(T counter) {
+    counters.put(counter.getName(), counter);
+    limits.incrCounters();
+  }
+
+  @Override
+  public synchronized T addCounter(String counterName, String displayName,
+                                   long value) {
+    String saveName = Limits.filterCounterName(counterName);
+    T counter = findCounterImpl(saveName, false);
+    if (counter == null) {
+      return addCounterImpl(saveName, displayName, value);
+    }
+    counter.setValue(value);
+    return counter;
+  }
+
+  private T addCounterImpl(String name, String displayName, long value) {
+    T counter = newCounter(name, displayName, value);
+    addCounter(counter);
+    return counter;
+  }
+
+  @Override
+  public synchronized T findCounter(String counterName, String displayName) {
+    // Take lock to avoid two threads not finding a counter and trying to add
+    // the same counter.
+    String saveName = Limits.filterCounterName(counterName);
+    T counter = findCounterImpl(saveName, false);
+    if (counter == null) {
+      return addCounterImpl(saveName, displayName, 0);
+    }
+    return counter;
+  }
+
+  @Override
+  public T findCounter(String counterName, boolean create) {
+    return findCounterImpl(Limits.filterCounterName(counterName), create);
+  }
+
+  // Lock the object. Cannot simply use concurrent constructs on the counters
+  // data-structure (like putIfAbsent) because of localization, limits etc.
+  private synchronized T findCounterImpl(String counterName, boolean create) {
+    T counter = counters.get(counterName);
+    if (counter == null && create) {
+      String localized =
+          ResourceBundles.getCounterName(getName(), counterName, counterName);
+      return addCounterImpl(counterName, localized, 0);
+    }
+    return counter;
+  }
+
+  @Override
+  public T findCounter(String counterName) {
+    return findCounter(counterName, true);
+  }
+
+  /**
+   * Abstract factory method to create a new counter of type T
+   * @param counterName of the counter
+   * @param displayName of the counter
+   * @param value of the counter
+   * @return a new counter
+   */
+  protected abstract T newCounter(String counterName, String displayName,
+                                  long value);
+
+  /**
+   * Abstract factory method to create a new counter of type T
+   * @return a new counter object
+   */
+  protected abstract T newCounter();
+
+  @Override
+  public Iterator<T> iterator() {
+    return counters.values().iterator();
+  }
+
+  /**
+   * GenericGroup ::= displayName #counter counter*
+   */
+  @Override
+  public synchronized void write(DataOutput out) throws IOException {
+    Text.writeString(out, displayName);
+    WritableUtils.writeVInt(out, counters.size());
+    for(TezCounter counter: counters.values()) {
+      counter.write(out);
+    }
+  }
+
+  @Override
+  public synchronized void readFields(DataInput in) throws IOException {
+    displayName = Text.readString(in);
+    counters.clear();
+    int size = WritableUtils.readVInt(in);
+    for (int i = 0; i < size; i++) {
+      T counter = newCounter();
+      counter.readFields(in);
+      counters.put(counter.getName(), counter);
+      limits.incrCounters();
+    }
+  }
+
+  @Override
+  public synchronized int size() {
+    return counters.size();
+  }
+
+  @Override
+  public synchronized boolean equals(Object genericRight) {
+    if (genericRight instanceof CounterGroupBase<?>) {
+      @SuppressWarnings("unchecked")
+      CounterGroupBase<T> right = (CounterGroupBase<T>) genericRight;
+      return Iterators.elementsEqual(iterator(), right.iterator());
+    }
+    return false;
+  }
+
+  @Override
+  public synchronized int hashCode() {
+    return counters.hashCode();
+  }
+
+  @Override
+  public void incrAllCounters(CounterGroupBase<T> rightGroup) {
+    try {
+      for (TezCounter right : rightGroup) {
+        TezCounter left = findCounter(right.getName(), right.getDisplayName());
+        left.increment(right.getValue());
+      }
+    } catch (LimitExceededException e) {
+      counters.clear();
+      throw e;
+    }
+  }
+}


[16/20] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/Constants.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/Constants.java b/tez-common/src/main/java/org/apache/tez/common/Constants.java
deleted file mode 100644
index 8ea2909..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/Constants.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.tez.common;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-
-
-public class Constants {
-
-  // TODO NEWTEZ Check which of these constants are expecting specific pieces of information which are being removed - like taskAttemptId
-  
-  public static final String TEZ = "tez";
-
-  public static final String MAP_OUTPUT_FILENAME_STRING = "file.out";
-  public static final String MAP_OUTPUT_INDEX_SUFFIX_STRING = ".index";
-  public static final String REDUCE_INPUT_FILE_FORMAT_STRING = "%s/map_%d.out";
-
-  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
-  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_ENGINE_TASK_OUTPUT_FILENAME_STRING = "file.out";
-
-  public static final String TEZ_ENGINE_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_ENGINE_JOB_CREDENTIALS = 
-      "tez.engine.job.credentials";
-  
-  @Private
-  public static final String TEZ_ENGINE_TASK_MEMORY =  "tez.engine.task.memory";
-  
-  public static final String TASK_OUTPUT_DIR = "output";
-  
-  public static final String TEZ_ENGINE_TASK_OUTPUT_MANAGER = 
-      "tez.engine.task.local.output.manager";
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/ContainerContext.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/ContainerContext.java b/tez-common/src/main/java/org/apache/tez/common/ContainerContext.java
deleted file mode 100644
index df92bdc..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/ContainerContext.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-// TODO EVENTUALLY move this over to PB. Fix package/module.
-// TODO EVENTUALLY unit tests for functionality.
-public class ContainerContext implements Writable {
-
-  String containerIdentifier;
-  String pid;
-
-  public ContainerContext() {
-    containerIdentifier = "";
-    pid = "";
-  }
-
-  public ContainerContext(String containerIdStr, String pid) {
-    this.containerIdentifier = containerIdStr;
-    this.pid = pid;
-  }
-
-  public String getContainerIdentifier() {
-    return containerIdentifier;
-  }
-
-  public String getPid() {
-    return pid;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    this.containerIdentifier = Text.readString(in);
-    this.pid = Text.readString(in);
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, containerIdentifier);
-    Text.writeString(out, pid);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/InputSpec.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/InputSpec.java b/tez-common/src/main/java/org/apache/tez/common/InputSpec.java
deleted file mode 100644
index e9faa26..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/InputSpec.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-public class InputSpec implements Writable {
-
-  private String vertexName;
-  private int inDegree;
-  private String inputClassName;
-  
-  public InputSpec() {
-  }
-  
-  public InputSpec(String vertexName, int inDegree,
-      String inputClassName) {
-    this.vertexName = vertexName;
-    this.inDegree = inDegree;
-    this.inputClassName = inputClassName;
-  }
-  
-  /**
-   * @return the name of the input vertex.
-   */
-  public String getVertexName() {
-    return this.vertexName;
-  }
-  
-  /**
-   * @return the number of inputs for this task, which will be available from
-   *         the specified vertex.
-   */
-  public int getNumInputs() {
-    return this.inDegree;
-  }
-
-  /**
-   * @return Input class name
-   */
-  public String getInputClassName() {
-    return this.inputClassName;
-  }
-  
-  @Override
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, vertexName);
-    out.writeInt(inDegree);
-    Text.writeString(out, inputClassName);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    vertexName = Text.readString(in);
-    this.inDegree = in.readInt();
-    inputClassName = Text.readString(in);
-  }
-  
-  @Override
-  public String toString() {
-    return "VertexName: " + vertexName + ", InDegree: " + inDegree
-        + ", InputClassName=" + inputClassName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/OutputSpec.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/OutputSpec.java b/tez-common/src/main/java/org/apache/tez/common/OutputSpec.java
deleted file mode 100644
index 52ec5d8..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/OutputSpec.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-
-public class OutputSpec implements Writable {
-
-  private String vertexName;
-  private int outDegree;
-  private String outputClassName;
-
-  public OutputSpec() {
-  }
-
-  public OutputSpec(String vertexName, int outDegree,
-      String outputClassName) {
-    this.vertexName = vertexName;
-    this.outDegree = outDegree;
-    this.outputClassName = outputClassName;
-  }
-
-  /**
-   * @return the name of the output vertex.
-   */
-  public String getVertexName() {
-    return this.vertexName;
-  }
-
-  /**
-   * @return the number of outputs to be generated by this task.
-   */
-  public int getNumOutputs() {
-    return this.outDegree;
-  }
-
-  /**
-   * @return Output class name
-   */
-  public String getOutputClassName() {
-    return this.outputClassName;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, vertexName);
-    out.writeInt(outDegree);
-    Text.writeString(out, outputClassName);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    vertexName = Text.readString(in);
-    this.outDegree = in.readInt();
-    outputClassName = Text.readString(in);
-  }
-  
-  @Override
-  public String toString() {
-    return "VertexName: " + vertexName + ", OutDegree: " + outDegree
-        + ", OutputClassName=" + outputClassName;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java b/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java
deleted file mode 100644
index 7c4540c..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/TezJobConfig.java
+++ /dev/null
@@ -1,314 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.common;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Evolving;
-
-
-/**
- * Meant for user configurable job properties. For others look at {@link Constants}
- *
- */
-
-// TODO EVENTUALLY A description for each property.
-@Private
-@Evolving
-public class TezJobConfig {
-
-
-
-
-  /** The number of milliseconds between progress reports. */
-  public static final int PROGRESS_INTERVAL = 3000;
-
-  public static final long DEFAULT_COMBINE_RECORDS_BEFORE_PROGRESS = 10000;
-
-  /**
-   * 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;
-
-  /**
-   * 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 =
-      4 * 1024 * 1024;
-
-  /**
-   * 
-   */
-  public static final String RECORDS_BEFORE_PROGRESS = 
-      "tez.task.merge.progress.records";
-  public static final long DEFAULT_RECORDS_BEFORE_PROGRESS = 10000; 
-
-  /**
-   * List of directories avialble to the engine. 
-   */
-  public static final String LOCAL_DIRS = "tez.engine.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";
-  
-  /**
-   * 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 DEFAULT_TASK_LOCAL_RESOURCE_DIR = "/tmp";
-  
-  public static final String TEZ_TASK_WORKING_DIR = "tez.engine.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_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_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_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 = 
-      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_ENGINE_SORT_THREADS = 
-	      "tez.engine.sort.threads";
-  public static final int DEFAULT_TEZ_ENGINE_SORT_THREADS = 1;
-
-  /**
-   * Specifies a partitioner class, which is used in Tez engine components like OnFileSortedOutput
-   */
-  public static final String TEZ_ENGINE_PARTITIONER_CLASS = "tez.engine.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_ENGINE_NUM_EXPECTED_PARTITIONS = "tez.engine.num.expected.partitions";
-  
-  /**
-   * 
-   */
-  public static final String COUNTERS_MAX_KEY = "tez.engine.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 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 int COUNTER_NAME_MAX_DEFAULT = 64;
-
-  /**
-   * 
-   */
-  public static final String COUNTER_GROUPS_MAX_KEY = "tez.engine.job.counters.groups.max";
-  public static final int COUNTER_GROUPS_MAX_DEFAULT = 50;
-
-  
-  /**
-   * Temporary interface for MR only (not chained Tez) to indicate whether
-   * 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;
-
-  // 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_ENGINE_SHUFFLE_PARALLEL_COPIES = 
-      "tez.engine.shuffle.parallel.copies";
-  public static final int DEFAULT_TEZ_ENGINE_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_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_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_ENGINE_SHUFFLE_CONNECT_TIMEOUT = 
-      "tez.engine.shuffle.connect.timeout";
-  public static final int DEFAULT_TEZ_ENGINE_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 = 
-      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_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT = 
-      "tez.engine.shuffle.input.buffer.percent";
-  public static final float DEFAULT_TEZ_ENGINE_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 = 
-      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;
-  
-  /**
-   * 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_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
-      "tez.engine.shuffle.memory-to-memory.enable";
-  public static final boolean DEFAULT_TEZ_ENGINE_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;
-
-  // TODO Rename. 
-  public static final String TEZ_ENGINE_GROUP_COMPARATOR_CLASS = 
-      "tez.engine.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_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_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_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_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_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_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS = 
-      "tez.engine.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
-   * credentials.
-   */
-  public static final String DAG_CREDENTIALS_BINARY =  "tez.dag.credentials.binary";
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/TezTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezTaskContext.java b/tez-common/src/main/java/org/apache/tez/common/TezTaskContext.java
deleted file mode 100644
index 4eed78b..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/TezTaskContext.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public abstract class TezTaskContext implements Writable {
-
-  // Serialized Fields
-  private TezTaskAttemptID taskAttemptId;
-  private String user;
-  private String jobName;
-  private String vertexName;
-
-  public TezTaskContext() {
-  }
-
-  public TezTaskContext(TezTaskAttemptID taskAttemptID, String user, String jobName,
-      String vertexName) {
-    this.taskAttemptId = taskAttemptID;
-    this.user = user;
-    this.jobName = jobName;
-    this.vertexName = vertexName;
-  }
-
-  public TezTaskAttemptID getTaskAttemptId() {
-    return taskAttemptId;
-  }
-
-  
-
-  public TezDAGID getDAGID() {
-    return taskAttemptId.getTaskID().getVertexID().getDAGId();
-  }
-
-  public String getUser() {
-    return user;
-  }
-
-  public String getJobName() {
-    return jobName;
-  }
-  
-  public String getVertexName() {
-    return this.vertexName;
-  }
-
-  public void statusUpdate() throws IOException, InterruptedException {
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    taskAttemptId.write(out);
-    Text.writeString(out, user);
-    Text.writeString(out, jobName);
-    Text.writeString(out, vertexName);
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    taskAttemptId = TezTaskAttemptID.read(in);
-    user = Text.readString(in);
-    jobName = Text.readString(in);
-    vertexName = Text.readString(in);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/TezTaskStatus.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezTaskStatus.java b/tez-common/src/main/java/org/apache/tez/common/TezTaskStatus.java
deleted file mode 100644
index de81f87..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/TezTaskStatus.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.common;
-
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public interface TezTaskStatus extends Writable {
-
-  //enumeration for reporting current phase of a task.
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public static enum Phase{STARTING, MAP, SHUFFLE, SORT, REDUCE, CLEANUP}
-
-  // what state is the task in?
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public static enum State {RUNNING, SUCCEEDED, FAILED, UNASSIGNED, KILLED, 
-                            COMMIT_PENDING, FAILED_UNCLEAN, KILLED_UNCLEAN}
-
-  public abstract TezTaskAttemptID getTaskAttemptId();
-
-  public abstract float getProgress();
-
-  public abstract void setProgress(float progress);
-
-  public abstract State getRunState();
-
-  public abstract void setRunState(State runState);
-
-  public abstract String getDiagnosticInfo();
-
-  public abstract void setDiagnosticInfo(String info);
-
-  // TODOTEZDAG Remove stateString / rename
-  public abstract String getStateString();
-
-  public abstract void setStateString(String stateString);
-
-  public abstract long getFinishTime();
-
-  public abstract void setFinishTime(long finishTime);
-  
-  // TODOTEZDAG Can shuffle / merge be made generic ? Otherwise just a single finish time.
-  public abstract long getShuffleFinishTime();
-
-  public abstract void setShuffleFinishTime(long shuffleFinishTime);
-  
-  public abstract long getMapFinishTime();
-
-  public abstract void setMapFinishTime(long mapFinishTime);
-  
-  public abstract long getSortFinishTime();
-  
-  public abstract void setSortFinishTime(long sortFinishTime);
-  
-  public abstract long getStartTime();
-  
-  public abstract void setStartTime(long startTime);
-
-  // TODOTEZDAG Remove phase
-  public abstract Phase getPhase();
-
-  public abstract void setPhase(Phase phase);
-
-  public abstract TezCounters getCounters();
-
-  public abstract void setCounters(TezCounters counters);
-
-  public abstract List<TezTaskAttemptID> getFailedDependencies();
-
-  public abstract void addFailedDependency(TezTaskAttemptID taskAttempttId);
-
-  public abstract void clearStatus();
-
-  public abstract void statusUpdate(float f, String string, TezCounters counters);
-
-  // TODOTEZDAG maybe remove ?
-  public abstract long getLocalOutputSize();
-
-  public abstract void setOutputSize(long l);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
deleted file mode 100644
index e64a26c..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import com.google.common.base.Objects;
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * An abstract counter class to provide common implementation of
- * the counter interface in both mapred and mapreduce packages.
- */
-@InterfaceAudience.Private
-public abstract class AbstractCounter implements TezCounter {
-
-  @Deprecated
-  @Override
-  public void setDisplayName(String name) {}
-
-  @Override
-  public synchronized boolean equals(Object genericRight) {
-    if (genericRight instanceof TezCounter) {
-      synchronized (genericRight) {
-        TezCounter right = (TezCounter) genericRight;
-        return getName().equals(right.getName()) &&
-               getDisplayName().equals(right.getDisplayName()) &&
-               getValue() == right.getValue();
-      }
-    }
-    return false;
-  }
-
-  @Override
-  public synchronized int hashCode() {
-    return Objects.hashCode(getName(), getDisplayName(), getValue());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java b/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
deleted file mode 100644
index d8896ed..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounterGroup.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableUtils;
-
-import com.google.common.collect.Iterators;
-
-/**
- * An abstract class to provide common implementation of the
- * generic counter group in both mapred and mapreduce package.
- *
- * @param <T> type of the counter for the group
- */
-@InterfaceAudience.Private
-public abstract class AbstractCounterGroup<T extends TezCounter>
-    implements CounterGroupBase<T> {
-
-  private final String name;
-  private String displayName;
-  private final ConcurrentMap<String, T> counters =
-      new ConcurrentSkipListMap<String, T>();
-  private final Limits limits;
-
-  public AbstractCounterGroup(String name, String displayName,
-                              Limits limits) {
-    this.name = name;
-    this.displayName = displayName;
-    this.limits = limits;
-  }
-
-  @Override
-  public String getName() {
-    return name;
-  }
-
-  @Override
-  public synchronized String getDisplayName() {
-    return displayName;
-  }
-
-  @Override
-  public synchronized void setDisplayName(String displayName) {
-    this.displayName = displayName;
-  }
-
-  @Override
-  public synchronized void addCounter(T counter) {
-    counters.put(counter.getName(), counter);
-    limits.incrCounters();
-  }
-
-  @Override
-  public synchronized T addCounter(String counterName, String displayName,
-                                   long value) {
-    String saveName = Limits.filterCounterName(counterName);
-    T counter = findCounterImpl(saveName, false);
-    if (counter == null) {
-      return addCounterImpl(saveName, displayName, value);
-    }
-    counter.setValue(value);
-    return counter;
-  }
-
-  private T addCounterImpl(String name, String displayName, long value) {
-    T counter = newCounter(name, displayName, value);
-    addCounter(counter);
-    return counter;
-  }
-
-  @Override
-  public synchronized T findCounter(String counterName, String displayName) {
-    // Take lock to avoid two threads not finding a counter and trying to add
-    // the same counter.
-    String saveName = Limits.filterCounterName(counterName);
-    T counter = findCounterImpl(saveName, false);
-    if (counter == null) {
-      return addCounterImpl(saveName, displayName, 0);
-    }
-    return counter;
-  }
-
-  @Override
-  public T findCounter(String counterName, boolean create) {
-    return findCounterImpl(Limits.filterCounterName(counterName), create);
-  }
-
-  // Lock the object. Cannot simply use concurrent constructs on the counters
-  // data-structure (like putIfAbsent) because of localization, limits etc.
-  private synchronized T findCounterImpl(String counterName, boolean create) {
-    T counter = counters.get(counterName);
-    if (counter == null && create) {
-      String localized =
-          ResourceBundles.getCounterName(getName(), counterName, counterName);
-      return addCounterImpl(counterName, localized, 0);
-    }
-    return counter;
-  }
-
-  @Override
-  public T findCounter(String counterName) {
-    return findCounter(counterName, true);
-  }
-
-  /**
-   * Abstract factory method to create a new counter of type T
-   * @param counterName of the counter
-   * @param displayName of the counter
-   * @param value of the counter
-   * @return a new counter
-   */
-  protected abstract T newCounter(String counterName, String displayName,
-                                  long value);
-
-  /**
-   * Abstract factory method to create a new counter of type T
-   * @return a new counter object
-   */
-  protected abstract T newCounter();
-
-  @Override
-  public Iterator<T> iterator() {
-    return counters.values().iterator();
-  }
-
-  /**
-   * GenericGroup ::= displayName #counter counter*
-   */
-  @Override
-  public synchronized void write(DataOutput out) throws IOException {
-    Text.writeString(out, displayName);
-    WritableUtils.writeVInt(out, counters.size());
-    for(TezCounter counter: counters.values()) {
-      counter.write(out);
-    }
-  }
-
-  @Override
-  public synchronized void readFields(DataInput in) throws IOException {
-    displayName = Text.readString(in);
-    counters.clear();
-    int size = WritableUtils.readVInt(in);
-    for (int i = 0; i < size; i++) {
-      T counter = newCounter();
-      counter.readFields(in);
-      counters.put(counter.getName(), counter);
-      limits.incrCounters();
-    }
-  }
-
-  @Override
-  public synchronized int size() {
-    return counters.size();
-  }
-
-  @Override
-  public synchronized boolean equals(Object genericRight) {
-    if (genericRight instanceof CounterGroupBase<?>) {
-      @SuppressWarnings("unchecked")
-      CounterGroupBase<T> right = (CounterGroupBase<T>) genericRight;
-      return Iterators.elementsEqual(iterator(), right.iterator());
-    }
-    return false;
-  }
-
-  @Override
-  public synchronized int hashCode() {
-    return counters.hashCode();
-  }
-
-  @Override
-  public void incrAllCounters(CounterGroupBase<T> rightGroup) {
-    try {
-      for (TezCounter right : rightGroup) {
-        TezCounter left = findCounter(right.getName(), right.getDisplayName());
-        left.increment(right.getValue());
-      }
-    } catch (LimitExceededException e) {
-      counters.clear();
-      throw e;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounters.java b/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
deleted file mode 100644
index fd4fdee..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
+++ /dev/null
@@ -1,385 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import static org.apache.tez.common.counters.CounterGroupFactory.getFrameworkGroupId;
-import static org.apache.tez.common.counters.CounterGroupFactory.isFrameworkGroup;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.ConcurrentSkipListMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Maps;
-
-/**
- * An abstract class to provide common implementation for the Counters
- * container in both mapred and mapreduce packages.
- *
- * @param <C> type of counter inside the counters
- * @param <G> type of group inside the counters
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public abstract class AbstractCounters<C extends TezCounter,
-                                       G extends CounterGroupBase<C>>
-    implements Writable, Iterable<G> {
-
-  protected static final Log LOG = LogFactory.getLog("mapreduce.Counters");
-
-  /**
-   * A cache from enum values to the associated counter.
-   */
-  private Map<Enum<?>, C> cache = Maps.newIdentityHashMap();
-  //framework & fs groups
-  private Map<String, G> fgroups = new ConcurrentSkipListMap<String, G>();
-  // other groups
-  private Map<String, G> groups = new ConcurrentSkipListMap<String, G>();
-  private final CounterGroupFactory<C, G> groupFactory;
-
-  // For framework counter serialization without strings
-  enum GroupType { FRAMEWORK, FILESYSTEM };
-
-  // Writes only framework and fs counters if false.
-  private boolean writeAllCounters = true;
-
-  private static final Map<String, String> legacyMap = Maps.newHashMap();
-  static {
-    legacyMap.put("org.apache.hadoop.mapred.Task$Counter",
-                  TaskCounter.class.getName());
-    legacyMap.put("org.apache.hadoop.mapred.JobInProgress$Counter",
-                  JobCounter.class.getName());
-    legacyMap.put("FileSystemCounters", FileSystemCounter.class.getName());
-  }
-
-  private final Limits limits = new Limits();
-
-  @InterfaceAudience.Private
-  public AbstractCounters(CounterGroupFactory<C, G> gf) {
-    groupFactory = gf;
-  }
-
-  /**
-   * Construct from another counters object.
-   * @param <C1> type of the other counter
-   * @param <G1> type of the other counter group
-   * @param counters the counters object to copy
-   * @param groupFactory the factory for new groups
-   */
-  @InterfaceAudience.Private
-  public <C1 extends TezCounter, G1 extends CounterGroupBase<C1>>
-  AbstractCounters(AbstractCounters<C1, G1> counters,
-                   CounterGroupFactory<C, G> groupFactory) {
-    this.groupFactory = groupFactory;
-    for(G1 group: counters) {
-      String name = group.getName();
-      G newGroup = groupFactory.newGroup(name, group.getDisplayName(), limits);
-      (isFrameworkGroup(name) ? fgroups : groups).put(name, newGroup);
-      for(TezCounter counter: group) {
-        newGroup.addCounter(counter.getName(), counter.getDisplayName(),
-                            counter.getValue());
-      }
-    }
-  }
-
-  /** Add a group.
-   * @param group object to add
-   * @return the group
-   */
-  @InterfaceAudience.Private
-  public synchronized G addGroup(G group) {
-    String name = group.getName();
-    if (isFrameworkGroup(name)) {
-      fgroups.put(name, group);
-    } else {
-      limits.checkGroups(groups.size() + 1);
-      groups.put(name, group);
-    }
-    return group;
-  }
-
-  /**
-   * Add a new group
-   * @param name of the group
-   * @param displayName of the group
-   * @return the group
-   */
-  @InterfaceAudience.Private
-  public G addGroup(String name, String displayName) {
-    return addGroup(groupFactory.newGroup(name, displayName, limits));
-  }
-
-  /**
-   * Find a counter, create one if necessary
-   * @param groupName of the counter
-   * @param counterName name of the counter
-   * @return the matching counter
-   */
-  public C findCounter(String groupName, String counterName) {
-    G grp = getGroup(groupName);
-    return grp.findCounter(counterName);
-  }
-
-  /**
-   * Find the counter for the given enum. The same enum will always return the
-   * same counter.
-   * @param key the counter key
-   * @return the matching counter object
-   */
-  public synchronized C findCounter(Enum<?> key) {
-    C counter = cache.get(key);
-    if (counter == null) {
-      counter = findCounter(key.getDeclaringClass().getName(), key.name());
-      cache.put(key, counter);
-    }
-    return counter;
-  }
-
-  /**
-   * Find the file system counter for the given scheme and enum.
-   * @param scheme of the file system
-   * @param key the enum of the counter
-   * @return the file system counter
-   */
-  @InterfaceAudience.Private
-  public synchronized C findCounter(String scheme, FileSystemCounter key) {
-    return ((FileSystemCounterGroup<C>) getGroup(
-        FileSystemCounter.class.getName()).getUnderlyingGroup()).
-        findCounter(scheme, key);
-  }
-
-  /**
-   * Returns the names of all counter classes.
-   * @return Set of counter names.
-   */
-  public synchronized Iterable<String> getGroupNames() {
-    HashSet<String> deprecated = new HashSet<String>();
-    for(Map.Entry<String, String> entry : legacyMap.entrySet()) {
-      String newGroup = entry.getValue();
-      boolean isFGroup = isFrameworkGroup(newGroup);
-      if(isFGroup ? fgroups.containsKey(newGroup) : groups.containsKey(newGroup)) {
-        deprecated.add(entry.getKey());
-      }
-    }
-    return Iterables.concat(fgroups.keySet(), groups.keySet(), deprecated);
-  }
-
-  @Override
-  public Iterator<G> iterator() {
-    return Iterators.concat(fgroups.values().iterator(),
-                            groups.values().iterator());
-  }
-
-  /**
-   * Returns the named counter group, or an empty group if there is none
-   * with the specified name.
-   * @param groupName name of the group
-   * @return the group
-   */
-  public synchronized G getGroup(String groupName) {
-
-    // filterGroupName
-    boolean groupNameInLegacyMap = true;
-    String newGroupName = legacyMap.get(groupName);
-    if (newGroupName == null) {
-      groupNameInLegacyMap = false;
-      newGroupName = Limits.filterGroupName(groupName);
-    }
-
-    boolean isFGroup = isFrameworkGroup(newGroupName);
-    G group = isFGroup ? fgroups.get(newGroupName) : groups.get(newGroupName);
-    if (group == null) {
-      group = groupFactory.newGroup(newGroupName, limits);
-      if (isFGroup) {
-        fgroups.put(newGroupName, group);
-      } else {
-        limits.checkGroups(groups.size() + 1);
-        groups.put(newGroupName, group);
-      }
-      if (groupNameInLegacyMap) {
-        LOG.warn("Group " + groupName + " is deprecated. Use " + newGroupName
-            + " instead");
-      }
-    }
-    return group;
-  }
-
-  /**
-   * Returns the total number of counters, by summing the number of counters
-   * in each group.
-   * @return the total number of counters
-   */
-  public synchronized int countCounters() {
-    int result = 0;
-    for (G group : this) {
-      result += group.size();
-    }
-    return result;
-  }
-
-  /**
-   * Write the set of groups.
-   * Counters ::= version #fgroups (groupId, group)* #groups (group)*
-   */
-  @Override
-  public synchronized void write(DataOutput out) throws IOException {
-    WritableUtils.writeVInt(out, groupFactory.version());
-    WritableUtils.writeVInt(out, fgroups.size());  // framework groups first
-    for (G group : fgroups.values()) {
-      if (group.getUnderlyingGroup() instanceof FrameworkCounterGroup<?, ?>) {
-        WritableUtils.writeVInt(out, GroupType.FRAMEWORK.ordinal());
-        WritableUtils.writeVInt(out, getFrameworkGroupId(group.getName()));
-        group.write(out);
-      } else if (group.getUnderlyingGroup() instanceof FileSystemCounterGroup<?>) {
-        WritableUtils.writeVInt(out, GroupType.FILESYSTEM.ordinal());
-        group.write(out);
-      }
-    }
-    if (writeAllCounters) {
-      WritableUtils.writeVInt(out, groups.size());
-      for (G group : groups.values()) {
-        Text.writeString(out, group.getName());
-        group.write(out);
-      }
-    } else {
-      WritableUtils.writeVInt(out, 0);
-    }
-  }
-
-  @Override
-  public synchronized void readFields(DataInput in) throws IOException {
-    int version = WritableUtils.readVInt(in);
-    if (version != groupFactory.version()) {
-      throw new IOException("Counters version mismatch, expected "+
-          groupFactory.version() +" got "+ version);
-    }
-    int numFGroups = WritableUtils.readVInt(in);
-    fgroups.clear();
-    GroupType[] groupTypes = GroupType.values();
-    while (numFGroups-- > 0) {
-      GroupType groupType = groupTypes[WritableUtils.readVInt(in)];
-      G group;
-      switch (groupType) {
-        case FILESYSTEM: // with nothing
-          group = groupFactory.newFileSystemGroup();
-          break;
-        case FRAMEWORK:  // with group id
-          group = groupFactory.newFrameworkGroup(WritableUtils.readVInt(in));
-          break;
-        default: // Silence dumb compiler, as it would've thrown earlier
-          throw new IOException("Unexpected counter group type: "+ groupType);
-      }
-      group.readFields(in);
-      fgroups.put(group.getName(), group);
-    }
-    int numGroups = WritableUtils.readVInt(in);
-    while (numGroups-- > 0) {
-      limits.checkGroups(groups.size() + 1);
-      G group = groupFactory.newGenericGroup(Text.readString(in), null, limits);
-      group.readFields(in);
-      groups.put(group.getName(), group);
-    }
-  }
-
-  /**
-   * Return textual representation of the counter values.
-   * @return the string
-   */
-  @Override
-  public synchronized String toString() {
-    StringBuilder sb = new StringBuilder("Counters: " + countCounters());
-    for (G group: this) {
-      sb.append("\n\t").append(group.getDisplayName());
-      for (TezCounter counter: group) {
-        sb.append("\n\t\t").append(counter.getDisplayName()).append("=")
-          .append(counter.getValue());
-      }
-    }
-    return sb.toString();
-  }
-
-  /**
-   * Increments multiple counters by their amounts in another Counters
-   * instance.
-   * @param other the other Counters instance
-   */
-  public synchronized void incrAllCounters(AbstractCounters<C, G> other) {
-    for(G right : other) {
-      String groupName = right.getName();
-      G left = (isFrameworkGroup(groupName) ? fgroups : groups).get(groupName);
-      if (left == null) {
-        left = addGroup(groupName, right.getDisplayName());
-      }
-      left.incrAllCounters(right);
-    }
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public boolean equals(Object genericRight) {
-    if (genericRight instanceof AbstractCounters<?, ?>) {
-      return Iterators.elementsEqual(iterator(),
-          ((AbstractCounters<C, G>)genericRight).iterator());
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return groups.hashCode();
-  }
-
-  /**
-   * Set the "writeAllCounters" option to true or false
-   * @param send  if true all counters would be serialized, otherwise only
-   *              framework counters would be serialized in
-   *              {@link #write(DataOutput)}
-   */
-  @InterfaceAudience.Private
-  public void setWriteAllCounters(boolean send) {
-    writeAllCounters = send;
-  }
-
-  /**
-   * Get the "writeAllCounters" option
-   * @return true of all counters would serialized
-   */
-  @InterfaceAudience.Private
-  public boolean getWriteAllCounters() {
-    return writeAllCounters;
-  }
-
-  @InterfaceAudience.Private
-  public Limits limits() {
-    return limits;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroup.java b/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroup.java
deleted file mode 100644
index bc7986d..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroup.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * A group of {@link TezCounter}s that logically belong together. Typically,
- * it is an {@link Enum} subclass and the counters are the values.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public interface CounterGroup extends CounterGroupBase<TezCounter> {
-  // essentially a typedef so user doesn't have to use generic syntax
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java b/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
deleted file mode 100644
index 3b702ba..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-
-/**
- * The common counter group interface.
- *
- * @param <T> type of the counter for the group
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public interface CounterGroupBase<T extends TezCounter>
-    extends Writable, Iterable<T> {
-
-  /**
-   * Get the internal name of the group
-   * @return the internal name
-   */
-  String getName();
-
-  /**
-   * Get the display name of the group.
-   * @return the human readable name
-   */
-  String getDisplayName();
-
-  /**
-   * Set the display name of the group
-   * @param displayName of the group
-   */
-  void setDisplayName(String displayName);
-
-  /** Add a counter to this group.
-   * @param counter to add
-   */
-  void addCounter(T counter);
-
-  /**
-   * Add a counter to this group
-   * @param name  of the counter
-   * @param displayName of the counter
-   * @param value of the counter
-   * @return the counter
-   */
-  T addCounter(String name, String displayName, long value);
-
-  /**
-   * Find a counter in the group.
-   * @param counterName the name of the counter
-   * @param displayName the display name of the counter
-   * @return the counter that was found or added
-   */
-  T findCounter(String counterName, String displayName);
-
-  /**
-   * Find a counter in the group
-   * @param counterName the name of the counter
-   * @param create create the counter if not found if true
-   * @return the counter that was found or added or null if create is false
-   */
-  T findCounter(String counterName, boolean create);
-
-  /**
-   * Find a counter in the group.
-   * @param counterName the name of the counter
-   * @return the counter that was found or added
-   */
-  T findCounter(String counterName);
-
-  /**
-   * @return the number of counters in this group.
-   */
-  int size();
-
-  /**
-   * Increment all counters by a group of counters
-   * @param rightGroup  the group to be added to this group
-   */
-  void incrAllCounters(CounterGroupBase<T> rightGroup);
-  
-  @Private
-  /**
-   * Exposes the underlying group type if a facade.
-   * @return the underlying object that this object is wrapping up.
-   */
-  CounterGroupBase<T> getUnderlyingGroup();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java b/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
deleted file mode 100644
index 45da0dd..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import java.util.List;
-import java.util.Map;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * An abstract class to provide common implementation of the
- * group factory in both mapred and mapreduce packages.
- *
- * @param <C> type of the counter
- * @param <G> type of the group
- */
-@InterfaceAudience.Private
-public abstract class CounterGroupFactory<C extends TezCounter,
-                                          G extends CounterGroupBase<C>> {
-
-  public interface FrameworkGroupFactory<F> {
-    F newGroup(String name);
-  }
-
-  // Integer mapping (for serialization) for framework groups
-  private static final Map<String, Integer> s2i = Maps.newHashMap();
-  private static final List<String> i2s = Lists.newArrayList();
-  private static final int VERSION = 1;
-  private static final String FS_GROUP_NAME = FileSystemCounter.class.getName();
-
-  private final Map<String, FrameworkGroupFactory<G>> fmap = Maps.newHashMap();
-  {
-    // Add builtin counter class here and the version when changed.
-    addFrameworkGroup(TaskCounter.class);
-    addFrameworkGroup(JobCounter.class);
-    addFrameworkGroup(DAGCounter.class);
-  }
-
-  // Initialize the framework counter group mapping
-  private synchronized <T extends Enum<T>>
-  void addFrameworkGroup(final Class<T> cls) {
-    updateFrameworkGroupMapping(cls);
-    fmap.put(cls.getName(), newFrameworkGroupFactory(cls));
-  }
-
-  // Update static mappings (c2i, i2s) of framework groups
-  private static synchronized void updateFrameworkGroupMapping(Class<?> cls) {
-    String name = cls.getName();
-    Integer i = s2i.get(name);
-    if (i != null) return;
-    i2s.add(name);
-    s2i.put(name, i2s.size() - 1);
-  }
-
-  /**
-   * Required override to return a new framework group factory
-   * @param <T> type of the counter enum class
-   * @param cls the counter enum class
-   * @return a new framework group factory
-   */
-  protected abstract <T extends Enum<T>>
-  FrameworkGroupFactory<G> newFrameworkGroupFactory(Class<T> cls);
-
-  /**
-   * Create a new counter group
-   * @param name of the group
-   * @param limits the counters limits policy object
-   * @return a new counter group
-   */
-  public G newGroup(String name, Limits limits) {
-    return newGroup(name, ResourceBundles.getCounterGroupName(name, name),
-                    limits);
-  }
-
-  /**
-   * Create a new counter group
-   * @param name of the group
-   * @param displayName of the group
-   * @param limits the counters limits policy object
-   * @return a new counter group
-   */
-  public G newGroup(String name, String displayName, Limits limits) {
-    FrameworkGroupFactory<G> gf = fmap.get(name);
-    if (gf != null) return gf.newGroup(name);
-    if (name.equals(FS_GROUP_NAME)) {
-      return newFileSystemGroup();
-    } else if (s2i.get(name) != null) {
-      return newFrameworkGroup(s2i.get(name));
-    }
-    return newGenericGroup(name, displayName, limits);
-  }
-
-  /**
-   * Create a new framework group
-   * @param id of the group
-   * @return a new framework group
-   */
-  public G newFrameworkGroup(int id) {
-    String name;
-    synchronized(CounterGroupFactory.class) {
-      if (id < 0 || id >= i2s.size()) throwBadFrameGroupIdException(id);
-      name = i2s.get(id); // should not throw here.
-    }
-    FrameworkGroupFactory<G> gf = fmap.get(name);
-    if (gf == null) throwBadFrameGroupIdException(id);
-    return gf.newGroup(name);
-  }
-
-  /**
-   * Get the id of a framework group
-   * @param name of the group
-   * @return the framework group id
-   */
-  public static synchronized int getFrameworkGroupId(String name) {
-    Integer i = s2i.get(name);
-    if (i == null) throwBadFrameworkGroupNameException(name);
-    return i;
-  }
-
-  /**
-   * @return the counter factory version
-   */
-  public int version() {
-    return VERSION;
-  }
-
-  /**
-   * Check whether a group name is a name of a framework group (including
-   * the filesystem group).
-   *
-   * @param name  to check
-   * @return true for framework group names
-   */
-  public static synchronized boolean isFrameworkGroup(String name) {
-    return s2i.get(name) != null || name.equals(FS_GROUP_NAME);
-  }
-
-  private static void throwBadFrameGroupIdException(int id) {
-    throw new IllegalArgumentException("bad framework group id: "+ id);
-  }
-
-  private static void throwBadFrameworkGroupNameException(String name) {
-    throw new IllegalArgumentException("bad framework group name: "+ name);
-  }
-
-  /**
-   * Abstract factory method to create a generic (vs framework) counter group
-   * @param name  of the group
-   * @param displayName of the group
-   * @param limits limits of the counters
-   * @return a new generic counter group
-   */
-  protected abstract G newGenericGroup(String name, String displayName,
-                                       Limits limits);
-
-  /**
-   * Abstract factory method to create a file system counter group
-   * @return a new file system counter group
-   */
-  protected abstract G newFileSystemGroup();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/DAGCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/DAGCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/DAGCounter.java
deleted file mode 100644
index 3598572..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/DAGCounter.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-// Per-job counters
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public enum DAGCounter {
-  NUM_FAILED_TASKS, 
-  NUM_KILLED_TASKS,
-  TOTAL_LAUNCHED_TASKS,
-  OTHER_LOCAL_TASKS,
-  DATA_LOCAL_TASKS,
-  RACK_LOCAL_TASKS,
-  SLOTS_MILLIS_TASKS,
-  FALLOW_SLOTS_MILLIS_TASKS,
-  TOTAL_LAUNCHED_UBERTASKS,
-  NUM_UBER_SUBTASKS,
-  NUM_FAILED_UBERTASKS
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
deleted file mode 100644
index 08f4c5d..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-@InterfaceAudience.Private
-public enum FileSystemCounter {
-  BYTES_READ,
-  BYTES_WRITTEN,
-  READ_OPS,
-  LARGE_READ_OPS,
-  WRITE_OPS,
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java b/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
deleted file mode 100644
index d4b167a..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
+++ /dev/null
@@ -1,327 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.Iterator;
-import java.util.Locale;
-import java.util.Map;
-
-import com.google.common.base.Joiner;
-import static com.google.common.base.Preconditions.*;
-import com.google.common.collect.AbstractIterator;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Maps;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * An abstract class to provide common implementation of the filesystem
- * counter group in both mapred and mapreduce packages.
- *
- * @param <C> the type of the Counter for the group
- */
-@InterfaceAudience.Private
-public abstract class FileSystemCounterGroup<C extends TezCounter>
-    implements CounterGroupBase<C> {
-
-  static final int MAX_NUM_SCHEMES = 100; // intern/sanity check
-  static final ConcurrentMap<String, String> schemes = Maps.newConcurrentMap();
-
-  // C[] would need Array.newInstance which requires a Class<C> reference.
-  // Just a few local casts probably worth not having to carry it around.
-  private final Map<String, Object[]> map =
-    new ConcurrentSkipListMap<String, Object[]>();
-  private String displayName;
-
-  private static final Joiner NAME_JOINER = Joiner.on('_');
-  private static final Joiner DISP_JOINER = Joiner.on(": ");
-
-  @InterfaceAudience.Private
-  public static class FSCounter extends AbstractCounter {
-    final String scheme;
-    final FileSystemCounter key;
-    private long value;
-
-    public FSCounter(String scheme, FileSystemCounter ref) {
-      this.scheme = scheme;
-      key = ref;
-    }
-
-    @Override
-    public String getName() {
-      return NAME_JOINER.join(scheme, key.name());
-    }
-
-    @Override
-    public String getDisplayName() {
-      return DISP_JOINER.join(scheme, localizeCounterName(key.name()));
-    }
-
-    protected String localizeCounterName(String counterName) {
-      return ResourceBundles.getCounterName(FileSystemCounter.class.getName(),
-                                            counterName, counterName);
-    }
-
-    @Override
-    public long getValue() {
-      return value;
-    }
-
-    @Override
-    public void setValue(long value) {
-      this.value = value;
-    }
-
-    @Override
-    public void increment(long incr) {
-      value += incr;
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-      assert false : "shouldn't be called";
-    }
-
-    @Override
-    public void readFields(DataInput in) throws IOException {
-      assert false : "shouldn't be called";
-    }
-
-    @Override
-    public TezCounter getUnderlyingCounter() {
-      return this;
-    }
-  }
-
-  @Override
-  public String getName() {
-    return FileSystemCounter.class.getName();
-  }
-
-  @Override
-  public String getDisplayName() {
-    if (displayName == null) {
-      displayName = ResourceBundles.getCounterGroupName(getName(),
-          "File System Counters");
-    }
-    return displayName;
-  }
-
-  @Override
-  public void setDisplayName(String displayName) {
-    this.displayName = displayName;
-  }
-
-  @Override
-  public void addCounter(C counter) {
-    C ours;
-    if (counter instanceof FileSystemCounterGroup.FSCounter) {
-      FSCounter c = (FSCounter) counter;
-      ours = findCounter(c.scheme, c.key);
-    }
-    else {
-      ours = findCounter(counter.getName());
-    }
-    ours.setValue(counter.getValue());
-  }
-
-  @Override
-  public C addCounter(String name, String displayName, long value) {
-    C counter = findCounter(name);
-    counter.setValue(value);
-    return counter;
-  }
-
-  // Parse generic counter name into [scheme, key]
-  private String[] parseCounterName(String counterName) {
-    int schemeEnd = counterName.indexOf('_');
-    if (schemeEnd < 0) {
-      throw new IllegalArgumentException("bad fs counter name");
-    }
-    return new String[]{counterName.substring(0, schemeEnd),
-                        counterName.substring(schemeEnd + 1)};
-  }
-
-  @Override
-  public C findCounter(String counterName, String displayName) {
-    return findCounter(counterName);
-  }
-
-  @Override
-  public C findCounter(String counterName, boolean create) {
-    try {
-      String[] pair = parseCounterName(counterName);
-      return findCounter(pair[0], FileSystemCounter.valueOf(pair[1]));
-    }
-    catch (Exception e) {
-      if (create) throw new IllegalArgumentException(e);
-      return null;
-    }
-  }
-
-  @Override
-  public C findCounter(String counterName) {
-    return findCounter(counterName, true);
-  }
-
-  @SuppressWarnings("unchecked")
-  public synchronized C findCounter(String scheme, FileSystemCounter key) {
-    final String canonicalScheme = checkScheme(scheme);
-    Object[] counters = map.get(canonicalScheme);
-    int ord = key.ordinal();
-    if (counters == null) {
-      counters = new Object[FileSystemCounter.values().length];
-      map.put(canonicalScheme, counters);
-      counters[ord] = newCounter(canonicalScheme, key);
-    }
-    else if (counters[ord] == null) {
-      counters[ord] = newCounter(canonicalScheme, key);
-    }
-    return (C) counters[ord];
-  }
-
-  private String checkScheme(String scheme) {
-    String fixed = scheme.toUpperCase(Locale.US);
-    String interned = schemes.putIfAbsent(fixed, fixed);
-    if (schemes.size() > MAX_NUM_SCHEMES) {
-      // mistakes or abuses
-      throw new IllegalArgumentException("too many schemes? "+ schemes.size() +
-                                         " when process scheme: "+ scheme);
-    }
-    return interned == null ? fixed : interned;
-  }
-
-  /**
-   * Abstract factory method to create a file system counter
-   * @param scheme of the file system
-   * @param key the enum of the file system counter
-   * @return a new file system counter
-   */
-  protected abstract C newCounter(String scheme, FileSystemCounter key);
-
-  @Override
-  public int size() {
-    int n = 0;
-    for (Object[] counters : map.values()) {
-      n += numSetCounters(counters);
-    }
-    return n;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public void incrAllCounters(CounterGroupBase<C> other) {
-    if (checkNotNull(other.getUnderlyingGroup(), "other group")
-        instanceof FileSystemCounterGroup<?>) {
-      for (TezCounter counter : other) {
-        FSCounter c = (FSCounter) ((TezCounter)counter).getUnderlyingCounter();
-        findCounter(c.scheme, c.key) .increment(counter.getValue());
-      }
-    }
-  }
-
-  /**
-   * FileSystemGroup ::= #scheme (scheme #counter (key value)*)*
-   */
-  @Override
-  public void write(DataOutput out) throws IOException {
-    WritableUtils.writeVInt(out, map.size()); // #scheme
-    for (Map.Entry<String, Object[]> entry : map.entrySet()) {
-      WritableUtils.writeString(out, entry.getKey()); // scheme
-      // #counter for the above scheme
-      WritableUtils.writeVInt(out, numSetCounters(entry.getValue()));
-      for (Object counter : entry.getValue()) {
-        if (counter == null) continue;
-        @SuppressWarnings("unchecked")
-        FSCounter c = (FSCounter) ((TezCounter)counter).getUnderlyingCounter();
-        WritableUtils.writeVInt(out, c.key.ordinal());  // key
-        WritableUtils.writeVLong(out, c.getValue());    // value
-      }
-    }
-  }
-
-  private int numSetCounters(Object[] counters) {
-    int n = 0;
-    for (Object counter : counters) if (counter != null) ++n;
-    return n;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int numSchemes = WritableUtils.readVInt(in);    // #scheme
-    FileSystemCounter[] enums = FileSystemCounter.values();
-    for (int i = 0; i < numSchemes; ++i) {
-      String scheme = WritableUtils.readString(in); // scheme
-      int numCounters = WritableUtils.readVInt(in); // #counter
-      for (int j = 0; j < numCounters; ++j) {
-        findCounter(scheme, enums[WritableUtils.readVInt(in)])  // key
-            .setValue(WritableUtils.readVLong(in)); // value
-      }
-    }
-  }
-
-  @Override
-  public Iterator<C> iterator() {
-    return new AbstractIterator<C>() {
-      Iterator<Object[]> it = map.values().iterator();
-      Object[] counters = it.hasNext() ? it.next() : null;
-      int i = 0;
-      @Override
-      protected C computeNext() {
-        while (counters != null) {
-          while (i < counters.length) {
-            @SuppressWarnings("unchecked")
-            C counter = (C) counters[i++];
-            if (counter != null) return counter;
-          }
-          i = 0;
-          counters = it.hasNext() ? it.next() : null;
-        }
-        return endOfData();
-      }
-    };
-  }
-
-  @Override
-  public synchronized boolean equals(Object genericRight) {
-    if (genericRight instanceof CounterGroupBase<?>) {
-      @SuppressWarnings("unchecked")
-      CounterGroupBase<C> right = (CounterGroupBase<C>) genericRight;
-      return Iterators.elementsEqual(iterator(), right.iterator());
-    }
-    return false;
-  }
-
-  @Override
-  public synchronized int hashCode() {
-    // need to be deep as counters is an array
-    int hash = FileSystemCounter.class.hashCode();
-    for (Object[] counters : map.values()) {
-      if (counters != null) hash ^= Arrays.hashCode(counters);
-    }
-    return hash;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java b/tez-common/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
deleted file mode 100644
index 42fb636..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
+++ /dev/null
@@ -1,275 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Iterator;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.WritableUtils;
-
-import com.google.common.collect.AbstractIterator;
-import com.google.common.collect.Iterators;
-
-/**
- * An abstract class to provide common implementation for the framework
- * counter group in both mapred and mapreduce packages.
- *
- * @param <T> type of the counter enum class
- * @param <C> type of the counter
- */
-@InterfaceAudience.Private
-public abstract class FrameworkCounterGroup<T extends Enum<T>,
-    C extends TezCounter> implements CounterGroupBase<C> {
-  private static final Log LOG = LogFactory.getLog(FrameworkCounterGroup.class);
-  
-  private final Class<T> enumClass; // for Enum.valueOf
-  private final Object[] counters;  // local casts are OK and save a class ref
-  private String displayName = null;
-
-  /**
-   * A counter facade for framework counters.
-   * Use old (which extends new) interface to make compatibility easier.
-   */
-  @InterfaceAudience.Private
-  public static class FrameworkCounter<T extends Enum<T>> extends AbstractCounter {
-    final T key;
-    final String groupName;
-    private long value;
-
-    public FrameworkCounter(T ref, String groupName) {
-      key = ref;
-      this.groupName = groupName;
-    }
-
-    @Override
-    public String getName() {
-      return key.name();
-    }
-
-    @Override
-    public String getDisplayName() {
-      return ResourceBundles.getCounterName(groupName, getName(), getName());
-    }
-
-    @Override
-    public long getValue() {
-      return value;
-    }
-
-    @Override
-    public void setValue(long value) {
-      this.value = value;
-    }
-
-    @Override
-    public void increment(long incr) {
-      value += incr;
-    }
-
-    @Override
-    public void write(DataOutput out) throws IOException {
-      assert false : "shouldn't be called";
-    }
-
-    @Override
-    public void readFields(DataInput in) throws IOException {
-      assert false : "shouldn't be called";
-    }
-
-    @Override
-    public TezCounter getUnderlyingCounter() {
-      return this;
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  public FrameworkCounterGroup(Class<T> enumClass) {
-    this.enumClass = enumClass;
-    T[] enums = enumClass.getEnumConstants();
-    counters = new Object[enums.length];
-  }
-
-  @Override
-  public String getName() {
-    return enumClass.getName();
-  }
-
-  @Override
-  public String getDisplayName() {
-    if (displayName == null) {
-      displayName = ResourceBundles.getCounterGroupName(getName(), getName());
-    }
-    return displayName;
-  }
-
-  @Override
-  public void setDisplayName(String displayName) {
-    this.displayName = displayName;
-  }
-
-  private T valueOf(String name) {
-    return Enum.valueOf(enumClass, name);
-  }
-
-  @Override
-  public void addCounter(C counter) {
-    C ours = findCounter(counter.getName());
-    ours.setValue(counter.getValue());
-  }
-
-  @Override
-  public C addCounter(String name, String displayName, long value) {
-    C counter = findCounter(name);
-    counter.setValue(value);
-    return counter;
-  }
-
-  @Override
-  public C findCounter(String counterName, String displayName) {
-    return findCounter(counterName);
-  }
-
-  @Override
-  public C findCounter(String counterName, boolean create) {
-    try {
-      return findCounter(valueOf(counterName));
-    }
-    catch (Exception e) {
-      if (create) throw new IllegalArgumentException(e);
-      return null;
-    }
-  }
-
-  @Override
-  public C findCounter(String counterName) {
-    return findCounter(valueOf(counterName));
-  }
-
-  @SuppressWarnings("unchecked")
-  private C findCounter(T key) {
-    int i = key.ordinal();
-    if (counters[i] == null) {
-      counters[i] = newCounter(key);
-    }
-    return (C) counters[i];
-  }
-
-  /**
-   * Abstract factory method for new framework counter
-   * @param key for the enum value of a counter
-   * @return a new counter for the key
-   */
-  protected abstract C newCounter(T key);
-
-  @Override
-  public int size() {
-    int n = 0;
-    for (int i = 0; i < counters.length; ++i) {
-      if (counters[i] != null) ++n;
-    }
-    return n;
-  }
-
-  @Override
-  @SuppressWarnings("unchecked")
-  public void incrAllCounters(CounterGroupBase<C> other) {
-    if (checkNotNull(other, "other counter group")
-        instanceof FrameworkCounterGroup<?, ?>) {
-      for (TezCounter counter : other) {
-        findCounter(((FrameworkCounter) counter).key.name())
-            .increment(counter.getValue());
-      }
-    }
-  }
-
-  /**
-   * FrameworkGroup ::= #counter (key value)*
-   */
-  @Override
-  @SuppressWarnings("unchecked")
-  public void write(DataOutput out) throws IOException {
-    WritableUtils.writeVInt(out, size());
-    for (int i = 0; i < counters.length; ++i) {
-      TezCounter counter = (C) counters[i];
-      if (counter != null) {
-        WritableUtils.writeVInt(out, i);
-        WritableUtils.writeVLong(out, counter.getValue());
-      }
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    clear();
-    int len = WritableUtils.readVInt(in);
-    T[] enums = enumClass.getEnumConstants();
-    for (int i = 0; i < len; ++i) {
-      int ord = WritableUtils.readVInt(in);
-      TezCounter counter = newCounter(enums[ord]);
-      counter.setValue(WritableUtils.readVLong(in));
-      counters[ord] = counter;
-    }
-  }
-
-  private void clear() {
-    for (int i = 0; i < counters.length; ++i) {
-      counters[i] = null;
-    }
-  }
-
-  @Override
-  public Iterator<C> iterator() {
-    return new AbstractIterator<C>() {
-      int i = 0;
-      @Override
-      protected C computeNext() {
-        while (i < counters.length) {
-          @SuppressWarnings("unchecked")
-          C counter = (C) counters[i++];
-          if (counter != null) return counter;
-        }
-        return endOfData();
-      }
-    };
-  }
-
-  @Override
-  public boolean equals(Object genericRight) {
-    if (genericRight instanceof CounterGroupBase<?>) {
-      @SuppressWarnings("unchecked")
-      CounterGroupBase<C> right = (CounterGroupBase<C>) genericRight;
-      return Iterators.elementsEqual(iterator(), right.iterator());
-    }
-    return false;
-  }
-
-  @Override
-  public synchronized int hashCode() {
-    // need to be deep as counters is an array
-    return Arrays.deepHashCode(new Object[]{enumClass, counters, displayName});
-  }
-}


[14/20] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/DAG.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/DAG.java
deleted file mode 100644
index 9cb602c..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/DAG.java
+++ /dev/null
@@ -1,377 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.dag.api;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Stack;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
-import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
-import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
-import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
-import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
-import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
-import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
-import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
-import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResource;
-import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
-import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
-import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
-import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
-
-
-public class DAG { // FIXME rename to Topology
-  final List<Vertex> vertices;
-  final List<Edge> edges;
-  final String name;
-
-  public DAG(String name) {
-    this.vertices = new ArrayList<Vertex>();
-    this.edges = new ArrayList<Edge>();
-    this.name = name;
-  }
-
-  public synchronized DAG addVertex(Vertex vertex) {
-    if (vertices.contains(vertex)) {
-      throw new IllegalArgumentException(
-          "Vertex " + vertex + " already defined!");
-    }
-    vertices.add(vertex);
-    return this;
-  }
-
-  @Private
-  public synchronized List<Vertex> getVertices() {
-    return Collections.unmodifiableList(this.vertices);
-  }
-
-  public synchronized DAG addEdge(Edge edge) {
-    // Sanity checks
-    if (!vertices.contains(edge.getInputVertex())) {
-      throw new IllegalArgumentException(
-          "Input vertex " + edge.getInputVertex() + " doesn't exist!");
-    }
-    if (!vertices.contains(edge.getOutputVertex())) {
-      throw new IllegalArgumentException(
-          "Output vertex " + edge.getOutputVertex() + " doesn't exist!");
-    }
-    if (edges.contains(edge)) {
-      throw new IllegalArgumentException(
-          "Edge " + edge + " already defined!");
-    }
-
-    // Inform the vertices
-    edge.getInputVertex().addOutputVertex(edge.getOutputVertex(), edge.getId());
-    edge.getOutputVertex().addInputVertex(edge.getInputVertex(), edge.getId());
-
-    edges.add(edge);
-    return this;
-  }
-
-  public String getName() {
-    return this.name;
-  }
-
-  // AnnotatedVertex is used by verify()
-  private static class AnnotatedVertex {
-    Vertex v;
-
-    int index; //for Tarjan's algorithm
-    int lowlink; //for Tarjan's algorithm
-    boolean onstack; //for Tarjan's algorithm
-
-    int outDegree;
-
-    private AnnotatedVertex(Vertex v){
-       this.v = v;
-       index = -1;
-       lowlink = -1;
-       outDegree = 0;
-    }
-  }
-
-  // verify()
-  //
-  // Default rules
-  //   Illegal:
-  //     - duplicate vertex id
-  //     - cycles
-  //
-  //   Ok:
-  //     - orphaned vertex.  Occurs in map-only
-  //     - islands.  Occurs if job has unrelated workflows.
-  //
-  //   Not yet categorized:
-  //     - orphaned vertex in DAG of >1 vertex.  Could be unrelated map-only job.
-  //     - v1->v2 via two edges.  perhaps some self-join job would use this?
-  //
-  // "restricted" mode:
-  //   In short term, the supported DAGs are limited. Call with restricted=true for these verifications.
-  //   Illegal:
-  //     - any vertex with more than one input or output edge. (n-ary input, n-ary merge)
-  public void verify() throws IllegalStateException {
-    verify(true);
-  }
-
-  public void verify(boolean restricted) throws IllegalStateException  {
-    if (vertices.isEmpty()) {
-      throw new IllegalStateException("Invalid dag containing 0 vertices");
-    }
-
-    Map<Vertex, List<Edge>> edgeMap = new HashMap<Vertex, List<Edge>>();
-    for(Edge e : edges){
-      Vertex inputVertex = e.getInputVertex();
-      List<Edge> edgeList = edgeMap.get(inputVertex);
-      if(edgeList == null){
-        edgeList = new ArrayList<Edge>();
-        edgeMap.put(inputVertex, edgeList);
-      }
-      edgeList.add(e);
-    }
-
-    // check for valid vertices, duplicate vertex names,
-    // and prepare for cycle detection
-    Map<String, AnnotatedVertex> vertexMap = new HashMap<String, AnnotatedVertex>();
-    for(Vertex v : vertices){
-      if(vertexMap.containsKey(v.getVertexName())){
-         throw new IllegalStateException("DAG contains multiple vertices"
-             + " with name: " + v.getVertexName());
-      }
-      vertexMap.put(v.getVertexName(), new AnnotatedVertex(v));
-    }
-
-    detectCycles(edgeMap, vertexMap);
-
-    if(restricted){
-      for(Edge e : edges){
-        vertexMap.get(e.getInputVertex().getVertexName()).outDegree++;
-        if (e.getEdgeProperty().getDataMovementType() !=
-            DataMovementType.SCATTER_GATHER) {
-          throw new IllegalStateException(
-              "Unsupported connection pattern on edge. " + e);
-        }
-        if (e.getEdgeProperty().getDataSourceType() !=
-            DataSourceType.PERSISTED) {
-          throw new IllegalStateException(
-              "Unsupported source type on edge. " + e);
-        }
-        if (e.getEdgeProperty().getSchedulingType() !=
-            SchedulingType.SEQUENTIAL) {
-          throw new IllegalStateException(
-              "Unsupported scheduling type on edge. " + e);
-        }
-      }
-      for(AnnotatedVertex av: vertexMap.values()){
-        if (av.outDegree > 1) {
-          throw new IllegalStateException("Vertex has outDegree>1: "
-              + av.v.getVertexName());
-        }
-      }
-    }
-  }
-
-  // Adaptation of Tarjan's algorithm for connected components.
-  // http://en.wikipedia.org/wiki/Tarjan%27s_strongly_connected_components_algorithm
-  private void detectCycles(Map<Vertex, List<Edge>> edgeMap, Map<String, AnnotatedVertex> vertexMap)
-      throws IllegalStateException{
-    Integer nextIndex = 0; // boxed integer so it is passed by reference.
-    Stack<AnnotatedVertex> stack = new Stack<DAG.AnnotatedVertex>();
-    for(AnnotatedVertex av: vertexMap.values()){
-      if(av.index == -1){
-        assert stack.empty();
-        strongConnect(av, vertexMap, edgeMap, stack, nextIndex);
-      }
-    }
-  }
-
-  // part of Tarjan's algorithm for connected components.
-  // http://en.wikipedia.org/wiki/Tarjan%27s_strongly_connected_components_algorithm
-  private void strongConnect(
-          AnnotatedVertex av,
-          Map<String, AnnotatedVertex> vertexMap,
-          Map<Vertex, List<Edge>> edgeMap,
-          Stack<AnnotatedVertex> stack, Integer nextIndex) throws IllegalStateException{
-    av.index = nextIndex;
-    av.lowlink = nextIndex;
-    nextIndex++;
-    stack.push(av);
-    av.onstack = true;
-
-    List<Edge> edges = edgeMap.get(av.v);
-    if(edges != null){
-      for(Edge e : edgeMap.get(av.v)){
-        AnnotatedVertex outVertex = vertexMap.get(e.getOutputVertex().getVertexName());
-        if(outVertex.index == -1){
-          strongConnect(outVertex, vertexMap, edgeMap, stack, nextIndex);
-          av.lowlink = Math.min(av.lowlink, outVertex.lowlink);
-        }
-        else if(outVertex.onstack){
-          // strongly connected component detected, but we will wait till later so that the full cycle can be displayed.
-          // update lowlink in case outputVertex should be considered the root of this component.
-          av.lowlink = Math.min(av.lowlink, outVertex.index);
-        }
-      }
-    }
-
-    if(av.lowlink == av.index ){
-       AnnotatedVertex pop = stack.pop();
-       pop.onstack = false;
-       if(pop != av){
-         // there was something on the stack other than this "av".
-         // this indicates there is a scc/cycle. It comprises all nodes from top of stack to "av"
-         StringBuilder message = new StringBuilder();
-         message.append(av.v.getVertexName() + " <- ");
-         for( ; pop != av; pop = stack.pop()){
-           message.append(pop.v.getVertexName() + " <- ");
-           pop.onstack = false;
-         }
-         message.append(av.v.getVertexName());
-         throw new IllegalStateException("DAG contains a cycle: " + message);
-       }
-    }
-  }
-
-
-  // create protobuf message describing DAG
-  @Private
-  public DAGPlan createDag(Configuration dagConf) {
-    verify(true);
-
-    DAGPlan.Builder dagBuilder = DAGPlan.newBuilder();
-
-    dagBuilder.setName(this.name);
-
-    for(Vertex vertex : vertices){
-      VertexPlan.Builder vertexBuilder = VertexPlan.newBuilder();
-      vertexBuilder.setName(vertex.getVertexName());
-      vertexBuilder.setType(PlanVertexType.NORMAL); // vertex type is implicitly NORMAL until  TEZ-46.
-      vertexBuilder.setProcessorDescriptor(DagTypeConverters
-          .convertToDAGPlan(vertex.getProcessorDescriptor()));
-
-      //task config
-      PlanTaskConfiguration.Builder taskConfigBuilder = PlanTaskConfiguration.newBuilder();
-      Resource resource = vertex.getTaskResource();
-      taskConfigBuilder.setNumTasks(vertex.getParallelism());
-      taskConfigBuilder.setMemoryMb(resource.getMemory());
-      taskConfigBuilder.setVirtualCores(resource.getVirtualCores());
-      taskConfigBuilder.setJavaOpts(vertex.getJavaOpts());
-
-      taskConfigBuilder.setTaskModule(vertex.getVertexName());
-      PlanLocalResource.Builder localResourcesBuilder = PlanLocalResource.newBuilder();
-      Map<String,LocalResource> lrs = vertex.getTaskLocalResources();
-      for(Entry<String, LocalResource> entry : lrs.entrySet()){
-        String key = entry.getKey();
-        LocalResource lr = entry.getValue();
-        localResourcesBuilder.setName(key);
-        localResourcesBuilder.setUri(
-            DagTypeConverters.convertToDAGPlan(lr.getResource()));
-        localResourcesBuilder.setSize(lr.getSize());
-        localResourcesBuilder.setTimeStamp(lr.getTimestamp());
-        localResourcesBuilder.setType(
-            DagTypeConverters.convertToDAGPlan(lr.getType()));
-        localResourcesBuilder.setVisibility(
-            DagTypeConverters.convertToDAGPlan(lr.getVisibility()));
-        if(lr.getType() == LocalResourceType.PATTERN){
-          if (lr.getPattern() == null || lr.getPattern().isEmpty()) {
-            throw new TezUncheckedException("LocalResource type set to pattern"
-                + " but pattern is null or empty");
-          }
-          localResourcesBuilder.setPattern(lr.getPattern());
-        }
-        taskConfigBuilder.addLocalResource(localResourcesBuilder);
-      }
-
-      if(vertex.getTaskEnvironment() != null){
-        for(String key : vertex.getTaskEnvironment().keySet()){
-          PlanKeyValuePair.Builder envSettingBuilder = PlanKeyValuePair.newBuilder();
-          envSettingBuilder.setKey(key);
-          envSettingBuilder.setValue(vertex.getTaskEnvironment().get(key));
-          taskConfigBuilder.addEnvironmentSetting(envSettingBuilder);
-        }
-      }
-
-      if(vertex.getTaskLocationsHint() != null ){
-        if(vertex.getTaskLocationsHint().getTaskLocationHints() != null){
-          for(TaskLocationHint hint : vertex.getTaskLocationsHint().getTaskLocationHints()){
-            PlanTaskLocationHint.Builder taskLocationHintBuilder = PlanTaskLocationHint.newBuilder();
-
-            if(hint.getDataLocalHosts() != null){
-              taskLocationHintBuilder.addAllHost(hint.getDataLocalHosts());
-            }
-            if(hint.getRacks() != null){
-              taskLocationHintBuilder.addAllRack(hint.getRacks());
-            }
-
-            vertexBuilder.addTaskLocationHint(taskLocationHintBuilder);
-          }
-        }
-      }
-
-      for(String inEdgeId : vertex.getInputEdgeIds()){
-        vertexBuilder.addInEdgeId(inEdgeId);
-      }
-
-      for(String outEdgeId : vertex.getOutputEdgeIds()){
-        vertexBuilder.addOutEdgeId(outEdgeId);
-      }
-
-      vertexBuilder.setTaskConfig(taskConfigBuilder);
-      dagBuilder.addVertex(vertexBuilder);
-    }
-
-    for(Edge edge : edges){
-      EdgePlan.Builder edgeBuilder = EdgePlan.newBuilder();
-      edgeBuilder.setId(edge.getId());
-      edgeBuilder.setInputVertexName(edge.getInputVertex().getVertexName());
-      edgeBuilder.setOutputVertexName(edge.getOutputVertex().getVertexName());
-      edgeBuilder.setDataMovementType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getDataMovementType()));
-      edgeBuilder.setDataSourceType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getDataSourceType()));
-      edgeBuilder.setSchedulingType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getSchedulingType()));
-      edgeBuilder.setEdgeSource(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getEdgeSource()));
-      edgeBuilder.setEdgeDestination(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getEdgeDestination()));
-      dagBuilder.addEdge(edgeBuilder);
-    }
-
-    if(dagConf != null) {
-      Iterator<Entry<String, String>> iter = dagConf.iterator();
-      ConfigurationProto.Builder confProtoBuilder =
-          ConfigurationProto.newBuilder();
-      while (iter.hasNext()) {
-        Entry<String, String> entry = iter.next();
-        PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder();
-        kvp.setKey(entry.getKey());
-        kvp.setValue(entry.getValue());
-        confProtoBuilder.addConfKeyValues(kvp);
-      }
-      dagBuilder.setDagKeyValues(confProtoBuilder);
-    }
-
-    return dagBuilder.build();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
deleted file mode 100644
index 1fd78f1..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
+++ /dev/null
@@ -1,278 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.dag.api;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.URL;
-import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
-import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
-import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
-import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
-import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
-import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
-import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataMovementType;
-import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataSourceType;
-import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeSchedulingType;
-import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
-import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResource;
-import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResourceType;
-import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResourceVisibility;
-import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
-import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-
-import com.google.protobuf.ByteString;
-
-
-public class DagTypeConverters {
-  
-  public static PlanLocalResourceVisibility convertToDAGPlan(LocalResourceVisibility visibility){
-    switch(visibility){
-      case PUBLIC : return PlanLocalResourceVisibility.PUBLIC;  
-      case PRIVATE : return PlanLocalResourceVisibility.PRIVATE;
-      case APPLICATION : return PlanLocalResourceVisibility.APPLICATION;
-      default : throw new RuntimeException("unknown 'visibility': " + visibility);
-    }
-  }
-  
-  public static LocalResourceVisibility convertFromDAGPlan(PlanLocalResourceVisibility visibility){
-    switch(visibility){
-      case PUBLIC : return LocalResourceVisibility.PUBLIC;  
-      case PRIVATE : return LocalResourceVisibility.PRIVATE;
-      case APPLICATION : return LocalResourceVisibility.APPLICATION;
-      default : throw new RuntimeException("unknown 'visibility': " + visibility);
-    }
-  }
-  
-  public static PlanEdgeDataSourceType convertToDAGPlan(DataSourceType sourceType){
-    switch(sourceType){
-      case PERSISTED : return PlanEdgeDataSourceType.PERSISTED;  
-      case PERSISTED_RELIABLE : return PlanEdgeDataSourceType.PERSISTED_RELIABLE;
-      case EPHEMERAL :  return PlanEdgeDataSourceType.EPHEMERAL;
-      default : throw new RuntimeException("unknown 'dataSourceType': " + sourceType);
-    }
-  }
-  
-  public static DataSourceType convertFromDAGPlan(PlanEdgeDataSourceType sourceType){
-    switch(sourceType){
-      case PERSISTED : return DataSourceType.PERSISTED;  
-      case PERSISTED_RELIABLE : return DataSourceType.PERSISTED_RELIABLE;
-      case EPHEMERAL :  return DataSourceType.EPHEMERAL;
-      default : throw new RuntimeException("unknown 'dataSourceType': " + sourceType);
-    }
-  }
-  
-  public static PlanEdgeDataMovementType convertToDAGPlan(DataMovementType type){
-    switch(type){
-      case ONE_TO_ONE : return PlanEdgeDataMovementType.ONE_TO_ONE;  
-      case BROADCAST : return PlanEdgeDataMovementType.BROADCAST;
-      case SCATTER_GATHER : return PlanEdgeDataMovementType.SCATTER_GATHER;
-      default : throw new RuntimeException("unknown 'dataMovementType': " + type);
-    }
-  }
-  
-  public static DataMovementType convertFromDAGPlan(PlanEdgeDataMovementType type){
-    switch(type){
-      case ONE_TO_ONE : return DataMovementType.ONE_TO_ONE;  
-      case BROADCAST : return DataMovementType.BROADCAST;
-      case SCATTER_GATHER : return DataMovementType.SCATTER_GATHER;
-      default : throw new IllegalArgumentException("unknown 'dataMovementType': " + type);
-    }
-  }
-  
-  public static PlanEdgeSchedulingType convertToDAGPlan(SchedulingType type){
-    switch(type){
-      case SEQUENTIAL : return PlanEdgeSchedulingType.SEQUENTIAL;  
-      case CONCURRENT : return PlanEdgeSchedulingType.CONCURRENT;
-      default : throw new RuntimeException("unknown 'SchedulingType': " + type);
-    }
-  }
-  
-  public static SchedulingType convertFromDAGPlan(PlanEdgeSchedulingType type){
-    switch(type){
-      case SEQUENTIAL : return SchedulingType.SEQUENTIAL;  
-      case CONCURRENT : return SchedulingType.CONCURRENT;
-      default : throw new IllegalArgumentException("unknown 'SchedulingType': " + type);
-    }
-  }
-  
-  public static PlanLocalResourceType convertToDAGPlan(LocalResourceType type) {
-    switch(type){
-    case ARCHIVE : return PlanLocalResourceType.ARCHIVE;
-    case FILE : return PlanLocalResourceType.FILE;
-    case PATTERN : return PlanLocalResourceType.PATTERN;
-    default : throw new IllegalArgumentException("unknown 'type': " + type);
-    }
-  }
-  
-  public static LocalResourceType convertFromDAGPlan(PlanLocalResourceType type) {
-    switch(type){
-    case ARCHIVE : return LocalResourceType.ARCHIVE;
-    case FILE : return LocalResourceType.FILE;
-    case PATTERN : return LocalResourceType.PATTERN;
-    default : throw new IllegalArgumentException("unknown 'type': " + type);
-    }
-  }
-
-  public static VertexLocationHint convertFromDAGPlan(
-      List<PlanTaskLocationHint> locationHints) {
-
-    List<TaskLocationHint> outputList = new ArrayList<TaskLocationHint>();  
-    
-    for(PlanTaskLocationHint inputHint : locationHints){
-      TaskLocationHint outputHint = new TaskLocationHint(
-          new HashSet<String>(inputHint.getHostList()),
-          new HashSet<String>(inputHint.getRackList()));
-      outputList.add(outputHint);
-    }
-    return new VertexLocationHint(outputList.size(), outputList);
-  }
-
-  // notes re HDFS URL handling:
-  //   Resource URLs in the protobuf message are strings of the form hdfs://host:port/path 
-  //   org.apache.hadoop.fs.Path.Path  is actually a URI type that allows any scheme
-  //   org.apache.hadoop.yarn.api.records.URL is a URL type used by YARN.
-  //   java.net.URL cannot be used out of the box as it rejects unknown schemes such as HDFS.
-  
-  public static String convertToDAGPlan(URL resource) {
-    // see above notes on HDFS URL handling
-    String out = resource.getScheme() + "://" + resource.getHost() + ":" + resource.getPort() 
-        + resource.getFile();
-    return out;
-  }
-
-  public static Map<String, LocalResource> createLocalResourceMapFromDAGPlan(
-      List<PlanLocalResource> localResourcesList) {
-    Map<String, LocalResource> map = new HashMap<String, LocalResource>();
-    for(PlanLocalResource res : localResourcesList){
-      LocalResource r = new LocalResourcePBImpl();
-      
-      //NOTE: have to check every optional field in protobuf generated classes for existence before accessing
-      //else we will receive a default value back, eg ""
-      if(res.hasPattern()){
-        r.setPattern(res.getPattern());
-      }
-      r.setResource(ConverterUtils.getYarnUrlFromPath(new Path(res.getUri())));  // see above notes on HDFS URL handling
-      r.setSize(res.getSize());
-      r.setTimestamp(res.getTimeStamp());
-      r.setType(DagTypeConverters.convertFromDAGPlan(res.getType()));
-      r.setVisibility(DagTypeConverters.convertFromDAGPlan(res.getVisibility()));
-      map.put(res.getName(), r);
-    }
-    return map;
-  }
-
-  public static Map<String, String> createEnvironmentMapFromDAGPlan(
-      List<PlanKeyValuePair> environmentSettingList) {  
-      
-    Map<String, String> map = new HashMap<String, String>();
-    for(PlanKeyValuePair setting : environmentSettingList){
-      map.put(setting.getKey(), setting.getValue());
-    }
-    
-    return map;
-  }
-  
-  public static Map<String, EdgePlan> createEdgePlanMapFromDAGPlan(List<EdgePlan> edgeList){
-    Map<String, EdgePlan> edgePlanMap =
-        new HashMap<String, EdgePlan>();
-    for(EdgePlan edgePlanItem : edgeList){
-      edgePlanMap.put(edgePlanItem.getId(), edgePlanItem);
-    }
-    return edgePlanMap;
-  }
-  
-  public static EdgeProperty createEdgePropertyMapFromDAGPlan(EdgePlan edge) {
-    return new EdgeProperty(
-        convertFromDAGPlan(edge.getDataMovementType()),
-        convertFromDAGPlan(edge.getDataSourceType()),
-        convertFromDAGPlan(edge.getSchedulingType()),
-        convertOutputDescriptorFromDAGPlan(edge.getEdgeSource()),
-        convertInputDescriptorFromDAGPlan(edge.getEdgeDestination())
-    );
-  }
-
-  public static Resource createResourceRequestFromTaskConfig(
-      PlanTaskConfiguration taskConfig) {
-    return Resource.newInstance(taskConfig.getMemoryMb(), taskConfig.getVirtualCores());
-  }
-
-  public static Map<String, String> convertConfFromProto(
-      ConfigurationProto confProto) {
-    List<PlanKeyValuePair> settingList = confProto.getConfKeyValuesList();
-    Map<String, String> map = new HashMap<String, String>();
-    for(PlanKeyValuePair setting: settingList){
-      map.put(setting.getKey(), setting.getValue());
-    }
-    return map;
-  }
-
-  public static TezEntityDescriptorProto convertToDAGPlan(
-      TezEntityDescriptor descriptor) {
-    TezEntityDescriptorProto.Builder builder = TezEntityDescriptorProto
-        .newBuilder();
-    builder.setClassName(descriptor.getClassName());
-    if (descriptor.getUserPayload() != null) {
-      builder
-          .setUserPayload(ByteString.copyFrom(descriptor.getUserPayload()));
-    }
-    return builder.build();
-  }
-
-  public static InputDescriptor convertInputDescriptorFromDAGPlan(
-      TezEntityDescriptorProto proto) {
-    String className = proto.getClassName();
-    byte[] bb = null;
-    if (proto.hasUserPayload()) {
-      bb = proto.getUserPayload().toByteArray();
-    }
-    return new InputDescriptor(className).setUserPayload(bb);
-  }
-
-  public static OutputDescriptor convertOutputDescriptorFromDAGPlan(
-      TezEntityDescriptorProto proto) {
-    String className = proto.getClassName();
-    byte[] bb = null;
-    if (proto.hasUserPayload()) {
-      bb =  proto.getUserPayload().toByteArray();
-    }
-    return new OutputDescriptor(className).setUserPayload(bb);
-  }
-
-  public static ProcessorDescriptor convertProcessorDescriptorFromDAGPlan(
-      TezEntityDescriptorProto proto) {
-    String className = proto.getClassName();
-    byte[] bb = null;
-    if (proto.hasUserPayload()) {
-      bb = proto.getUserPayload().toByteArray();
-    }
-    return new ProcessorDescriptor(className).setUserPayload(bb);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/Edge.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/Edge.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/Edge.java
deleted file mode 100644
index a893bc3..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/Edge.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.dag.api;
-
-public class Edge{
-  
-  private final Vertex inputVertex;
-  private final Vertex outputVertex;
-  private final EdgeProperty edgeProperty;
-    
-  public Edge(Vertex inputVertex, 
-               Vertex outputVertex, 
-               EdgeProperty edgeProperty) {
-    this.inputVertex = inputVertex;
-    this.outputVertex = outputVertex;
-    this.edgeProperty = edgeProperty;
-  }
-  
-  // RENAME to source and destination
-  public Vertex getInputVertex() {
-    return inputVertex;
-  }
-  
-  public Vertex getOutputVertex() {
-    return outputVertex;
-  }
-  
-  public EdgeProperty getEdgeProperty() {
-    return edgeProperty;
-  }
-  
-  /*
-   * Used to identify the edge in the configuration
-   */
-  public String getId() {
-    return String.valueOf(this.hashCode());
-  }
- 
-  @Override
-  public String toString() {
-    return inputVertex + " -> " + outputVertex + " (" + edgeProperty + ")";
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
deleted file mode 100644
index 326d3d0..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.dag.api;
-
-public class EdgeProperty {
-  
-  /**
-   * Defines the manner of data movement between source and destination tasks.
-   * Determines which destination tasks have access to data produced on this
-   * edge by a source task. A destination task may choose to read any portion of
-   * the data available to it.
-   */
-  public enum DataMovementType {
-    /**
-     * Output on this edge produced by the i-th source task is available to the 
-     * i-th destination task.
-     */
-    ONE_TO_ONE,
-    /**
-     * Output on this edge produced by any source task is available to all
-     * destination tasks.
-     */
-    BROADCAST,
-    /**
-     * The i-th output on this edge produced by all source tasks is available to
-     * the same destination task. Source tasks scatter their outputs and they
-     * are gathered by designated destination tasks.
-     */
-    SCATTER_GATHER
-  }
-  
-  /**
-   * Determines the lifetime of the data produced on this edge by a source task.
-   */
-  public enum DataSourceType {
-    /**
-     * Data produced by the source is persisted and available even when the
-     * task is not running. The data may become unavailable and may cause the 
-     * source task to be re-executed.
-     */
-    PERSISTED,
-    /**
-     * Source data is stored reliably and will always be available
-     */
-    PERSISTED_RELIABLE,
-    /**
-     * Data produced by the source task is available only while the source task
-     * is running. This requires the destination task to run concurrently with 
-     * the source task.
-     */
-    EPHEMERAL
-  }
-  
-  /**
-   * Determines when the destination task is eligible to run, once the source  
-   * task is eligible to run.
-   */
-  public enum SchedulingType {
-    /**
-     * Destination task is eligible to run after one or more of its source tasks 
-     * have started or completed.
-     */
-    SEQUENTIAL,
-    /**
-     * Destination task must run concurrently with the source task
-     */
-    CONCURRENT
-  }
-  
-  DataMovementType dataMovementType;
-  DataSourceType dataSourceType;
-  SchedulingType schedulingType;
-  InputDescriptor inputDescriptor;
-  OutputDescriptor outputDescriptor;
-  
-  /**
-   * @param dataMovementType
-   * @param dataSourceType
-   * @param edgeSource
-   *          The {@link OutputDescriptor} that generates data on the edge.
-   * @param edgeDestination
-   *          The {@link InputDescriptor} which will consume data from the edge.
-   */
-  public EdgeProperty(DataMovementType dataMovementType, 
-                       DataSourceType dataSourceType,
-                       SchedulingType schedulingType,
-                       OutputDescriptor edgeSource,
-                       InputDescriptor edgeDestination) {
-    this.dataMovementType = dataMovementType;
-    this.dataSourceType = dataSourceType;
-    this.schedulingType = schedulingType;
-    this.inputDescriptor = edgeDestination;
-    this.outputDescriptor = edgeSource;
-  }
-  
-  public DataMovementType getDataMovementType() {
-    return dataMovementType;
-  }
-  
-  public DataSourceType getDataSourceType() {
-    return dataSourceType;
-  }
-  
-  public SchedulingType getSchedulingType() {
-    return schedulingType;
-  }
-  
-  /**
-   * Returns the {@link InputDescriptor} which will consume data from the edge.
-   * 
-   * @return
-   */
-  public InputDescriptor getEdgeDestination() {
-    return inputDescriptor;
-  }
-  
-  /**
-   * Returns the {@link OutputDescriptor} which produces data on the edge.
-   * 
-   * @return
-   */
-  public OutputDescriptor getEdgeSource() {
-    return outputDescriptor;
-  }
-  
-  @Override
-  public String toString() {
-    return "{ " + dataMovementType + " : " + inputDescriptor.getClassName()
-        + " >> " + dataSourceType + " >> " + outputDescriptor.getClassName() + " }";
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
deleted file mode 100644
index dea9001..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-public class InputDescriptor extends TezEntityDescriptor {
-
-  public InputDescriptor(String inputClassName) {
-    super(inputClassName);
-  }
-
-  @Override
-  public InputDescriptor setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
deleted file mode 100644
index 16fb9b1..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-public class OutputDescriptor extends TezEntityDescriptor {
-
-  public OutputDescriptor(String outputClassName) {
-    super(outputClassName);
-  }
-
-  @Override
-  public OutputDescriptor setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
deleted file mode 100644
index 092147d..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-public class ProcessorDescriptor extends TezEntityDescriptor {
-
-  public ProcessorDescriptor(String processorClassName) {
-    super(processorClassName);
-  }
-
-  public ProcessorDescriptor setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
deleted file mode 100644
index 7447974..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ /dev/null
@@ -1,223 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-import org.apache.hadoop.conf.Configuration;
-
-public class TezConfiguration extends Configuration {
-
-  public final static String TEZ_SITE_XML = "tez-site.xml";
-
-  static {
-    addDefaultResource(TEZ_SITE_XML);
-  }
-
-  public TezConfiguration() {
-    super();
-  }
-
-  public TezConfiguration(Configuration conf) {
-    super(conf);
-  }
-
-  public static final String TEZ_PREFIX = "tez.";
-  public static final String TEZ_AM_PREFIX = TEZ_PREFIX + "am.";
-  public static final String TEZ_TASK_PREFIX = TEZ_PREFIX + "task.";
-
-  public static final String TEZ_AM_STAGING_DIR = TEZ_PREFIX + "staging-dir";
-  public static final String TEZ_AM_STAGING_DIR_DEFAULT = "/tmp/tez/staging";
-
-  // TODO Should not be required once all tokens are handled via AppSubmissionContext
-  public static final String JOB_SUBMIT_DIR = TEZ_PREFIX + "jobSubmitDir";
-  public static final String APPLICATION_TOKENS_FILE = "appTokens";
-  public static final String TEZ_APPLICATION_MASTER_CLASS =
-      "org.apache.tez.dag.app.DAGAppMaster";
-
-  /** Root Logging level passed to the Tez app master.*/
-  public static final String TEZ_AM_LOG_LEVEL = TEZ_AM_PREFIX+"log.level";
-  public static final String TEZ_AM_LOG_LEVEL_DEFAULT = "INFO";
-
-  public static final String TEZ_AM_JAVA_OPTS = TEZ_AM_PREFIX
-      + "java.opts";
-  public static final String DEFAULT_TEZ_AM_JAVA_OPTS = " -Xmx1024m ";
-
-  public static final String TEZ_AM_CANCEL_DELEGATION_TOKEN = TEZ_AM_PREFIX +
-      "am.complete.cancel.delegation.tokens";
-  public static final boolean TEZ_AM_CANCEL_DELEGATION_TOKEN_DEFAULT = true;
-
-  public static final String TEZ_AM_TASK_LISTENER_THREAD_COUNT =
-      TEZ_AM_PREFIX + "task.listener.thread-count";
-  public static final int TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT = 30;
-
-  public static final String TEZ_AM_CONTAINER_LISTENER_THREAD_COUNT =
-      TEZ_AM_PREFIX + "container.listener.thread-count";
-  public static final int TEZ_AM_CONTAINER_LISTENER_THREAD_COUNT_DEFAULT = 30;
-
-  // TODO Some of the DAG properties are job specific and not AM specific. Rename accordingly.
-  // TODO Are any of these node blacklisting properties required. (other than for MR compat)
-  public static final String TEZ_AM_MAX_TASK_FAILURES_PER_NODE = TEZ_AM_PREFIX
-      + "maxtaskfailures.per.node";
-  public static final int TEZ_AM_MAX_TASK_FAILURES_PER_NODE_DEFAULT = 3;
-
-  public static final String TEZ_AM_MAX_TASK_ATTEMPTS =
-      TEZ_AM_PREFIX + "max.task.attempts";
-  public static final int TEZ_AM_MAX_TASK_ATTEMPTS_DEFAULT = 4;
-
-  public static final String TEZ_AM_NODE_BLACKLISTING_ENABLED = TEZ_AM_PREFIX
-      + "node-blacklisting.enabled";
-  public static final boolean TEZ_AM_NODE_BLACKLISTING_ENABLED_DEFAULT = true;
-  public static final String TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD = TEZ_AM_PREFIX
-      + "node-blacklisting.ignore-threshold-node-percent";
-  public static final int TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD_DEFAULT = 33;
-
-  /** Number of threads to handle job client RPC requests.*/
-  public static final String TEZ_AM_CLIENT_THREAD_COUNT =
-      TEZ_AM_PREFIX + "client.am.thread-count";
-  public static final int TEZ_AM_CLIENT_THREAD_COUNT_DEFAULT = 1;
-  /**
-   * Range of ports that the AM can use when binding. Leave blank
-   * if you want all possible ports.
-   */
-  public static final String TEZ_AM_CLIENT_AM_PORT_RANGE =
-      TEZ_AM_PREFIX + "client.am.port-range";
-
-
-  public static final String TEZ_AM_RESOURCE_MEMORY_MB = TEZ_AM_PREFIX
-      + "resource.memory.mb";
-  public static final int TEZ_AM_RESOURCE_MEMORY_MB_DEFAULT = 1536;
-
-  public static final String TEZ_AM_RESOURCE_CPU_VCORES = TEZ_AM_PREFIX
-      + "resource.cpu.vcores";
-  public static final int TEZ_AM_RESOURCE_CPU_VCORES_DEFAULT = 1;
-
-  public static final String
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_SRC_FRACTION = TEZ_AM_PREFIX
-          + "shuffle-vertex-manager.min-src-fraction";
-  public static final float
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_SRC_FRACTION_DEFAULT = 0.25f;
-
-  public static final String
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MAX_SRC_FRACTION = TEZ_AM_PREFIX
-          + "shuffle-vertex-manager.max-src-fraction";
-  public static final float
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MAX_SRC_FRACTION_DEFAULT = 0.75f;
-
-  public static final String
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_ENABLE_AUTO_PARALLEL = TEZ_AM_PREFIX +
-          "shuffle-vertex-manager.enable.auto-parallel";
-  public static final boolean
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_ENABLE_AUTO_PARALLEL_DEFAULT = false;
-
-  public static final String
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_DESIRED_TASK_INPUT_SIZE = TEZ_AM_PREFIX +
-          "shuffle-vertex-manager.desired-task-input-size";
-  public static final long
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_DESIRED_TASK_INPUT_SIZE_DEFAULT =
-          1024*1024*100L;
-
-  public static final String
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_TASK_PARALLELISM = TEZ_AM_PREFIX +
-          "shuffle-vertex-manager.min-task-parallelism";
-  public static final int
-          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_TASK_PARALLELISM_DEFAULT = 1;
-
-  public static final String
-          TEZ_AM_SLOWSTART_DAG_SCHEDULER_MIN_SHUFFLE_RESOURCE_FRACTION = TEZ_AM_PREFIX
-          + "slowstart-dag-scheduler.min-resource-fraction";
-  public static final float
-          TEZ_AM_SLOWSTART_DAG_SCHEDULER_MIN_SHUFFLE_RESOURCE_FRACTION_DEFAULT = 0.5f;
-
-  public static final String TEZ_AM_AGGRESSIVE_SCHEDULING = TEZ_AM_PREFIX +
-      "aggressive.scheduling";
-  public static boolean TEZ_AM_AGGRESSIVE_SCHEDULING_DEFAULT = false;
-
-  /**
-   * The complete path to the serialized dag plan file
-   * <code>TEZ_AM_PLAN_PB_BINARY</code>. Used to make the plan available to
-   * individual tasks if needed. This will typically be a path in the job submit
-   * directory.
-   */
-  public static final String TEZ_AM_PLAN_REMOTE_PATH = TEZ_AM_PREFIX
-      + "dag-am-plan.remote.path";
-
-  public static final String TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX = TEZ_AM_PREFIX
-      + "am-rm.heartbeat.interval-ms.max";
-  public static final int TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX_DEFAULT = 1000;
-
-  public static final String TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX = TEZ_TASK_PREFIX
-      + "get-task.sleep.interval-ms.max";
-  public static final int TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX_DEFAULT = 500;
-
-  public static final String TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS = TEZ_TASK_PREFIX
-      + "am.heartbeat.interval-ms.max";
-  public static final int TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS_DEFAULT = 100;
-
-  public static final String TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT = TEZ_TASK_PREFIX
-      + "max-events-per-heartbeat.max";
-  public static final int TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT_DEFAULT = 100;
-
-  /**
-   * Configuration to specify whether container should be reused.
-   */
-  public static final String TEZ_AM_CONTAINER_REUSE_ENABLED = TEZ_AM_PREFIX
-      + "container.reuse.enabled";
-  public static final boolean TEZ_AM_CONTAINER_REUSE_ENABLED_DEFAULT = true;
-
-  /**
-   * Whether to reuse containers for rack local tasks. Active only if reuse is
-   * enabled.
-   */
-  public static final String TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED = TEZ_AM_PREFIX
-      + "container.reuse.rack-fallback.enabled";
-  public static final boolean TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED_DEFAULT = true;
-
-  /**
-   * Whether to reuse containers for non-local tasks. Active only if reuse is
-   * enabled.
-   */
-  public static final String TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED = TEZ_AM_PREFIX
-      + "container.reuse.non-local-fallback.enabled";
-  public static final boolean TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED_DEFAULT = false;
-
-  public static final String TEZ_AM_CONTAINER_REUSE_DELAY_ALLOCATION_MILLIS = TEZ_AM_PREFIX
-      + "container.reuse.delay-allocation-millis";
-  public static final long TEZ_AM_CONTAINER_REUSE_DELAY_ALLOCATION_MILLIS_DEFAULT = 3000l;
-
-  public static final String TEZ_PB_BINARY_CONF_NAME = "tez-conf.pb";
-  public static final String TEZ_PB_PLAN_BINARY_NAME = "tez-dag.pb";
-  public static final String TEZ_PB_PLAN_TEXT_NAME = "tez-dag.pb.txt";
-
-  /*
-   * Logger properties
-   */
-  public static final String TEZ_CONTAINER_LOG4J_PROPERTIES_FILE = "tez-container-log4j.properties";
-  public static final String TEZ_CONTAINER_LOGGER_NAME = "CLA";
-  public static final String TEZ_ROOT_LOGGER_NAME = "tez.root.logger";
-  public static final String TEZ_CONTAINER_LOG_FILE_NAME = "syslog";
-  public static final String TEZ_CONTAINER_ERR_FILE_NAME = "stderr";
-  public static final String TEZ_CONTAINER_OUT_FILE_NAME = "stdout";
-
-
-  public static final String TEZ_LIB_URIS =
-      TEZ_PREFIX + "lib.uris";
-
-  public static final String TEZ_APPLICATION_TYPE = "TEZ-MR*";
-
-  public static final String LOCAL_FRAMEWORK_NAME = "local-tez";
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConstants.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
deleted file mode 100644
index 5463d65..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-/**
- * Specifies all constant values in Tez
- */
-public class TezConstants {
-
-  // Env variable names
-  public static final String TEZ_AM_IS_SESSION_ENV = "TEZ_AM_IS_SESSION";
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
deleted file mode 100644
index 9d4b2c4..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-public abstract class TezEntityDescriptor {
-
-  protected byte[] userPayload;
-  private String className;
-
-  public TezEntityDescriptor(String className) {
-    this.className = className;
-  }
-
-  public byte[] getUserPayload() {
-    return this.userPayload;
-  }
-
-  public TezEntityDescriptor setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-    return this;
-  }
-
-  public String getClassName() {
-    return this.className;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezException.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezException.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezException.java
deleted file mode 100644
index e3b14e7..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezException.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.dag.api;
-
-/**
- *  Base TezException
- */
-public class TezException extends Exception {
-  private static final long serialVersionUID = 6337442733802964447L;
-  public TezException(Throwable cause) { super(cause); }
-  public TezException(String message) { super(message); }
-  public TezException(String message, Throwable cause) {
-    super(message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
deleted file mode 100644
index f55f6dd..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.dag.api;
-
-/**
- * Base Tez Unchecked Exception
- */
-public class TezUncheckedException extends RuntimeException {
-
-  private static final long serialVersionUID = -4956339297375386184L;
-  
-  public TezUncheckedException(Throwable cause) { super(cause); }
-  public TezUncheckedException(String message) { super(message); }
-  public TezUncheckedException(String message, Throwable cause) {
-    super(message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/Vertex.java
deleted file mode 100644
index 900822b..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/Vertex.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
-
-public class Vertex { // FIXME rename to Task
-
-  private final String vertexName;
-  private final ProcessorDescriptor processorDescriptor;
-
-  private final int parallelism;
-  private VertexLocationHint taskLocationsHint;
-  private final Resource taskResource;
-  private Map<String, LocalResource> taskLocalResources;
-  private Map<String, String> taskEnvironment;
-
-  private final List<Vertex> inputVertices = new ArrayList<Vertex>();
-  private final List<Vertex> outputVertices = new ArrayList<Vertex>();
-  private final List<String> inputEdgeIds = new ArrayList<String>();
-  private final List<String> outputEdgeIds = new ArrayList<String>();
-  private String javaOpts = "";
-
-
-  public Vertex(String vertexName,
-      ProcessorDescriptor processorDescriptor,
-      int parallelism,
-      Resource taskResource) {
-    this.vertexName = vertexName;
-    this.processorDescriptor = processorDescriptor;
-    this.parallelism = parallelism;
-    this.taskResource = taskResource;
-    if (parallelism == 0) {
-      throw new IllegalArgumentException("Parallelism cannot be 0");
-    }
-    if (taskResource == null) {
-      throw new IllegalArgumentException("Resource cannot be null");
-    }
-  }
-
-  public String getVertexName() { // FIXME rename to getName()
-    return vertexName;
-  }
-
-  public ProcessorDescriptor getProcessorDescriptor() {
-    return this.processorDescriptor;
-  }
-
-  public int getParallelism() {
-    return parallelism;
-  }
-
-  public Resource getTaskResource() {
-    return taskResource;
-  }
-
-  public Vertex setTaskLocationsHint(List<TaskLocationHint> locations) {
-    if (locations == null) {
-      return this;
-    }
-    assert locations.size() == parallelism;
-    taskLocationsHint = new VertexLocationHint(parallelism, locations);
-    return this;
-  }
-
-  // used internally to create parallelism location resource file
-  VertexLocationHint getTaskLocationsHint() {
-    return taskLocationsHint;
-  }
-
-  public Vertex setTaskLocalResources(Map<String, LocalResource> localResources) {
-    this.taskLocalResources = localResources;
-    return this;
-  }
-
-  public Map<String, LocalResource> getTaskLocalResources() {
-    return taskLocalResources;
-  }
-
-  public Vertex setTaskEnvironment(Map<String, String> environment) {
-    this.taskEnvironment = environment;
-    return this;
-  }
-
-  public Map<String, String> getTaskEnvironment() {
-    return taskEnvironment;
-  }
-
-  public Vertex setJavaOpts(String javaOpts){
-     this. javaOpts = javaOpts;
-     return this;
-  }
-
-  public String getJavaOpts(){
-	  return javaOpts;
-  }
-
-  @Override
-  public String toString() {
-    return "[" + vertexName + " : " + processorDescriptor.getClassName() + "]";
-  }
-
-  void addInputVertex(Vertex inputVertex, String edgeId) {
-    inputVertices.add(inputVertex);
-    inputEdgeIds.add(edgeId);
-  }
-
-  void addOutputVertex(Vertex outputVertex, String edgeId) {
-    outputVertices.add(outputVertex);
-    outputEdgeIds.add(edgeId);
-  }
-
-  List<Vertex> getInputVertices() {
-    return inputVertices;
-  }
-
-  List<Vertex> getOutputVertices() {
-    return outputVertices;
-  }
-
-  List<String> getInputEdgeIds() {
-    return inputEdgeIds;
-  }
-
-  List<String> getOutputEdgeIds() {
-    return outputEdgeIds;
-  }
-
-  // FIXME how do we support profiling? Can't profile all tasks.
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
deleted file mode 100644
index 4f19314..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.Set;
-
-public class VertexLocationHint  {
-
-  private final int numTasks;
-  private final List<TaskLocationHint> taskLocationHints;
-
-  public VertexLocationHint(int numTasks,
-      List<TaskLocationHint> taskLocationHints) {
-    this.numTasks = numTasks;
-    if (taskLocationHints != null) {
-      this.taskLocationHints = Collections.unmodifiableList(taskLocationHints);
-    } else {
-      this.taskLocationHints = null;
-    }
-  }
-
-  public int getNumTasks() {
-    return numTasks;
-  }
-
-  public List<TaskLocationHint> getTaskLocationHints() {
-    return taskLocationHints;
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 7883;
-    int result = 1;
-    result = prime * result + numTasks;
-    if (taskLocationHints != null) {
-      result = prime * result + taskLocationHints.hashCode();
-    }
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    VertexLocationHint other = (VertexLocationHint) obj;
-    if (numTasks != other.numTasks) {
-      return false;
-    }
-    if (taskLocationHints != null) {
-      if (!taskLocationHints.equals(other.taskLocationHints)) {
-        return false;
-      }
-    } else if (other.taskLocationHints != null) {
-      return false;
-    }
-    return true;
-  }
-
-  public static class TaskLocationHint {
-
-    // Host names if any to be used
-    private final Set<String> hosts;
-    // Rack names if any to be used
-    private final Set<String> racks;
-
-    public TaskLocationHint(Set<String> hosts, Set<String> racks) {
-      if (hosts != null) {
-        this.hosts = Collections.unmodifiableSet(hosts);
-      } else {
-        this.hosts = null;
-      }
-      if (racks != null) {
-        this.racks = Collections.unmodifiableSet(racks);
-      } else {
-        this.racks = null;
-      }
-    }
-
-    public Set<String> getDataLocalHosts() {
-      return hosts;
-    }
-
-    public Set<String> getRacks() {
-      return racks;
-    }
-
-    @Override
-    public int hashCode() {
-      final int prime = 9397;
-      int result = 1;
-      result = ( hosts != null) ?
-          prime * result + hosts.hashCode() :
-          result + prime;
-      result = ( racks != null) ?
-          prime * result + racks.hashCode() :
-          result + prime;
-      return result;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-      if (this == obj) {
-        return true;
-      }
-      if (obj == null) {
-        return false;
-      }
-      if (getClass() != obj.getClass()) {
-        return false;
-      }
-      TaskLocationHint other = (TaskLocationHint) obj;
-      if (hosts != null) {
-        if (!hosts.equals(other.hosts)) {
-          return false;
-        }
-      } else if (other.hosts != null) {
-        return false;
-      }
-      if (racks != null) {
-        if (!racks.equals(other.racks)) {
-          return false;
-        }
-      } else if (other.racks != null) {
-        return false;
-      }
-      return true;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
deleted file mode 100644
index 9062e8e..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.dag.api.client;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.tez.dag.api.TezException;
-
-/*
- * Interface class for monitoring the <code>DAG</code> running in a Tez DAG
- * Application Master.
- */
-public interface DAGClient extends Closeable {
-
-  /**
-   * Get the YARN ApplicationId for the app running the DAG
-   * @return <code>ApplicationId</code>
-   */
-  public ApplicationId getApplicationId();
-
-  @Private
-  /**
-   * Get the YARN ApplicationReport for the app running the DAG. For performance
-   * reasons this may be stale copy and should be used to access static info. It
-   * may be null.
-   * @return <code>ApplicationReport</code> or null
-   */
-  public ApplicationReport getApplicationReport();
-
-  /**
-   * Get the status of the specified DAG
-   */
-  public DAGStatus getDAGStatus() throws IOException, TezException;
-
-  /**
-   * Get the status of a Vertex of a DAG
-   */
-  public VertexStatus getVertexStatus(String vertexName)
-      throws IOException, TezException;
-
-  /**
-   * Kill a running DAG
-   *
-   */
-  public void tryKillDAG() throws TezException, IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
deleted file mode 100644
index d61173d..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.dag.api.client;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProtoOrBuilder;
-import org.apache.tez.dag.api.records.DAGProtos.StringProgressPairProto;
-import org.apache.tez.dag.api.TezUncheckedException;
-
-public class DAGStatus {
-
-  public enum State {
-    SUBMITTED,
-    INITING,
-    RUNNING,
-    SUCCEEDED,
-    KILLED,
-    FAILED,
-    ERROR,
-  };
-
-  DAGStatusProtoOrBuilder proxy = null;
-  Progress progress = null;
-  Map<String, Progress> vertexProgress = null;
-
-  public DAGStatus(DAGStatusProtoOrBuilder proxy) {
-    this.proxy = proxy;
-  }
-
-  public State getState() {
-    switch(proxy.getState()) {
-    case DAG_SUBMITTED:
-      return DAGStatus.State.SUBMITTED;
-    // For simplicity, initing/terminating states are presented as running
-    case DAG_INITING:
-    case DAG_TERMINATING:
-    case DAG_RUNNING:
-      return DAGStatus.State.RUNNING;
-    case DAG_SUCCEEDED:
-      return DAGStatus.State.SUCCEEDED;
-    case DAG_FAILED:
-      return DAGStatus.State.FAILED;
-    case DAG_KILLED:
-      return DAGStatus.State.KILLED;
-    case DAG_ERROR:
-      return DAGStatus.State.ERROR;
-    default:
-      throw new TezUncheckedException("Unsupported value for DAGStatus.State : " +
-                              proxy.getState());
-    }
-  }
-
-  public boolean isCompleted() {
-    State state = getState();
-    return (state == State.SUCCEEDED ||
-             state == State.FAILED ||
-             state == State.KILLED ||
-             state == State.ERROR);
-  }
-
-  public List<String> getDiagnostics() {
-    return proxy.getDiagnosticsList();
-  }
-
-  /**
-   * Gets overall progress value of the DAG.
-   *
-   * @return Progress of the DAG. Maybe null when the DAG is not running. Maybe
-   *         null when the DAG is running and the application master cannot be
-   *         reached - e.g. when the execution platform has restarted the
-   *         application master.
-   * @see Progress
-   */
-  public Progress getDAGProgress() {
-    if(progress == null && proxy.hasDAGProgress()) {
-      progress = new Progress(proxy.getDAGProgress());
-    }
-    return progress;
-  }
-
-  /**
-   * Get the progress of a vertex in the DAG
-   *
-   * @return Progress of the vertex. May be null when the DAG is not running.
-   *         Maybe null when the DAG is running and the application master
-   *         cannot be reached - e.g. when the execution platform has restarted
-   *         the application master.
-   * @see Progress
-   */
-  public Map<String, Progress> getVertexProgress() {
-    if(vertexProgress == null) {
-      if(proxy.getVertexProgressList() != null) {
-        List<StringProgressPairProto> kvList = proxy.getVertexProgressList();
-        vertexProgress = new HashMap<String, Progress>(kvList.size());
-        for(StringProgressPairProto kv : kvList){
-          vertexProgress.put(kv.getKey(), new Progress(kv.getProgress()));
-        }
-      }
-    }
-    return vertexProgress;
-  }
-
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("status=" + getState()
-        + ", progress=" + getDAGProgress());
-    return sb.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/Progress.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
deleted file mode 100644
index 9577320..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.dag.api.client;
-
-import org.apache.tez.dag.api.records.DAGProtos.ProgressProtoOrBuilder;
-
-public class Progress {
-  
-  ProgressProtoOrBuilder proxy = null;
-  
-  Progress(ProgressProtoOrBuilder proxy) {
-    this.proxy = proxy;
-  }
-  
-  public int getTotalTaskCount() {
-    return proxy.getTotalTaskCount();
-  }
-
-  public int getSucceededTaskCount() {
-    return proxy.getSucceededTaskCount();
-  }
-
-  public int getRunningTaskCount() {
-    return proxy.getRunningTaskCount();
-  }
-
-  public int getFailedTaskCount() {
-    return proxy.getFailedTaskCount();
-  }
-
-  public int getKilledTaskCount() {
-    return proxy.getKilledTaskCount();
-  }
-  
-  @Override
-  public String toString() {
-    StringBuilder sb = new StringBuilder();
-    sb.append("TotalTasks: ");
-    sb.append(getTotalTaskCount());
-    sb.append(" Succeeded: ");
-    sb.append(getSucceededTaskCount());
-    sb.append(" Running: ");
-    sb.append(getRunningTaskCount());
-    sb.append(" Failed: "); 
-    sb.append(getFailedTaskCount());
-    sb.append(" Killed: "); 
-    sb.append(getKilledTaskCount());
-    return sb.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
deleted file mode 100644
index ce5dbe0..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.dag.api.client;
-
-import java.util.List;
-
-import org.apache.tez.dag.api.records.DAGProtos.VertexStatusProtoOrBuilder;
-import org.apache.tez.dag.api.TezUncheckedException;
-
-public class VertexStatus {
-  
-  public enum State {
-    INITED,
-    RUNNING,
-    SUCCEEDED,
-    KILLED,
-    FAILED,
-    ERROR,
-    TERMINATING,
-  };
-  
-  VertexStatusProtoOrBuilder proxy = null;
-  Progress progress = null;
-  
-  public VertexStatus(VertexStatusProtoOrBuilder proxy) {
-    this.proxy = proxy;
-  }
-
-  public State getState() {
-    switch(proxy.getState()) {
-    case VERTEX_INITED:
-      return VertexStatus.State.INITED;
-    case VERTEX_RUNNING:
-      return VertexStatus.State.RUNNING;
-    case VERTEX_SUCCEEDED:
-      return VertexStatus.State.SUCCEEDED;
-    case VERTEX_FAILED:
-      return VertexStatus.State.FAILED;
-    case VERTEX_KILLED:
-      return VertexStatus.State.KILLED;
-    case VERTEX_ERROR:
-      return VertexStatus.State.ERROR;
-    case VERTEX_TERMINATING:
-      return VertexStatus.State.TERMINATING;
-    default:
-      throw new TezUncheckedException("Unsupported value for VertexStatus.State : " + 
-                              proxy.getState());
-    }    
-  }
-
-  public List<String> getDiagnostics() {
-    return proxy.getDiagnosticsList();
-  }
-
-  public Progress getProgress() {
-    if(progress == null && proxy.hasProgress()) {
-      progress = new Progress(proxy.getProgress());
-    }
-    return progress;    
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
deleted file mode 100644
index a1ee18f..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api.client.rpc;
-
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.DAGClientAMProtocol;
-
-@ProtocolInfo(
-    protocolName = "org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolPB",
-    protocolVersion = 1)
-public interface DAGClientAMProtocolBlockingPB 
-                              extends DAGClientAMProtocol.BlockingInterface {
-
-}


[17/20] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
new file mode 100644
index 0000000..dae5625
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
@@ -0,0 +1,291 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api.client.rpc;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Collections;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.dag.api.client.VertexStatus;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusRequestProto;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusRequestProto;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TryKillDAGRequestProto;
+import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProto;
+import org.apache.tez.dag.api.records.DAGProtos.DAGStatusStateProto;
+
+import com.google.protobuf.ServiceException;
+
+public class DAGClientRPCImpl implements DAGClient {
+  private static final Log LOG = LogFactory.getLog(DAGClientRPCImpl.class);
+
+  private final ApplicationId appId;
+  private final String dagId;
+  private final TezConfiguration conf;
+  private ApplicationReport appReport;
+  private YarnClient yarnClient;
+  private DAGClientAMProtocolBlockingPB proxy = null;
+
+  public DAGClientRPCImpl(ApplicationId appId, String dagId,
+      TezConfiguration conf) {
+    this.appId = appId;
+    this.dagId = dagId;
+    this.conf = conf;
+    yarnClient = new YarnClientImpl();
+    yarnClient.init(new YarnConfiguration(conf));
+    yarnClient.start();
+    appReport = null;
+  }
+
+  @Override
+  public ApplicationId getApplicationId() {
+    return appId;
+  }
+
+  @Override
+  public DAGStatus getDAGStatus() throws IOException, TezException {
+    if(createAMProxyIfNeeded()) {
+      try {
+        return getDAGStatusViaAM();
+      } catch (TezException e) {
+        resetProxy(e); // create proxy again
+      }
+    }
+
+    // Later maybe from History
+    return getDAGStatusViaRM();
+  }
+
+  @Override
+  public VertexStatus getVertexStatus(String vertexName)
+                                    throws IOException, TezException {
+    if(createAMProxyIfNeeded()) {
+      try {
+        return getVertexStatusViaAM(vertexName);
+      } catch (TezException e) {
+        resetProxy(e); // create proxy again
+      }
+    }
+
+    // need AM for this. Later maybe from History
+    return null;
+  }
+
+  @Override
+  public void tryKillDAG() throws TezException, IOException {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("TryKill for app: " + appId + " dag:" + dagId);
+    }
+    if(createAMProxyIfNeeded()) {
+      TryKillDAGRequestProto requestProto =
+          TryKillDAGRequestProto.newBuilder().setDagId(dagId).build();
+      try {
+        proxy.tryKillDAG(null, requestProto);
+      } catch (ServiceException e) {
+        resetProxy(e);
+      }
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (this.proxy != null) {
+      RPC.stopProxy(this.proxy);
+    }
+    if(yarnClient != null) {
+      yarnClient.stop();
+    }
+  }
+
+  @Override
+  public ApplicationReport getApplicationReport() {
+    return appReport;
+  }
+
+  void resetProxy(Exception e) {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Resetting AM proxy for app: " + appId + " dag:" + dagId +
+          " due to exception :", e);
+    }
+    proxy = null;
+  }
+
+  DAGStatus getDAGStatusViaAM() throws IOException, TezException {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId);
+    }
+    GetDAGStatusRequestProto requestProto =
+        GetDAGStatusRequestProto.newBuilder().setDagId(dagId).build();
+    try {
+      return new DAGStatus(
+                 proxy.getDAGStatus(null, requestProto).getDagStatus());
+    } catch (ServiceException e) {
+      // TEZ-151 retrieve wrapped TezException
+      throw new TezException(e);
+    }
+  }
+
+
+
+  DAGStatus getDAGStatusViaRM() throws TezException, IOException {
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId);
+    }
+    ApplicationReport appReport;
+    try {
+      appReport = yarnClient.getApplicationReport(appId);
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+
+    if(appReport == null) {
+      throw new TezException("Unknown/Invalid appId: " + appId);
+    }
+
+    DAGStatusProto.Builder builder = DAGStatusProto.newBuilder();
+    DAGStatus dagStatus = new DAGStatus(builder);
+    DAGStatusStateProto dagState = null;
+    switch (appReport.getYarnApplicationState()) {
+    case NEW:
+    case NEW_SAVING:
+    case SUBMITTED:
+    case ACCEPTED:
+      dagState = DAGStatusStateProto.DAG_SUBMITTED;
+      break;
+    case RUNNING:
+      dagState = DAGStatusStateProto.DAG_RUNNING;
+      break;
+    case FAILED:
+      dagState = DAGStatusStateProto.DAG_FAILED;
+      break;
+    case KILLED:
+      dagState = DAGStatusStateProto.DAG_KILLED;
+      break;
+    case FINISHED:
+      switch(appReport.getFinalApplicationStatus()) {
+      case UNDEFINED:
+      case FAILED:
+        dagState = DAGStatusStateProto.DAG_FAILED;
+        break;
+      case KILLED:
+        dagState = DAGStatusStateProto.DAG_KILLED;
+        break;
+      case SUCCEEDED:
+        dagState = DAGStatusStateProto.DAG_SUCCEEDED;
+        break;
+      default:
+        throw new TezUncheckedException("Encountered unknown final application"
+          + " status from YARN"
+          + ", appState=" + appReport.getYarnApplicationState()
+          + ", finalStatus=" + appReport.getFinalApplicationStatus());
+      }
+      break;
+    default:
+      throw new TezUncheckedException("Encountered unknown application state"
+          + " from YARN, appState=" + appReport.getYarnApplicationState());
+    }
+
+    builder.setState(dagState);
+    if(appReport.getDiagnostics() != null) {
+      builder.addAllDiagnostics(Collections.singleton(appReport.getDiagnostics()));
+    }
+
+    return dagStatus;
+  }
+
+  VertexStatus getVertexStatusViaAM(String vertexName) throws TezException {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("GetVertexStatus via AM for app: " + appId + " dag: " + dagId
+          + " vertex: " + vertexName);
+    }
+    GetVertexStatusRequestProto requestProto =
+        GetVertexStatusRequestProto.newBuilder().
+                        setDagId(dagId).setVertexName(vertexName).build();
+
+    try {
+      return new VertexStatus(
+                 proxy.getVertexStatus(null, requestProto).getVertexStatus());
+    } catch (ServiceException e) {
+      // TEZ-151 retrieve wrapped TezException
+      throw new TezException(e);
+    }
+  }
+
+  ApplicationReport getAppReport() throws IOException, TezException {
+    try {
+      ApplicationReport appReport = yarnClient.getApplicationReport(appId);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("App: " + appId + " in state: "
+            + appReport.getYarnApplicationState());
+      }
+      return appReport;
+    } catch (YarnException e) {
+      throw new TezException(e);
+    }
+  }
+
+  boolean createAMProxyIfNeeded() throws IOException, TezException {
+    if(proxy != null) {
+      // if proxy exist optimistically use it assuming there is no retry
+      return true;
+    }
+    appReport = getAppReport();
+
+    if(appReport == null) {
+      return false;
+    }
+    YarnApplicationState appState = appReport.getYarnApplicationState();
+    if(appState != YarnApplicationState.RUNNING) {
+      return false;
+    }
+
+    // YARN-808. Cannot ascertain if AM is ready until we connect to it.
+    // workaround check the default string set by YARN
+    if(appReport.getHost() == null || appReport.getHost().equals("N/A") ||
+        appReport.getRpcPort() == 0){
+      // attempt not running
+      return false;
+    }
+
+    InetSocketAddress addr = new InetSocketAddress(appReport.getHost(),
+        appReport.getRpcPort());
+
+    RPC.setProtocolEngine(conf, DAGClientAMProtocolBlockingPB.class,
+        ProtobufRpcEngine.class);
+    proxy = (DAGClientAMProtocolBlockingPB) RPC.getProxy(
+        DAGClientAMProtocolBlockingPB.class, 0, addr, conf);
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Event.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Event.java b/tez-api/src/main/java/org/apache/tez/engine/api/Event.java
new file mode 100644
index 0000000..80da655
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Event.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+/**
+ * Base class for all events generated within the Tez execution engine.
+ * Used as the primary mode of communication between the AM, Inputs, Processors
+ * and Outputs.
+ */
+public abstract class Event {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Input.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Input.java b/tez-api/src/main/java/org/apache/tez/engine/api/Input.java
new file mode 100644
index 0000000..e333075
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Input.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+import java.util.List;
+
+/**
+ * Represents an input through which a TezProcessor receives data on an edge.
+ * </p>
+ *
+ * <code>Input</code> classes must have a 0 argument public constructor for Tez
+ * to construct the <code>Input</code>. Tez will take care of initializing and
+ * closing the Input after a {@link Processor} completes. </p>
+ */
+public interface Input {
+
+  /**
+   * Initializes the <code>Input</code>.
+   *
+   * @param inputContext
+   *          the {@link TezInputContext}
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> initialize(TezInputContext inputContext)
+      throws Exception;
+
+  /**
+   * Gets an instance of the {@link Reader} for this <code>Output</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public Reader getReader() throws Exception;
+
+  /**
+   * Handles user and system generated {@link Events}s, which typically carry
+   * information such as an output being available on the previous vertex.
+   *
+   * @param inputEvents
+   *          the list of {@link Event}s
+   */
+  public void handleEvents(List<Event> inputEvents);
+
+  /**
+   * Closes the <code>Input</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> close() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java
new file mode 100644
index 0000000..90be09e
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalIOProcessor.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+import java.util.Map;
+
+/**
+ * Represents a processor which consumes {@link LogicalInput}s and produces
+ * {@link LogicalOutput}s
+ */
+public interface LogicalIOProcessor extends Processor {
+
+  /**
+   * Runs the {@link LogicalProcessor}
+   * 
+   * @param inputs
+   *          a map of the source vertex name to {@link LogicalInput} - one per
+   *          incoming edge.
+   * @param outputs
+   *          a map of the destination vertex name to {@link LogicalOutput} -
+   *          one per outgoing edge
+   * @throws Exception TODO
+   */
+  public void run(Map<String, LogicalInput> inputs,
+      Map<String, LogicalOutput> outputs) throws Exception;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java
new file mode 100644
index 0000000..4a47ccf
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalInput.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+/**
+ * An @link {@link Input} which handles all incoming physical connections on an
+ * edge. A {@link LogicalIOProcessor} sees a single Logical Input per incoming
+ * edge.
+ */
+public interface LogicalInput extends Input {
+
+  /**
+   * Sets the number of physical inputs that this <code>LogicalInput</code> will
+   * receive. This will be called by the Tez framework before initializing the
+   * <code>LogicalInput</code>
+   * 
+   * @param numInputs
+   *          the number of physical inputs.
+   */
+  public void setNumPhysicalInputs(int numInputs);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java
new file mode 100644
index 0000000..4626fbd
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/LogicalOutput.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+/**
+ * An @link {@link Output} which handles all outgoing physical connections on an
+ * edge. A {@link LogicalIOProcessor} sees a single Logical Output per outgoing
+ * edge.
+ */
+public interface LogicalOutput extends Output {
+  /**
+   * Sets the number of physical ouputs that this <code>LogicalOutput</code>
+   * will receive. This will be called by the Tez framework before initializing
+   * the <code>LogicalOutput</code>
+   * 
+   * @param numOutputs
+   *          the number of physical outputs
+   */
+  public void setNumPhysicalOutputs(int numOutputs);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Output.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Output.java b/tez-api/src/main/java/org/apache/tez/engine/api/Output.java
new file mode 100644
index 0000000..ec679ed
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Output.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+import java.util.List;
+
+/**
+ * Represents an Output through which a TezProcessor writes information on an
+ * edge. </p>
+ *
+ * <code>Output</code> implementations must have a 0 argument public constructor
+ * for Tez to construct the <code>Output</code>. Tez will take care of
+ * initializing and closing the Input after a {@link Processor} completes. </p>
+ */
+public interface Output {
+
+  /**
+   * Initializes the <code>Output</code>
+   *
+   * @param outputContext
+   *          the {@link TezOutputContext}
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws Exception;
+
+  /**
+   * Gets an instance of the {@link Writer} in an <code>Output</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public Writer getWriter() throws Exception;
+
+  /**
+   * Handles user and system generated {@link Events}s, which typically carry
+   * information such as a downstream vertex being ready to consume input.
+   *
+   * @param outputEvents
+   *          the list of {@link Event}s
+   */
+  public void handleEvents(List<Event> outputEvents);
+
+  /**
+   * Closes the <code>Output</code>
+   *
+   * @return
+   * @throws Exception
+   *           if an error occurs
+   */
+  public List<Event> close() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java b/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java
new file mode 100644
index 0000000..05e6b84
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Processor.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * {@link Processor} represents the <em>Tez</em> entity responsible for
+ * consuming {@link Input} and producing {@link Output}.
+ */
+public interface Processor {
+
+  /**
+   * Initializes the <code>Processor</code>
+   *
+   * @param processorContext
+   * @throws IOException
+   *           if an error occurs
+   */
+  public void initialize(TezProcessorContext processorContext)
+      throws Exception;
+
+  /**
+   * Handles user and system generated {@link Events}s.
+   *
+   * @param processorEvents
+   *          the list of {@link Event}s
+   */
+  public void handleEvents(List<Event> processorEvents);
+
+  /**
+   * Closes the <code>Processor</code>
+   *
+   * @throws IOException
+   *           if an error occurs
+   */
+  public void close() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java b/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java
new file mode 100644
index 0000000..502c5f2
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Reader.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+/**
+ * A <code>Reader</code> represents the data being read in an {@link Input}
+ */
+public interface Reader {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java
new file mode 100644
index 0000000..ddf1ff8
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/TezInputContext.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+/**
+ * Context handle for the Input to initialize itself.
+ */
+public interface TezInputContext extends TezTaskContext {
+
+  /**
+   * Get the Vertex Name of the Source that generated data for this Input
+   * @return Name of the Source Vertex
+   */
+  public String getSourceVertexName();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java
new file mode 100644
index 0000000..791a0f0
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/TezOutputContext.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+/**
+ * Context handle for the Output to initialize itself.
+ */
+public interface TezOutputContext extends TezTaskContext {
+
+  /**
+   * Get the Vertex Name of the Destination that is the recipient of this
+   * Output's data
+   * @return Name of the Destination Vertex
+   */
+  public String getDestinationVertexName();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java
new file mode 100644
index 0000000..2bbbe81
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/TezProcessorContext.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+import java.io.IOException;
+
+/**
+ * Context handle for the Processor to initialize itself.
+ */
+public interface TezProcessorContext extends TezTaskContext {
+
+  /**
+   * Set the overall progress of this Task Attempt
+   * @param progress Progress in the range from [0.0 - 1.0f]
+   */
+  public void setProgress(float progress);
+
+  /**
+   * Check whether this attempt can commit its output
+   * @return true if commit allowed
+   * @throws IOException
+   */
+  public boolean canCommit() throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java b/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java
new file mode 100644
index 0000000..706e646
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/TezTaskContext.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.common.counters.TezCounters;
+
+/**
+ * Base interface for Context classes used to initialize the Input, Output
+ * and Processor instances.
+ */
+public interface TezTaskContext {
+
+  // TODO NEWTEZ
+  // Scale the maximum events we fetch per RPC call to mitigate OOM issues
+  // on the ApplicationMaster when a thundering herd of reducers fetch events
+  // This should not be necessary after HADOOP-8942
+
+  /**
+   * Get the {@link ApplicationId} for the running app
+   * @return the {@link ApplicationId}
+   */
+  public ApplicationId getApplicationId();
+
+  /**
+   * Get the current DAG Attempt Number
+   * @return DAG Attempt Number
+   */
+  public int getDAGAttemptNumber();
+
+  /**
+   * Get the index of this Task
+   * @return Task Index
+   */
+  public int getTaskIndex();
+
+  /**
+   * Get the current Task Attempt Number
+   * @return Task Attempt Number
+   */
+  public int getTaskAttemptNumber();
+
+  /**
+   * Get the name of the DAG
+   * @return the DAG name
+   */
+  public String getDAGName();
+
+  /**
+   * Get the name of the Vertex in which the task is running
+   * @return Vertex Name
+   */
+  public String getTaskVertexName();
+
+  public TezCounters getCounters();
+
+  /**
+   * Send Events to the AM and/or dependent Vertices
+   * @param events Events to be sent
+   */
+  public void sendEvents(List<Event> events);
+
+  /**
+   * Get the User Payload for the Input/Output/Processor
+   * @return User Payload
+   */
+  public byte[] getUserPayload();
+
+  /**
+   * Get the work diectories for the Input/Output/Processor
+   * @return an array of work dirs
+   */
+  public String[] getWorkDirs();
+
+  /**
+   * Returns an identifier which is unique to the specific Input, Processor or
+   * Output
+   *
+   * @return
+   */
+  public String getUniqueIdentifier();
+
+  /**
+   * Report a fatal error to the framework. This will cause the entire task to
+   * fail and should not be used for reporting temporary or recoverable errors
+   *
+   * @param exception an exception representing the error
+   */
+  public void fatalError(Throwable exception, String message);
+
+  /**
+   * Returns meta-data for the specified service. As an example, when the MR
+   * ShuffleHandler is used - this would return the jobToken serialized as bytes
+   *
+   * @param serviceName
+   *          the name of the service for which meta-data is required
+   * @return a ByteBuffer representing the meta-data
+   */
+  public ByteBuffer getServiceConsumerMetaData(String serviceName);
+
+  /**
+   * Return Provider meta-data for the specified service As an example, when the
+   * MR ShuffleHandler is used - this would return the shuffle port serialized
+   * as bytes
+   *
+   * @param serviceName
+   *          the name of the service for which provider meta-data is required
+   * @return a ByteBuffer representing the meta-data
+   */
+  public ByteBuffer getServiceProviderMetaData(String serviceName);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java b/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java
new file mode 100644
index 0000000..c9503a3
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/Writer.java
@@ -0,0 +1,26 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+/**
+ * A <code>Writer</code> represents the data being written by an {@link Output}
+ */
+public interface Writer {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java
----------------------------------------------------------------------
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/engine/api/events/DataMovementEvent.java
new file mode 100644
index 0000000..3f35555
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/events/DataMovementEvent.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.events;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.engine.api.Event;
+
+/**
+ * Event used by user code to send information between tasks. An output can
+ * generate an Event of this type to sending information regarding output data
+ * ( such as URI for file-based output data, port info in case of
+ * streaming-based data transfers ) to the Input on the destination vertex.
+ */
+public final class DataMovementEvent extends Event {
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that generated an Event.
+   * For a Processor-generated event, this is ignored.
+   */
+  private final int sourceIndex;
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that is meant to receive
+   * this Event. For a Processor event, this is ignored.
+   */
+  private int targetIndex;
+
+  /**
+   * User Payload for this Event
+   */
+  private final byte[] userPayload;
+
+  /**
+   * Version number to indicate what attempt generated this Event
+   */
+  private int version;
+
+  /**
+   * User Event constructor
+   * @param sourceIndex Index to identify the physical edge of the input/output
+   * that generated the event
+   * @param userPayload User Payload of the User Event
+   */
+  public DataMovementEvent(int sourceIndex,
+      byte[] userPayload) {
+    this.userPayload = userPayload;
+    this.sourceIndex = sourceIndex;
+  }
+
+  @Private
+  public DataMovementEvent(int sourceIndex,
+      int targetIndex,
+      byte[] userPayload) {
+    this.userPayload = userPayload;
+    this.sourceIndex = sourceIndex;
+    this.targetIndex = targetIndex;
+  }
+
+  /**
+   * Constructor for Processor-generated User Events
+   * @param userPayload
+   */
+  public DataMovementEvent(byte[] userPayload) {
+    this(-1, userPayload);
+  }
+
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  public int getSourceIndex() {
+    return sourceIndex;
+  }
+
+  public int getTargetIndex() {
+    return targetIndex;
+  }
+
+  @Private
+  public void setTargetIndex(int targetIndex) {
+    this.targetIndex = targetIndex;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  @Private
+  public void setVersion(int version) {
+    this.version = version;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java
----------------------------------------------------------------------
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/engine/api/events/InputFailedEvent.java
new file mode 100644
index 0000000..57de09b
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputFailedEvent.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.events;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.engine.api.Event;
+
+/**
+ * Event sent from the AM to an Input to indicate that one of it's sources has
+ * failed - effectively the input is no longer available from the particular
+ * source.
+ * Users are not expected to send this event.
+ */
+public class InputFailedEvent extends Event{
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that generated the data.
+   * For a Processor-generated event, this is ignored.
+   */
+  private final int sourceIndex;
+
+  /**
+   * Index(i) of the i-th (physical) Input or Output that is meant to receive
+   * this Event. For a Processor event, this is ignored.
+   */
+  private int targetIndex;
+
+  /**
+   * Version number to indicate what attempt generated this Event
+   */
+  private int version;
+
+  /**
+   * User Event constructor
+   * @param sourceIndex Index to identify the physical edge of the input/output
+   * that generated the event
+   * @param userPayload User Payload of the User Event
+   */
+  public InputFailedEvent(int sourceIndex) {
+    this.sourceIndex = sourceIndex;
+  }
+
+  @Private
+  public InputFailedEvent(int sourceIndex,
+      int targetIndex,
+      int version) {
+    this.sourceIndex = sourceIndex;
+    this.targetIndex = targetIndex;
+    this.version = version;
+  }
+
+  public int getSourceIndex() {
+    return sourceIndex;
+  }
+
+  public int getTargetIndex() {
+    return targetIndex;
+  }
+
+  @Private
+  public void setTargetIndex(int targetIndex) {
+    this.targetIndex = targetIndex;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  @Private
+  public void setVersion(int version) {
+    this.version = version;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java
----------------------------------------------------------------------
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/engine/api/events/InputInformationEvent.java
new file mode 100644
index 0000000..3656d45
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputInformationEvent.java
@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.events;
+
+import org.apache.tez.engine.api.Event;
+
+/**
+ * Event used to send user specific data from the user 
+ * code in the AM to the task input
+ */
+public class InputInformationEvent extends Event {
+
+  /**
+   * User Payload for this Event
+   */
+  private final byte[] userPayload;
+  public InputInformationEvent(byte[] userPayload) {
+    this.userPayload = userPayload;
+  }
+
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java
----------------------------------------------------------------------
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/engine/api/events/InputReadErrorEvent.java
new file mode 100644
index 0000000..fa49b79
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/api/events/InputReadErrorEvent.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.events;
+
+import org.apache.tez.engine.api.Event;
+
+/**
+ * Event generated by an Input to indicate error when trying to retrieve data.
+ * This is not necessarily a fatal event - it's an indication to the AM to retry
+ * source data generation.
+ */
+public final class InputReadErrorEvent extends Event {
+
+  /**
+   * Diagnostics/trace of the error that occurred on the Input's edge.
+   */
+  private final String diagnostics;
+
+  /**
+   * Index of the physical edge on which the error occurred.
+   */
+  private final int index;
+
+  /**
+   * Version of the data on which the error occurred.
+   */
+  private final int version;
+
+  public InputReadErrorEvent(String diagnostics, int index,
+      int version) {
+    super();
+    this.diagnostics = diagnostics;
+    this.index = index;
+    this.version = version;
+  }
+
+  public String getDiagnostics() {
+    return diagnostics;
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
----------------------------------------------------------------------
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/engine/common/objectregistry/ObjectLifeCycle.java
new file mode 100644
index 0000000..7099299
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.common.objectregistry;
+
+/**
+ * Defines the valid lifecycle and scope of Objects stored in ObjectRegistry.
+ * Objects are guaranteed to not be valid outside of their defined life-cycle
+ * period. Objects are not guaranteed to be retained through the defined period
+ * as they may be evicted for various reasons.
+ */
+public enum ObjectLifeCycle {
+  /** Objects are valid for the lifetime of the Tez JVM/Session
+   */
+  SESSION,
+  /** Objects are valid for the lifetime of the DAG.
+   */
+  DAG,
+  /** Objects are valid for the lifetime of the Vertex.
+   */
+  VERTEX,
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
----------------------------------------------------------------------
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/engine/common/objectregistry/ObjectRegistry.java
new file mode 100644
index 0000000..a27903d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.common.objectregistry;
+
+/**
+ * Preliminary version of a simple shared object cache to re-use
+ * objects across multiple tasks within the same container/JVM.
+ */
+public interface ObjectRegistry {
+
+  /**
+   * Insert or update object into the registry. This will remove an object
+   * associated with the same key with a different life-cycle as there is only
+   * one instance of an Object stored for a given key irrespective of the
+   * life-cycle attached to the Object.
+   * @param lifeCycle What life-cycle is the Object valid for
+   * @param key Key to identify the Object
+   * @param value Object to be inserted
+   * @return Previous Object associated with the key attached if present
+   * else null. Could return the same object if the object was associated with
+   * the same key for a different life-cycle.
+   */
+  public Object add(ObjectLifeCycle lifeCycle, String key, Object value);
+
+  /**
+   * Return the object associated with the provided key
+   * @param key Key to find object
+   * @return Object if found else null
+   */
+  public Object get(String key);
+
+  /**
+   * Delete the object associated with the provided key
+   * @param lifeCycle What life-cycle is the Object valid for
+   * @param key Key to find object
+   * @return True if an object was found and removed
+   */
+  public boolean delete(String key);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
----------------------------------------------------------------------
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/engine/common/objectregistry/ObjectRegistryFactory.java
new file mode 100644
index 0000000..94352b3
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.common.objectregistry;
+
+import com.google.inject.Inject;
+
+public class ObjectRegistryFactory {
+
+  @Inject
+  private static ObjectRegistry objectRegistry;
+
+  public static ObjectRegistry getObjectRegistry() {
+    return objectRegistry;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/proto/DAGApiRecords.proto
----------------------------------------------------------------------
diff --git a/tez-api/src/main/proto/DAGApiRecords.proto b/tez-api/src/main/proto/DAGApiRecords.proto
new file mode 100644
index 0000000..4385749
--- /dev/null
+++ b/tez-api/src/main/proto/DAGApiRecords.proto
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+option java_package = "org.apache.tez.dag.api.records";
+option java_outer_classname = "DAGProtos";
+option java_generate_equals_and_hash = true;
+
+// DAG plan messages
+
+// Many of these types have a dual in the Tez-api.  To reduce confusion, these types have prefix or suffix 
+// of "Plan" to indicate they are to be used in the dag-plan.
+// The big types use a suffix:  JobPlan, VertexPlan, EdgePlan 
+//   --> these get more direct use in the runtime and the naming is natural.
+// The enums and utility types use prefix: PlanVertexType, PlanEdgeConnectionPaatern, etc
+//   --> there is not great naming choice for these that avoids ambiguity, but this one seems acceptable. 
+
+enum PlanVertexType {
+  INPUT = 0;
+  NORMAL = 1;
+  OUTPUT = 2;
+}
+
+enum PlanEdgeDataMovementType {
+  ONE_TO_ONE = 0;
+  BROADCAST = 1;
+  SCATTER_GATHER = 2;
+}
+
+enum PlanEdgeDataSourceType {
+  PERSISTED = 0;
+  PERSISTED_RELIABLE = 1;
+  EPHEMERAL = 2;
+}
+
+enum PlanEdgeSchedulingType {
+  SEQUENTIAL = 0;
+  CONCURRENT = 1;
+}
+
+message PlanKeyValuePair {
+  required string key = 1;
+  required string value = 2;
+}
+
+enum PlanLocalResourceType {
+  FILE = 0;
+  ARCHIVE = 1;
+  PATTERN = 2;
+}
+
+enum PlanLocalResourceVisibility {
+  PUBLIC = 0;
+  PRIVATE = 1;
+  APPLICATION = 2;
+}
+
+message PlanLocalResource {
+  required string name = 1;
+  required string uri = 2;
+  required int64 size = 3;
+  required int64 timeStamp = 4;
+  required PlanLocalResourceType type = 5;
+  required PlanLocalResourceVisibility visibility = 6;
+  optional string pattern = 7; // only used if type=PATTERN
+}
+
+// Each taskLocationHint represents a single split in in the input.
+// It is the list of [{rack,machines}] that host a replica of each particular split.
+// For now it is represented as pair-of-arrays rather than array-of-pairs.
+message PlanTaskLocationHint {
+  repeated string rack = 1;
+  repeated string host = 2;
+}
+
+message PlanTaskConfiguration {
+  required int32 numTasks = 1;
+  required int32 memoryMb = 2;
+  required int32 virtualCores = 3;
+  required string javaOpts = 4;
+  required string taskModule = 5;
+  repeated PlanLocalResource localResource = 6;
+  repeated PlanKeyValuePair environmentSetting = 8;  
+}
+
+message TezEntityDescriptorProto {
+  optional string class_name = 1;
+  optional bytes user_payload = 2;
+}
+
+message VertexPlan {
+  required string name = 1;
+  required PlanVertexType type = 2;
+  optional TezEntityDescriptorProto processor_descriptor = 3;
+  required PlanTaskConfiguration taskConfig = 4;
+  repeated PlanTaskLocationHint taskLocationHint = 7;
+  repeated string inEdgeId = 8;
+  repeated string outEdgeId = 9;
+}
+
+message EdgePlan {
+  required string id = 1;
+  required string inputVertexName = 2;
+  required string outputVertexName = 3;
+  required PlanEdgeDataMovementType dataMovementType = 4;
+  required PlanEdgeDataSourceType dataSourceType = 5;
+  required PlanEdgeSchedulingType schedulingType = 6;
+  optional TezEntityDescriptorProto edge_source = 7;
+  optional TezEntityDescriptorProto edge_destination = 8;
+}
+
+message ConfigurationProto {
+  repeated PlanKeyValuePair confKeyValues = 1;
+}
+
+message DAGPlan {
+  required string name = 1;
+  repeated VertexPlan vertex = 2;
+  repeated EdgePlan edge = 3;
+  optional ConfigurationProto dagKeyValues = 4;
+}
+
+// DAG monitoring messages
+message ProgressProto {
+  optional int32 totalTaskCount = 1;
+  optional int32 succeededTaskCount = 2;
+  optional int32 runningTaskCount = 3;
+  optional int32 failedTaskCount = 4;
+  optional int32 killedTaskCount = 5;
+}
+
+enum VertexStatusStateProto {
+  VERTEX_INITED = 0;
+  VERTEX_RUNNING = 1;
+  VERTEX_SUCCEEDED = 2;
+  VERTEX_KILLED = 3;
+  VERTEX_FAILED = 4;
+  VERTEX_ERROR = 5;
+  VERTEX_TERMINATING = 6;
+}
+
+message VertexStatusProto {
+  optional VertexStatusStateProto state = 1;
+  repeated string diagnostics = 2;
+  optional ProgressProto progress = 3;
+}
+
+enum DAGStatusStateProto {
+  DAG_SUBMITTED = 0;
+  DAG_INITING = 1;
+  DAG_RUNNING = 2;
+  DAG_SUCCEEDED = 3;
+  DAG_KILLED = 4;
+  DAG_FAILED = 5;
+  DAG_ERROR = 6;
+  DAG_TERMINATING = 7;
+}
+
+message StringProgressPairProto {
+  required string key = 1;
+  required ProgressProto progress = 2;
+}
+
+message DAGStatusProto {
+  optional DAGStatusStateProto state = 1;
+  repeated string diagnostics = 2;
+  optional ProgressProto DAGProgress = 3;
+  repeated StringProgressPairProto vertexProgress = 4;  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/proto/DAGClientAMProtocol.proto
----------------------------------------------------------------------
diff --git a/tez-api/src/main/proto/DAGClientAMProtocol.proto b/tez-api/src/main/proto/DAGClientAMProtocol.proto
new file mode 100644
index 0000000..6fcd1f8
--- /dev/null
+++ b/tez-api/src/main/proto/DAGClientAMProtocol.proto
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+option java_package = "org.apache.tez.dag.api.client.rpc";
+option java_outer_classname = "DAGClientAMProtocolRPC";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+
+//import "DAGClientAMProtocolRecords.proto";
+
+import "DAGApiRecords.proto";
+
+message GetAllDAGsRequestProto {
+}
+
+message GetAllDAGsResponseProto {
+  repeated string dagId = 1;
+}
+
+message GetDAGStatusRequestProto {
+  optional string dagId = 1;
+}
+
+message GetDAGStatusResponseProto {
+  optional DAGStatusProto dagStatus = 1;
+}
+
+message GetVertexStatusRequestProto {
+  optional string dagId = 1;
+  optional string vertexName = 2;
+}
+
+message GetVertexStatusResponseProto {
+  optional VertexStatusProto vertexStatus = 1;
+}
+
+message TryKillDAGRequestProto {
+  optional string dagId = 1;
+}
+
+message TryKillDAGResponseProto {
+  //nothing yet
+}
+
+message SubmitDAGRequestProto {
+  optional DAGPlan d_a_g_plan = 1;
+}
+
+message SubmitDAGResponseProto {
+  optional string dagId = 1;
+}
+
+message ShutdownSessionRequestProto {
+}
+
+message ShutdownSessionResponseProto {
+}
+
+service DAGClientAMProtocol {
+  rpc getAllDAGs (GetAllDAGsRequestProto) returns (GetAllDAGsResponseProto);
+  rpc getDAGStatus (GetDAGStatusRequestProto) returns (GetDAGStatusResponseProto);
+  rpc getVertexStatus (GetVertexStatusRequestProto) returns (GetVertexStatusResponseProto);
+  rpc tryKillDAG (TryKillDAGRequestProto) returns (TryKillDAGResponseProto);
+  rpc submitDAG (SubmitDAGRequestProto) returns (SubmitDAGResponseProto);
+  rpc shutdownSession (ShutdownSessionRequestProto) returns (ShutdownSessionResponseProto);
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/proto/Events.proto
----------------------------------------------------------------------
diff --git a/tez-api/src/main/proto/Events.proto b/tez-api/src/main/proto/Events.proto
new file mode 100644
index 0000000..21cacf6
--- /dev/null
+++ b/tez-api/src/main/proto/Events.proto
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+option java_package = "org.apache.tez.engine.api.events";
+option java_outer_classname = "EventProtos";
+option java_generate_equals_and_hash = true;
+
+message DataMovementEventProto {
+  optional int32 source_index = 1;
+  optional int32 target_index = 2;
+  optional bytes user_payload = 3;
+  optional int32 version = 4;
+}
+
+message InputReadErrorEventProto {
+  optional int32 index = 1;
+  optional string diagnostics = 2;
+  optional int32 version = 3;
+}
+
+message InputFailedEventProto {
+  optional int32 source_index = 1;
+  optional int32 target_index = 2;
+  optional int32 version = 4;
+}
+
+message InputInformationEventProto {
+  optional bytes user_payload = 1;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
new file mode 100644
index 0000000..53ec357
--- /dev/null
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+import static org.junit.Assert.*;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
+import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
+import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
+import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
+import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+// based on TestDAGLocationHint
+public class TestDAGPlan {
+  @Rule
+  public TemporaryFolder tempFolder = new TemporaryFolder(); //TODO: doesn't seem to be deleting this folder automatically as expected.
+
+  @Test
+  public void testBasicJobPlanSerde() throws IOException {
+
+    DAGPlan job = DAGPlan.newBuilder()
+       .setName("test")
+       .addVertex(
+           VertexPlan.newBuilder()
+             .setName("vertex1")
+             .setType(PlanVertexType.NORMAL)
+             .addTaskLocationHint(PlanTaskLocationHint.newBuilder().addHost("machineName").addRack("rack1").build())
+             .setTaskConfig(
+                 PlanTaskConfiguration.newBuilder()
+                   .setNumTasks(2)
+                   .setVirtualCores(4)
+                   .setMemoryMb(1024)
+                   .setJavaOpts("")
+                   .setTaskModule("x.y")
+                   .build())
+             .build())
+        .build();
+   File file = tempFolder.newFile("jobPlan");
+   FileOutputStream outStream = null;
+   try {
+     outStream = new FileOutputStream(file);
+     job.writeTo(outStream);
+   }
+   finally {
+     if(outStream != null){
+       outStream.close();
+     }
+   }
+
+   DAGPlan inJob;
+   FileInputStream inputStream;
+   try {
+     inputStream = new FileInputStream(file);
+     inJob = DAGPlan.newBuilder().mergeFrom(inputStream).build();
+   }
+   finally {
+     outStream.close();
+   }
+
+   Assert.assertEquals(job, inJob);
+  }
+
+  @Test
+  public void testUserPayloadSerde() {
+    DAG dag = new DAG("testDag");
+    ProcessorDescriptor pd1 = new ProcessorDescriptor("processor1").
+        setUserPayload("processor1Bytes".getBytes());
+    ProcessorDescriptor pd2 = new ProcessorDescriptor("processor2").
+        setUserPayload("processor2Bytes".getBytes());
+    Vertex v1 = new Vertex("v1", pd1, 10, Resource.newInstance(1024, 1));
+    Vertex v2 = new Vertex("v2", pd2, 1, Resource.newInstance(1024, 1));
+    v1.setJavaOpts("").setTaskEnvironment(new HashMap<String, String>())
+        .setTaskLocalResources(new HashMap<String, LocalResource>());
+    v2.setJavaOpts("").setTaskEnvironment(new HashMap<String, String>())
+        .setTaskLocalResources(new HashMap<String, LocalResource>());
+
+    InputDescriptor inputDescriptor = new InputDescriptor("input").
+        setUserPayload("inputBytes".getBytes());
+    OutputDescriptor outputDescriptor = new OutputDescriptor("output").
+        setUserPayload("outputBytes".getBytes());
+    Edge edge = new Edge(v1, v2, new EdgeProperty(
+        DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
+        SchedulingType.SEQUENTIAL, outputDescriptor, inputDescriptor));
+
+    dag.addVertex(v1).addVertex(v2).addEdge(edge);
+
+    DAGPlan dagProto = dag.createDag(new TezConfiguration());
+
+    assertEquals(2, dagProto.getVertexCount());
+    assertEquals(1, dagProto.getEdgeCount());
+
+    VertexPlan v1Proto = dagProto.getVertex(0);
+    VertexPlan v2Proto = dagProto.getVertex(1);
+    EdgePlan edgeProto = dagProto.getEdge(0);
+
+    assertEquals("processor1Bytes", new String(v1Proto.getProcessorDescriptor()
+        .getUserPayload().toByteArray()));
+    assertEquals("processor1", v1Proto.getProcessorDescriptor().getClassName());
+
+    assertEquals("processor2Bytes", new String(v2Proto.getProcessorDescriptor()
+        .getUserPayload().toByteArray()));
+    assertEquals("processor2", v2Proto.getProcessorDescriptor().getClassName());
+
+    assertEquals("inputBytes", new String(edgeProto.getEdgeDestination()
+        .getUserPayload().toByteArray()));
+    assertEquals("input", edgeProto.getEdgeDestination().getClassName());
+
+    assertEquals("outputBytes", new String(edgeProto.getEdgeSource()
+        .getUserPayload().toByteArray()));
+    assertEquals("output", edgeProto.getEdgeSource().getClassName());
+
+    EdgeProperty edgeProperty = DagTypeConverters
+        .createEdgePropertyMapFromDAGPlan(dagProto.getEdgeList().get(0));
+
+    byte[] ib = edgeProperty.getEdgeDestination().getUserPayload();
+    assertEquals("inputBytes", new String(ib));
+    assertEquals("input", edgeProperty.getEdgeDestination().getClassName());
+
+    byte[] ob = edgeProperty.getEdgeSource().getUserPayload();
+    assertEquals("outputBytes", new String(ob));
+    assertEquals("output", edgeProperty.getEdgeSource().getClassName());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
new file mode 100644
index 0000000..b33f3a6
--- /dev/null
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
@@ -0,0 +1,417 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
+import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
+import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestDAGVerify {
+
+  private final String dummyProcessorClassName = TestDAGVerify.class.getName();
+  private final String dummyInputClassName = TestDAGVerify.class.getName();
+  private final String dummyOutputClassName = TestDAGVerify.class.getName();
+  private final int dummyTaskCount = 2;
+  private final Resource dummyTaskResource = Resource.newInstance(1, 1);
+
+  //    v1
+  //    |
+  //    v2
+  @Test
+  public void testVerify1() {
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor(dummyProcessorClassName),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor(dummyOutputClassName),
+            new InputDescriptor(dummyInputClassName)));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addEdge(e1);
+    dag.verify();
+  }
+
+  @Test(expected = IllegalStateException.class)  
+  public void testVerify2() {
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor(dummyProcessorClassName),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.ONE_TO_ONE, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor(dummyOutputClassName),
+            new InputDescriptor(dummyInputClassName)));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addEdge(e1);
+    dag.verify();
+  }
+
+  @Test(expected = IllegalStateException.class)  
+  public void testVerify3() {
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor(dummyProcessorClassName),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.EPHEMERAL, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor(dummyOutputClassName),
+            new InputDescriptor(dummyInputClassName)));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addEdge(e1);
+    dag.verify();
+  }
+
+  @Test(expected = IllegalStateException.class)  
+  public void testVerify4() {
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor(dummyProcessorClassName),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.EPHEMERAL, SchedulingType.CONCURRENT, 
+            new OutputDescriptor(dummyOutputClassName),
+            new InputDescriptor(dummyInputClassName)));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addEdge(e1);
+    dag.verify();
+  }
+
+  //    v1 <----
+  //      |     ^
+  //       v2   ^
+  //      |  |  ^
+  //    v3    v4
+  @Test
+  public void testCycle1() {
+    IllegalStateException ex=null;
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v3 = new Vertex("v3",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v4 = new Vertex("v4",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e2 = new Edge(v2, v3,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e3 = new Edge(v2, v4,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e4 = new Edge(v4, v1,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addVertex(v3);
+    dag.addVertex(v4);
+    dag.addEdge(e1);
+    dag.addEdge(e2);
+    dag.addEdge(e3);
+    dag.addEdge(e4);
+    try{
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage().startsWith("DAG contains a cycle"));
+  }
+
+  //     v1
+  //      |
+  //    -> v2
+  //    ^  | |
+  //    v3    v4
+  @Test
+  public void testCycle2() {
+    IllegalStateException ex=null;
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v3 = new Vertex("v3",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v4 = new Vertex("v4",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e2 = new Edge(v2, v3,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e3 = new Edge(v2, v4,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e4 = new Edge(v3, v2,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addVertex(v3);
+    dag.addVertex(v4);
+    dag.addEdge(e1);
+    dag.addEdge(e2);
+    dag.addEdge(e3);
+    dag.addEdge(e4);
+    try{
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage().startsWith("DAG contains a cycle"));
+  }
+
+  @Test
+  public void repeatedVertexName() {
+    IllegalStateException ex=null;
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v1repeat = new Vertex("v1",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v1repeat);
+    try {
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage().startsWith("DAG contains multiple vertices with name"));
+  }
+
+  //  v1  v2
+  //   |  |
+  //    v3
+  @Test
+  public void BinaryInputDisallowed() {
+    IllegalStateException ex=null;
+    try {
+      Vertex v1 = new Vertex("v1",
+          new ProcessorDescriptor("MapProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Vertex v2 = new Vertex("v2",
+          new ProcessorDescriptor("MapProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Vertex v3 = new Vertex("v3",
+          new ProcessorDescriptor("ReduceProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Edge e1 = new Edge(v1, v3,
+          new EdgeProperty(DataMovementType.ONE_TO_ONE, 
+              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+              new OutputDescriptor("dummy output class"),
+              new InputDescriptor("dummy input class")));
+      Edge e2 = new Edge(v2, v3,
+          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+              new OutputDescriptor("dummy output class"),
+              new InputDescriptor("dummy input class")));
+      DAG dag = new DAG("testDag");
+      dag.addVertex(v1);
+      dag.addVertex(v2);
+      dag.addVertex(v3);
+      dag.addEdge(e1);
+      dag.addEdge(e2);
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage().startsWith(
+        "Unsupported connection pattern on edge"));
+  }
+
+  //  v1  v2
+  //   |  |
+  //    v3
+  @Test
+  public void BinaryInputAllowed() {
+    Vertex v1 = new Vertex("v1",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v2 = new Vertex("v2",
+        new ProcessorDescriptor("MapProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Vertex v3 = new Vertex("v3",
+        new ProcessorDescriptor("ReduceProcessor"),
+        dummyTaskCount, dummyTaskResource);
+    Edge e1 = new Edge(v1, v3,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL,
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    Edge e2 = new Edge(v2, v3,
+        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+            new OutputDescriptor("dummy output class"),
+            new InputDescriptor("dummy input class")));
+    DAG dag = new DAG("testDag");
+    dag.addVertex(v1);
+    dag.addVertex(v2);
+    dag.addVertex(v3);
+    dag.addEdge(e1);
+    dag.addEdge(e2);
+    dag.verify();
+  }
+
+  //   v1
+  //  |  |
+  //  v2  v3
+  @Test
+  public void BinaryOutput() {
+    IllegalStateException ex=null;
+    try {
+      Vertex v1 = new Vertex("v1",
+          new ProcessorDescriptor("MapProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Vertex v2 = new Vertex("v2",
+          new ProcessorDescriptor("MapProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Vertex v3 = new Vertex("v3",
+          new ProcessorDescriptor("MapProcessor"),
+          dummyTaskCount, dummyTaskResource);
+      Edge e1 = new Edge(v1, v2,
+          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+              new OutputDescriptor("dummy output class"),
+              new InputDescriptor("dummy input class")));
+      Edge e2 = new Edge(v1, v2,
+          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
+              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+              new OutputDescriptor("dummy output class"),
+              new InputDescriptor("dummy input class")));
+      DAG dag = new DAG("testDag");
+      dag.addVertex(v1);
+      dag.addVertex(v2);
+      dag.addVertex(v3);
+      dag.addEdge(e1);
+      dag.addEdge(e2);
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage().startsWith("Vertex has outDegree>1"));
+  }
+
+  @Test
+  public void testDagWithNoVertices() {
+    IllegalStateException ex=null;
+    try {
+      DAG dag = new DAG("testDag");
+      dag.verify();
+    }
+    catch (IllegalStateException e){
+      ex = e;
+    }
+    Assert.assertNotNull(ex);
+    System.out.println(ex.getMessage());
+    Assert.assertTrue(ex.getMessage()
+        .startsWith("Invalid dag containing 0 vertices"));
+  }
+
+  @SuppressWarnings("unused")
+  @Test
+  public void testInvalidVertexConstruction() {
+    try {
+      Vertex v1 = new Vertex("v1",
+          new ProcessorDescriptor("MapProcessor"),
+          0, dummyTaskResource);
+      Assert.fail("Expected exception for 0 parallelism");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Parallelism cannot be 0"));
+    }
+    try {
+      Vertex v1 = new Vertex("v1",
+          new ProcessorDescriptor("MapProcessor"),
+          1, null);
+      Assert.fail("Expected exception for 0 parallelism");
+    } catch (IllegalArgumentException e) {
+      Assert.assertTrue(e.getMessage().startsWith("Resource cannot be null"));
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/pom.xml
----------------------------------------------------------------------
diff --git a/tez-common/pom.xml b/tez-common/pom.xml
index 87f18d6..b4882bb 100644
--- a/tez-common/pom.xml
+++ b/tez-common/pom.xml
@@ -39,7 +39,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-dag-api</artifactId>
+      <artifactId>tez-api</artifactId>
     </dependency>
   </dependencies>
 


[04/20] TEZ-444. Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
index 6817151..1e0b146 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
@@ -32,23 +32,29 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobContext;
 import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
 import org.apache.hadoop.mapreduce.split.SplitMetaInfoReaderTez;
 import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
 import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezEngineTaskContext;
+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.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.mapreduce.processor.MRTask;
-import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.KVReader;
+import org.apache.tez.engine.newapi.LogicalInput;
+import org.apache.tez.engine.newapi.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 com.google.common.base.Preconditions;
 
 /**
  * {@link SimpleInput} is an {@link Input} which provides key/values pairs
@@ -57,161 +63,152 @@ import org.apache.tez.mapreduce.processor.MRTaskReporter;
  * It is compatible with all standard Apache Hadoop MapReduce 
  * {@link InputFormat} implementations.
  */
-@SuppressWarnings({ "unchecked", "rawtypes" })
-public class SimpleInput implements Input {
+
+public class SimpleInput implements LogicalInput {
 
   private static final Log LOG = LogFactory.getLog(SimpleInput.class);
   
-  MRTask task;
   
-  boolean useNewApi;
+  private TezInputContext inputContext;
   
-  JobConf jobConf;
+  private JobConf jobConf;
+  private Configuration incrementalConf;
+  private boolean recordReaderCreated = false;
+  
+  boolean useNewApi;
   
   org.apache.hadoop.mapreduce.TaskAttemptContext taskAttemptContext;
 
-  org.apache.hadoop.mapreduce.InputFormat newInputFormat;
-  org.apache.hadoop.mapreduce.RecordReader newRecordReader;
+  @SuppressWarnings("rawtypes")
+  private org.apache.hadoop.mapreduce.InputFormat newInputFormat;
+  @SuppressWarnings("rawtypes")
+  private org.apache.hadoop.mapreduce.RecordReader newRecordReader;
+  protected org.apache.hadoop.mapreduce.InputSplit newInputSplit;
   
-  org.apache.hadoop.mapred.InputFormat oldInputFormat;
-  org.apache.hadoop.mapred.RecordReader oldRecordReader;
+  @SuppressWarnings("rawtypes")
+  private InputFormat oldInputFormat;
+  @SuppressWarnings("rawtypes")
+  protected RecordReader oldRecordReader;
 
   protected TaskSplitIndex splitMetaInfo = new TaskSplitIndex();
   
-  Object key;
-  Object value;
-  
   private TezCounter inputRecordCounter;
   private TezCounter fileInputByteCounter; 
   private List<Statistics> fsStats;
-  private MRTaskReporter reporter;
 
-  public SimpleInput(TezEngineTaskContext task, int index)
-  {}
-  
-  public void setTask(MRTask task) {
-    this.task = task;
-  }
+  @Override
+  public List<Event> initialize(TezInputContext inputContext) throws IOException {
+    this.inputContext = inputContext;
+    Configuration conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
+    this.jobConf = new JobConf(conf);
 
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-    if (task == null) {
-      return;
-    }
-    
-    if (conf instanceof JobConf) {
-      jobConf = (JobConf)conf;
-    } else {
-      jobConf = new JobConf(conf);
-    }
-    
     // Read split information.
-    TaskSplitMetaInfo[] allMetaInfo = readSplits(jobConf);
-    TaskSplitMetaInfo thisTaskMetaInfo = allMetaInfo[task.getTaskAttemptId()
-        .getTaskID().getId()];
-    splitMetaInfo = new TaskSplitIndex(thisTaskMetaInfo.getSplitLocation(),
+    TaskSplitMetaInfo[] allMetaInfo = readSplits(conf);
+    TaskSplitMetaInfo thisTaskMetaInfo = allMetaInfo[inputContext.getTaskIndex()];
+    this.splitMetaInfo = new TaskSplitIndex(thisTaskMetaInfo.getSplitLocation(),
         thisTaskMetaInfo.getStartOffset());
     
+    // TODO NEWTEZ Rename this to be specific to SimpleInput. This Input, in
+    // theory, can be used by the MapProcessor, ReduceProcessor or a custom
+    // processor. (The processor could provide the counter though)
+    this.inputRecordCounter = inputContext.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS);
+    this.fileInputByteCounter = inputContext.getCounters().findCounter(FileInputFormatCounter.BYTES_READ);
     
-    useNewApi = jobConf.getUseNewMapper();
-    taskAttemptContext = task.getTaskAttemptContext();
-    
-    inputRecordCounter = task.getInputRecordsCounter();
-    fileInputByteCounter = task.getFileInputBytesCounter();
-
-    reporter = task.getMRReporter();
+    useNewApi = this.jobConf.getUseNewMapper();
 
     if (useNewApi) {
+      TaskAttemptContext taskAttemptContext = createTaskAttemptContext();
+      Class<? extends org.apache.hadoop.mapreduce.InputFormat<?, ?>> inputFormatClazz;
       try {
-        newInputFormat = 
-            ReflectionUtils.newInstance(
-                taskAttemptContext.getInputFormatClass(), jobConf);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException(cnfe);
+        inputFormatClazz = taskAttemptContext.getInputFormatClass();
+      } catch (ClassNotFoundException e) {
+        throw new IOException("Unable to instantiate InputFormat class", e);
       }
-      
+
+      newInputFormat = ReflectionUtils.newInstance(inputFormatClazz, this.jobConf);
+
       newInputSplit = getNewSplitDetails(splitMetaInfo);
+
       List<Statistics> matchedStats = null;
       if (newInputSplit instanceof org.apache.hadoop.mapreduce.lib.input.FileSplit) {
-        matchedStats = MRTask.getFsStatistics(
+        matchedStats = Utils.getFsStatistics(
             ((org.apache.hadoop.mapreduce.lib.input.FileSplit)
-                newInputSplit).getPath(), jobConf);
+                newInputSplit).getPath(), this.jobConf);
       }
       fsStats = matchedStats;
-      newRecordReader = 
-          newInputFormat.createRecordReader(newInputSplit, taskAttemptContext);
-    } else {
-      oldInputFormat = jobConf.getInputFormat();
-      org.apache.hadoop.mapred.InputSplit oldInputSplit =
+      
+      try {
+        newRecordReader = newInputFormat.createRecordReader(newInputSplit, taskAttemptContext);
+        newRecordReader.initialize(newInputSplit, taskAttemptContext);
+      } catch (InterruptedException e) {
+        throw new IOException("Interrupted while creating record reader", e);
+      }
+    } else { // OLD API
+      oldInputFormat = this.jobConf.getInputFormat();
+      InputSplit oldInputSplit =
           getOldSplitDetails(splitMetaInfo);
       
+      
       List<Statistics> matchedStats = null;
       if (oldInputSplit instanceof FileSplit) {
-        matchedStats = 
-            MRTask.getFsStatistics(
-                ((FileSplit)oldInputSplit).getPath(), jobConf);
+        matchedStats = Utils.getFsStatistics(((FileSplit) oldInputSplit).getPath(), this.jobConf);
       }
       fsStats = matchedStats;
-
+      
       long bytesInPrev = getInputBytes();
-      oldRecordReader = 
-          jobConf.getInputFormat().getRecordReader(
-              oldInputSplit, jobConf, reporter);
+      oldRecordReader = oldInputFormat.getRecordReader(oldInputSplit,
+          this.jobConf, new MRReporter(inputContext, oldInputSplit));
       long bytesInCurr = getInputBytes();
       fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-
-      updateJobWithSplit(jobConf, oldInputSplit);
-    }
+      setIncrementalConfigParams(oldInputSplit);
+    }    
+    return null;
   }
 
-  public boolean hasNext() throws IOException, InterruptedException {
-    boolean hasNext = false;
-    long bytesInPrev = getInputBytes();
+  @Override
+  public KVReader getReader() throws IOException {
+    Preconditions
+        .checkState(recordReaderCreated == false,
+            "Only a single instance of record reader can be created for this input.");
+    recordReaderCreated = true;
+    return new MRInputKVReader();
+  }
 
-    if (useNewApi) { 
-        hasNext = newRecordReader.nextKeyValue();
-    } else {
-      hasNext = oldRecordReader.next(key, value);
-    }
-    
-    long bytesInCurr = getInputBytes();
-    fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-    reporter.setProgress(getProgress());
 
-    if (hasNext) {
-      inputRecordCounter.increment(1);
-    }
-    
-    return hasNext;
+  @Override
+  public void handleEvents(List<Event> inputEvents) {
+    // Not expecting any events at the moment.
   }
 
-  private SimpleValueIterator vIter = new SimpleValueIterator();
-  private SimpleIterable valuesIterable =
-      new SimpleIterable(vIter);
-
-  private org.apache.hadoop.mapreduce.InputSplit newInputSplit;
 
-  public void setKey(Object key) {
-    this.key = key;
-  }
-  
-  public void setValue(Object value) {
-    this.value = value;
+  @Override
+  public void setNumPhysicalInputs(int numInputs) {
+    // Not required at the moment. May be required if splits are sent via events.
   }
 
-  public Object getNextKey() throws IOException, InterruptedException {
+  @Override
+  public List<Event> close() throws IOException {
+    long bytesInPrev = getInputBytes();
     if (useNewApi) {
-      return newRecordReader.getCurrentKey();
+      newRecordReader.close();
     } else {
-      return key;
+      oldRecordReader.close();
     }
+    long bytesInCurr = getInputBytes();
+    fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
+    
+    return null;
   }
 
-  public Iterable getNextValues() throws IOException,
-      InterruptedException {
-    value = newRecordReader.getCurrentValue();
-    vIter.setValue(value);
-    return valuesIterable;
+  /**
+   * SimpleInputs sets some additional parameters like split location when using
+   * the new API. This methods returns the list of additional updates, and
+   * should be used by Processors using the old MapReduce API with SimpleInput.
+   * 
+   * @return the additional fields set by SimpleInput
+   */
+  public Configuration getConfigUpdates() {
+    return new Configuration(incrementalConf);
   }
 
   public float getProgress() throws IOException, InterruptedException {
@@ -222,27 +219,22 @@ public class SimpleInput implements Input {
     }
   }
 
-  public void close() throws IOException {
-    long bytesInPrev = getInputBytes();
-    if (useNewApi) {
-      newRecordReader.close();
-    } else {
-      oldRecordReader.close();
-    }
-    long bytesInCurr = getInputBytes();
-    fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
+  
+  private TaskAttemptContext createTaskAttemptContext() {
+    return new TaskAttemptContextImpl(this.jobConf, inputContext, true);
   }
+  
 
-  static class SimpleValueIterator implements Iterator {
+  private static class SimpleValueIterator implements Iterator<Object> {
 
     private Object value;
-    
+
     public void setValue(Object value) {
       this.value = value;
     }
-    
+
     public boolean hasNext() {
-      return false;
+      return value != null;
     }
 
     public Object next() {
@@ -256,28 +248,23 @@ public class SimpleInput implements Input {
     }
   }
 
-  static class SimpleIterable implements Iterable {
-    private final Iterator iterator;
-    public SimpleIterable(Iterator iterator) {
+  private static class SimpleIterable implements Iterable<Object> {
+    private final Iterator<Object> iterator;
+    public SimpleIterable(Iterator<Object> iterator) {
       this.iterator = iterator;
     }
-    
-    public Iterator iterator() {
+
+    @Override
+    public Iterator<Object> iterator() {
       return iterator;
     }
   }
-  
 
-  public RecordReader getOldRecordReader() {
-    return oldRecordReader;
-  }
-  
-  public org.apache.hadoop.mapreduce.RecordReader getNewRecordReader() {
-    return newRecordReader;
-  }
+
+
   
-  public org.apache.hadoop.mapred.InputSplit 
-  getOldSplitDetails(TaskSplitIndex splitMetaInfo) 
+  @SuppressWarnings("unchecked")
+  private InputSplit getOldSplitDetails(TaskSplitIndex splitMetaInfo)
       throws IOException {
     Path file = new Path(splitMetaInfo.getSplitLocation());
     FileSystem fs = FileSystem.getLocal(jobConf);
@@ -306,14 +293,15 @@ public class SimpleInput implements Input {
     deserializer.open(inFile);
     org.apache.hadoop.mapred.InputSplit split = deserializer.deserialize(null);
     long pos = inFile.getPos();
-    reporter.getCounter(TaskCounter.SPLIT_RAW_BYTES).increment(pos - offset);
+    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
+        .increment(pos - offset);
     inFile.close();
     return split;
   }
 
-  public org.apache.hadoop.mapreduce.InputSplit 
-  getNewSplitDetails(TaskSplitIndex splitMetaInfo) 
-      throws IOException {
+  @SuppressWarnings("unchecked")
+  private org.apache.hadoop.mapreduce.InputSplit getNewSplitDetails(
+      TaskSplitIndex splitMetaInfo) throws IOException {
     Path file = new Path(splitMetaInfo.getSplitLocation());
     long offset = splitMetaInfo.getStartOffset();
     
@@ -343,17 +331,23 @@ public class SimpleInput implements Input {
     org.apache.hadoop.mapreduce.InputSplit split = 
         deserializer.deserialize(null);
     long pos = inFile.getPos();
-    reporter.getCounter(TaskCounter.SPLIT_RAW_BYTES).increment(pos - offset);
+    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
+        .increment(pos - offset);
     inFile.close();
     return split;
   }
 
-  private void updateJobWithSplit(final JobConf job, InputSplit inputSplit) {
+  private void setIncrementalConfigParams(InputSplit inputSplit) {
     if (inputSplit instanceof FileSplit) {
       FileSplit fileSplit = (FileSplit) inputSplit;
-      job.set(JobContext.MAP_INPUT_FILE, fileSplit.getPath().toString());
-      job.setLong(JobContext.MAP_INPUT_START, fileSplit.getStart());
-      job.setLong(JobContext.MAP_INPUT_PATH, fileSplit.getLength());
+      this.incrementalConf = new Configuration(false);
+
+      this.incrementalConf.set(JobContext.MAP_INPUT_FILE, fileSplit.getPath()
+          .toString());
+      this.incrementalConf.setLong(JobContext.MAP_INPUT_START,
+          fileSplit.getStart());
+      this.incrementalConf.setLong(JobContext.MAP_INPUT_PATH,
+          fileSplit.getLength());
     }
     LOG.info("Processing split: " + inputSplit);
   }
@@ -367,16 +361,6 @@ public class SimpleInput implements Input {
     return bytesRead;
   }
 
-  public void initializeNewRecordReader(
-      org.apache.hadoop.mapreduce.InputSplit split, TaskAttemptContext context) 
-  throws IOException, InterruptedException {
-    newRecordReader.initialize(split, context);
-  }
-  
-  public org.apache.hadoop.mapreduce.InputSplit getNewInputSplit() {
-    return newInputSplit;
-  }
-
   protected TaskSplitMetaInfo[] readSplits(Configuration conf)
       throws IOException {
     TaskSplitMetaInfo[] allTaskSplitMetaInfo;
@@ -384,4 +368,71 @@ public class SimpleInput implements Input {
         FileSystem.getLocal(conf));
     return allTaskSplitMetaInfo;
   }
+  
+  private class MRInputKVReader implements KVReader {
+    
+    Object key;
+    Object value;
+
+    private SimpleValueIterator valueIterator = new SimpleValueIterator();
+    private SimpleIterable valueIterable = new SimpleIterable(valueIterator);
+
+    private final boolean localNewApi;
+    
+    MRInputKVReader() {
+      localNewApi = useNewApi;
+      if (!localNewApi) {
+        key = oldRecordReader.createKey();
+        value =oldRecordReader.createValue();
+      }
+    }
+    
+    // Setup the values iterator once, and set value on the same object each time
+    // to prevent lots of objects being created.
+
+    
+    @SuppressWarnings("unchecked")
+    @Override
+    public boolean next() throws IOException {
+      boolean hasNext = false;
+      long bytesInPrev = getInputBytes();
+      if (localNewApi) {
+        try {
+          hasNext = newRecordReader.nextKeyValue();
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException("Interrupted while checking for next key-value", e);
+        }
+      } else {
+        hasNext = oldRecordReader.next(key, value);
+      }
+      long bytesInCurr = getInputBytes();
+      fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
+      
+      if (hasNext) {
+        inputRecordCounter.increment(1);
+      }
+      
+      return hasNext;
+    }
+
+    @Override
+    public KVRecord getCurrentKV() throws IOException {
+      KVRecord kvRecord = null;
+      if (localNewApi) {
+        try {
+          valueIterator.setValue(newRecordReader.getCurrentValue());
+          kvRecord = new KVRecord(newRecordReader.getCurrentKey(), valueIterable);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          throw new IOException("Interrupted while fetching next key-value", e);
+        }
+        
+      } else {
+        valueIterator.setValue(value);
+        kvRecord = new KVRecord(key, valueIterable);
+      }
+      return kvRecord;
+    }
+  };
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java
new file mode 100644
index 0000000..4e61aa7
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInputLegacy.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.mapreduce.input;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.mapred.RecordReader;
+
+public class SimpleInputLegacy extends SimpleInput {
+
+  @Private
+  public org.apache.hadoop.mapreduce.InputSplit getNewInputSplit() {
+    return this.newInputSplit;
+  }  
+  
+  @SuppressWarnings("rawtypes")
+  @Private
+  public RecordReader getOldRecordReader() {
+    return this.oldRecordReader;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
deleted file mode 100644
index 5566fd8..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newcombine/MRCombiner.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.newcombine;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.RawKeyValueIterator;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.ReduceContext;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskCounter;
-import org.apache.hadoop.mapreduce.TaskID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
-import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
-import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.common.counters.TezCounter;
-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.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
-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.newprocessor.MRTaskReporter;
-
-@SuppressWarnings({"rawtypes", "unchecked"})
-public class MRCombiner implements Combiner {
-
-  private static Log LOG = LogFactory.getLog(MRCombiner.class);
-  
-  private final Configuration conf;
-  private final Class<?> keyClass;
-  private final Class<?> valClass;
-  private final RawComparator<?> comparator;
-  private final boolean useNewApi;
-  
-  private final TezCounter combineInputKeyCounter;
-  private final TezCounter combineInputValueCounter;
-  
-  private final MRTaskReporter reporter;
-  private final TaskAttemptID mrTaskAttemptID;
-
-  public MRCombiner(TezTaskContext taskContext) throws IOException {
-    this.conf = TezUtils.createConfFromUserPayload(taskContext.getUserPayload());
-
-    assert(taskContext instanceof TezInputContext || taskContext instanceof TezOutputContext);
-    if (taskContext instanceof TezOutputContext) {
-      this.keyClass = ConfigUtils.getIntermediateOutputKeyClass(conf);
-      this.valClass = ConfigUtils.getIntermediateOutputValueClass(conf);
-      this.comparator = ConfigUtils.getIntermediateOutputKeyComparator(conf);
-      this.reporter = new MRTaskReporter((TezOutputContext)taskContext);
-    } else {
-      this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
-      this.valClass = ConfigUtils.getIntermediateInputValueClass(conf);
-      this.comparator = ConfigUtils.getIntermediateInputKeyComparator(conf);
-      this.reporter = new MRTaskReporter((TezInputContext)taskContext);
-    }
-
-    this.useNewApi = ConfigUtils.useNewApi(conf);
-    
-    combineInputKeyCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
-    combineInputValueCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_OUTPUT_RECORDS);
-    
-    boolean isMap = conf.getBoolean(MRConfig.IS_MAP_PROCESSOR,false);
-    this.mrTaskAttemptID = new TaskAttemptID(
-        new TaskID(String.valueOf(taskContext.getApplicationId()
-            .getClusterTimestamp()), taskContext.getApplicationId().getId(),
-            isMap ? TaskType.MAP : TaskType.REDUCE,
-            taskContext.getTaskIndex()), taskContext.getTaskAttemptNumber());
-    
-    LOG.info("Using combineKeyClass: " + keyClass + ", combineValueClass: " + valClass + ", combineComparator: " +comparator + ", useNewApi: " + useNewApi);
-  }
-
-  @Override
-  public void combine(TezRawKeyValueIterator rawIter, Writer writer)
-      throws InterruptedException, IOException {
-    if (useNewApi) {
-      runNewCombiner(rawIter, writer);
-    } else {
-      runOldCombiner(rawIter, writer);
-    }
-    
-  }
-
-  ///////////////// Methods for old API //////////////////////
-  
-  private void runOldCombiner(final TezRawKeyValueIterator rawIter, final Writer writer) throws IOException {
-    Class<? extends Reducer> reducerClazz = (Class<? extends Reducer>) conf.getClass("mapred.combiner.class", null, Reducer.class);
-    
-    Reducer combiner = ReflectionUtils.newInstance(reducerClazz, conf);
-    
-    OutputCollector collector = new OutputCollector() {
-      @Override
-      public void collect(Object key, Object value) throws IOException {
-        writer.append(key, value);
-      }
-    };
-    
-    CombinerValuesIterator values = new CombinerValuesIterator(rawIter, keyClass, valClass, comparator);
-    
-    while (values.moveToNext()) {
-      combiner.reduce(values.getKey(), values.getValues().iterator(), collector, reporter);
-    }
-  }
-  
-  private final class CombinerValuesIterator<KEY,VALUE> extends ValuesIterator<KEY, VALUE> {
-    public CombinerValuesIterator(TezRawKeyValueIterator rawIter,
-        Class<KEY> keyClass, Class<VALUE> valClass,
-        RawComparator<KEY> comparator) throws IOException {
-      super(rawIter, comparator, keyClass, valClass, conf,
-          combineInputKeyCounter, combineInputValueCounter);
-    }
-  }
-  
-  ///////////////// End of methods for old API //////////////////////
-  
-  ///////////////// Methods for new API //////////////////////
-  
-  private void runNewCombiner(final TezRawKeyValueIterator rawIter, final Writer writer) throws InterruptedException, IOException {
-    
-    RecordWriter recordWriter = new RecordWriter() {
-
-      @Override
-      public void write(Object key, Object value) throws IOException,
-          InterruptedException {
-        writer.append(key, value);
-      }
-
-      @Override
-      public void close(TaskAttemptContext context) throws IOException,
-          InterruptedException {
-        // Will be closed by whoever invokes the combiner.
-      }
-    };
-    
-    Class<? extends org.apache.hadoop.mapreduce.Reducer> reducerClazz = (Class<? extends org.apache.hadoop.mapreduce.Reducer>) conf
-        .getClass(MRJobConfig.COMBINE_CLASS_ATTR, null,
-            org.apache.hadoop.mapreduce.Reducer.class);
-    org.apache.hadoop.mapreduce.Reducer reducer = ReflectionUtils.newInstance(reducerClazz, conf);
-    
-    org.apache.hadoop.mapreduce.Reducer.Context reducerContext =
-        createReduceContext(
-            conf,
-            mrTaskAttemptID,
-            rawIter,
-            new MRCounters.MRCounter(combineInputKeyCounter),
-            new MRCounters.MRCounter(combineInputValueCounter),
-            recordWriter,
-            reporter,
-            (RawComparator)comparator,
-            keyClass,
-            valClass);
-    
-    reducer.run(reducerContext);
-    recordWriter.close(reducerContext);
-  }
-
-  private static <KEYIN, VALUEIN, KEYOUT, VALUEOUT> org.apache.hadoop.mapreduce.Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context createReduceContext(
-      Configuration conf,
-      TaskAttemptID mrTaskAttemptID,
-      final TezRawKeyValueIterator rawIter,
-      Counter combineInputKeyCounter,
-      Counter combineInputValueCounter,
-      RecordWriter<KEYOUT, VALUEOUT> recordWriter,
-      MRTaskReporter reporter,
-      RawComparator<KEYIN> comparator,
-      Class<KEYIN> keyClass,
-      Class<VALUEIN> valClass) throws InterruptedException, IOException {
-
-    RawKeyValueIterator r = new RawKeyValueIterator() {
-
-      @Override
-      public boolean next() throws IOException {
-        return rawIter.next();
-      }
-
-      @Override
-      public DataInputBuffer getValue() throws IOException {
-        return rawIter.getValue();
-      }
-
-      @Override
-      public Progress getProgress() {
-        return rawIter.getProgress();
-      }
-
-      @Override
-      public DataInputBuffer getKey() throws IOException {
-        return rawIter.getKey();
-      }
-
-      @Override
-      public void close() throws IOException {
-        rawIter.close();
-      }
-    };
-
-    ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> rContext = new ReduceContextImpl<KEYIN, VALUEIN, KEYOUT, VALUEOUT>(
-        conf, mrTaskAttemptID, r, combineInputKeyCounter,
-        combineInputValueCounter, recordWriter, null, reporter, comparator,
-        keyClass, valClass);
-
-    org.apache.hadoop.mapreduce.Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context reducerContext = new WrappedReducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>()
-        .getReducerContext(rContext);
-    return reducerContext;
-  }
-
-  
- 
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/ShuffledMergedInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/ShuffledMergedInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/ShuffledMergedInputLegacy.java
deleted file mode 100644
index a83620e..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/ShuffledMergedInputLegacy.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
- * intermediate sorted data, merges them and provides key/<values> to the
- * consumer.
- * 
- * The Copy and Merge will be triggered by the initialization - which is handled
- * by the Tez framework. Input is not consumable until the Copy and Merge are
- * complete. Methods are provided to check for this, as well as to wait for
- * completion. Attempting to get a reader on a non-complete input will block.
- * 
- */
-
-package org.apache.tez.mapreduce.newinput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.lib.input.ShuffledMergedInput;
-
-public class ShuffledMergedInputLegacy extends ShuffledMergedInput {
-
-  @Private
-  public TezRawKeyValueIterator getIterator() throws IOException, InterruptedException {
-    // wait for input so that iterator is available
-    waitForInputReady();
-    return rawIter;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInput.java
deleted file mode 100644
index 73d8cc7..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInput.java
+++ /dev/null
@@ -1,438 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.mapreduce.newinput;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileSystem.Statistics;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
-import org.apache.hadoop.mapreduce.split.SplitMetaInfoReaderTez;
-import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitIndex;
-import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
-import org.apache.hadoop.util.ReflectionUtils;
-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.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.mapreduce.common.Utils;
-import org.apache.tez.mapreduce.hadoop.newmapred.MRReporter;
-import org.apache.tez.mapreduce.hadoop.newmapreduce.TaskAttemptContextImpl;
-
-import com.google.common.base.Preconditions;
-
-/**
- * {@link SimpleInput} is an {@link Input} which provides key/values pairs
- * for the consumer.
- *
- * It is compatible with all standard Apache Hadoop MapReduce 
- * {@link InputFormat} implementations.
- */
-
-public class SimpleInput implements LogicalInput {
-
-  private static final Log LOG = LogFactory.getLog(SimpleInput.class);
-  
-  
-  private TezInputContext inputContext;
-  
-  private JobConf jobConf;
-  private Configuration incrementalConf;
-  private boolean recordReaderCreated = false;
-  
-  boolean useNewApi;
-  
-  org.apache.hadoop.mapreduce.TaskAttemptContext taskAttemptContext;
-
-  @SuppressWarnings("rawtypes")
-  private org.apache.hadoop.mapreduce.InputFormat newInputFormat;
-  @SuppressWarnings("rawtypes")
-  private org.apache.hadoop.mapreduce.RecordReader newRecordReader;
-  protected org.apache.hadoop.mapreduce.InputSplit newInputSplit;
-  
-  @SuppressWarnings("rawtypes")
-  private InputFormat oldInputFormat;
-  @SuppressWarnings("rawtypes")
-  protected RecordReader oldRecordReader;
-
-  protected TaskSplitIndex splitMetaInfo = new TaskSplitIndex();
-  
-  private TezCounter inputRecordCounter;
-  private TezCounter fileInputByteCounter; 
-  private List<Statistics> fsStats;
-
-  @Override
-  public List<Event> initialize(TezInputContext inputContext) throws IOException {
-    this.inputContext = inputContext;
-    Configuration conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
-    this.jobConf = new JobConf(conf);
-
-    // Read split information.
-    TaskSplitMetaInfo[] allMetaInfo = readSplits(conf);
-    TaskSplitMetaInfo thisTaskMetaInfo = allMetaInfo[inputContext.getTaskIndex()];
-    this.splitMetaInfo = new TaskSplitIndex(thisTaskMetaInfo.getSplitLocation(),
-        thisTaskMetaInfo.getStartOffset());
-    
-    // TODO NEWTEZ Rename this to be specific to SimpleInput. This Input, in
-    // theory, can be used by the MapProcessor, ReduceProcessor or a custom
-    // processor. (The processor could provide the counter though)
-    this.inputRecordCounter = inputContext.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS);
-    this.fileInputByteCounter = inputContext.getCounters().findCounter(FileInputFormatCounter.BYTES_READ);
-    
-    useNewApi = this.jobConf.getUseNewMapper();
-
-    if (useNewApi) {
-      TaskAttemptContext taskAttemptContext = createTaskAttemptContext();
-      Class<? extends org.apache.hadoop.mapreduce.InputFormat<?, ?>> inputFormatClazz;
-      try {
-        inputFormatClazz = taskAttemptContext.getInputFormatClass();
-      } catch (ClassNotFoundException e) {
-        throw new IOException("Unable to instantiate InputFormat class", e);
-      }
-
-      newInputFormat = ReflectionUtils.newInstance(inputFormatClazz, this.jobConf);
-
-      newInputSplit = getNewSplitDetails(splitMetaInfo);
-
-      List<Statistics> matchedStats = null;
-      if (newInputSplit instanceof org.apache.hadoop.mapreduce.lib.input.FileSplit) {
-        matchedStats = Utils.getFsStatistics(
-            ((org.apache.hadoop.mapreduce.lib.input.FileSplit)
-                newInputSplit).getPath(), this.jobConf);
-      }
-      fsStats = matchedStats;
-      
-      try {
-        newRecordReader = newInputFormat.createRecordReader(newInputSplit, taskAttemptContext);
-        newRecordReader.initialize(newInputSplit, taskAttemptContext);
-      } catch (InterruptedException e) {
-        throw new IOException("Interrupted while creating record reader", e);
-      }
-    } else { // OLD API
-      oldInputFormat = this.jobConf.getInputFormat();
-      InputSplit oldInputSplit =
-          getOldSplitDetails(splitMetaInfo);
-      
-      
-      List<Statistics> matchedStats = null;
-      if (oldInputSplit instanceof FileSplit) {
-        matchedStats = Utils.getFsStatistics(((FileSplit) oldInputSplit).getPath(), this.jobConf);
-      }
-      fsStats = matchedStats;
-      
-      long bytesInPrev = getInputBytes();
-      oldRecordReader = oldInputFormat.getRecordReader(oldInputSplit,
-          this.jobConf, new MRReporter(inputContext, oldInputSplit));
-      long bytesInCurr = getInputBytes();
-      fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-      setIncrementalConfigParams(oldInputSplit);
-    }    
-    return null;
-  }
-
-  @Override
-  public KVReader getReader() throws IOException {
-    Preconditions
-        .checkState(recordReaderCreated == false,
-            "Only a single instance of record reader can be created for this input.");
-    recordReaderCreated = true;
-    return new MRInputKVReader();
-  }
-
-
-  @Override
-  public void handleEvents(List<Event> inputEvents) {
-    // Not expecting any events at the moment.
-  }
-
-
-  @Override
-  public void setNumPhysicalInputs(int numInputs) {
-    // Not required at the moment. May be required if splits are sent via events.
-  }
-
-  @Override
-  public List<Event> close() throws IOException {
-    long bytesInPrev = getInputBytes();
-    if (useNewApi) {
-      newRecordReader.close();
-    } else {
-      oldRecordReader.close();
-    }
-    long bytesInCurr = getInputBytes();
-    fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-    
-    return null;
-  }
-
-  /**
-   * SimpleInputs sets some additional parameters like split location when using
-   * the new API. This methods returns the list of additional updates, and
-   * should be used by Processors using the old MapReduce API with SimpleInput.
-   * 
-   * @return the additional fields set by SimpleInput
-   */
-  public Configuration getConfigUpdates() {
-    return new Configuration(incrementalConf);
-  }
-
-  public float getProgress() throws IOException, InterruptedException {
-    if (useNewApi) {
-      return newRecordReader.getProgress();
-    } else {
-      return oldRecordReader.getProgress();
-    }
-  }
-
-  
-  private TaskAttemptContext createTaskAttemptContext() {
-    return new TaskAttemptContextImpl(this.jobConf, inputContext, true);
-  }
-  
-
-  private static class SimpleValueIterator implements Iterator<Object> {
-
-    private Object value;
-
-    public void setValue(Object value) {
-      this.value = value;
-    }
-
-    public boolean hasNext() {
-      return value != null;
-    }
-
-    public Object next() {
-      Object value = this.value;
-      this.value = null;
-      return value;
-    }
-
-    public void remove() {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  private static class SimpleIterable implements Iterable<Object> {
-    private final Iterator<Object> iterator;
-    public SimpleIterable(Iterator<Object> iterator) {
-      this.iterator = iterator;
-    }
-
-    @Override
-    public Iterator<Object> iterator() {
-      return iterator;
-    }
-  }
-
-
-
-  
-  @SuppressWarnings("unchecked")
-  private InputSplit getOldSplitDetails(TaskSplitIndex splitMetaInfo)
-      throws IOException {
-    Path file = new Path(splitMetaInfo.getSplitLocation());
-    FileSystem fs = FileSystem.getLocal(jobConf);
-    file = fs.makeQualified(file);
-    LOG.info("Reading input split file from : " + file);
-    long offset = splitMetaInfo.getStartOffset();
-    
-    FSDataInputStream inFile = fs.open(file);
-    inFile.seek(offset);
-    String className = Text.readString(inFile);
-    Class<org.apache.hadoop.mapred.InputSplit> cls;
-    try {
-      cls = 
-          (Class<org.apache.hadoop.mapred.InputSplit>) 
-          jobConf.getClassByName(className);
-    } catch (ClassNotFoundException ce) {
-      IOException wrap = new IOException("Split class " + className + 
-          " not found");
-      wrap.initCause(ce);
-      throw wrap;
-    }
-    SerializationFactory factory = new SerializationFactory(jobConf);
-    Deserializer<org.apache.hadoop.mapred.InputSplit> deserializer = 
-        (Deserializer<org.apache.hadoop.mapred.InputSplit>) 
-        factory.getDeserializer(cls);
-    deserializer.open(inFile);
-    org.apache.hadoop.mapred.InputSplit split = deserializer.deserialize(null);
-    long pos = inFile.getPos();
-    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
-        .increment(pos - offset);
-    inFile.close();
-    return split;
-  }
-
-  @SuppressWarnings("unchecked")
-  private org.apache.hadoop.mapreduce.InputSplit getNewSplitDetails(
-      TaskSplitIndex splitMetaInfo) throws IOException {
-    Path file = new Path(splitMetaInfo.getSplitLocation());
-    long offset = splitMetaInfo.getStartOffset();
-    
-    // Split information read from local filesystem.
-    FileSystem fs = FileSystem.getLocal(jobConf);
-    file = fs.makeQualified(file);
-    LOG.info("Reading input split file from : " + file);
-    FSDataInputStream inFile = fs.open(file);
-    inFile.seek(offset);
-    String className = Text.readString(inFile);
-    Class<org.apache.hadoop.mapreduce.InputSplit> cls;
-    try {
-      cls = 
-          (Class<org.apache.hadoop.mapreduce.InputSplit>) 
-          jobConf.getClassByName(className);
-    } catch (ClassNotFoundException ce) {
-      IOException wrap = new IOException("Split class " + className + 
-          " not found");
-      wrap.initCause(ce);
-      throw wrap;
-    }
-    SerializationFactory factory = new SerializationFactory(jobConf);
-    Deserializer<org.apache.hadoop.mapreduce.InputSplit> deserializer = 
-        (Deserializer<org.apache.hadoop.mapreduce.InputSplit>) 
-        factory.getDeserializer(cls);
-    deserializer.open(inFile);
-    org.apache.hadoop.mapreduce.InputSplit split = 
-        deserializer.deserialize(null);
-    long pos = inFile.getPos();
-    inputContext.getCounters().findCounter(TaskCounter.SPLIT_RAW_BYTES)
-        .increment(pos - offset);
-    inFile.close();
-    return split;
-  }
-
-  private void setIncrementalConfigParams(InputSplit inputSplit) {
-    if (inputSplit instanceof FileSplit) {
-      FileSplit fileSplit = (FileSplit) inputSplit;
-      this.incrementalConf = new Configuration(false);
-
-      this.incrementalConf.set(JobContext.MAP_INPUT_FILE, fileSplit.getPath()
-          .toString());
-      this.incrementalConf.setLong(JobContext.MAP_INPUT_START,
-          fileSplit.getStart());
-      this.incrementalConf.setLong(JobContext.MAP_INPUT_PATH,
-          fileSplit.getLength());
-    }
-    LOG.info("Processing split: " + inputSplit);
-  }
-
-  private long getInputBytes() {
-    if (fsStats == null) return 0;
-    long bytesRead = 0;
-    for (Statistics stat: fsStats) {
-      bytesRead = bytesRead + stat.getBytesRead();
-    }
-    return bytesRead;
-  }
-
-  protected TaskSplitMetaInfo[] readSplits(Configuration conf)
-      throws IOException {
-    TaskSplitMetaInfo[] allTaskSplitMetaInfo;
-    allTaskSplitMetaInfo = SplitMetaInfoReaderTez.readSplitMetaInfo(conf,
-        FileSystem.getLocal(conf));
-    return allTaskSplitMetaInfo;
-  }
-  
-  private class MRInputKVReader implements KVReader {
-    
-    Object key;
-    Object value;
-
-    private SimpleValueIterator valueIterator = new SimpleValueIterator();
-    private SimpleIterable valueIterable = new SimpleIterable(valueIterator);
-
-    private final boolean localNewApi;
-    
-    MRInputKVReader() {
-      localNewApi = useNewApi;
-      if (!localNewApi) {
-        key = oldRecordReader.createKey();
-        value =oldRecordReader.createValue();
-      }
-    }
-    
-    // Setup the values iterator once, and set value on the same object each time
-    // to prevent lots of objects being created.
-
-    
-    @SuppressWarnings("unchecked")
-    @Override
-    public boolean next() throws IOException {
-      boolean hasNext = false;
-      long bytesInPrev = getInputBytes();
-      if (localNewApi) {
-        try {
-          hasNext = newRecordReader.nextKeyValue();
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new IOException("Interrupted while checking for next key-value", e);
-        }
-      } else {
-        hasNext = oldRecordReader.next(key, value);
-      }
-      long bytesInCurr = getInputBytes();
-      fileInputByteCounter.increment(bytesInCurr - bytesInPrev);
-      
-      if (hasNext) {
-        inputRecordCounter.increment(1);
-      }
-      
-      return hasNext;
-    }
-
-    @Override
-    public KVRecord getCurrentKV() throws IOException {
-      KVRecord kvRecord = null;
-      if (localNewApi) {
-        try {
-          valueIterator.setValue(newRecordReader.getCurrentValue());
-          kvRecord = new KVRecord(newRecordReader.getCurrentKey(), valueIterable);
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new IOException("Interrupted while fetching next key-value", e);
-        }
-        
-      } else {
-        valueIterator.setValue(value);
-        kvRecord = new KVRecord(key, valueIterable);
-      }
-      return kvRecord;
-    }
-  };
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInputLegacy.java
deleted file mode 100644
index 8f07a38..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newinput/SimpleInputLegacy.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.newinput;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.mapred.RecordReader;
-
-public class SimpleInputLegacy extends SimpleInput {
-
-  @Private
-  public org.apache.hadoop.mapreduce.InputSplit getNewInputSplit() {
-    return this.newInputSplit;
-  }  
-  
-  @SuppressWarnings("rawtypes")
-  @Private
-  public RecordReader getOldRecordReader() {
-    return this.oldRecordReader;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
deleted file mode 100644
index a8fb900..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newoutput/SimpleOutput.java
+++ /dev/null
@@ -1,326 +0,0 @@
-package org.apache.tez.mapreduce.newoutput;
-
-import java.io.IOException;
-import java.text.NumberFormat;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileSystem.Statistics;
-import org.apache.hadoop.mapred.FileOutputCommitter;
-import org.apache.hadoop.mapred.FileOutputFormat;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.mapred.TaskID;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
-import org.apache.hadoop.util.ReflectionUtils;
-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.newapi.Event;
-import org.apache.tez.engine.newapi.KVWriter;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.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.newmapred.MRReporter;
-import org.apache.tez.mapreduce.hadoop.newmapreduce.TaskAttemptContextImpl;
-import org.apache.tez.mapreduce.newprocessor.MRTaskReporter;
-
-public class SimpleOutput implements LogicalOutput {
-
-  private static final Log LOG = LogFactory.getLog(SimpleOutput.class);
-
-  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
-  static {
-    NUMBER_FORMAT.setMinimumIntegerDigits(5);
-    NUMBER_FORMAT.setGroupingUsed(false);
-  }
-
-  private TezOutputContext outputContext;
-  private JobConf jobConf;
-  boolean useNewApi;
-  private AtomicBoolean closed = new AtomicBoolean(false);
-
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapreduce.OutputFormat newOutputFormat;
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapreduce.RecordWriter newRecordWriter;
-
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapred.OutputFormat oldOutputFormat;
-  @SuppressWarnings("rawtypes")
-  org.apache.hadoop.mapred.RecordWriter oldRecordWriter;
-
-  private TezCounter outputRecordCounter;
-  private TezCounter fileOutputByteCounter;
-  private List<Statistics> fsStats;
-
-  private TaskAttemptContext newApiTaskAttemptContext;
-  private org.apache.hadoop.mapred.TaskAttemptContext oldApiTaskAttemptContext;
-
-  private boolean isMapperOutput;
-
-  private OutputCommitter committer;
-
-  @Override
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws IOException, InterruptedException {
-    LOG.info("Initializing Simple Output");
-    this.outputContext = outputContext;
-    Configuration conf = TezUtils.createConfFromUserPayload(
-        outputContext.getUserPayload());
-    this.jobConf = new JobConf(conf);
-    this.useNewApi = this.jobConf.getUseNewMapper();
-    this.isMapperOutput = jobConf.getBoolean(MRConfig.IS_MAP_PROCESSOR,
-        false);
-    jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
-        outputContext.getDAGAttemptNumber());
-
-    outputRecordCounter = outputContext.getCounters().findCounter(
-        TaskCounter.MAP_OUTPUT_RECORDS);
-    fileOutputByteCounter = outputContext.getCounters().findCounter(
-        FileOutputFormatCounter.BYTES_WRITTEN);
-
-    if (useNewApi) {
-      newApiTaskAttemptContext = createTaskAttemptContext();
-      try {
-        newOutputFormat =
-            ReflectionUtils.newInstance(
-                newApiTaskAttemptContext.getOutputFormatClass(), jobConf);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException(cnfe);
-      }
-
-      List<Statistics> matchedStats = null;
-      if (newOutputFormat instanceof
-          org.apache.hadoop.mapreduce.lib.output.FileOutputFormat) {
-        matchedStats =
-            Utils.getFsStatistics(
-                org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
-                    .getOutputPath(newApiTaskAttemptContext),
-                jobConf);
-      }
-      fsStats = matchedStats;
-
-      long bytesOutPrev = getOutputBytes();
-      try {
-        newRecordWriter =
-            newOutputFormat.getRecordWriter(newApiTaskAttemptContext);
-      } catch (InterruptedException e) {
-        throw new IOException("Interrupted while creating record writer", e);
-      }
-      long bytesOutCurr = getOutputBytes();
-      fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-    } else {
-      TaskAttemptID taskAttemptId = new TaskAttemptID(new TaskID(Long.toString(
-          outputContext.getApplicationId().getClusterTimestamp()),
-          outputContext.getApplicationId().getId(),
-          (isMapperOutput ? TaskType.MAP : TaskType.REDUCE),
-          outputContext.getTaskIndex()),
-          outputContext.getTaskAttemptNumber());
-      jobConf.set(JobContext.TASK_ATTEMPT_ID, taskAttemptId.toString());
-      jobConf.set(JobContext.TASK_ID, taskAttemptId.getTaskID().toString());
-      jobConf.setBoolean(JobContext.TASK_ISMAP, isMapperOutput);
-      jobConf.setInt(JobContext.TASK_PARTITION,
-          taskAttemptId.getTaskID().getId());
-      jobConf.set(JobContext.ID, taskAttemptId.getJobID().toString());
-
-      oldApiTaskAttemptContext =
-          new org.apache.tez.mapreduce.hadoop.newmapred.TaskAttemptContextImpl(
-              jobConf, taskAttemptId,
-              new MRTaskReporter(outputContext));
-      oldOutputFormat = jobConf.getOutputFormat();
-
-      List<Statistics> matchedStats = null;
-      if (oldOutputFormat
-          instanceof org.apache.hadoop.mapred.FileOutputFormat) {
-        matchedStats =
-            Utils.getFsStatistics(
-                org.apache.hadoop.mapred.FileOutputFormat.getOutputPath(
-                    jobConf),
-                jobConf);
-      }
-      fsStats = matchedStats;
-
-      FileSystem fs = FileSystem.get(jobConf);
-      String finalName = getOutputName();
-
-      long bytesOutPrev = getOutputBytes();
-      oldRecordWriter =
-          oldOutputFormat.getRecordWriter(
-              fs, jobConf, finalName, new MRReporter(outputContext));
-      long bytesOutCurr = getOutputBytes();
-      fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-    }
-    initCommitter(jobConf, useNewApi);
-
-    LOG.info("Initialized Simple Output"
-        + ", using_new_api: " + useNewApi);
-    return null;
-  }
-
-  public void initCommitter(JobConf job, boolean useNewApi)
-      throws IOException, InterruptedException {
-
-    if (useNewApi) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("using new api for output committer");
-      }
-
-      OutputFormat<?, ?> outputFormat = null;
-      try {
-        outputFormat = ReflectionUtils.newInstance(
-            newApiTaskAttemptContext.getOutputFormatClass(), job);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException("Unknown OutputFormat", cnfe);
-      }
-      this.committer = outputFormat.getOutputCommitter(
-          newApiTaskAttemptContext);
-    } else {
-      this.committer = job.getOutputCommitter();
-    }
-
-    Path outputPath = FileOutputFormat.getOutputPath(job);
-    if (outputPath != null) {
-      if ((this.committer instanceof FileOutputCommitter)) {
-        FileOutputFormat.setWorkOutputPath(job,
-            ((FileOutputCommitter) this.committer).getTaskAttemptPath(
-                oldApiTaskAttemptContext));
-      } else {
-        FileOutputFormat.setWorkOutputPath(job, outputPath);
-      }
-    }
-    if (useNewApi) {
-      this.committer.setupTask(newApiTaskAttemptContext);
-    } else {
-      this.committer.setupTask(oldApiTaskAttemptContext);
-    }
-  }
-
-  public boolean isCommitRequired() throws IOException {
-    if (useNewApi) {
-      return committer.needsTaskCommit(newApiTaskAttemptContext);
-    } else {
-      return committer.needsTaskCommit(oldApiTaskAttemptContext);
-    }
-  }
-
-  private TaskAttemptContext createTaskAttemptContext() {
-    return new TaskAttemptContextImpl(this.jobConf, outputContext,
-        isMapperOutput);
-  }
-
-  private long getOutputBytes() {
-    if (fsStats == null) return 0;
-    long bytesWritten = 0;
-    for (Statistics stat: fsStats) {
-      bytesWritten = bytesWritten + stat.getBytesWritten();
-    }
-    return bytesWritten;
-  }
-
-  private String getOutputName() {
-    return "part-" + NUMBER_FORMAT.format(outputContext.getTaskIndex());
-  }
-
-  @Override
-  public KVWriter getWriter() throws IOException {
-    return new KVWriter() {
-      private final boolean useNewWriter = useNewApi;
-
-      @SuppressWarnings("unchecked")
-      @Override
-      public void write(Object key, Object value) throws IOException {
-        long bytesOutPrev = getOutputBytes();
-        if (useNewWriter) {
-          try {
-            newRecordWriter.write(key, value);
-          } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw new IOException("Interrupted while writing next key-value",e);
-          }
-        } else {
-          oldRecordWriter.write(key, value);
-        }
-
-        long bytesOutCurr = getOutputBytes();
-        fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-        outputRecordCounter.increment(1);
-      }
-    };
-  }
-
-  @Override
-  public void handleEvents(List<Event> outputEvents) {
-    // Not expecting any events at the moment.
-  }
-
-  @Override
-  public synchronized List<Event> close() throws IOException {
-    if (closed.getAndSet(true)) {
-      return null;
-    }
-
-    LOG.info("Closing Simple Output");
-    long bytesOutPrev = getOutputBytes();
-    if (useNewApi) {
-      try {
-        newRecordWriter.close(newApiTaskAttemptContext);
-      } catch (InterruptedException e) {
-        throw new IOException("Interrupted while closing record writer", e);
-      }
-    } else {
-      oldRecordWriter.close(null);
-    }
-    long bytesOutCurr = getOutputBytes();
-    fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-    LOG.info("Closed Simple Output");
-    return null;
-  }
-
-  @Override
-  public void setNumPhysicalOutputs(int numOutputs) {
-    // Nothing to do for now
-  }
-
-  /**
-   * SimpleOutput expects that a Processor call commit prior to the
-   * Processor's completion
-   * @throws IOException
-   */
-  public void commit() throws IOException {
-    close();
-    if (useNewApi) {
-      committer.commitTask(newApiTaskAttemptContext);
-    } else {
-      committer.commitTask(oldApiTaskAttemptContext);
-    }
-  }
-
-
-  /**
-   * SimpleOutput expects that a Processor call abort in case of any error
-   * ( including an error during commit ) prior to the Processor's completion
-   * @throws IOException
-   */
-  public void abort() throws IOException {
-    close();
-    if (useNewApi) {
-      committer.abortTask(newApiTaskAttemptContext);
-    } else {
-      committer.abortTask(oldApiTaskAttemptContext);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java
deleted file mode 100644
index dcea35c..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newpartition/MRPartitioner.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.newpartition;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-
-@SuppressWarnings({ "rawtypes", "unchecked" })
-public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
-
-  static final Log LOG = LogFactory.getLog(MRPartitioner.class);
-
-  private final boolean useNewApi;
-  private int partitions = 1;
-
-  private org.apache.hadoop.mapreduce.Partitioner newPartitioner;
-  private org.apache.hadoop.mapred.Partitioner oldPartitioner;
-
-  public MRPartitioner(Configuration conf) {
-    this.useNewApi = ConfigUtils.useNewApi(conf);
-    this.partitions = conf.getInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, 1);
-
-    if (useNewApi) {
-      if (partitions > 1) {
-        newPartitioner = (org.apache.hadoop.mapreduce.Partitioner) ReflectionUtils
-            .newInstance(
-                (Class<? extends org.apache.hadoop.mapreduce.Partitioner<?, ?>>) conf
-                    .getClass(MRJobConfig.PARTITIONER_CLASS_ATTR,
-                        org.apache.hadoop.mapreduce.lib.partition.HashPartitioner.class), conf);
-      } else {
-        newPartitioner = new org.apache.hadoop.mapreduce.Partitioner() {
-          @Override
-          public int getPartition(Object key, Object value, int numPartitions) {
-            return numPartitions - 1;
-          }
-        };
-      }
-    } else {
-      if (partitions > 1) {
-        oldPartitioner = (org.apache.hadoop.mapred.Partitioner) ReflectionUtils.newInstance(
-            (Class<? extends org.apache.hadoop.mapred.Partitioner>) conf.getClass(
-                "mapred.partitioner.class", org.apache.hadoop.mapred.lib.HashPartitioner.class), conf);
-      } else {
-        oldPartitioner = new org.apache.hadoop.mapred.Partitioner() {
-          @Override
-          public void configure(JobConf job) {
-          }
-
-          @Override
-          public int getPartition(Object key, Object value, int numPartitions) {
-            return numPartitions - 1;
-          }
-        };
-      }
-    }
-  }
-
-  @Override
-  public int getPartition(Object key, Object value, int numPartitions) {
-    if (useNewApi) {
-      return newPartitioner.getPartition(key, value, numPartitions);
-    } else {
-      return oldPartitioner.getPartition(key, value, numPartitions);
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/FileSystemStatisticsUpdater.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/FileSystemStatisticsUpdater.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/FileSystemStatisticsUpdater.java
deleted file mode 100644
index 5f97049..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/FileSystemStatisticsUpdater.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.tez.mapreduce.newprocessor;
-
-import java.util.List;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.tez.common.counters.FileSystemCounter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.common.counters.TezCounters;
-
-  /**
-   * An updater that tracks the last number reported for a given file
-   * system and only creates the counters when they are needed.
-   */
-  class FileSystemStatisticUpdater {
-    private List<FileSystem.Statistics> stats;
-    private TezCounter readBytesCounter, writeBytesCounter,
-        readOpsCounter, largeReadOpsCounter, writeOpsCounter;
-    private String scheme;
-    private TezCounters counters;
-
-    FileSystemStatisticUpdater(TezCounters counters, List<FileSystem.Statistics> stats, String scheme) {
-      this.stats = stats;
-      this.scheme = scheme;
-      this.counters = counters;
-    }
-
-    void updateCounters() {
-      if (readBytesCounter == null) {
-        readBytesCounter = counters.findCounter(scheme,
-            FileSystemCounter.BYTES_READ);
-      }
-      if (writeBytesCounter == null) {
-        writeBytesCounter = counters.findCounter(scheme,
-            FileSystemCounter.BYTES_WRITTEN);
-      }
-      if (readOpsCounter == null) {
-        readOpsCounter = counters.findCounter(scheme,
-            FileSystemCounter.READ_OPS);
-      }
-      if (largeReadOpsCounter == null) {
-        largeReadOpsCounter = counters.findCounter(scheme,
-            FileSystemCounter.LARGE_READ_OPS);
-      }
-      if (writeOpsCounter == null) {
-        writeOpsCounter = counters.findCounter(scheme,
-            FileSystemCounter.WRITE_OPS);
-      }
-      long readBytes = 0;
-      long writeBytes = 0;
-      long readOps = 0;
-      long largeReadOps = 0;
-      long writeOps = 0;
-      for (FileSystem.Statistics stat: stats) {
-        readBytes = readBytes + stat.getBytesRead();
-        writeBytes = writeBytes + stat.getBytesWritten();
-        readOps = readOps + stat.getReadOps();
-        largeReadOps = largeReadOps + stat.getLargeReadOps();
-        writeOps = writeOps + stat.getWriteOps();
-      }
-      readBytesCounter.setValue(readBytes);
-      writeBytesCounter.setValue(writeBytes);
-      readOpsCounter.setValue(readOps);
-      largeReadOpsCounter.setValue(largeReadOps);
-      writeOpsCounter.setValue(writeOps);
-    }
-  }
-  

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/GcTimeUpdater.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/GcTimeUpdater.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/GcTimeUpdater.java
deleted file mode 100644
index b05f011..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/GcTimeUpdater.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.tez.mapreduce.newprocessor;
-
-import java.lang.management.GarbageCollectorMXBean;
-import java.lang.management.ManagementFactory;
-import java.util.List;
-
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.common.counters.TaskCounter;
-
-/**
-   * An updater that tracks the amount of time this task has spent in GC.
-   */
-  class GcTimeUpdater {
-    private long lastGcMillis = 0;
-    private List<GarbageCollectorMXBean> gcBeans = null;
-    TezCounters counters;
-
-    public GcTimeUpdater(TezCounters counters) {
-      this.gcBeans = ManagementFactory.getGarbageCollectorMXBeans();
-      getElapsedGc(); // Initialize 'lastGcMillis' with the current time spent.
-      this.counters = counters;
-    }
-
-    /**
-     * @return the number of milliseconds that the gc has used for CPU
-     * since the last time this method was called.
-     */
-    protected long getElapsedGc() {
-      long thisGcMillis = 0;
-      for (GarbageCollectorMXBean gcBean : gcBeans) {
-        thisGcMillis += gcBean.getCollectionTime();
-      }
-
-      long delta = thisGcMillis - lastGcMillis;
-      this.lastGcMillis = thisGcMillis;
-      return delta;
-    }
-
-    /**
-     * Increment the gc-elapsed-time counter.
-     */
-    public void incrementGcCounter() {
-      if (null == counters) {
-        return; // nothing to do.
-      }
-
-      TezCounter gcCounter =
-        counters.findCounter(TaskCounter.GC_TIME_MILLIS);
-      if (null != gcCounter) {
-        gcCounter.increment(getElapsedGc());
-      }
-    }
-  }


[09/20] Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module - tez-engine module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
new file mode 100644
index 0000000..2e10a93
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+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.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.TezInputContext;
+import org.apache.tez.engine.newruntime.RuntimeTask;
+
+public class TezInputContextImpl extends TezTaskContextImpl
+    implements TezInputContext {
+
+  private final byte[] userPayload;
+  private final String sourceVertexName;
+  private final EventMetaData sourceInfo;
+
+  @Private
+  public TezInputContextImpl(Configuration conf, int appAttemptNumber,
+      TezUmbilical tezUmbilical, String taskVertexName,
+      String sourceVertexName, TezTaskAttemptID taskAttemptID,
+      TezCounters counters, byte[] userPayload,
+      RuntimeTask runtimeTask, Map<String, ByteBuffer> serviceConsumerMetadata) {
+    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
+        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
+    this.userPayload = userPayload;
+    this.sourceVertexName = sourceVertexName;
+    this.sourceInfo = new EventMetaData(
+        EventProducerConsumerType.INPUT, taskVertexName, sourceVertexName,
+        taskAttemptID);
+    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
+        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
+        getTaskIndex(), getTaskAttemptNumber(), sourceVertexName);
+  }
+
+  @Override
+  public void sendEvents(List<Event> events) {
+    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
+    for (Event e : events) {
+      TezEvent tEvt = new TezEvent(e, sourceInfo);
+      tezEvents.add(tEvt);
+    }
+    tezUmbilical.addEvents(tezEvents);
+  }
+
+  @Override
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  @Override
+  public String getSourceVertexName() {
+    return sourceVertexName;
+  }
+
+  @Override
+  public void fatalError(Throwable exception, String message) {
+    super.signalFatalError(exception, message, sourceInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
new file mode 100644
index 0000000..ef58de2
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+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.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.TezOutputContext;
+import org.apache.tez.engine.newruntime.RuntimeTask;
+
+public class TezOutputContextImpl extends TezTaskContextImpl
+    implements TezOutputContext {
+
+  private final byte[] userPayload;
+  private final String destinationVertexName;
+  private final EventMetaData sourceInfo;
+
+  @Private
+  public TezOutputContextImpl(Configuration conf, int appAttemptNumber,
+      TezUmbilical tezUmbilical, String taskVertexName,
+      String destinationVertexName,
+      TezTaskAttemptID taskAttemptID, TezCounters counters,
+      byte[] userPayload, RuntimeTask runtimeTask,
+      Map<String, ByteBuffer> serviceConsumerMetadata) {
+    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
+        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
+    this.userPayload = userPayload;
+    this.destinationVertexName = destinationVertexName;
+    this.sourceInfo = new EventMetaData(EventProducerConsumerType.OUTPUT,
+        taskVertexName, destinationVertexName, taskAttemptID);
+    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
+        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
+        getTaskIndex(), getTaskAttemptNumber(), destinationVertexName);
+  }
+
+  @Override
+  public void sendEvents(List<Event> events) {
+    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
+    for (Event e : events) {
+      TezEvent tEvt = new TezEvent(e, sourceInfo);
+      tezEvents.add(tEvt);
+    }
+    tezUmbilical.addEvents(tezEvents);
+  }
+
+  @Override
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  @Override
+  public String getDestinationVertexName() {
+    return destinationVertexName;
+  }
+
+  @Override
+  public void fatalError(Throwable exception, String message) {
+    super.signalFatalError(exception, message, sourceInfo);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
new file mode 100644
index 0000000..3f20d5c
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.nio.ByteBuffer;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+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.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.TezProcessorContext;
+import org.apache.tez.engine.newruntime.RuntimeTask;
+
+public class TezProcessorContextImpl extends TezTaskContextImpl
+  implements TezProcessorContext {
+
+  private final byte[] userPayload;
+  private final EventMetaData sourceInfo;
+
+  public TezProcessorContextImpl(Configuration conf, int appAttemptNumber,
+      TezUmbilical tezUmbilical, String vertexName,
+      TezTaskAttemptID taskAttemptID, TezCounters counters,
+      byte[] userPayload, RuntimeTask runtimeTask,
+      Map<String, ByteBuffer> serviceConsumerMetadata) {
+    super(conf, appAttemptNumber, vertexName, taskAttemptID,
+        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
+    this.userPayload = userPayload;
+    this.sourceInfo = new EventMetaData(EventProducerConsumerType.PROCESSOR,
+        taskVertexName, "", taskAttemptID);
+    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d", taskAttemptID
+        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
+        getTaskIndex(), getTaskAttemptNumber());
+  }
+
+  @Override
+  public void sendEvents(List<Event> events) {
+    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
+    for (Event e : events) {
+      TezEvent tEvt = new TezEvent(e, sourceInfo);
+      tezEvents.add(tEvt);
+    }
+    tezUmbilical.addEvents(tezEvents);
+  }
+
+  @Override
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  @Override
+  public void setProgress(float progress) {
+    runtimeTask.setProgress(progress);
+  }
+
+  @Override
+  public void fatalError(Throwable exception, String message) {
+    super.signalFatalError(exception, message, sourceInfo);
+  }
+
+  @Override
+  public boolean canCommit() throws IOException {
+    return tezUmbilical.canCommit(this.taskAttemptID);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
new file mode 100644
index 0000000..2312c49
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
@@ -0,0 +1,145 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
+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.newapi.TezTaskContext;
+import org.apache.tez.engine.newruntime.RuntimeTask;
+
+public abstract class TezTaskContextImpl implements TezTaskContext {
+
+  private final Configuration conf;
+  protected final String taskVertexName;
+  protected final TezTaskAttemptID taskAttemptID;
+  private final TezCounters counters;
+  private String[] workDirs;
+  protected String uniqueIdentifier;
+  protected final RuntimeTask runtimeTask;
+  protected final TezUmbilical tezUmbilical;
+  private final Map<String, ByteBuffer> serviceConsumerMetadata;
+  private final int appAttemptNumber;
+
+  @Private
+  public TezTaskContextImpl(Configuration conf, int appAttemptNumber,
+      String taskVertexName, TezTaskAttemptID taskAttemptID,
+      TezCounters counters, RuntimeTask runtimeTask,
+      TezUmbilical tezUmbilical, Map<String, ByteBuffer> serviceConsumerMetadata) {
+    this.conf = conf;
+    this.taskVertexName = taskVertexName;
+    this.taskAttemptID = taskAttemptID;
+    this.counters = counters;
+    // TODO Maybe change this to be task id specific at some point. For now
+    // Shuffle code relies on this being a path specified by YARN
+    this.workDirs = this.conf.getStrings(TezJobConfig.LOCAL_DIRS);
+    this.runtimeTask = runtimeTask;
+    this.tezUmbilical = tezUmbilical;
+    this.serviceConsumerMetadata = serviceConsumerMetadata;
+    // TODO NEWTEZ at some point dag attempt should not map to app attempt
+    this.appAttemptNumber = appAttemptNumber;
+  }
+
+  @Override
+  public ApplicationId getApplicationId() {
+    return taskAttemptID.getTaskID().getVertexID().getDAGId()
+        .getApplicationId();
+  }
+
+  @Override
+  public int getTaskIndex() {
+    return taskAttemptID.getTaskID().getId();
+  }
+
+  @Override
+  public int getDAGAttemptNumber() {
+    return appAttemptNumber;
+  }
+
+  @Override
+  public int getTaskAttemptNumber() {
+    return taskAttemptID.getId();
+  }
+
+  @Override
+  public String getDAGName() {
+    // TODO NEWTEZ Change to some form of the DAG name, for now using dagId as
+    // the unique identifier.
+    return taskAttemptID.getTaskID().getVertexID().getDAGId().toString();
+  }
+
+  @Override
+  public String getTaskVertexName() {
+    return taskVertexName;
+  }
+
+
+  @Override
+  public TezCounters getCounters() {
+    return counters;
+  }
+
+  @Override
+  public String[] getWorkDirs() {
+    return Arrays.copyOf(workDirs, workDirs.length);
+  }
+
+  @Override
+  public String getUniqueIdentifier() {
+    return uniqueIdentifier;
+  }
+
+  @Override
+  public ByteBuffer getServiceConsumerMetaData(String serviceName) {
+    return (ByteBuffer) serviceConsumerMetadata.get(serviceName)
+        .asReadOnlyBuffer().rewind();
+  }
+
+  @Override
+  public ByteBuffer getServiceProviderMetaData(String serviceName) {
+    return AuxiliaryServiceHelper.getServiceDataFromEnv(
+        serviceName, System.getenv());
+  }
+
+  protected void signalFatalError(Throwable t, String message,
+      EventMetaData sourceInfo) {
+    runtimeTask.setFatalError(t, message);
+    String diagnostics;
+    if (t != null && message != null) {
+      diagnostics = "exceptionThrown=" + StringUtils.stringifyException(t)
+          + ", errorMessage=" + message;
+    } else if (t == null && message == null) {
+      diagnostics = "Unknown error";
+    } else {
+      diagnostics = t != null ?
+          "exceptionThrown=" + StringUtils.stringifyException(t)
+          : " errorMessage=" + message;
+    }
+    tezUmbilical.signalFatalError(taskAttemptID, diagnostics, sourceInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java
new file mode 100644
index 0000000..925d87b
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezUmbilical.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+public interface TezUmbilical {
+
+  public void addEvents(Collection<TezEvent> events);
+
+  public void signalFatalError(TezTaskAttemptID taskAttemptID,
+      String diagnostics,
+      EventMetaData sourceInfo);
+
+  public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
index 78d2e0c..927f0ad 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
@@ -26,7 +26,7 @@ import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutputFiles;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
 import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.engine.shuffle.common.DiskFetchedInput;
 import org.apache.tez.engine.shuffle.common.FetchedInput;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
index b36c240..0b86a8e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
@@ -30,10 +30,10 @@ import org.apache.hadoop.io.compress.DefaultCodec;
 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.newapi.KVReader;
 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;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
index 9f3dbbe..84ddd28 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
@@ -29,13 +29,13 @@ import org.apache.hadoop.io.compress.DefaultCodec;
 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.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.newoutput.TezTaskOutput;
-import org.apache.tez.engine.newapi.KVWriter;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
 import org.apache.tez.engine.newapi.TezOutputContext;
 
 public class FileBasedKVWriter implements KVWriter {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
index 3920ce6..ab78d82 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
@@ -30,8 +30,8 @@ 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.common.combine.Combiner;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutput;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutputFiles;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
 import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.newapi.TezTaskContext;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineInput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineInput.java
deleted file mode 100644
index bf504bb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineInput.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.common.combine;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-@SuppressWarnings({ "unchecked", "rawtypes" })
-public class CombineInput implements Input {
-
-  private final TezRawKeyValueIterator input;
-  private TezCounter inputValueCounter;
-  private TezCounter inputKeyCounter;
-  private RawComparator<Object> comparator;
-  private Object key;                                  // current key
-  private Object value;                              // current value
-  private boolean firstValue = false;                 // first value in key
-  private boolean nextKeyIsSame = false;              // more w/ this key
-  private boolean hasMore;                            // more in file
-  protected Progressable reporter;
-  private Deserializer keyDeserializer;
-  private Deserializer valueDeserializer;
-  private DataInputBuffer buffer = new DataInputBuffer();
-  private BytesWritable currentRawKey = new BytesWritable();
-  private ValueIterable iterable = new ValueIterable();
-  
-  public CombineInput(TezRawKeyValueIterator kvIter) {
-    this.input = kvIter;
-  }
-
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-  }
-
-  public boolean hasNext() throws IOException, InterruptedException {
-    while (hasMore && nextKeyIsSame) {
-      nextKeyValue();
-    }
-    if (hasMore) {
-      if (inputKeyCounter != null) {
-        inputKeyCounter.increment(1);
-      }
-      return nextKeyValue();
-    } else {
-      return false;
-    }
-  }
-
-  private boolean nextKeyValue() throws IOException, InterruptedException {
-    if (!hasMore) {
-      key = null;
-      value = null;
-      return false;
-    }
-    firstValue = !nextKeyIsSame;
-    DataInputBuffer nextKey = input.getKey();
-    currentRawKey.set(nextKey.getData(), nextKey.getPosition(), 
-                      nextKey.getLength() - nextKey.getPosition());
-    buffer.reset(currentRawKey.getBytes(), 0, currentRawKey.getLength());
-    key = keyDeserializer.deserialize(key);
-    DataInputBuffer nextVal = input.getValue();
-    buffer.reset(nextVal.getData(), nextVal.getPosition(), nextVal.getLength());
-    value = valueDeserializer.deserialize(value);
-
-    hasMore = input.next();
-    if (hasMore) {
-      nextKey = input.getKey();
-      nextKeyIsSame = comparator.compare(currentRawKey.getBytes(), 0, 
-                                     currentRawKey.getLength(),
-                                     nextKey.getData(),
-                                     nextKey.getPosition(),
-                                     nextKey.getLength() - nextKey.getPosition()
-                                         ) == 0;
-    } else {
-      nextKeyIsSame = false;
-    }
-    inputValueCounter.increment(1);
-    return true;
-  }
-
-  public Object getNextKey() throws IOException, InterruptedException {
-    return key;
-  }
-
-  public Iterable getNextValues() throws IOException,
-      InterruptedException {
-    return iterable;
-  }
-
-  public float getProgress() throws IOException, InterruptedException {
-    return input.getProgress().getProgress();
-  }
-
-  public void close() throws IOException {
-    input.close();
-  }
-
-  public TezRawKeyValueIterator getIterator() {
-    return this.input;
-  }
-
-  protected class ValueIterator implements Iterator<Object> {
-
-
-    public boolean hasNext() {
-      return firstValue || nextKeyIsSame;
-    }
-
-    public Object next() {
-
-      // if this is the first record, we don't need to advance
-      if (firstValue) {
-        firstValue = false;
-        return value;
-      }
-      // if this isn't the first record and the next key is different, they
-      // can't advance it here.
-      if (!nextKeyIsSame) {
-        throw new NoSuchElementException("iterate past last value");
-      }
-      // otherwise, go to the next key/value pair
-      try {
-        nextKeyValue();
-        return value;
-      } catch (IOException ie) {
-        throw new RuntimeException("next value iterator failed", ie);
-      } catch (InterruptedException ie) {
-        // this is bad, but we can't modify the exception list of java.util
-        throw new RuntimeException("next value iterator interrupted", ie);        
-      }
-    }
-
-    public void remove() {
-      throw new UnsupportedOperationException("remove not implemented");
-    }
-  }
-
-
-  
-  protected class ValueIterable implements Iterable<Object> {
-    private ValueIterator iterator = new ValueIterator();
-    public Iterator<Object> iterator() {
-      return iterator;
-    } 
-  }
-  
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineOutput.java
deleted file mode 100644
index 10a1b90..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/CombineOutput.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.common.combine;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.records.OutputContext;
-
-public class CombineOutput implements Output {
-
-  private final Writer writer;
-  
-  public CombineOutput(Writer writer) {
-    this.writer = writer;
-  }
-
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-    // TODO Auto-generated method stub
-
-  }
-
-  public void write(Object key, Object value) throws IOException,
-      InterruptedException {
-    writer.append(key, value);
-  }
-
-  @Override
-  public OutputContext getOutputContext() {
-    return null;
-  }
-  
-  public void close() throws IOException, InterruptedException {
-    writer.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java b/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
index 38b04d3..1cb89a7 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
@@ -35,8 +35,8 @@ import org.apache.tez.common.counters.TezCounter;
 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.newoutput.TezLocalTaskOutputFiles;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutput;
+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.newapi.TezInputContext;
 
 @SuppressWarnings({"rawtypes"})

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
index 46851c7..b2a0b54 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
@@ -30,7 +30,7 @@ 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.newoutput.TezTaskOutputFiles;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
 
 
 class MapOutput {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
index ad9bb5f..bf2be4e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
@@ -54,7 +54,7 @@ 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.newoutput.TezTaskOutputFiles;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
 import org.apache.tez.engine.hadoop.compat.NullProgressable;
 import org.apache.tez.engine.newapi.TezInputContext;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/sort/SortingOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/SortingOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/SortingOutput.java
deleted file mode 100644
index 35d7723..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/SortingOutput.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.common.sort;
-
-import org.apache.tez.common.RunningTaskContext;
-import org.apache.tez.engine.api.Output;
-
-/**
- * {@link SortingOutput} is an {@link Output} which sorts incoming key/value
- * pairs.
- */
-public interface SortingOutput extends Output {
-  
-  // TODO PreCommit rename
-  public void setTask(RunningTaskContext runningTaskContext);
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
index 1b5e015..8b4bd4e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
@@ -48,7 +48,7 @@ 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.newoutput.TezTaskOutput;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
 import org.apache.tez.engine.hadoop.compat.NullProgressable;
 import org.apache.tez.engine.newapi.TezOutputContext;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/CombineValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/CombineValuesIterator.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/CombineValuesIterator.java
deleted file mode 100644
index ae6a371..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/CombineValuesIterator.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.engine.common.task.impl;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.tez.common.TezTaskReporter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-/** Iterator to return Combined values */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class CombineValuesIterator<KEY,VALUE>
-extends ValuesIterator<KEY,VALUE> {
-
-  private final TezCounter combineInputCounter;
-
-  public CombineValuesIterator(TezRawKeyValueIterator in,
-      RawComparator<KEY> comparator, Class<KEY> keyClass,
-      Class<VALUE> valClass, Configuration conf, TezTaskReporter reporter,
-      TezCounter combineInputCounter) throws IOException {
-    super(in, comparator, keyClass, valClass, conf, reporter);
-    this.combineInputCounter = combineInputCounter;
-  }
-
-  public VALUE next() {
-    combineInputCounter.increment(1);
-    return super.next();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezLocalTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezLocalTaskOutputFiles.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezLocalTaskOutputFiles.java
deleted file mode 100644
index bbe4e34..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezLocalTaskOutputFiles.java
+++ /dev/null
@@ -1,249 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.common.task.local.newoutput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.Path;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-/**
- * Manipulate the working area for the transient store for maps and reduces.
- *
- * This class is used by map and reduce tasks to identify the directories that
- * they need to write to/read from for intermediate files. The callers of
- * these methods are from the Child running the Task.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TezLocalTaskOutputFiles extends TezTaskOutput {
-
-  public TezLocalTaskOutputFiles(Configuration conf, String uniqueId) {
-    super(conf, uniqueId);
-  }
-
-  private LocalDirAllocator lDirAlloc =
-    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-
-
-  /**
-   * Return the path to local map output file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputFile()
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, conf);
-  }
-
-  /**
-   * Create a local map output file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @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);
-  }
-  
-  /**
-   * Create a local map output file name. This should *only* be used if the size
-   * of the file is not known. Otherwise use the equivalent which accepts a size
-   * parameter.
-   * 
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getOutputFileForWrite() throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR
-        + Path.SEPARATOR + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING,
-        conf);
-  }
-
-  /**
-   * Create a local map output file name on the same volume.
-   */
-  @Override
-  public Path getOutputFileForWriteInVolume(Path existing) {
-    return new Path(existing.getParent(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-  }
-
-  /**
-   * Return the path to a local map output index file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  @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,
-        conf);
-  }
-
-  /**
-   * Create a local map output index file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @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,
-        size, conf);
-  }
-
-  /**
-   * Create a local map output index file name on the same volume.
-   */
-  @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);
-  }
-
-  /**
-   * Return a local map spill file created earlier.
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillFile(int spillNumber)
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out", conf);
-  }
-
-  /**
-   * Create a local map spill file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out", size, conf);
-  }
-
-  /**
-   * Return a local map spill index file created earlier
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillIndexFile(int spillNumber)
-      throws IOException {
-    return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out.index", conf);
-  }
-
-  /**
-   * Create a local map spill index file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getSpillIndexFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out.index", size, conf);
-  }
-
-  /**
-   * Return a local reduce input file created earlier
-   *
-   * @param mapId a map task id
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  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);
-  }
-
-  /**
-   * Create a local reduce input file name.
-   *
-   * @param mapId a map task id
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  @Override
-  public Path getInputFileForWrite(int taskId,
-                                   long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(String.format(
-        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, Constants.TASK_OUTPUT_DIR, taskId),
-        size, conf);
-  }
-
-  /** Removes all of the files related to a task. */
-  @Override
-  public void removeAll()
-      throws IOException {
-    deleteLocalFiles(Constants.TASK_OUTPUT_DIR);
-  }
-
-  private String[] getLocalDirs() throws IOException {
-    return conf.getStrings(TezJobConfig.LOCAL_DIRS);
-  }
-
-  @SuppressWarnings("deprecation")
-  private void deleteLocalFiles(String subdir) throws IOException {
-    String[] localDirs = getLocalDirs();
-    for (int i = 0; i < localDirs.length; i++) {
-      FileSystem.getLocal(conf).delete(new Path(localDirs[i], subdir));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutput.java
deleted file mode 100644
index 87a5aec..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutput.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.common.task.local.newoutput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-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;
-
-/**
- * Manipulate the working area for the transient store for maps and reduces.
- *
- * This class is used by map and reduce tasks to identify the directories that
- * they need to write to/read from for intermediate files. The callers of
- * these methods are from child space and see mapreduce.cluster.local.dir as
- * taskTracker/jobCache/jobId/attemptId
- * This class should not be used from TaskTracker space.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public abstract class TezTaskOutput {
-
-  protected Configuration conf;
-  protected String uniqueId;
-
-  public TezTaskOutput(Configuration conf, String uniqueId) {
-    this.conf = conf;
-    this.uniqueId = uniqueId;
-  }
-
-  /**
-   * Return the path to local map output file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputFile() throws IOException;
-
-  /**
-   * Create a local map output file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputFileForWrite(long size) throws IOException;
-
-  /**
-   * Create a local output file name. This method is meant to be used *only* if
-   * the size of the file is not know up front.
-   * 
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputFileForWrite() throws IOException;
-  
-  /**
-   * Create a local map output file name on the same volume.
-   */
-  public abstract Path getOutputFileForWriteInVolume(Path existing);
-
-  /**
-   * Return the path to a local map output index file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputIndexFile() throws IOException;
-
-  /**
-   * Create a local map output index file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getOutputIndexFileForWrite(long size) throws IOException;
-
-  /**
-   * Create a local map output index file name on the same volume.
-   */
-  public abstract Path getOutputIndexFileForWriteInVolume(Path existing);
-
-  /**
-   * Return a local map spill file created earlier.
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillFile(int spillNumber) throws IOException;
-
-  /**
-   * Create a local map spill file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillFileForWrite(int spillNumber, long size)
-      throws IOException;
-
-  /**
-   * Return a local map spill index file created earlier
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillIndexFile(int spillNumber) throws IOException;
-
-  /**
-   * Create a local map spill index file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getSpillIndexFileForWrite(int spillNumber, long size)
-      throws IOException;
-
-  /**
-   * Return a local reduce input file created earlier
-   *
-   * @param attemptIdentifier The identifier for the source task
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException;
-
-  /**
-   * Create a local reduce input file name.
-   *
-   * @param taskIdentifier The identifier for the source task
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public abstract Path getInputFileForWrite(
-      int taskIdentifier, long size) throws IOException;
-
-  /** Removes all of the files related to a task. */
-  public abstract void removeAll() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutputFiles.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutputFiles.java
deleted file mode 100644
index a37f05f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/newoutput/TezTaskOutputFiles.java
+++ /dev/null
@@ -1,246 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.common.task.local.newoutput;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.Path;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-/**
- * Manipulate the working area for the transient store for maps and reduces.
- *
- * This class is used by map and reduce tasks to identify the directories that
- * they need to write to/read from for intermediate files. The callers of
- * these methods are from child space and see mapreduce.cluster.local.dir as
- * taskTracker/jobCache/jobId/attemptId
- * This class should not be used from TaskTracker space.
- */
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TezTaskOutputFiles extends TezTaskOutput {
-  
-  public TezTaskOutputFiles(Configuration conf, String uniqueId) {
-    super(conf, uniqueId);
-  }
-
-  private static final Log LOG = LogFactory.getLog(TezTaskOutputFiles.class);
-
-  private static final String SPILL_FILE_PATTERN = "%s_spill_%d.out";
-  private static final String SPILL_INDEX_FILE_PATTERN = SPILL_FILE_PATTERN
-      + ".index";
-
-  
-
-  // assume configured to $localdir/usercache/$user/appcache/$appId
-  private LocalDirAllocator lDirAlloc =
-    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-  
-
-  private Path getAttemptOutputDir() {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("getAttemptOutputDir: "
-          + Constants.TASK_OUTPUT_DIR + "/"
-          + uniqueId);
-    }
-    return new Path(Constants.TASK_OUTPUT_DIR, uniqueId);
-  }
-
-  /**
-   * Return the path to local map output file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFile() throws IOException {
-    Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathToRead(attemptOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFileForWrite(long size) throws IOException {
-    Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), size, conf);
-  }
-
-  /**
-   * Create a local map output file name. This should *only* be used if the size
-   * of the file is not known. Otherwise use the equivalent which accepts a size
-   * parameter.
-   * 
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputFileForWrite() throws IOException {
-    Path attemptOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), conf);
-  }
-
-  /**
-   * 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 attemptOutputDir = new Path(outputDir, uniqueId);
-    return new Path(attemptOutputDir, Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
-  }
-
-  /**
-   * Return the path to a local map output index file created earlier
-   *
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputIndexFile() throws IOException {
-    Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
-                                      Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
-    return lDirAlloc.getLocalPathToRead(attemptIndexOutput.toString(), conf);
-  }
-
-  /**
-   * Create a local map output index file name.
-   *
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getOutputIndexFileForWrite(long size) throws IOException {
-    Path attemptIndexOutput =
-      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING +
-                                      Constants.TEZ_ENGINE_TASK_OUTPUT_INDEX_SUFFIX_STRING);
-    return lDirAlloc.getLocalPathForWrite(attemptIndexOutput.toString(),
-        size, conf);
-  }
-
-  /**
-   * Create a local map output index file name on the same volume.
-   */
-  public Path getOutputIndexFileForWriteInVolume(Path existing) {
-    Path outputDir = new Path(existing.getParent(), Constants.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 a local map spill file created earlier.
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillFile(int spillNumber) throws IOException {
-    return lDirAlloc.getLocalPathToRead(
-        String.format(SPILL_FILE_PATTERN,
-            uniqueId, spillNumber), conf);
-  }
-
-  /**
-   * Create a local map spill file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(
-        String.format(String.format(SPILL_FILE_PATTERN,
-            uniqueId, spillNumber)), size, conf);
-  }
-
-  /**
-   * Return a local map spill index file created earlier
-   *
-   * @param spillNumber the number
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillIndexFile(int spillNumber) throws IOException {
-    return lDirAlloc.getLocalPathToRead(
-        String.format(SPILL_INDEX_FILE_PATTERN,
-            uniqueId, spillNumber), conf);
-  }
-
-  /**
-   * Create a local map spill index file name.
-   *
-   * @param spillNumber the number
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getSpillIndexFileForWrite(int spillNumber, long size)
-      throws IOException {
-    return lDirAlloc.getLocalPathForWrite(
-        String.format(SPILL_INDEX_FILE_PATTERN,
-            uniqueId, spillNumber), size, conf);
-  }
-
-  /**
-   * Return a local reduce input file created earlier
-   *
-   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
-   * @return path
-   * @throws IOException
-   */
-  public Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException {
-    throw new UnsupportedOperationException("Incompatible with LocalRunner");
-  }
-
-  /**
-   * Create a local reduce input file name.
-   *
-   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
-   * @param size the size of the file
-   * @return path
-   * @throws IOException
-   */
-  public Path getInputFileForWrite(int srcTaskId,
-      long size) throws IOException {
-    return lDirAlloc.getLocalPathForWrite(String.format(
-        uniqueId, getAttemptOutputDir().toString(), srcTaskId),
-        size, conf);
-  }
-
-  /** Removes all of the files related to a task. */
-  public void removeAll() throws IOException {
-    throw new UnsupportedOperationException("Incompatible with LocalRunner");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
index 69484af..40e6b1a 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezLocalTaskOutputFiles.java
@@ -28,7 +28,7 @@ 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.dag.records.TezTaskID;
+import org.apache.tez.engine.common.InputAttemptIdentifier;
 
 /**
  * Manipulate the working area for the transient store for maps and reduces.
@@ -41,11 +41,13 @@ import org.apache.tez.dag.records.TezTaskID;
 @InterfaceStability.Unstable
 public class TezLocalTaskOutputFiles extends TezTaskOutput {
 
+  public TezLocalTaskOutputFiles(Configuration conf, String uniqueId) {
+    super(conf, uniqueId);
+  }
+
   private LocalDirAllocator lDirAlloc =
     new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
 
-  public TezLocalTaskOutputFiles() {
-  }
 
   /**
    * Return the path to local map output file created earlier
@@ -57,7 +59,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getOutputFile()
       throws IOException {
     return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, getConf());
+        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, conf);
   }
 
   /**
@@ -71,7 +73,22 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getOutputFileForWrite(long size)
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + Path.SEPARATOR
-        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, size, getConf());
+        + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING, size, conf);
+  }
+  
+  /**
+   * Create a local map output file name. This should *only* be used if the size
+   * of the file is not known. Otherwise use the equivalent which accepts a size
+   * parameter.
+   * 
+   * @return path
+   * @throws IOException
+   */
+  @Override
+  public Path getOutputFileForWrite() throws IOException {
+    return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR
+        + Path.SEPARATOR + Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING,
+        conf);
   }
 
   /**
@@ -93,7 +110,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
       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,
-        getConf());
+        conf);
   }
 
   /**
@@ -108,7 +125,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
       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,
-        size, getConf());
+        size, conf);
   }
 
   /**
@@ -131,7 +148,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getSpillFile(int spillNumber)
       throws IOException {
     return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out", getConf());
+        + spillNumber + ".out", conf);
   }
 
   /**
@@ -146,7 +163,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getSpillFileForWrite(int spillNumber, long size)
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out", size, getConf());
+        + spillNumber + ".out", size, conf);
   }
 
   /**
@@ -160,7 +177,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getSpillIndexFile(int spillNumber)
       throws IOException {
     return lDirAlloc.getLocalPathToRead(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out.index", getConf());
+        + spillNumber + ".out.index", conf);
   }
 
   /**
@@ -175,7 +192,7 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
   public Path getSpillIndexFileForWrite(int spillNumber, long size)
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(Constants.TASK_OUTPUT_DIR + "/spill"
-        + spillNumber + ".out.index", size, getConf());
+        + spillNumber + ".out.index", size, conf);
   }
 
   /**
@@ -186,11 +203,11 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
    * @throws IOException
    */
   @Override
-  public Path getInputFile(int mapId)
+  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)), getConf());
+        Constants.TASK_OUTPUT_DIR, Integer.valueOf(mapId.getInputIdentifier().getSrcTaskIndex())), conf);
   }
 
   /**
@@ -202,12 +219,12 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
    * @throws IOException
    */
   @Override
-  public Path getInputFileForWrite(TezTaskID mapId,
+  public Path getInputFileForWrite(int taskId,
                                    long size)
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(String.format(
-        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, Constants.TASK_OUTPUT_DIR, mapId.getId()),
-        size, getConf());
+        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING, Constants.TASK_OUTPUT_DIR, taskId),
+        size, conf);
   }
 
   /** Removes all of the files related to a task. */
@@ -217,20 +234,15 @@ public class TezLocalTaskOutputFiles extends TezTaskOutput {
     deleteLocalFiles(Constants.TASK_OUTPUT_DIR);
   }
 
-  @Override
-  public void setConf(Configuration conf) {
-    super.setConf(conf);
-  }
-
   private String[] getLocalDirs() throws IOException {
-    return getConf().getStrings(TezJobConfig.LOCAL_DIRS);
+    return conf.getStrings(TezJobConfig.LOCAL_DIRS);
   }
 
   @SuppressWarnings("deprecation")
   private void deleteLocalFiles(String subdir) throws IOException {
     String[] localDirs = getLocalDirs();
     for (int i = 0; i < localDirs.length; i++) {
-      FileSystem.getLocal(getConf()).delete(new Path(localDirs[i], subdir));
+      FileSystem.getLocal(conf).delete(new Path(localDirs[i], subdir));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
index 50d270b..e1d83ad 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutput.java
@@ -23,9 +23,8 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.fs.Path;
-import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.engine.common.InputAttemptIdentifier;
 
 /**
  * Manipulate the working area for the transient store for maps and reduces.
@@ -38,12 +37,14 @@ import org.apache.tez.dag.records.TezTaskID;
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public abstract class TezTaskOutput implements Configurable {
+public abstract class TezTaskOutput {
 
   protected Configuration conf;
   protected String uniqueId;
 
-  public TezTaskOutput() {
+  public TezTaskOutput(Configuration conf, String uniqueId) {
+    this.conf = conf;
+    this.uniqueId = uniqueId;
   }
 
   /**
@@ -64,6 +65,15 @@ public abstract class TezTaskOutput implements Configurable {
   public abstract Path getOutputFileForWrite(long size) throws IOException;
 
   /**
+   * Create a local output file name. This method is meant to be used *only* if
+   * the size of the file is not know up front.
+   * 
+   * @return path
+   * @throws IOException
+   */
+  public abstract Path getOutputFileForWrite() throws IOException;
+  
+  /**
    * Create a local map output file name on the same volume.
    */
   public abstract Path getOutputFileForWriteInVolume(Path existing);
@@ -133,42 +143,23 @@ public abstract class TezTaskOutput implements Configurable {
   /**
    * Return a local reduce input file created earlier
    *
-   * @param mapId a map task id
+   * @param attemptIdentifier The identifier for the source task
    * @return path
    * @throws IOException
    */
-  public abstract Path getInputFile(int mapId) throws IOException;
+  public abstract Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException;
 
   /**
    * Create a local reduce input file name.
    *
-   * @param mapId a map task id
+   * @param taskIdentifier The identifier for the source task
    * @param size the size of the file
    * @return path
    * @throws IOException
    */
   public abstract Path getInputFileForWrite(
-      TezTaskID mapId, long size) throws IOException;
+      int taskIdentifier, long size) throws IOException;
 
   /** Removes all of the files related to a task. */
   public abstract void removeAll() throws IOException;
-
-  public void setUniqueIdentifier(String uniqueId) {
-    this.uniqueId = uniqueId;
-  }
-  
-  public String getUniqueIdentifier() {
-    return this.uniqueId;
-  }
-  
-  @Override
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  @Override
-  public Configuration getConf() {
-    return conf;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
index b7874f0..b8f051b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/TezTaskOutputFiles.java
@@ -29,7 +29,7 @@ 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.dag.records.TezTaskID;
+import org.apache.tez.engine.common.InputAttemptIdentifier;
 
 /**
  * Manipulate the working area for the transient store for maps and reduces.
@@ -40,32 +40,35 @@ import org.apache.tez.dag.records.TezTaskID;
  * taskTracker/jobCache/jobId/attemptId
  * This class should not be used from TaskTracker space.
  */
+
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class TezTaskOutputFiles extends TezTaskOutput {
+  
+  public TezTaskOutputFiles(Configuration conf, String uniqueId) {
+    super(conf, uniqueId);
+  }
 
   private static final Log LOG = LogFactory.getLog(TezTaskOutputFiles.class);
-  private Configuration conf;
 
   private static final String SPILL_FILE_PATTERN = "%s_spill_%d.out";
   private static final String SPILL_INDEX_FILE_PATTERN = SPILL_FILE_PATTERN
       + ".index";
 
-  public TezTaskOutputFiles() {
-  }
+  
 
   // assume configured to $localdir/usercache/$user/appcache/$appId
   private LocalDirAllocator lDirAlloc =
     new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+  
 
   private Path getAttemptOutputDir() {
     if (LOG.isDebugEnabled()) {
       LOG.debug("getAttemptOutputDir: "
           + Constants.TASK_OUTPUT_DIR + "/"
-          + conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID));
+          + uniqueId);
     }
-    return new Path(Constants.TASK_OUTPUT_DIR,
-        conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID));
+    return new Path(Constants.TASK_OUTPUT_DIR, uniqueId);
   }
 
   /**
@@ -94,12 +97,25 @@ public class TezTaskOutputFiles extends TezTaskOutput {
   }
 
   /**
+   * Create a local map output file name. This should *only* be used if the size
+   * of the file is not known. Otherwise use the equivalent which accepts a size
+   * parameter.
+   * 
+   * @return path
+   * @throws IOException
+   */
+  public Path getOutputFileForWrite() throws IOException {
+    Path attemptOutput =
+      new Path(getAttemptOutputDir(), Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
+    return lDirAlloc.getLocalPathForWrite(attemptOutput.toString(), conf);
+  }
+
+  /**
    * 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 attemptOutputDir = new Path(outputDir,
-        conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID));
+    Path attemptOutputDir = new Path(outputDir, uniqueId);
     return new Path(attemptOutputDir, Constants.TEZ_ENGINE_TASK_OUTPUT_FILENAME_STRING);
   }
 
@@ -136,8 +152,7 @@ public class TezTaskOutputFiles extends TezTaskOutput {
    */
   public Path getOutputIndexFileForWriteInVolume(Path existing) {
     Path outputDir = new Path(existing.getParent(), Constants.TASK_OUTPUT_DIR);
-    Path attemptOutputDir = new Path(outputDir,
-        conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID));
+    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);
   }
@@ -152,7 +167,7 @@ public class TezTaskOutputFiles extends TezTaskOutput {
   public Path getSpillFile(int spillNumber) throws IOException {
     return lDirAlloc.getLocalPathToRead(
         String.format(SPILL_FILE_PATTERN,
-            conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID), spillNumber), conf);
+            uniqueId, spillNumber), conf);
   }
 
   /**
@@ -167,7 +182,7 @@ public class TezTaskOutputFiles extends TezTaskOutput {
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(
         String.format(String.format(SPILL_FILE_PATTERN,
-            conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID), spillNumber)), size, conf);
+            uniqueId, spillNumber)), size, conf);
   }
 
   /**
@@ -180,7 +195,7 @@ public class TezTaskOutputFiles extends TezTaskOutput {
   public Path getSpillIndexFile(int spillNumber) throws IOException {
     return lDirAlloc.getLocalPathToRead(
         String.format(SPILL_INDEX_FILE_PATTERN,
-            conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID), spillNumber), conf);
+            uniqueId, spillNumber), conf);
   }
 
   /**
@@ -195,33 +210,32 @@ public class TezTaskOutputFiles extends TezTaskOutput {
       throws IOException {
     return lDirAlloc.getLocalPathForWrite(
         String.format(SPILL_INDEX_FILE_PATTERN,
-            conf.get(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID), spillNumber), size, conf);
+            uniqueId, spillNumber), size, conf);
   }
 
   /**
    * Return a local reduce input file created earlier
    *
-   * @param mapId a map task id
+   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
    * @return path
    * @throws IOException
    */
-  public Path getInputFile(int mapId) throws IOException {
+  public Path getInputFile(InputAttemptIdentifier attemptIdentifier) throws IOException {
     throw new UnsupportedOperationException("Incompatible with LocalRunner");
   }
 
   /**
    * Create a local reduce input file name.
    *
-   * @param mapId a map task id
+   * @param attemptIdentifier an identifier for a task. The attempt information is ignored.
    * @param size the size of the file
    * @return path
    * @throws IOException
    */
-  public Path getInputFileForWrite(TezTaskID mapId,
+  public Path getInputFileForWrite(int srcTaskId,
       long size) throws IOException {
     return lDirAlloc.getLocalPathForWrite(String.format(
-        Constants.TEZ_ENGINE_TASK_INPUT_FILE_FORMAT_STRING,
-        getAttemptOutputDir().toString(), mapId.getId()),
+        uniqueId, getAttemptOutputDir().toString(), srcTaskId),
         size, conf);
   }
 
@@ -229,13 +243,4 @@ public class TezTaskOutputFiles extends TezTaskOutput {
   public void removeAll() throws IOException {
     throw new UnsupportedOperationException("Incompatible with LocalRunner");
   }
-
-  public void setConf(Configuration conf) {
-    this.conf = conf;
-  }
-
-  public Configuration getConf() {
-    return conf;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
index eccd119..c719fba 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
@@ -29,12 +29,12 @@ import org.apache.tez.common.TezJobConfig;
 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.KVReader;
 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.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
 import org.apache.tez.engine.newapi.LogicalInput;
 import org.apache.tez.engine.newapi.TezInputContext;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/LocalMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/LocalMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/LocalMergedInput.java
deleted file mode 100644
index 269fe81..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/LocalMergedInput.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.lib.oldinput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-/**
- * {@link LocalMergedInput} in an {@link Input} which shuffles intermediate
- * sorted data, merges them and provides key/<values> to the consumer. 
- */
-public class LocalMergedInput extends OldShuffledMergedInput {
-
-  public LocalMergedInput(TezEngineTaskContext task, int index) {
-    super(task, index);
-  }
-
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-  }
-
-  public boolean hasNext() throws IOException, InterruptedException {
-    return false;
-  }
-
-  public Object getNextKey() throws IOException, InterruptedException {
-    return null;
-  }
-
-  @SuppressWarnings({ "unchecked", "rawtypes" })
-  public Iterable getNextValues() 
-      throws IOException, InterruptedException {
-    return null;
-  }
-
-  public float getProgress() throws IOException, InterruptedException {
-    return 0f;
-  }
-
-  public void close() throws IOException {
-  }
-
-  public TezRawKeyValueIterator getIterator() {
-    return null;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/OldShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/OldShuffledMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/OldShuffledMergedInput.java
deleted file mode 100644
index c046a27..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldinput/OldShuffledMergedInput.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.lib.oldinput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.RunningTaskContext;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-/**
- * {@link OldShuffledMergedInput} in an {@link Input} which shuffles intermediate
- * sorted data, merges them and provides key/<values> to the consumer. 
- */
-public class OldShuffledMergedInput implements Input {
-
-
-  public OldShuffledMergedInput(TezEngineTaskContext task, int index) {
-  }
-
-  public void mergeWith(OldShuffledMergedInput other) {
-  }
-  
-  public void setTask(RunningTaskContext runningTaskContext) {
-  }
-  
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-  }
-
-  public boolean hasNext() throws IOException, InterruptedException {
-    return false;
-  }
-
-  public Object getNextKey() throws IOException, InterruptedException {
-    return null;
-  }
-
-  @SuppressWarnings({ "unchecked", "rawtypes" })
-  public Iterable getNextValues() 
-      throws IOException, InterruptedException {
-    return null;
-  }
-
-  public float getProgress() throws IOException, InterruptedException {
-    return 0f;
-  }
-
-  public void close() throws IOException {
-  }
-
-  public TezRawKeyValueIterator getIterator() {
-    return null;
-  }
-  
-}


[19/20] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
new file mode 100644
index 0000000..fd4fdee
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/AbstractCounters.java
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import static org.apache.tez.common.counters.CounterGroupFactory.getFrameworkGroupId;
+import static org.apache.tez.common.counters.CounterGroupFactory.isFrameworkGroup;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentSkipListMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Maps;
+
+/**
+ * An abstract class to provide common implementation for the Counters
+ * container in both mapred and mapreduce packages.
+ *
+ * @param <C> type of counter inside the counters
+ * @param <G> type of group inside the counters
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public abstract class AbstractCounters<C extends TezCounter,
+                                       G extends CounterGroupBase<C>>
+    implements Writable, Iterable<G> {
+
+  protected static final Log LOG = LogFactory.getLog("mapreduce.Counters");
+
+  /**
+   * A cache from enum values to the associated counter.
+   */
+  private Map<Enum<?>, C> cache = Maps.newIdentityHashMap();
+  //framework & fs groups
+  private Map<String, G> fgroups = new ConcurrentSkipListMap<String, G>();
+  // other groups
+  private Map<String, G> groups = new ConcurrentSkipListMap<String, G>();
+  private final CounterGroupFactory<C, G> groupFactory;
+
+  // For framework counter serialization without strings
+  enum GroupType { FRAMEWORK, FILESYSTEM };
+
+  // Writes only framework and fs counters if false.
+  private boolean writeAllCounters = true;
+
+  private static final Map<String, String> legacyMap = Maps.newHashMap();
+  static {
+    legacyMap.put("org.apache.hadoop.mapred.Task$Counter",
+                  TaskCounter.class.getName());
+    legacyMap.put("org.apache.hadoop.mapred.JobInProgress$Counter",
+                  JobCounter.class.getName());
+    legacyMap.put("FileSystemCounters", FileSystemCounter.class.getName());
+  }
+
+  private final Limits limits = new Limits();
+
+  @InterfaceAudience.Private
+  public AbstractCounters(CounterGroupFactory<C, G> gf) {
+    groupFactory = gf;
+  }
+
+  /**
+   * Construct from another counters object.
+   * @param <C1> type of the other counter
+   * @param <G1> type of the other counter group
+   * @param counters the counters object to copy
+   * @param groupFactory the factory for new groups
+   */
+  @InterfaceAudience.Private
+  public <C1 extends TezCounter, G1 extends CounterGroupBase<C1>>
+  AbstractCounters(AbstractCounters<C1, G1> counters,
+                   CounterGroupFactory<C, G> groupFactory) {
+    this.groupFactory = groupFactory;
+    for(G1 group: counters) {
+      String name = group.getName();
+      G newGroup = groupFactory.newGroup(name, group.getDisplayName(), limits);
+      (isFrameworkGroup(name) ? fgroups : groups).put(name, newGroup);
+      for(TezCounter counter: group) {
+        newGroup.addCounter(counter.getName(), counter.getDisplayName(),
+                            counter.getValue());
+      }
+    }
+  }
+
+  /** Add a group.
+   * @param group object to add
+   * @return the group
+   */
+  @InterfaceAudience.Private
+  public synchronized G addGroup(G group) {
+    String name = group.getName();
+    if (isFrameworkGroup(name)) {
+      fgroups.put(name, group);
+    } else {
+      limits.checkGroups(groups.size() + 1);
+      groups.put(name, group);
+    }
+    return group;
+  }
+
+  /**
+   * Add a new group
+   * @param name of the group
+   * @param displayName of the group
+   * @return the group
+   */
+  @InterfaceAudience.Private
+  public G addGroup(String name, String displayName) {
+    return addGroup(groupFactory.newGroup(name, displayName, limits));
+  }
+
+  /**
+   * Find a counter, create one if necessary
+   * @param groupName of the counter
+   * @param counterName name of the counter
+   * @return the matching counter
+   */
+  public C findCounter(String groupName, String counterName) {
+    G grp = getGroup(groupName);
+    return grp.findCounter(counterName);
+  }
+
+  /**
+   * Find the counter for the given enum. The same enum will always return the
+   * same counter.
+   * @param key the counter key
+   * @return the matching counter object
+   */
+  public synchronized C findCounter(Enum<?> key) {
+    C counter = cache.get(key);
+    if (counter == null) {
+      counter = findCounter(key.getDeclaringClass().getName(), key.name());
+      cache.put(key, counter);
+    }
+    return counter;
+  }
+
+  /**
+   * Find the file system counter for the given scheme and enum.
+   * @param scheme of the file system
+   * @param key the enum of the counter
+   * @return the file system counter
+   */
+  @InterfaceAudience.Private
+  public synchronized C findCounter(String scheme, FileSystemCounter key) {
+    return ((FileSystemCounterGroup<C>) getGroup(
+        FileSystemCounter.class.getName()).getUnderlyingGroup()).
+        findCounter(scheme, key);
+  }
+
+  /**
+   * Returns the names of all counter classes.
+   * @return Set of counter names.
+   */
+  public synchronized Iterable<String> getGroupNames() {
+    HashSet<String> deprecated = new HashSet<String>();
+    for(Map.Entry<String, String> entry : legacyMap.entrySet()) {
+      String newGroup = entry.getValue();
+      boolean isFGroup = isFrameworkGroup(newGroup);
+      if(isFGroup ? fgroups.containsKey(newGroup) : groups.containsKey(newGroup)) {
+        deprecated.add(entry.getKey());
+      }
+    }
+    return Iterables.concat(fgroups.keySet(), groups.keySet(), deprecated);
+  }
+
+  @Override
+  public Iterator<G> iterator() {
+    return Iterators.concat(fgroups.values().iterator(),
+                            groups.values().iterator());
+  }
+
+  /**
+   * Returns the named counter group, or an empty group if there is none
+   * with the specified name.
+   * @param groupName name of the group
+   * @return the group
+   */
+  public synchronized G getGroup(String groupName) {
+
+    // filterGroupName
+    boolean groupNameInLegacyMap = true;
+    String newGroupName = legacyMap.get(groupName);
+    if (newGroupName == null) {
+      groupNameInLegacyMap = false;
+      newGroupName = Limits.filterGroupName(groupName);
+    }
+
+    boolean isFGroup = isFrameworkGroup(newGroupName);
+    G group = isFGroup ? fgroups.get(newGroupName) : groups.get(newGroupName);
+    if (group == null) {
+      group = groupFactory.newGroup(newGroupName, limits);
+      if (isFGroup) {
+        fgroups.put(newGroupName, group);
+      } else {
+        limits.checkGroups(groups.size() + 1);
+        groups.put(newGroupName, group);
+      }
+      if (groupNameInLegacyMap) {
+        LOG.warn("Group " + groupName + " is deprecated. Use " + newGroupName
+            + " instead");
+      }
+    }
+    return group;
+  }
+
+  /**
+   * Returns the total number of counters, by summing the number of counters
+   * in each group.
+   * @return the total number of counters
+   */
+  public synchronized int countCounters() {
+    int result = 0;
+    for (G group : this) {
+      result += group.size();
+    }
+    return result;
+  }
+
+  /**
+   * Write the set of groups.
+   * Counters ::= version #fgroups (groupId, group)* #groups (group)*
+   */
+  @Override
+  public synchronized void write(DataOutput out) throws IOException {
+    WritableUtils.writeVInt(out, groupFactory.version());
+    WritableUtils.writeVInt(out, fgroups.size());  // framework groups first
+    for (G group : fgroups.values()) {
+      if (group.getUnderlyingGroup() instanceof FrameworkCounterGroup<?, ?>) {
+        WritableUtils.writeVInt(out, GroupType.FRAMEWORK.ordinal());
+        WritableUtils.writeVInt(out, getFrameworkGroupId(group.getName()));
+        group.write(out);
+      } else if (group.getUnderlyingGroup() instanceof FileSystemCounterGroup<?>) {
+        WritableUtils.writeVInt(out, GroupType.FILESYSTEM.ordinal());
+        group.write(out);
+      }
+    }
+    if (writeAllCounters) {
+      WritableUtils.writeVInt(out, groups.size());
+      for (G group : groups.values()) {
+        Text.writeString(out, group.getName());
+        group.write(out);
+      }
+    } else {
+      WritableUtils.writeVInt(out, 0);
+    }
+  }
+
+  @Override
+  public synchronized void readFields(DataInput in) throws IOException {
+    int version = WritableUtils.readVInt(in);
+    if (version != groupFactory.version()) {
+      throw new IOException("Counters version mismatch, expected "+
+          groupFactory.version() +" got "+ version);
+    }
+    int numFGroups = WritableUtils.readVInt(in);
+    fgroups.clear();
+    GroupType[] groupTypes = GroupType.values();
+    while (numFGroups-- > 0) {
+      GroupType groupType = groupTypes[WritableUtils.readVInt(in)];
+      G group;
+      switch (groupType) {
+        case FILESYSTEM: // with nothing
+          group = groupFactory.newFileSystemGroup();
+          break;
+        case FRAMEWORK:  // with group id
+          group = groupFactory.newFrameworkGroup(WritableUtils.readVInt(in));
+          break;
+        default: // Silence dumb compiler, as it would've thrown earlier
+          throw new IOException("Unexpected counter group type: "+ groupType);
+      }
+      group.readFields(in);
+      fgroups.put(group.getName(), group);
+    }
+    int numGroups = WritableUtils.readVInt(in);
+    while (numGroups-- > 0) {
+      limits.checkGroups(groups.size() + 1);
+      G group = groupFactory.newGenericGroup(Text.readString(in), null, limits);
+      group.readFields(in);
+      groups.put(group.getName(), group);
+    }
+  }
+
+  /**
+   * Return textual representation of the counter values.
+   * @return the string
+   */
+  @Override
+  public synchronized String toString() {
+    StringBuilder sb = new StringBuilder("Counters: " + countCounters());
+    for (G group: this) {
+      sb.append("\n\t").append(group.getDisplayName());
+      for (TezCounter counter: group) {
+        sb.append("\n\t\t").append(counter.getDisplayName()).append("=")
+          .append(counter.getValue());
+      }
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Increments multiple counters by their amounts in another Counters
+   * instance.
+   * @param other the other Counters instance
+   */
+  public synchronized void incrAllCounters(AbstractCounters<C, G> other) {
+    for(G right : other) {
+      String groupName = right.getName();
+      G left = (isFrameworkGroup(groupName) ? fgroups : groups).get(groupName);
+      if (left == null) {
+        left = addGroup(groupName, right.getDisplayName());
+      }
+      left.incrAllCounters(right);
+    }
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public boolean equals(Object genericRight) {
+    if (genericRight instanceof AbstractCounters<?, ?>) {
+      return Iterators.elementsEqual(iterator(),
+          ((AbstractCounters<C, G>)genericRight).iterator());
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return groups.hashCode();
+  }
+
+  /**
+   * Set the "writeAllCounters" option to true or false
+   * @param send  if true all counters would be serialized, otherwise only
+   *              framework counters would be serialized in
+   *              {@link #write(DataOutput)}
+   */
+  @InterfaceAudience.Private
+  public void setWriteAllCounters(boolean send) {
+    writeAllCounters = send;
+  }
+
+  /**
+   * Get the "writeAllCounters" option
+   * @return true of all counters would serialized
+   */
+  @InterfaceAudience.Private
+  public boolean getWriteAllCounters() {
+    return writeAllCounters;
+  }
+
+  @InterfaceAudience.Private
+  public Limits limits() {
+    return limits;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroup.java
new file mode 100644
index 0000000..bc7986d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroup.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A group of {@link TezCounter}s that logically belong together. Typically,
+ * it is an {@link Enum} subclass and the counters are the values.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public interface CounterGroup extends CounterGroupBase<TezCounter> {
+  // essentially a typedef so user doesn't have to use generic syntax
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
new file mode 100644
index 0000000..3b702ba
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupBase.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * The common counter group interface.
+ *
+ * @param <T> type of the counter for the group
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface CounterGroupBase<T extends TezCounter>
+    extends Writable, Iterable<T> {
+
+  /**
+   * Get the internal name of the group
+   * @return the internal name
+   */
+  String getName();
+
+  /**
+   * Get the display name of the group.
+   * @return the human readable name
+   */
+  String getDisplayName();
+
+  /**
+   * Set the display name of the group
+   * @param displayName of the group
+   */
+  void setDisplayName(String displayName);
+
+  /** Add a counter to this group.
+   * @param counter to add
+   */
+  void addCounter(T counter);
+
+  /**
+   * Add a counter to this group
+   * @param name  of the counter
+   * @param displayName of the counter
+   * @param value of the counter
+   * @return the counter
+   */
+  T addCounter(String name, String displayName, long value);
+
+  /**
+   * Find a counter in the group.
+   * @param counterName the name of the counter
+   * @param displayName the display name of the counter
+   * @return the counter that was found or added
+   */
+  T findCounter(String counterName, String displayName);
+
+  /**
+   * Find a counter in the group
+   * @param counterName the name of the counter
+   * @param create create the counter if not found if true
+   * @return the counter that was found or added or null if create is false
+   */
+  T findCounter(String counterName, boolean create);
+
+  /**
+   * Find a counter in the group.
+   * @param counterName the name of the counter
+   * @return the counter that was found or added
+   */
+  T findCounter(String counterName);
+
+  /**
+   * @return the number of counters in this group.
+   */
+  int size();
+
+  /**
+   * Increment all counters by a group of counters
+   * @param rightGroup  the group to be added to this group
+   */
+  void incrAllCounters(CounterGroupBase<T> rightGroup);
+  
+  @Private
+  /**
+   * Exposes the underlying group type if a facade.
+   * @return the underlying object that this object is wrapping up.
+   */
+  CounterGroupBase<T> getUnderlyingGroup();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
new file mode 100644
index 0000000..45da0dd
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/CounterGroupFactory.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * An abstract class to provide common implementation of the
+ * group factory in both mapred and mapreduce packages.
+ *
+ * @param <C> type of the counter
+ * @param <G> type of the group
+ */
+@InterfaceAudience.Private
+public abstract class CounterGroupFactory<C extends TezCounter,
+                                          G extends CounterGroupBase<C>> {
+
+  public interface FrameworkGroupFactory<F> {
+    F newGroup(String name);
+  }
+
+  // Integer mapping (for serialization) for framework groups
+  private static final Map<String, Integer> s2i = Maps.newHashMap();
+  private static final List<String> i2s = Lists.newArrayList();
+  private static final int VERSION = 1;
+  private static final String FS_GROUP_NAME = FileSystemCounter.class.getName();
+
+  private final Map<String, FrameworkGroupFactory<G>> fmap = Maps.newHashMap();
+  {
+    // Add builtin counter class here and the version when changed.
+    addFrameworkGroup(TaskCounter.class);
+    addFrameworkGroup(JobCounter.class);
+    addFrameworkGroup(DAGCounter.class);
+  }
+
+  // Initialize the framework counter group mapping
+  private synchronized <T extends Enum<T>>
+  void addFrameworkGroup(final Class<T> cls) {
+    updateFrameworkGroupMapping(cls);
+    fmap.put(cls.getName(), newFrameworkGroupFactory(cls));
+  }
+
+  // Update static mappings (c2i, i2s) of framework groups
+  private static synchronized void updateFrameworkGroupMapping(Class<?> cls) {
+    String name = cls.getName();
+    Integer i = s2i.get(name);
+    if (i != null) return;
+    i2s.add(name);
+    s2i.put(name, i2s.size() - 1);
+  }
+
+  /**
+   * Required override to return a new framework group factory
+   * @param <T> type of the counter enum class
+   * @param cls the counter enum class
+   * @return a new framework group factory
+   */
+  protected abstract <T extends Enum<T>>
+  FrameworkGroupFactory<G> newFrameworkGroupFactory(Class<T> cls);
+
+  /**
+   * Create a new counter group
+   * @param name of the group
+   * @param limits the counters limits policy object
+   * @return a new counter group
+   */
+  public G newGroup(String name, Limits limits) {
+    return newGroup(name, ResourceBundles.getCounterGroupName(name, name),
+                    limits);
+  }
+
+  /**
+   * Create a new counter group
+   * @param name of the group
+   * @param displayName of the group
+   * @param limits the counters limits policy object
+   * @return a new counter group
+   */
+  public G newGroup(String name, String displayName, Limits limits) {
+    FrameworkGroupFactory<G> gf = fmap.get(name);
+    if (gf != null) return gf.newGroup(name);
+    if (name.equals(FS_GROUP_NAME)) {
+      return newFileSystemGroup();
+    } else if (s2i.get(name) != null) {
+      return newFrameworkGroup(s2i.get(name));
+    }
+    return newGenericGroup(name, displayName, limits);
+  }
+
+  /**
+   * Create a new framework group
+   * @param id of the group
+   * @return a new framework group
+   */
+  public G newFrameworkGroup(int id) {
+    String name;
+    synchronized(CounterGroupFactory.class) {
+      if (id < 0 || id >= i2s.size()) throwBadFrameGroupIdException(id);
+      name = i2s.get(id); // should not throw here.
+    }
+    FrameworkGroupFactory<G> gf = fmap.get(name);
+    if (gf == null) throwBadFrameGroupIdException(id);
+    return gf.newGroup(name);
+  }
+
+  /**
+   * Get the id of a framework group
+   * @param name of the group
+   * @return the framework group id
+   */
+  public static synchronized int getFrameworkGroupId(String name) {
+    Integer i = s2i.get(name);
+    if (i == null) throwBadFrameworkGroupNameException(name);
+    return i;
+  }
+
+  /**
+   * @return the counter factory version
+   */
+  public int version() {
+    return VERSION;
+  }
+
+  /**
+   * Check whether a group name is a name of a framework group (including
+   * the filesystem group).
+   *
+   * @param name  to check
+   * @return true for framework group names
+   */
+  public static synchronized boolean isFrameworkGroup(String name) {
+    return s2i.get(name) != null || name.equals(FS_GROUP_NAME);
+  }
+
+  private static void throwBadFrameGroupIdException(int id) {
+    throw new IllegalArgumentException("bad framework group id: "+ id);
+  }
+
+  private static void throwBadFrameworkGroupNameException(String name) {
+    throw new IllegalArgumentException("bad framework group name: "+ name);
+  }
+
+  /**
+   * Abstract factory method to create a generic (vs framework) counter group
+   * @param name  of the group
+   * @param displayName of the group
+   * @param limits limits of the counters
+   * @return a new generic counter group
+   */
+  protected abstract G newGenericGroup(String name, String displayName,
+                                       Limits limits);
+
+  /**
+   * Abstract factory method to create a file system counter group
+   * @return a new file system counter group
+   */
+  protected abstract G newFileSystemGroup();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
new file mode 100644
index 0000000..3598572
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+// Per-job counters
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum DAGCounter {
+  NUM_FAILED_TASKS, 
+  NUM_KILLED_TASKS,
+  TOTAL_LAUNCHED_TASKS,
+  OTHER_LOCAL_TASKS,
+  DATA_LOCAL_TASKS,
+  RACK_LOCAL_TASKS,
+  SLOTS_MILLIS_TASKS,
+  FALLOW_SLOTS_MILLIS_TASKS,
+  TOTAL_LAUNCHED_UBERTASKS,
+  NUM_UBER_SUBTASKS,
+  NUM_FAILED_UBERTASKS
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
new file mode 100644
index 0000000..08f4c5d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public enum FileSystemCounter {
+  BYTES_READ,
+  BYTES_WRITTEN,
+  READ_OPS,
+  LARGE_READ_OPS,
+  WRITE_OPS,
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
new file mode 100644
index 0000000..d4b167a
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounterGroup.java
@@ -0,0 +1,327 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.Iterator;
+import java.util.Locale;
+import java.util.Map;
+
+import com.google.common.base.Joiner;
+import static com.google.common.base.Preconditions.*;
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Maps;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * An abstract class to provide common implementation of the filesystem
+ * counter group in both mapred and mapreduce packages.
+ *
+ * @param <C> the type of the Counter for the group
+ */
+@InterfaceAudience.Private
+public abstract class FileSystemCounterGroup<C extends TezCounter>
+    implements CounterGroupBase<C> {
+
+  static final int MAX_NUM_SCHEMES = 100; // intern/sanity check
+  static final ConcurrentMap<String, String> schemes = Maps.newConcurrentMap();
+
+  // C[] would need Array.newInstance which requires a Class<C> reference.
+  // Just a few local casts probably worth not having to carry it around.
+  private final Map<String, Object[]> map =
+    new ConcurrentSkipListMap<String, Object[]>();
+  private String displayName;
+
+  private static final Joiner NAME_JOINER = Joiner.on('_');
+  private static final Joiner DISP_JOINER = Joiner.on(": ");
+
+  @InterfaceAudience.Private
+  public static class FSCounter extends AbstractCounter {
+    final String scheme;
+    final FileSystemCounter key;
+    private long value;
+
+    public FSCounter(String scheme, FileSystemCounter ref) {
+      this.scheme = scheme;
+      key = ref;
+    }
+
+    @Override
+    public String getName() {
+      return NAME_JOINER.join(scheme, key.name());
+    }
+
+    @Override
+    public String getDisplayName() {
+      return DISP_JOINER.join(scheme, localizeCounterName(key.name()));
+    }
+
+    protected String localizeCounterName(String counterName) {
+      return ResourceBundles.getCounterName(FileSystemCounter.class.getName(),
+                                            counterName, counterName);
+    }
+
+    @Override
+    public long getValue() {
+      return value;
+    }
+
+    @Override
+    public void setValue(long value) {
+      this.value = value;
+    }
+
+    @Override
+    public void increment(long incr) {
+      value += incr;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      assert false : "shouldn't be called";
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      assert false : "shouldn't be called";
+    }
+
+    @Override
+    public TezCounter getUnderlyingCounter() {
+      return this;
+    }
+  }
+
+  @Override
+  public String getName() {
+    return FileSystemCounter.class.getName();
+  }
+
+  @Override
+  public String getDisplayName() {
+    if (displayName == null) {
+      displayName = ResourceBundles.getCounterGroupName(getName(),
+          "File System Counters");
+    }
+    return displayName;
+  }
+
+  @Override
+  public void setDisplayName(String displayName) {
+    this.displayName = displayName;
+  }
+
+  @Override
+  public void addCounter(C counter) {
+    C ours;
+    if (counter instanceof FileSystemCounterGroup.FSCounter) {
+      FSCounter c = (FSCounter) counter;
+      ours = findCounter(c.scheme, c.key);
+    }
+    else {
+      ours = findCounter(counter.getName());
+    }
+    ours.setValue(counter.getValue());
+  }
+
+  @Override
+  public C addCounter(String name, String displayName, long value) {
+    C counter = findCounter(name);
+    counter.setValue(value);
+    return counter;
+  }
+
+  // Parse generic counter name into [scheme, key]
+  private String[] parseCounterName(String counterName) {
+    int schemeEnd = counterName.indexOf('_');
+    if (schemeEnd < 0) {
+      throw new IllegalArgumentException("bad fs counter name");
+    }
+    return new String[]{counterName.substring(0, schemeEnd),
+                        counterName.substring(schemeEnd + 1)};
+  }
+
+  @Override
+  public C findCounter(String counterName, String displayName) {
+    return findCounter(counterName);
+  }
+
+  @Override
+  public C findCounter(String counterName, boolean create) {
+    try {
+      String[] pair = parseCounterName(counterName);
+      return findCounter(pair[0], FileSystemCounter.valueOf(pair[1]));
+    }
+    catch (Exception e) {
+      if (create) throw new IllegalArgumentException(e);
+      return null;
+    }
+  }
+
+  @Override
+  public C findCounter(String counterName) {
+    return findCounter(counterName, true);
+  }
+
+  @SuppressWarnings("unchecked")
+  public synchronized C findCounter(String scheme, FileSystemCounter key) {
+    final String canonicalScheme = checkScheme(scheme);
+    Object[] counters = map.get(canonicalScheme);
+    int ord = key.ordinal();
+    if (counters == null) {
+      counters = new Object[FileSystemCounter.values().length];
+      map.put(canonicalScheme, counters);
+      counters[ord] = newCounter(canonicalScheme, key);
+    }
+    else if (counters[ord] == null) {
+      counters[ord] = newCounter(canonicalScheme, key);
+    }
+    return (C) counters[ord];
+  }
+
+  private String checkScheme(String scheme) {
+    String fixed = scheme.toUpperCase(Locale.US);
+    String interned = schemes.putIfAbsent(fixed, fixed);
+    if (schemes.size() > MAX_NUM_SCHEMES) {
+      // mistakes or abuses
+      throw new IllegalArgumentException("too many schemes? "+ schemes.size() +
+                                         " when process scheme: "+ scheme);
+    }
+    return interned == null ? fixed : interned;
+  }
+
+  /**
+   * Abstract factory method to create a file system counter
+   * @param scheme of the file system
+   * @param key the enum of the file system counter
+   * @return a new file system counter
+   */
+  protected abstract C newCounter(String scheme, FileSystemCounter key);
+
+  @Override
+  public int size() {
+    int n = 0;
+    for (Object[] counters : map.values()) {
+      n += numSetCounters(counters);
+    }
+    return n;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void incrAllCounters(CounterGroupBase<C> other) {
+    if (checkNotNull(other.getUnderlyingGroup(), "other group")
+        instanceof FileSystemCounterGroup<?>) {
+      for (TezCounter counter : other) {
+        FSCounter c = (FSCounter) ((TezCounter)counter).getUnderlyingCounter();
+        findCounter(c.scheme, c.key) .increment(counter.getValue());
+      }
+    }
+  }
+
+  /**
+   * FileSystemGroup ::= #scheme (scheme #counter (key value)*)*
+   */
+  @Override
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeVInt(out, map.size()); // #scheme
+    for (Map.Entry<String, Object[]> entry : map.entrySet()) {
+      WritableUtils.writeString(out, entry.getKey()); // scheme
+      // #counter for the above scheme
+      WritableUtils.writeVInt(out, numSetCounters(entry.getValue()));
+      for (Object counter : entry.getValue()) {
+        if (counter == null) continue;
+        @SuppressWarnings("unchecked")
+        FSCounter c = (FSCounter) ((TezCounter)counter).getUnderlyingCounter();
+        WritableUtils.writeVInt(out, c.key.ordinal());  // key
+        WritableUtils.writeVLong(out, c.getValue());    // value
+      }
+    }
+  }
+
+  private int numSetCounters(Object[] counters) {
+    int n = 0;
+    for (Object counter : counters) if (counter != null) ++n;
+    return n;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    int numSchemes = WritableUtils.readVInt(in);    // #scheme
+    FileSystemCounter[] enums = FileSystemCounter.values();
+    for (int i = 0; i < numSchemes; ++i) {
+      String scheme = WritableUtils.readString(in); // scheme
+      int numCounters = WritableUtils.readVInt(in); // #counter
+      for (int j = 0; j < numCounters; ++j) {
+        findCounter(scheme, enums[WritableUtils.readVInt(in)])  // key
+            .setValue(WritableUtils.readVLong(in)); // value
+      }
+    }
+  }
+
+  @Override
+  public Iterator<C> iterator() {
+    return new AbstractIterator<C>() {
+      Iterator<Object[]> it = map.values().iterator();
+      Object[] counters = it.hasNext() ? it.next() : null;
+      int i = 0;
+      @Override
+      protected C computeNext() {
+        while (counters != null) {
+          while (i < counters.length) {
+            @SuppressWarnings("unchecked")
+            C counter = (C) counters[i++];
+            if (counter != null) return counter;
+          }
+          i = 0;
+          counters = it.hasNext() ? it.next() : null;
+        }
+        return endOfData();
+      }
+    };
+  }
+
+  @Override
+  public synchronized boolean equals(Object genericRight) {
+    if (genericRight instanceof CounterGroupBase<?>) {
+      @SuppressWarnings("unchecked")
+      CounterGroupBase<C> right = (CounterGroupBase<C>) genericRight;
+      return Iterators.elementsEqual(iterator(), right.iterator());
+    }
+    return false;
+  }
+
+  @Override
+  public synchronized int hashCode() {
+    // need to be deep as counters is an array
+    int hash = FileSystemCounter.class.hashCode();
+    for (Object[] counters : map.values()) {
+      if (counters != null) hash ^= Arrays.hashCode(counters);
+    }
+    return hash;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java b/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
new file mode 100644
index 0000000..42fb636
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/FrameworkCounterGroup.java
@@ -0,0 +1,275 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.WritableUtils;
+
+import com.google.common.collect.AbstractIterator;
+import com.google.common.collect.Iterators;
+
+/**
+ * An abstract class to provide common implementation for the framework
+ * counter group in both mapred and mapreduce packages.
+ *
+ * @param <T> type of the counter enum class
+ * @param <C> type of the counter
+ */
+@InterfaceAudience.Private
+public abstract class FrameworkCounterGroup<T extends Enum<T>,
+    C extends TezCounter> implements CounterGroupBase<C> {
+  private static final Log LOG = LogFactory.getLog(FrameworkCounterGroup.class);
+  
+  private final Class<T> enumClass; // for Enum.valueOf
+  private final Object[] counters;  // local casts are OK and save a class ref
+  private String displayName = null;
+
+  /**
+   * A counter facade for framework counters.
+   * Use old (which extends new) interface to make compatibility easier.
+   */
+  @InterfaceAudience.Private
+  public static class FrameworkCounter<T extends Enum<T>> extends AbstractCounter {
+    final T key;
+    final String groupName;
+    private long value;
+
+    public FrameworkCounter(T ref, String groupName) {
+      key = ref;
+      this.groupName = groupName;
+    }
+
+    @Override
+    public String getName() {
+      return key.name();
+    }
+
+    @Override
+    public String getDisplayName() {
+      return ResourceBundles.getCounterName(groupName, getName(), getName());
+    }
+
+    @Override
+    public long getValue() {
+      return value;
+    }
+
+    @Override
+    public void setValue(long value) {
+      this.value = value;
+    }
+
+    @Override
+    public void increment(long incr) {
+      value += incr;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      assert false : "shouldn't be called";
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      assert false : "shouldn't be called";
+    }
+
+    @Override
+    public TezCounter getUnderlyingCounter() {
+      return this;
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public FrameworkCounterGroup(Class<T> enumClass) {
+    this.enumClass = enumClass;
+    T[] enums = enumClass.getEnumConstants();
+    counters = new Object[enums.length];
+  }
+
+  @Override
+  public String getName() {
+    return enumClass.getName();
+  }
+
+  @Override
+  public String getDisplayName() {
+    if (displayName == null) {
+      displayName = ResourceBundles.getCounterGroupName(getName(), getName());
+    }
+    return displayName;
+  }
+
+  @Override
+  public void setDisplayName(String displayName) {
+    this.displayName = displayName;
+  }
+
+  private T valueOf(String name) {
+    return Enum.valueOf(enumClass, name);
+  }
+
+  @Override
+  public void addCounter(C counter) {
+    C ours = findCounter(counter.getName());
+    ours.setValue(counter.getValue());
+  }
+
+  @Override
+  public C addCounter(String name, String displayName, long value) {
+    C counter = findCounter(name);
+    counter.setValue(value);
+    return counter;
+  }
+
+  @Override
+  public C findCounter(String counterName, String displayName) {
+    return findCounter(counterName);
+  }
+
+  @Override
+  public C findCounter(String counterName, boolean create) {
+    try {
+      return findCounter(valueOf(counterName));
+    }
+    catch (Exception e) {
+      if (create) throw new IllegalArgumentException(e);
+      return null;
+    }
+  }
+
+  @Override
+  public C findCounter(String counterName) {
+    return findCounter(valueOf(counterName));
+  }
+
+  @SuppressWarnings("unchecked")
+  private C findCounter(T key) {
+    int i = key.ordinal();
+    if (counters[i] == null) {
+      counters[i] = newCounter(key);
+    }
+    return (C) counters[i];
+  }
+
+  /**
+   * Abstract factory method for new framework counter
+   * @param key for the enum value of a counter
+   * @return a new counter for the key
+   */
+  protected abstract C newCounter(T key);
+
+  @Override
+  public int size() {
+    int n = 0;
+    for (int i = 0; i < counters.length; ++i) {
+      if (counters[i] != null) ++n;
+    }
+    return n;
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void incrAllCounters(CounterGroupBase<C> other) {
+    if (checkNotNull(other, "other counter group")
+        instanceof FrameworkCounterGroup<?, ?>) {
+      for (TezCounter counter : other) {
+        findCounter(((FrameworkCounter) counter).key.name())
+            .increment(counter.getValue());
+      }
+    }
+  }
+
+  /**
+   * FrameworkGroup ::= #counter (key value)*
+   */
+  @Override
+  @SuppressWarnings("unchecked")
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeVInt(out, size());
+    for (int i = 0; i < counters.length; ++i) {
+      TezCounter counter = (C) counters[i];
+      if (counter != null) {
+        WritableUtils.writeVInt(out, i);
+        WritableUtils.writeVLong(out, counter.getValue());
+      }
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    clear();
+    int len = WritableUtils.readVInt(in);
+    T[] enums = enumClass.getEnumConstants();
+    for (int i = 0; i < len; ++i) {
+      int ord = WritableUtils.readVInt(in);
+      TezCounter counter = newCounter(enums[ord]);
+      counter.setValue(WritableUtils.readVLong(in));
+      counters[ord] = counter;
+    }
+  }
+
+  private void clear() {
+    for (int i = 0; i < counters.length; ++i) {
+      counters[i] = null;
+    }
+  }
+
+  @Override
+  public Iterator<C> iterator() {
+    return new AbstractIterator<C>() {
+      int i = 0;
+      @Override
+      protected C computeNext() {
+        while (i < counters.length) {
+          @SuppressWarnings("unchecked")
+          C counter = (C) counters[i++];
+          if (counter != null) return counter;
+        }
+        return endOfData();
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object genericRight) {
+    if (genericRight instanceof CounterGroupBase<?>) {
+      @SuppressWarnings("unchecked")
+      CounterGroupBase<C> right = (CounterGroupBase<C>) genericRight;
+      return Iterators.elementsEqual(iterator(), right.iterator());
+    }
+    return false;
+  }
+
+  @Override
+  public synchronized int hashCode() {
+    // need to be deep as counters is an array
+    return Arrays.deepHashCode(new Object[]{enumClass, counters, displayName});
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/GenericCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/GenericCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/GenericCounter.java
new file mode 100644
index 0000000..5477606
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/GenericCounter.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * A generic counter implementation
+ */
+@InterfaceAudience.Private
+public class GenericCounter extends AbstractCounter {
+
+  private String name;
+  private String displayName;
+  private long value = 0;
+
+  public GenericCounter() {
+    // mostly for readFields
+  }
+
+  public GenericCounter(String name, String displayName) {
+    this.name = name;
+    this.displayName = displayName;
+  }
+
+  public GenericCounter(String name, String displayName, long value) {
+    this.name = name;
+    this.displayName = displayName;
+    this.value = value;
+  }
+
+  @Override @Deprecated
+  public synchronized void setDisplayName(String displayName) {
+    this.displayName = displayName;
+  }
+
+  @Override
+  public synchronized void readFields(DataInput in) throws IOException {
+    name = Text.readString(in);
+    displayName = in.readBoolean() ? Text.readString(in) : name;
+    value = WritableUtils.readVLong(in);
+  }
+
+  /**
+   * GenericCounter ::= keyName isDistinctDisplayName [displayName] value
+   */
+  @Override
+  public synchronized void write(DataOutput out) throws IOException {
+    Text.writeString(out, name);
+    boolean distinctDisplayName = ! name.equals(displayName);
+    out.writeBoolean(distinctDisplayName);
+    if (distinctDisplayName) {
+      Text.writeString(out, displayName);
+    }
+    WritableUtils.writeVLong(out, value);
+  }
+
+  @Override
+  public synchronized String getName() {
+    return name;
+  }
+
+  @Override
+  public synchronized String getDisplayName() {
+    return displayName;
+  }
+
+  @Override
+  public synchronized long getValue() {
+    return value;
+  }
+
+  @Override
+  public synchronized void setValue(long value) {
+    this.value = value;
+  }
+
+  @Override
+  public synchronized void increment(long incr) {
+    value += incr;
+  }
+
+  @Override
+  public TezCounter getUnderlyingCounter() {
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/JobCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/JobCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/JobCounter.java
new file mode 100644
index 0000000..1eb2be8
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/JobCounter.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+// Per-job counters
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum JobCounter {
+  NUM_FAILED_MAPS, 
+  NUM_FAILED_REDUCES,
+  NUM_KILLED_MAPS,
+  NUM_KILLED_REDUCES,
+  TOTAL_LAUNCHED_MAPS,
+  TOTAL_LAUNCHED_REDUCES,
+  OTHER_LOCAL_MAPS,
+  DATA_LOCAL_MAPS,
+  RACK_LOCAL_MAPS,
+  SLOTS_MILLIS_MAPS,
+  SLOTS_MILLIS_REDUCES,
+  FALLOW_SLOTS_MILLIS_MAPS,
+  FALLOW_SLOTS_MILLIS_REDUCES,
+  TOTAL_LAUNCHED_UBERTASKS,
+  NUM_UBER_SUBMAPS,
+  NUM_UBER_SUBREDUCES,
+  NUM_FAILED_UBERTASKS
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/LimitExceededException.java b/tez-api/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
new file mode 100644
index 0000000..e50bd81
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class LimitExceededException extends RuntimeException {
+
+  private static final long serialVersionUID = 1L;
+
+  public LimitExceededException(String msg) {
+    super(msg);
+  }
+
+  // Only allows chaining of related exceptions
+  public LimitExceededException(LimitExceededException cause) {
+    super(cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java b/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java
new file mode 100644
index 0000000..aacce87
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/Limits.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.TezJobConfig;
+
+@InterfaceAudience.Private
+public class Limits {
+
+  private static Configuration conf = null;
+  private static int GROUP_NAME_MAX;
+  private static int COUNTER_NAME_MAX;
+  private static int GROUPS_MAX;
+  private static int COUNTERS_MAX;
+  private static boolean initialized = false;
+
+  private static synchronized void ensureInitialized() {
+    if (initialized) {
+      return;
+    }
+    if (conf == null) {
+      conf = new Configuration();
+    }
+    GROUP_NAME_MAX =
+        conf.getInt(TezJobConfig.COUNTER_GROUP_NAME_MAX_KEY,
+            TezJobConfig.COUNTER_GROUP_NAME_MAX_DEFAULT);
+    COUNTER_NAME_MAX =
+        conf.getInt(TezJobConfig.COUNTER_NAME_MAX_KEY,
+            TezJobConfig.COUNTER_NAME_MAX_DEFAULT);
+    GROUPS_MAX =
+        conf.getInt(TezJobConfig.COUNTER_GROUPS_MAX_KEY,
+            TezJobConfig.COUNTER_GROUPS_MAX_DEFAULT);
+    COUNTERS_MAX =
+        conf.getInt(TezJobConfig.COUNTERS_MAX_KEY, TezJobConfig.
+            COUNTERS_MAX_DEFAULT);
+    initialized = true;
+  }
+
+  private int totalCounters;
+  private LimitExceededException firstViolation;
+
+  public static String filterName(String name, int maxLen) {
+    return name.length() > maxLen ? name.substring(0, maxLen - 1) : name;
+  }
+
+  public static String filterCounterName(String name) {
+    ensureInitialized();
+    return filterName(name, COUNTER_NAME_MAX);
+  }
+
+  public static String filterGroupName(String name) {
+    ensureInitialized();
+    return filterName(name, GROUP_NAME_MAX);
+  }
+
+  public synchronized void checkCounters(int size) {
+    ensureInitialized();
+    if (firstViolation != null) {
+      throw new LimitExceededException(firstViolation);
+    }
+    if (size > COUNTERS_MAX) {
+      firstViolation = new LimitExceededException("Too many counters: "+ size +
+                                                  " max="+ COUNTERS_MAX);
+      throw firstViolation;
+    }
+  }
+
+  public synchronized void incrCounters() {
+    checkCounters(totalCounters + 1);
+    ++totalCounters;
+  }
+
+  public synchronized void checkGroups(int size) {
+    ensureInitialized();
+    if (firstViolation != null) {
+      throw new LimitExceededException(firstViolation);
+    }
+    if (size > GROUPS_MAX) {
+      firstViolation = new LimitExceededException("Too many counter groups: "+
+                                                  size +" max="+ GROUPS_MAX);
+    }
+  }
+
+  public synchronized LimitExceededException violation() {
+    return firstViolation;
+  }
+
+  public synchronized static void setConfiguration(Configuration conf) {
+    if (Limits.conf == null && conf != null) {
+      Limits.conf = conf;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/ResourceBundles.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/ResourceBundles.java b/tez-api/src/main/java/org/apache/tez/common/counters/ResourceBundles.java
new file mode 100644
index 0000000..8113cab
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/ResourceBundles.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import java.util.Locale;
+import java.util.ResourceBundle;
+import java.util.MissingResourceException;
+
+/**
+ * Helper class to handle resource bundles in a saner way
+ */
+public class ResourceBundles {
+
+  /**
+   * Get a resource bundle
+   * @param bundleName of the resource
+   * @return the resource bundle
+   * @throws MissingResourceException
+   */
+  public static ResourceBundle getBundle(String bundleName) {
+    return ResourceBundle.getBundle(bundleName.replace('$', '_'),
+        Locale.getDefault(), Thread.currentThread().getContextClassLoader());
+  }
+
+  /**
+   * Get a resource given bundle name and key
+   * @param <T> type of the resource
+   * @param bundleName name of the resource bundle
+   * @param key to lookup the resource
+   * @param suffix for the key to lookup
+   * @param defaultValue of the resource
+   * @return the resource or the defaultValue
+   * @throws ClassCastException if the resource found doesn't match T
+   */
+  @SuppressWarnings("unchecked")
+  public static synchronized <T> T getValue(String bundleName, String key,
+                                            String suffix, T defaultValue) {
+    T value;
+    try {
+      ResourceBundle bundle = getBundle(bundleName);
+      value = (T) bundle.getObject(getLookupKey(key, suffix));
+      if (value != null) {
+        return value;
+      }
+    }
+    catch (Exception e) {
+      // Ignore
+    }
+    return defaultValue;
+  }
+
+  private static String getLookupKey(String key, String suffix) {
+    if (suffix == null || suffix.isEmpty()) return key;
+    return key + suffix;
+  }
+
+  /**
+   * Get the counter group display name
+   * @param group the group name to lookup
+   * @param defaultValue of the group
+   * @return the group display name
+   */
+  public static String getCounterGroupName(String group, String defaultValue) {
+    return getValue(group, "CounterGroupName", "", defaultValue);
+  }
+
+  /**
+   * Get the counter display name
+   * @param group the counter group name for the counter
+   * @param counter the counter name to lookup
+   * @param defaultValue of the counter
+   * @return the counter display name
+   */
+  public static String getCounterName(String group, String counter,
+                                      String defaultValue) {
+    return getValue(group, counter, ".name", defaultValue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java
new file mode 100644
index 0000000..b6fca27
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/TaskCounter.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+// TODO TEZAM5 For MR compatibility, a conversion from tez.TaskCounters to
+// mapreduce.TaskCounters will likely be required somewhere.
+// Similarly for FileSystemCounters and others.
+
+// Counters used by Task classes
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public enum TaskCounter {
+  // TODO Eventually, rename counters to be non-MR specific and map them to MR equivalent.
+  MAP_INPUT_RECORDS, 
+  MAP_OUTPUT_RECORDS,
+  MAP_SKIPPED_RECORDS,
+  MAP_OUTPUT_BYTES,
+  MAP_OUTPUT_MATERIALIZED_BYTES,
+  SPLIT_RAW_BYTES,
+  COMBINE_INPUT_RECORDS,
+  COMBINE_OUTPUT_RECORDS,
+  REDUCE_INPUT_GROUPS,
+  REDUCE_SHUFFLE_BYTES,
+  REDUCE_INPUT_RECORDS,
+  REDUCE_OUTPUT_RECORDS,
+  REDUCE_SKIPPED_GROUPS,
+  REDUCE_SKIPPED_RECORDS,
+  SPILLED_RECORDS,
+  SHUFFLED_MAPS, 
+  FAILED_SHUFFLE,
+  MERGED_MAP_OUTPUTS,
+  GC_TIME_MILLIS,
+  CPU_MILLISECONDS,
+  PHYSICAL_MEMORY_BYTES,
+  VIRTUAL_MEMORY_BYTES,
+  COMMITTED_HEAP_BYTES,
+  
+  INPUT_RECORDS, 
+  OUTPUT_RECORDS,
+  SKIPPED_RECORDS,
+  OUTPUT_BYTES,
+  OUTPUT_MATERIALIZED_BYTES,
+  INPUT_GROUPS,
+  SHUFFLE_BYTES,
+  SHUFFLED_TASKS, 
+  MERGED_TASK_OUTPUTS,
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java
new file mode 100644
index 0000000..394c820
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounter.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.io.Writable;
+
+/**
+ * A named counter that tracks the progress of a map/reduce job.
+ *
+ * <p><code>Counters</code> represent global counters, defined either by the
+ * Map-Reduce framework or applications. Each <code>Counter</code> is named by
+ * an {@link Enum} and has a long for the value.</p>
+ *
+ * <p><code>Counters</code> are bunched into Groups, each comprising of
+ * counters from a particular <code>Enum</code> class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public interface TezCounter extends Writable {
+
+  /**
+   * Set the display name of the counter
+   * @param displayName of the counter
+   * @deprecated (and no-op by default)
+   */
+  @Deprecated
+  void setDisplayName(String displayName);
+
+  /**
+   * @return the name of the counter
+   */
+  String getName();
+
+  /**
+   * Get the display name of the counter.
+   * @return the user facing name of the counter
+   */
+  String getDisplayName();
+
+  /**
+   * What is the current value of this counter?
+   * @return the current value
+   */
+  long getValue();
+
+  /**
+   * Set this counter by the given value
+   * @param value the value to set
+   */
+  void setValue(long value);
+
+  /**
+   * Increment this counter by the given value
+   * @param incr the value to increase this counter by
+   */
+  void increment(long incr);
+ 
+  /**
+   * Return the underlying object if this is a facade.
+   * @return the undelying object.
+   */
+  @Private
+  TezCounter getUnderlyingCounter();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java
new file mode 100644
index 0000000..1c9521a
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/TezCounters.java
@@ -0,0 +1,144 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.common.counters;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * <p><code>Counters</code> holds per job/task counters, defined either by the
+ * Map-Reduce framework or applications. Each <code>Counter</code> can be of
+ * any {@link Enum} type.</p>
+ *
+ * <p><code>Counters</code> are bunched into {@link CounterGroup}s, each
+ * comprising of counters from a particular <code>Enum</code> class.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class TezCounters extends AbstractCounters<TezCounter, CounterGroup> {
+
+  // Mix framework group implementation into CounterGroup interface
+  private static class FrameworkGroupImpl<T extends Enum<T>>
+      extends FrameworkCounterGroup<T, TezCounter> implements CounterGroup {
+
+    FrameworkGroupImpl(Class<T> cls) {
+      super(cls);
+    }
+
+    @Override
+    protected FrameworkCounter<T> newCounter(T key) {
+      return new FrameworkCounter<T>(key, getName());
+    }
+
+    @Override
+    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
+      return this;
+    }
+  }
+
+  // Mix generic group implementation into CounterGroup interface
+  // and provide some mandatory group factory methods.
+  private static class GenericGroup extends AbstractCounterGroup<TezCounter>
+      implements CounterGroup {
+
+    GenericGroup(String name, String displayName, Limits limits) {
+      super(name, displayName, limits);
+    }
+
+    @Override
+    protected TezCounter newCounter(String name, String displayName, long value) {
+      return new GenericCounter(name, displayName, value);
+    }
+
+    @Override
+    protected TezCounter newCounter() {
+      return new GenericCounter();
+    }
+
+    @Override
+    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
+      return this;
+    }
+  }
+
+  // Mix file system group implementation into the CounterGroup interface
+  private static class FileSystemGroup extends FileSystemCounterGroup<TezCounter>
+      implements CounterGroup {
+
+    @Override
+    protected TezCounter newCounter(String scheme, FileSystemCounter key) {
+      return new FSCounter(scheme, key);
+    }
+
+    @Override
+    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
+      return this;
+    }
+  }
+
+  /**
+   * Provide factory methods for counter group factory implementation.
+   * See also the GroupFactory in
+   *  {@link org.apache.hadoop.TezCounters.Counters mapred.Counters}
+   */
+  private static class GroupFactory
+      extends CounterGroupFactory<TezCounter, CounterGroup> {
+
+    @Override
+    protected <T extends Enum<T>>
+    FrameworkGroupFactory<CounterGroup>
+        newFrameworkGroupFactory(final Class<T> cls) {
+      return new FrameworkGroupFactory<CounterGroup>() {
+        @Override public CounterGroup newGroup(String name) {
+          return new FrameworkGroupImpl<T>(cls); // impl in this package
+        }
+      };
+    }
+
+    @Override
+    protected CounterGroup newGenericGroup(String name, String displayName,
+                                           Limits limits) {
+      return new GenericGroup(name, displayName, limits);
+    }
+
+    @Override
+    protected CounterGroup newFileSystemGroup() {
+      return new FileSystemGroup();
+    }
+  }
+
+  private static final GroupFactory groupFactory = new GroupFactory();
+
+  /**
+   * Default constructor
+   */
+  public TezCounters() {
+    super(groupFactory);
+  }
+
+  /**
+   * Construct the Counters object from the another counters object
+   * @param <C> the type of counter
+   * @param <G> the type of counter group
+   * @param counters the old counters object
+   */
+  public <C extends TezCounter, G extends CounterGroupBase<C>>
+  TezCounters(AbstractCounters<C, G> counters) {
+    super(counters, groupFactory);
+  }
+}


[10/20] git commit: Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module - tez-engine module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
 Rename *.new* packages back to what they should be, remove dead code
 from the old packages - mapreduce module - tez-engine module (part of
 TEZ-398). (sseth)


Project: http://git-wip-us.apache.org/repos/asf/incubator-tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tez/commit/b4950f98
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tez/tree/b4950f98
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tez/diff/b4950f98

Branch: refs/heads/TEZ-398
Commit: b4950f98a7a0c62d4c6d53ab75bfb857e8f6b551
Parents: 3d60945
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Sep 23 10:44:14 2013 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Sep 23 10:44:14 2013 -0700

----------------------------------------------------------------------
 .../apache/hadoop/mapred/YarnTezDagChild.java   |   24 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |    6 +-
 .../java/org/apache/tez/dag/app/dag/Task.java   |    2 +-
 .../java/org/apache/tez/dag/app/dag/Vertex.java |    4 +-
 .../dag/event/TaskAttemptEventStatusUpdate.java |    2 +-
 .../dag/app/dag/event/TaskEventAddTezEvent.java |    2 +-
 .../app/dag/event/VertexEventRouteEvent.java    |    2 +-
 .../org/apache/tez/dag/app/dag/impl/Edge.java   |   10 +-
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |    4 +-
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   |    2 +-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |   12 +-
 .../app/rm/AMSchedulerEventTALaunchRequest.java |    2 +-
 .../rm/container/AMContainerEventAssignTA.java  |    2 +-
 .../dag/app/rm/container/AMContainerImpl.java   |    2 +-
 .../dag/app/rm/container/AMContainerTask.java   |    2 +-
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |    2 +-
 .../tez/dag/app/rm/TestContainerReuse.java      |    7 +-
 .../org/apache/tez/common/ContainerTask.java    |    2 +-
 .../apache/tez/common/RunningTaskContext.java   |   70 -
 .../apache/tez/common/TezEngineTaskContext.java |  141 --
 .../org/apache/tez/common/TezTaskReporter.java  |  116 --
 .../tez/common/TezTaskUmbilicalProtocol.java    |   14 +-
 .../org/apache/tez/engine/api/KVReader.java     |   81 +
 .../org/apache/tez/engine/api/KVWriter.java     |   40 +
 .../api/events/TaskAttemptCompletedEvent.java   |   28 +
 .../api/events/TaskAttemptFailedEvent.java      |   35 +
 .../api/events/TaskStatusUpdateEvent.java       |   70 +
 .../tez/engine/api/impl/EventMetaData.java      |  152 ++
 .../apache/tez/engine/api/impl/EventType.java   |   29 +
 .../apache/tez/engine/api/impl/InputSpec.java   |   88 +
 .../apache/tez/engine/api/impl/OutputSpec.java  |   87 +
 .../apache/tez/engine/api/impl/TaskSpec.java    |  146 ++
 .../apache/tez/engine/api/impl/TezEvent.java    |  248 +++
 .../engine/api/impl/TezHeartbeatRequest.java    |  137 ++
 .../engine/api/impl/TezHeartbeatResponse.java   |  105 ++
 .../engine/api/impl/TezInputContextImpl.java    |   84 +
 .../engine/api/impl/TezOutputContextImpl.java   |   85 +
 .../api/impl/TezProcessorContextImpl.java       |   86 +
 .../tez/engine/api/impl/TezTaskContextImpl.java |  145 ++
 .../tez/engine/api/impl/TezUmbilical.java       |   36 +
 .../broadcast/input/BroadcastInputManager.java  |    2 +-
 .../broadcast/input/BroadcastKVReader.java      |    2 +-
 .../broadcast/output/FileBasedKVWriter.java     |    4 +-
 .../tez/engine/common/TezEngineUtils.java       |    4 +-
 .../tez/engine/common/combine/CombineInput.java |  176 --
 .../engine/common/combine/CombineOutput.java    |   55 -
 .../common/localshuffle/LocalShuffle.java       |    4 +-
 .../engine/common/shuffle/impl/MapOutput.java   |    2 +-
 .../common/shuffle/impl/MergeManager.java       |    2 +-
 .../tez/engine/common/sort/SortingOutput.java   |   32 -
 .../engine/common/sort/impl/ExternalSorter.java |    2 +-
 .../common/task/impl/CombineValuesIterator.java |   51 -
 .../newoutput/TezLocalTaskOutputFiles.java      |  249 ---
 .../task/local/newoutput/TezTaskOutput.java     |  165 --
 .../local/newoutput/TezTaskOutputFiles.java     |  246 ---
 .../local/output/TezLocalTaskOutputFiles.java   |   58 +-
 .../common/task/local/output/TezTaskOutput.java |   45 +-
 .../task/local/output/TezTaskOutputFiles.java   |   65 +-
 .../engine/lib/input/ShuffledMergedInput.java   |    2 +-
 .../engine/lib/oldinput/LocalMergedInput.java   |   67 -
 .../lib/oldinput/OldShuffledMergedInput.java    |   74 -
 .../lib/oldoutput/OldInMemorySortedOutput.java  |   58 -
 .../oldoutput/OldLocalOnFileSorterOutput.java   |   38 -
 .../lib/oldoutput/OldOnFileSortedOutput.java    |   62 -
 .../engine/lib/output/InMemorySortedOutput.java |    2 +-
 .../lib/output/LocalOnFileSorterOutput.java     |    2 +-
 .../engine/lib/output/OnFileSortedOutput.java   |    2 +-
 .../lib/output/OnFileUnorderedKVOutput.java     |    2 +-
 .../org/apache/tez/engine/newapi/KVReader.java  |   79 -
 .../org/apache/tez/engine/newapi/KVWriter.java  |   38 -
 .../events/TaskAttemptCompletedEvent.java       |   28 -
 .../newapi/events/TaskAttemptFailedEvent.java   |   35 -
 .../newapi/events/TaskStatusUpdateEvent.java    |   70 -
 .../tez/engine/newapi/impl/EventMetaData.java   |  152 --
 .../tez/engine/newapi/impl/EventType.java       |   29 -
 .../tez/engine/newapi/impl/InputSpec.java       |   88 -
 .../tez/engine/newapi/impl/OutputSpec.java      |   87 -
 .../apache/tez/engine/newapi/impl/TaskSpec.java |  146 --
 .../apache/tez/engine/newapi/impl/TezEvent.java |  248 ---
 .../engine/newapi/impl/TezHeartbeatRequest.java |  137 --
 .../newapi/impl/TezHeartbeatResponse.java       |  105 --
 .../engine/newapi/impl/TezInputContextImpl.java |   84 -
 .../newapi/impl/TezOutputContextImpl.java       |   85 -
 .../newapi/impl/TezProcessorContextImpl.java    |   86 -
 .../engine/newapi/impl/TezTaskContextImpl.java  |  145 --
 .../tez/engine/newapi/impl/TezUmbilical.java    |   36 -
 .../LogicalIOProcessorRuntimeTask.java          |   20 +-
 .../tez/engine/newruntime/RuntimeTask.java      |    6 +-
 .../apache/tez/engine/runtime/RuntimeUtils.java |  164 --
 .../engine/shuffle/common/DiskFetchedInput.java |    2 +-
 .../org/apache/tez/engine/task/RuntimeTask.java |   92 -
 .../mapred/LocalClientProtocolProviderTez.java  |  108 +-
 .../hadoop/mapred/LocalJobRunnerMetricsTez.java |  196 +-
 .../apache/hadoop/mapred/LocalJobRunnerTez.java | 1753 +++++++++---------
 .../apache/tez/mapreduce/input/SimpleInput.java |    2 +-
 .../tez/mapreduce/output/SimpleOutput.java      |    2 +-
 .../mapreduce/processor/map/MapProcessor.java   |    4 +-
 .../processor/reduce/ReduceProcessor.java       |    4 +-
 .../tez/mapreduce/TestUmbilicalProtocol.java    |    4 +-
 .../tez/mapreduce/processor/MapUtils.java       |   43 +-
 .../processor/map/TestMapProcessor.java         |   45 +-
 .../processor/reduce/TestReduceProcessor.java   |   76 +-
 102 files changed, 2984 insertions(+), 4800 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
----------------------------------------------------------------------
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 31898a3..f32fa6b 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,23 +67,23 @@ import org.apache.tez.dag.api.TezConfiguration;
 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.newapi.events.TaskAttemptCompletedEvent;
-import org.apache.tez.engine.newapi.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.newapi.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.newapi.impl.EventMetaData;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
-import org.apache.tez.engine.newapi.impl.TezEvent;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
-import org.apache.tez.engine.newapi.impl.TezUmbilical;
 import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.input.SimpleInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
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 2be9c5f..36486c9 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
@@ -52,10 +52,10 @@ import org.apache.tez.dag.app.rm.container.AMContainerTask;
 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.engine.newapi.impl.TezEvent;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
 import org.apache.tez.engine.records.OutputContext;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
----------------------------------------------------------------------
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 0947a41..088a195 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.TaskReport;
 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.newapi.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezEvent;
 
 /**
  * Read only view of Task.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
----------------------------------------------------------------------
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 85240e7..42ff8de 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,8 +32,8 @@ import org.apache.tez.dag.app.dag.impl.Edge;
 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.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
+import org.apache.tez.engine.api.impl.InputSpec;
+import org.apache.tez.engine.api.impl.OutputSpec;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
----------------------------------------------------------------------
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 3a8c489..0b8db76 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.DAGCounter;
 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.newapi.events.TaskStatusUpdateEvent;
+import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
 
 public class TaskAttemptEventStatusUpdate extends TaskAttemptEvent {
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
----------------------------------------------------------------------
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 51f6d53..4154bd0 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.newapi.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezEvent;
 
 public class TaskEventAddTezEvent extends TaskEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
----------------------------------------------------------------------
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 c851ae0..37478cb 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 @@ package org.apache.tez.dag.app.dag.event;
 import java.util.List;
 
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.newapi.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezEvent;
 
 public class VertexEventRouteEvent extends VertexEvent {
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
----------------------------------------------------------------------
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 d565978..3605857 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.Vertex;
 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.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.engine.newapi.events.DataMovementEvent;
 import org.apache.tez.engine.newapi.events.InputFailedEvent;
 import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.impl.EventMetaData;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
-import org.apache.tez.engine.newapi.impl.TezEvent;
 
 public class Edge {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
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 1ae9dcd..f2b2776 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.TezTaskAttemptID;
 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.newapi.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
----------------------------------------------------------------------
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 92a1859..13fa915 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,7 +75,7 @@ import org.apache.tez.dag.history.events.TaskStartedEvent;
 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.newapi.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezEvent;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
----------------------------------------------------------------------
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 0bcba4f..1ec1225 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,14 +106,14 @@ import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.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.newapi.events.DataMovementEvent;
 import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.newapi.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.newapi.impl.EventMetaData;
-import org.apache.tez.engine.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
-import org.apache.tez.engine.newapi.impl.TezEvent;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
----------------------------------------------------------------------
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 1c30b0b..14edd96 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.hadoop.yarn.api.records.Resource;
 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.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
----------------------------------------------------------------------
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 dd178fb..76e80f5 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 @@ package org.apache.tez.dag.app.rm.container;
 
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 public class AMContainerEventAssignTA extends AMContainerEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
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 eccf92a..94dd580 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.NMCommunicatorLaunchRequestEvent;
 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.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 @SuppressWarnings("rawtypes")
 public class AMContainerImpl implements AMContainer {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
----------------------------------------------------------------------
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 be1c08e..c0ef524 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.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 public class AMContainerTask {
   private final boolean shouldDie;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
----------------------------------------------------------------------
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 b5e283b..676e747 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.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
----------------------------------------------------------------------
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 408f88a..3a6e008 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
@@ -29,7 +29,6 @@ import org.apache.hadoop.yarn.client.api.AMRMClient;
 import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.util.RackResolver;
-import org.apache.tez.common.TezEngineTaskContext;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.OutputDescriptor;
 import org.apache.tez.dag.api.ProcessorDescriptor;
@@ -57,9 +56,9 @@ import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+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.junit.Test;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java b/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
index 3c18d9f..e90f7fa 100644
--- a/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
+++ b/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.java
@@ -22,7 +22,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.hadoop.io.Writable;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TaskSpec;
 
 public class ContainerTask implements Writable {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/common/RunningTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/RunningTaskContext.java b/tez-engine/src/main/java/org/apache/tez/common/RunningTaskContext.java
deleted file mode 100644
index aac4095..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/RunningTaskContext.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.common;
-
-import java.io.IOException;
-
-import javax.crypto.SecretKey;
-
-import org.apache.hadoop.util.Progress;
-import org.apache.tez.engine.api.Partitioner;
-import org.apache.tez.engine.api.Processor;
-
-public class RunningTaskContext {
-  
-  protected SecretKey jobTokenSecret;
-  protected TezTaskReporter reporter;
-  protected Partitioner partitioner;
-  protected Processor combineProcessor;
-  protected TezTaskStatus status;
-  protected Progress progress = new Progress();
-
-  public Progress getProgress() {
-    return progress;
-  }
-
-  public void setJobTokenSecret(SecretKey jobTokenSecret) {
-    this.jobTokenSecret = jobTokenSecret;
-  }
-
-  public TezTaskStatus getStatus() {
-    return status;
-  }
-
-  public TezTaskReporter getTaskReporter() {
-    return reporter;
-  }
-
-  // TODO Doesn't belong here.
-  public Processor getCombineProcessor() {
-    return combineProcessor;
-  }
-
-  // TODO Doesn't belong here.
-  public Partitioner getPartitioner() {
-    return partitioner;
-  }
-
-  // TODO Doesn't belong here.
-  public SecretKey getJobTokenSecret() {
-    return jobTokenSecret;
-  }
-  
-  public void statusUpdate() throws IOException, InterruptedException {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/common/TezEngineTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/TezEngineTaskContext.java b/tez-engine/src/main/java/org/apache/tez/common/TezEngineTaskContext.java
deleted file mode 100644
index c012928..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/TezEngineTaskContext.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public class TezEngineTaskContext extends TezTaskContext {
-
-  // These two could be replaced by a TezConfiguration / DagSpec.
-  private List<InputSpec> inputSpecList;
-  private List<OutputSpec> outputSpecList;
-  private ProcessorDescriptor processorDescriptor;
-  
-  public TezEngineTaskContext() {
-    super();
-  }
-
-  public TezEngineTaskContext(TezTaskAttemptID taskAttemptID, String user,
-      String jobName, String vertexName, ProcessorDescriptor processorDescriptor,
-      List<InputSpec> inputSpecList, List<OutputSpec> outputSpecList) {
-    super(taskAttemptID, user, jobName, vertexName);
-    this.inputSpecList = inputSpecList;
-    this.outputSpecList = outputSpecList;
-    if (this.inputSpecList == null) {
-      inputSpecList = new ArrayList<InputSpec>(0);
-    }
-    if (this.outputSpecList == null) {
-      outputSpecList = new ArrayList<OutputSpec>(0);
-    }
-    this.inputSpecList = inputSpecList;
-    this.outputSpecList = outputSpecList;
-    this.processorDescriptor = processorDescriptor;
-  }
-
-  public String getRuntimeName() {
-    // FIXME. Add this to the DAG configuration, and fetch from there.
-    return "org.apache.tez.mapreduce.task.MRRuntimeTask";
-  }
-
-  public String getProcessorName() {
-    return processorDescriptor.getClassName();
-  }
-  
-  public byte[] getProcessorUserPayload() {
-    return processorDescriptor.getUserPayload();
-  }
-  
-  public List<InputSpec> getInputSpecList() {
-    return this.inputSpecList;
-  }
-  
-  public List<OutputSpec> getOutputSpecList() {
-    return this.outputSpecList;
-  }
-  
-  @Override
-  public void write(DataOutput out) throws IOException {
-    super.write(out);
-    byte[] procDesc = 
-        DagTypeConverters.convertToDAGPlan(processorDescriptor).toByteArray();
-    out.writeInt(procDesc.length);
-    out.write(procDesc);
-    out.writeInt(inputSpecList.size());
-    for (InputSpec inputSpec : inputSpecList) {
-      inputSpec.write(out);
-    }
-    out.writeInt(outputSpecList.size());
-    for (OutputSpec outputSpec : outputSpecList) {
-      outputSpec.write(out);
-    }
-  }
-  
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
-    int procDescLength = in.readInt();
-    // TODO at least 3 buffer copies here. Need to convert this to full PB
-    // TEZ-305
-    byte[] procDescBytes = new byte[procDescLength];
-    in.readFully(procDescBytes);
-    processorDescriptor = DagTypeConverters.convertProcessorDescriptorFromDAGPlan(
-        TezEntityDescriptorProto.parseFrom(procDescBytes)); 
-    int numInputSpecs = in.readInt();
-    inputSpecList = new ArrayList<InputSpec>(numInputSpecs);
-    for (int i = 0; i < numInputSpecs; i++) {
-      InputSpec inputSpec = new InputSpec();
-      inputSpec.readFields(in);
-      inputSpecList.add(inputSpec);
-    }
-    int numOutputSpecs = in.readInt();
-    outputSpecList = new ArrayList<OutputSpec>(numOutputSpecs);
-    for (int i = 0; i < numOutputSpecs; i++) {
-      OutputSpec outputSpec = new OutputSpec();
-      outputSpec.readFields(in);
-      outputSpecList.add(outputSpec);
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuffer sb = new StringBuffer();
-    sb.append("processorName=" + getProcessorName()
-        + ", inputSpecListSize=" + inputSpecList.size()
-        + ", outputSpecListSize=" + outputSpecList.size());
-    sb.append(", inputSpecList=[");
-    for (InputSpec i : inputSpecList) {
-      sb.append("{" + i.toString() + "}, ");
-    }
-    sb.append("], outputSpecList=[");
-    for (OutputSpec i : outputSpecList) {
-      sb.append("{" + i.toString() + "}, ");
-    }
-    sb.append("]");
-    return sb.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/common/TezTaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/TezTaskReporter.java b/tez-engine/src/main/java/org/apache/tez/common/TezTaskReporter.java
deleted file mode 100644
index 1931e31..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/TezTaskReporter.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.common;
-
-import java.io.IOException;
-
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
-
-public interface TezTaskReporter extends Progressable, Master {
-
-  public void setStatus(String status);
-
-  public float getProgress();
-
-  public void setProgress(float progress);
-  
-  public void progress();
-
-  public TezCounter getCounter(String group, String name);
-
-  public TezCounter getCounter(Enum<?> name);
-
-  public void incrCounter(String group, String counter, long amount);
-
-  public void incrCounter(Enum<?> key, long amount);
-
-  public void reportFatalError(TezTaskAttemptID taskAttemptId, 
-      Throwable exception, String logMsg);
-
-  public final TezTaskReporter NULL = new TezTaskReporter() {
-
-    @Override
-    public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-        int fromEventIdx, int maxEventsToFetch,
-        TezTaskAttemptID reduce) {
-      return null;
-    }
-    
-    @Override
-    public void setStatus(String status) {
-    }
-    
-    @Override
-    public void setProgress(float progress) {
-    }
-    
-    @Override
-    public void progress() {
-    }
-    
-    @Override
-    public void incrCounter(Enum<?> key, long amount) {
-    }
-    
-    @Override
-    public void incrCounter(String group, String counter, long amount) {
-    }
-    
-    @Override
-    public float getProgress() {
-      return 0.0f;
-    }
-    
-    @Override
-    public TezCounter getCounter(Enum<?> name) {
-      return null;
-    }
-    
-    @Override
-    public TezCounter getCounter(String group, String name) {
-      return null;
-    }
-
-    @Override
-    public void reportFatalError(TezTaskAttemptID taskAttemptId,
-        Throwable exception, String logMsg) {
-      // TODO Auto-generated method stub
-      
-    }
-
-    @Override
-    public long getProtocolVersion(String protocol, long clientVersion)
-        throws IOException {
-      // TODO TEZAM3
-      return 0;
-    }
-
-    @Override
-    public ProtocolSignature getProtocolSignature(String protocol,
-        long clientVersion, int clientMethodsHash) throws IOException {
-      // TODO TEZAM3
-      return null;
-    }
-  };
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java b/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
index 28991a8..7d81b4c 100644
--- a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
+++ b/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
@@ -22,13 +22,15 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.ipc.VersionedProtocol;
 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.Master;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
+import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
+import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
 import org.apache.tez.engine.records.OutputContext;
+import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 
 /** 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
@@ -36,7 +38,8 @@ import org.apache.tez.engine.records.OutputContext;
  * and parent is via this protocol. */
 @InterfaceAudience.Private
 @InterfaceStability.Stable
-public interface TezTaskUmbilicalProtocol extends Master {
+@ProtocolInfo(protocolName = "TezTaskUmbilicalProtocol", protocolVersion = 1)
+public interface TezTaskUmbilicalProtocol extends VersionedProtocol {
 
   public static final long versionID = 19L;
 
@@ -68,4 +71,7 @@ public interface TezTaskUmbilicalProtocol extends Master {
   public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
       throws IOException, TezException;
 
+  public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
+      int fromEventIdx, int maxEventsToFetch,
+      TezTaskAttemptID taskAttemptId);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java b/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
new file mode 100644
index 0000000..150b598
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+import java.io.IOException;
+
+import org.apache.tez.engine.newapi.Reader;
+
+/**
+ * A key/value(s) pair based {@link Reader}.
+ * 
+ * Example usage
+ * <code>
+ * while (kvReader.next()) {
+ *   KVRecord kvRecord = getCurrentKV();
+ *   Object key =  kvRecord.getKey();
+ *   Iterable values = kvRecord.getValues();
+ * </code>
+ *
+ */
+public interface KVReader extends Reader {
+
+  /**
+   * Moves to the next key/values(s) pair
+   * 
+   * @return true if another key/value(s) pair exists, false if there are no more.
+   * @throws IOException
+   *           if an error occurs
+   */
+  public boolean next() throws IOException;
+
+  /**
+   * Return the current key/value(s) pair. Use moveToNext() to advance.
+   * @return
+   * @throws IOException
+   */
+  public KVRecord getCurrentKV() throws IOException;
+  
+  // TODO NEWTEZ Move this to getCurrentKey and getCurrentValue independently. Otherwise usage pattern seems to be getCurrentKV.getKey, getCurrentKV.getValue
+  
+  // TODO NEWTEZ KVRecord which does not need to return a list!
+  // TODO NEWTEZ Parameterize this
+  /**
+   * Represents a key and an associated set of values
+   *
+   */
+  public static class KVRecord {
+
+    private Object key;
+    private Iterable<Object> values;
+
+    public KVRecord(Object key, Iterable<Object> values) {
+      this.key = key;
+      this.values = values;
+    }
+
+    public Object getKey() {
+      return this.key;
+    }
+
+    public Iterable<Object> getValues() {
+      return this.values;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
new file mode 100644
index 0000000..079d488
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api;
+
+import java.io.IOException;
+
+import org.apache.tez.engine.newapi.Writer;
+
+/**
+ * A key/value(s) pair based {@link Writer}
+ */
+public interface KVWriter extends Writer {
+  /**
+   * Writes a key/value pair.
+   * 
+   * @param key
+   *          the key to write
+   * @param value
+   *          the value to write
+   * @throws IOException
+   *           if an error occurs
+   */
+  public void write(Object key, Object value) throws IOException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
new file mode 100644
index 0000000..3a90f56
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
@@ -0,0 +1,28 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.events;
+
+import org.apache.tez.engine.newapi.Event;
+
+public class TaskAttemptCompletedEvent extends Event {
+
+  public TaskAttemptCompletedEvent() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
new file mode 100644
index 0000000..bd0bc04
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.events;
+
+import org.apache.tez.engine.newapi.Event;
+
+public class TaskAttemptFailedEvent extends Event {
+
+  private final String diagnostics;
+
+  public TaskAttemptFailedEvent(String diagnostics) {
+    this.diagnostics = diagnostics;
+  }
+
+  public String getDiagnostics() {
+    return diagnostics;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
new file mode 100644
index 0000000..c0d77da
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.events;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.engine.newapi.Event;
+
+public class TaskStatusUpdateEvent extends Event implements Writable {
+
+  private TezCounters tezCounters;
+  private float progress;
+
+  public TaskStatusUpdateEvent() {
+  }
+
+  public TaskStatusUpdateEvent(TezCounters tezCounters, float progress) {
+    this.tezCounters = tezCounters;
+    this.progress = progress;
+  }
+
+  public TezCounters getCounters() {
+    return tezCounters;
+  }
+
+  public float getProgress() {
+    return progress;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeFloat(progress);
+    if (tezCounters != null) {
+      out.writeBoolean(true);
+      tezCounters.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    progress = in.readFloat();
+    if (in.readBoolean()) {
+      tezCounters = new TezCounters();
+      tezCounters.readFields(in);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java
new file mode 100644
index 0000000..64df7bb
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventMetaData.java
@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+/**
+ * Class that encapsulates all the information to identify the unique
+ * object that either generated an Event or is the recipient of an Event.
+ */
+public class EventMetaData implements Writable {
+
+  public static enum EventProducerConsumerType {
+    INPUT,
+    PROCESSOR,
+    OUTPUT,
+    SYSTEM
+  }
+
+  /**
+   * Producer Type ( one of Input/Output/Processor ) that generated the Event
+   * or Consumer Type that will consume the Event.
+   */
+  private EventProducerConsumerType producerConsumerType;
+
+  /**
+   * Name of the vertex where the event was generated.
+   */
+  private String taskVertexName;
+
+  /**
+   * Name of the vertex to which the Input or Output is connected to.
+   */
+  private String edgeVertexName;
+
+  /**
+   * i'th physical input/output that this event maps to.
+   */
+  private int index;
+
+  /**
+   * Task Attempt ID
+   */
+  private TezTaskAttemptID taskAttemptID;
+
+  public EventMetaData() {
+  }
+
+  public EventMetaData(EventProducerConsumerType generator,
+      String taskVertexName, String edgeVertexName,
+      TezTaskAttemptID taskAttemptID) {
+    this.producerConsumerType = generator;
+    this.taskVertexName = taskVertexName;
+    this.edgeVertexName = edgeVertexName;
+    this.taskAttemptID = taskAttemptID;
+  }
+
+  public EventProducerConsumerType getEventGenerator() {
+    return producerConsumerType;
+  }
+
+  public TezTaskAttemptID getTaskAttemptID() {
+    return taskAttemptID;
+  }
+
+  public String getTaskVertexName() {
+    return taskVertexName;
+  }
+
+  public String getEdgeVertexName() {
+    return edgeVertexName;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(producerConsumerType.ordinal());
+    if (taskVertexName != null) {
+      out.writeBoolean(true);
+      out.writeUTF(taskVertexName);
+    } else {
+      out.writeBoolean(false);
+    }
+    if (edgeVertexName != null) {
+      out.writeBoolean(true);
+      out.writeUTF(edgeVertexName);
+    } else {
+      out.writeBoolean(false);
+    }
+    if(taskAttemptID != null) {
+      out.writeBoolean(true);
+      taskAttemptID.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+    
+    out.writeInt(index);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    producerConsumerType = EventProducerConsumerType.values()[in.readInt()];
+    if (in.readBoolean()) {
+      taskVertexName = in.readUTF();
+    }
+    if (in.readBoolean()) {
+      edgeVertexName = in.readUTF();
+    }
+    if (in.readBoolean()) {
+      taskAttemptID = new TezTaskAttemptID();
+      taskAttemptID.readFields(in);
+    }
+    index = in.readInt();
+  }
+
+  public int getIndex() {
+    return index;
+  }
+
+  public void setIndex(int index) {
+    this.index = index;
+  }
+
+  @Override
+  public String toString() {
+    return "{ producerConsumerType=" + producerConsumerType
+        + ", taskVertexName=" + taskVertexName
+        + ", edgeVertexName=" + edgeVertexName
+        + ", taskAttemptId=" + taskAttemptID
+        + ", index=" + index + " }";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java
new file mode 100644
index 0000000..52fc10d
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/EventType.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+public enum EventType {
+  TASK_ATTEMPT_COMPLETED_EVENT,
+  TASK_ATTEMPT_FAILED_EVENT,
+  DATA_MOVEMENT_EVENT,
+  INPUT_READ_ERROR_EVENT,
+  INPUT_FAILED_EVENT,
+  INTPUT_INFORMATION_EVENT,
+  TASK_STATUS_UPDATE_EVENT
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java
new file mode 100644
index 0000000..a9ef333
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/InputSpec.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+
+public class InputSpec implements Writable {
+
+  private String sourceVertexName;
+  private InputDescriptor inputDescriptor;
+  private int physicalEdgeCount;
+
+  public InputSpec() {
+  }
+
+  public InputSpec(String sourceVertexName, InputDescriptor inputDescriptor,
+      int physicalEdgeCount) {
+    this.sourceVertexName = sourceVertexName;
+    this.inputDescriptor = inputDescriptor;
+    this.physicalEdgeCount = physicalEdgeCount;
+  }
+
+  public String getSourceVertexName() {
+    return sourceVertexName;
+  }
+
+  public InputDescriptor getInputDescriptor() {
+    return inputDescriptor;
+  }
+
+  public int getPhysicalEdgeCount() {
+    return physicalEdgeCount;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    // TODONEWTEZ convert to PB
+    out.writeUTF(sourceVertexName);
+    out.writeInt(physicalEdgeCount);
+    byte[] inputDescBytes =
+        DagTypeConverters.convertToDAGPlan(inputDescriptor).toByteArray();
+    out.writeInt(inputDescBytes.length);
+    out.write(inputDescBytes);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    sourceVertexName = in.readUTF();
+    physicalEdgeCount = in.readInt();
+    int inputDescLen = in.readInt();
+    byte[] inputDescBytes = new byte[inputDescLen];
+    in.readFully(inputDescBytes);
+    inputDescriptor =
+        DagTypeConverters.convertInputDescriptorFromDAGPlan(
+            TezEntityDescriptorProto.parseFrom(inputDescBytes));
+  }
+
+  public String toString() {
+    return "{ sourceVertexName=" + sourceVertexName
+        + ", physicalEdgeCount" + physicalEdgeCount
+        + ", inputClassName=" + inputDescriptor.getClassName()
+        + " }";
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java
new file mode 100644
index 0000000..3a1d5d8
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/OutputSpec.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+
+public class OutputSpec implements Writable {
+
+  private String destinationVertexName;
+  private OutputDescriptor outputDescriptor;
+  private int physicalEdgeCount;
+
+  public OutputSpec() {
+  }
+
+  public OutputSpec(String destinationVertexName,
+      OutputDescriptor outputDescriptor, int physicalEdgeCount) {
+    this.destinationVertexName = destinationVertexName;
+    this.outputDescriptor = outputDescriptor;
+    this.physicalEdgeCount = physicalEdgeCount;
+  }
+
+  public String getDestinationVertexName() {
+    return destinationVertexName;
+  }
+
+  public OutputDescriptor getOutputDescriptor() {
+    return outputDescriptor;
+  }
+
+  public int getPhysicalEdgeCount() {
+    return physicalEdgeCount;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    // TODONEWTEZ convert to PB
+    out.writeUTF(destinationVertexName);
+    out.writeInt(physicalEdgeCount);
+    byte[] inputDescBytes =
+        DagTypeConverters.convertToDAGPlan(outputDescriptor).toByteArray();
+    out.writeInt(inputDescBytes.length);
+    out.write(inputDescBytes);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    destinationVertexName = in.readUTF();
+    physicalEdgeCount = in.readInt();
+    int inputDescLen = in.readInt();
+    byte[] inputDescBytes = new byte[inputDescLen];
+    in.readFully(inputDescBytes);
+    outputDescriptor =
+        DagTypeConverters.convertOutputDescriptorFromDAGPlan(
+            TezEntityDescriptorProto.parseFrom(inputDescBytes));
+  }
+
+  public String toString() {
+    return "{ destinationVertexName=" + destinationVertexName
+        + ", physicalEdgeCount" + physicalEdgeCount
+        + ", outputClassName=" + outputDescriptor.getClassName()
+        + " }";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java
new file mode 100644
index 0000000..6527777
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TaskSpec.java
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+public class TaskSpec implements Writable {
+
+  private TezTaskAttemptID taskAttemptId;
+  private String vertexName;
+  private String user;
+  private ProcessorDescriptor processorDescriptor;
+  private List<InputSpec> inputSpecList;
+  private List<OutputSpec> outputSpecList;
+
+  public TaskSpec() {
+  }
+
+  // TODO NEWTEZ Remove user
+  public TaskSpec(TezTaskAttemptID taskAttemptID, String user,
+      String vertexName, ProcessorDescriptor processorDescriptor,
+      List<InputSpec> inputSpecList, List<OutputSpec> outputSpecList) {
+    this.taskAttemptId = taskAttemptID;
+    this.vertexName = vertexName;
+    this.user = user;
+    this.processorDescriptor = processorDescriptor;
+    this.inputSpecList = inputSpecList;
+    this.outputSpecList = outputSpecList;
+  }
+
+  public String getVertexName() {
+    return vertexName;
+  }
+
+  public TezTaskAttemptID getTaskAttemptID() {
+    return taskAttemptId;
+  }
+
+  public String getUser() {
+    return user;
+  }
+
+  public ProcessorDescriptor getProcessorDescriptor() {
+    return processorDescriptor;
+  }
+
+  public List<InputSpec> getInputs() {
+    return inputSpecList;
+  }
+
+  public List<OutputSpec> getOutputs() {
+    return outputSpecList;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    taskAttemptId.write(out);
+    out.writeUTF(vertexName);
+    byte[] procDesc =
+        DagTypeConverters.convertToDAGPlan(processorDescriptor).toByteArray();
+    out.writeInt(procDesc.length);
+    out.write(procDesc);
+    out.writeInt(inputSpecList.size());
+    for (InputSpec inputSpec : inputSpecList) {
+      inputSpec.write(out);
+    }
+    out.writeInt(outputSpecList.size());
+    for (OutputSpec outputSpec : outputSpecList) {
+      outputSpec.write(out);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    taskAttemptId = new TezTaskAttemptID();
+    taskAttemptId.readFields(in);
+    vertexName = in.readUTF();
+    int procDescLength = in.readInt();
+    // TODO at least 3 buffer copies here. Need to convert this to full PB
+    // TEZ-305
+    byte[] procDescBytes = new byte[procDescLength];
+    in.readFully(procDescBytes);
+    processorDescriptor =
+        DagTypeConverters.convertProcessorDescriptorFromDAGPlan(
+            TezEntityDescriptorProto.parseFrom(procDescBytes));
+    int numInputSpecs = in.readInt();
+    inputSpecList = new ArrayList<InputSpec>(numInputSpecs);
+    for (int i = 0; i < numInputSpecs; i++) {
+      InputSpec inputSpec = new InputSpec();
+      inputSpec.readFields(in);
+      inputSpecList.add(inputSpec);
+    }
+    int numOutputSpecs = in.readInt();
+    outputSpecList = new ArrayList<OutputSpec>(numOutputSpecs);
+    for (int i = 0; i < numOutputSpecs; i++) {
+      OutputSpec outputSpec = new OutputSpec();
+      outputSpec.readFields(in);
+      outputSpecList.add(outputSpec);
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append("TaskAttemptID:" + taskAttemptId);
+    sb.append("processorName=" + processorDescriptor.getClassName()
+        + ", inputSpecListSize=" + inputSpecList.size()
+        + ", outputSpecListSize=" + outputSpecList.size());
+    sb.append(", inputSpecList=[");
+    for (InputSpec i : inputSpecList) {
+      sb.append("{" + i.toString() + "}, ");
+    }
+    sb.append("], outputSpecList=[");
+    for (OutputSpec i : outputSpecList) {
+      sb.append("{" + i.toString() + "}, ");
+    }
+    sb.append("]");
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
new file mode 100644
index 0000000..9d0228d
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
@@ -0,0 +1,248 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.api.TezUncheckedException;
+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.engine.newapi.Event;
+import org.apache.tez.engine.newapi.events.DataMovementEvent;
+import org.apache.tez.engine.newapi.events.InputFailedEvent;
+import org.apache.tez.engine.newapi.events.InputInformationEvent;
+import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
+
+import com.google.protobuf.ByteString;
+
+public class TezEvent implements Writable {
+
+  private EventType eventType;
+
+  private Event event;
+
+  private EventMetaData sourceInfo;
+
+  private EventMetaData destinationInfo;
+
+  public TezEvent() {
+  }
+
+  public TezEvent(Event event, EventMetaData sourceInfo) {
+    this.event = event;
+    this.setSourceInfo(sourceInfo);
+    if (event instanceof DataMovementEvent) {
+      eventType = EventType.DATA_MOVEMENT_EVENT;
+    } else if (event instanceof InputReadErrorEvent) {
+      eventType = EventType.INPUT_READ_ERROR_EVENT;
+    } else if (event instanceof TaskAttemptFailedEvent) {
+      eventType = EventType.TASK_ATTEMPT_FAILED_EVENT;
+    } else if (event instanceof TaskAttemptCompletedEvent) {
+      eventType = EventType.TASK_ATTEMPT_COMPLETED_EVENT;
+    } else if (event instanceof InputInformationEvent) {
+      eventType = EventType.INTPUT_INFORMATION_EVENT;
+    } else if (event instanceof InputFailedEvent) {
+      eventType = EventType.INPUT_FAILED_EVENT;
+    } else if (event instanceof TaskStatusUpdateEvent) {
+      eventType = EventType.TASK_STATUS_UPDATE_EVENT;
+    } else {
+      throw new TezUncheckedException("Unknown event, event="
+          + event.getClass().getName());
+    }
+  }
+
+  public Event getEvent() {
+    return event;
+  }
+
+  public EventMetaData getSourceInfo() {
+    return sourceInfo;
+  }
+
+  public void setSourceInfo(EventMetaData sourceInfo) {
+    this.sourceInfo = sourceInfo;
+  }
+
+  public EventMetaData getDestinationInfo() {
+    return destinationInfo;
+  }
+
+  public void setDestinationInfo(EventMetaData destinationInfo) {
+    this.destinationInfo = destinationInfo;
+  }
+
+  public EventType getEventType() {
+    return eventType;
+  }
+
+  private void serializeEvent(DataOutput out) throws IOException {
+    if (event == null) {
+      out.writeBoolean(false);
+      return;
+    }
+    out.writeBoolean(true);
+    out.writeInt(eventType.ordinal());
+    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
+      // TODO NEWTEZ convert to PB
+      TaskStatusUpdateEvent sEvt = (TaskStatusUpdateEvent) event;
+      sEvt.write(out);
+    } else {
+      byte[] eventBytes = null;
+      switch (eventType) {
+      case DATA_MOVEMENT_EVENT:
+        DataMovementEvent dmEvt = (DataMovementEvent) event;
+        eventBytes = DataMovementEventProto.newBuilder()
+          .setSourceIndex(dmEvt.getSourceIndex())
+          .setTargetIndex(dmEvt.getTargetIndex())
+          .setUserPayload(ByteString.copyFrom(dmEvt.getUserPayload()))
+          .build().toByteArray();
+        break;
+      case INPUT_READ_ERROR_EVENT:
+        InputReadErrorEvent ideEvt = (InputReadErrorEvent) event;
+        eventBytes = InputReadErrorEventProto.newBuilder()
+            .setIndex(ideEvt.getIndex())
+            .setDiagnostics(ideEvt.getDiagnostics())
+            .build().toByteArray();
+        break;
+      case TASK_ATTEMPT_FAILED_EVENT:
+        TaskAttemptFailedEvent tfEvt = (TaskAttemptFailedEvent) event;
+        eventBytes = TaskAttemptFailedEventProto.newBuilder()
+            .setDiagnostics(tfEvt.getDiagnostics())
+            .build().toByteArray();
+        break;
+      case TASK_ATTEMPT_COMPLETED_EVENT:
+        eventBytes = TaskAttemptCompletedEventProto.newBuilder()
+            .build().toByteArray();
+        break;
+      case INPUT_FAILED_EVENT:
+        InputFailedEvent ifEvt = (InputFailedEvent) event;
+        eventBytes = InputFailedEventProto.newBuilder()
+            .setSourceIndex(ifEvt.getSourceIndex())
+            .setTargetIndex(ifEvt.getTargetIndex())
+            .setVersion(ifEvt.getVersion()).build().toByteArray();
+      case INTPUT_INFORMATION_EVENT:
+        InputInformationEvent iEvt = (InputInformationEvent) event;
+        eventBytes = InputInformationEventProto.newBuilder()
+            .setUserPayload(ByteString.copyFrom(iEvt.getUserPayload()))
+            .build().toByteArray();
+      default:
+        throw new TezUncheckedException("Unknown TezEvent"
+           + ", type=" + eventType);
+      }
+      out.writeInt(eventBytes.length);
+      out.write(eventBytes);
+    }
+  }
+
+  private void deserializeEvent(DataInput in) throws IOException {
+    if (!in.readBoolean()) {
+      event = null;
+      return;
+    }
+    eventType = EventType.values()[in.readInt()];
+    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
+      // TODO NEWTEZ convert to PB
+      event = new TaskStatusUpdateEvent();
+      ((TaskStatusUpdateEvent)event).readFields(in);
+    } else {
+      int eventBytesLen = in.readInt();
+      byte[] eventBytes = new byte[eventBytesLen];
+      in.readFully(eventBytes);
+      switch (eventType) {
+      case DATA_MOVEMENT_EVENT:
+        DataMovementEventProto dmProto =
+            DataMovementEventProto.parseFrom(eventBytes);
+        event = new DataMovementEvent(dmProto.getSourceIndex(),
+            dmProto.getTargetIndex(),
+            dmProto.getUserPayload().toByteArray());
+        break;
+      case INPUT_READ_ERROR_EVENT:
+        InputReadErrorEventProto ideProto =
+            InputReadErrorEventProto.parseFrom(eventBytes);
+        event = new InputReadErrorEvent(ideProto.getDiagnostics(),
+            ideProto.getIndex(), ideProto.getVersion());
+        break;
+      case TASK_ATTEMPT_FAILED_EVENT:
+        TaskAttemptFailedEventProto tfProto =
+            TaskAttemptFailedEventProto.parseFrom(eventBytes);
+        event = new TaskAttemptFailedEvent(tfProto.getDiagnostics());
+        break;
+      case TASK_ATTEMPT_COMPLETED_EVENT:
+        event = new TaskAttemptCompletedEvent();
+        break;
+      case INPUT_FAILED_EVENT:
+        InputFailedEventProto ifProto =
+            InputFailedEventProto.parseFrom(eventBytes);
+        event = new InputFailedEvent(ifProto.getSourceIndex(),
+            ifProto.getTargetIndex(), ifProto.getVersion());
+        break;
+      case INTPUT_INFORMATION_EVENT:
+        InputInformationEventProto infoProto =
+            InputInformationEventProto.parseFrom(eventBytes);
+        event = new InputInformationEvent(
+            infoProto.getUserPayload().toByteArray());
+        break;
+      default:
+        throw new TezUncheckedException("Unknown TezEvent"
+           + ", type=" + eventType);
+      }
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    serializeEvent(out);
+    if (sourceInfo != null) {
+      out.writeBoolean(true);
+      sourceInfo.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+    if (destinationInfo != null) {
+      out.writeBoolean(true);
+      destinationInfo.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    deserializeEvent(in);
+    if (in.readBoolean()) {
+      sourceInfo = new EventMetaData();
+      sourceInfo.readFields(in);
+    }
+    if (in.readBoolean()) {
+      destinationInfo = new EventMetaData();
+      destinationInfo.readFields(in);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java
new file mode 100644
index 0000000..dc1a447
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatRequest.java
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+
+public class TezHeartbeatRequest implements Writable {
+
+  private String containerIdentifier;
+  private List<TezEvent> events;
+  private TezTaskAttemptID currentTaskAttemptID;
+  private int startIndex;
+  private int maxEvents;
+  private long requestId;
+
+  public TezHeartbeatRequest() {
+  }
+
+  public TezHeartbeatRequest(long requestId, List<TezEvent> events,
+      String containerIdentifier, TezTaskAttemptID taskAttemptID,
+      int startIndex, int maxEvents) {
+    this.containerIdentifier = containerIdentifier;
+    this.requestId = requestId;
+    this.events = Collections.unmodifiableList(events);
+    this.startIndex = startIndex;
+    this.maxEvents = maxEvents;
+    this.currentTaskAttemptID = taskAttemptID;
+  }
+
+  public String getContainerIdentifier() {
+    return containerIdentifier;
+  }
+
+  public List<TezEvent> getEvents() {
+    return events;
+  }
+
+  public int getStartIndex() {
+    return startIndex;
+  }
+
+  public int getMaxEvents() {
+    return maxEvents;
+  }
+
+  public long getRequestId() {
+    return requestId;
+  }
+
+  public TezTaskAttemptID getCurrentTaskAttemptID() {
+    return currentTaskAttemptID;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    if (events != null) {
+      out.writeBoolean(true);
+      out.writeInt(events.size());
+      for (TezEvent e : events) {
+        e.write(out);
+      }
+    } else {
+      out.writeBoolean(false);
+    }
+    if (currentTaskAttemptID != null) {
+      out.writeBoolean(true);
+      currentTaskAttemptID.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+    out.writeInt(startIndex);
+    out.writeInt(maxEvents);
+    out.writeLong(requestId);
+    Text.writeString(out, containerIdentifier);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    if (in.readBoolean()) {
+      int eventsCount = in.readInt();
+      events = new ArrayList<TezEvent>(eventsCount);
+      for (int i = 0; i < eventsCount; ++i) {
+        TezEvent e = new TezEvent();
+        e.readFields(in);
+        events.add(e);
+      }
+    }
+    if (in.readBoolean()) {
+      currentTaskAttemptID = new TezTaskAttemptID();
+      currentTaskAttemptID.readFields(in);
+    } else {
+      currentTaskAttemptID = null;
+    }
+    startIndex = in.readInt();
+    maxEvents = in.readInt();
+    requestId = in.readLong();
+    containerIdentifier = Text.readString(in);
+  }
+
+  @Override
+  public String toString() {
+    return "{ "
+        + " containerId=" + containerIdentifier
+        + ", requestId=" + requestId
+        + ", startIndex=" + startIndex
+        + ", maxEventsToGet=" + maxEvents
+        + ", taskAttemptId" + currentTaskAttemptID
+        + ", eventCount=" + (events != null ? events.size() : 0)
+        + " }";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java
new file mode 100644
index 0000000..22ae7eb
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezHeartbeatResponse.java
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.api.impl;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.io.Writable;
+
+public class TezHeartbeatResponse implements Writable {
+
+  private long lastRequestId;
+  private boolean shouldDie = false;
+  private List<TezEvent> events;
+
+  public TezHeartbeatResponse() {
+  }
+
+  public TezHeartbeatResponse(List<TezEvent> events) {
+    this.events = Collections.unmodifiableList(events);
+  }
+
+  public List<TezEvent> getEvents() {
+    return events;
+  }
+
+  public boolean shouldDie() {
+    return shouldDie;
+  }
+
+  public long getLastRequestId() {
+    return lastRequestId;
+  }
+
+  public void setEvents(List<TezEvent> events) {
+    this.events = Collections.unmodifiableList(events);
+  }
+
+  public void setLastRequestId(long lastRequestId ) {
+    this.lastRequestId = lastRequestId;
+  }
+
+  public void setShouldDie() {
+    this.shouldDie = true;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(lastRequestId);
+    out.writeBoolean(shouldDie);
+    if(events != null) {
+      out.writeBoolean(true);
+      out.writeInt(events.size());
+      for (TezEvent e : events) {
+        e.write(out);
+      }
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    lastRequestId = in.readLong();
+    shouldDie = in.readBoolean();
+    if(in.readBoolean()) {
+      int eventCount = in.readInt();
+      events = new ArrayList<TezEvent>(eventCount);
+      for (int i = 0; i < eventCount; ++i) {
+        TezEvent e = new TezEvent();
+        e.readFields(in);
+        events.add(e);
+      }
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "{ "
+        + " lastRequestId=" + lastRequestId
+        + ", shouldDie=" + shouldDie
+        + ", eventCount=" + (events != null ? events.size() : 0)
+        + " }";
+  }
+}


[12/20] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/api/Task.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Task.java b/tez-engine-api/src/main/java/org/apache/tez/engine/api/Task.java
deleted file mode 100644
index 8dc9601..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Task.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.api;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * {@link Task} is the base <em>tez</em> entity which consumes 
- * input key/value pairs through an {@link Input} pipe, 
- * processes them via a {@link Processor} and 
- * produces output key/value pairs for an {@link Output} pipe.
- */
-public interface Task {
-  
-  /**
-   * Initialize the {@link Task}.
-   * 
-   * @param conf Tez configuration. TODO: Replace with a context object.
-   * @param master master controlling the task
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void initialize(Configuration conf, byte[] userPayload,
-      Master master) throws IOException, InterruptedException;
-  
-  /**
-   * Get {@link Input} of the task.
-   * @return <code>Input</code> of the task
-   */
-  public Input[] getInputs();
-
-  /**
-   * Get {@link Processor}s of the task.
-   * @return <code>Processor</code>s of the task
-   */
-  public Processor getProcessor();
-
-  /**
-   * Get {@link Output}s of the task.
-   * @return <code>Output</code>s of the task
-   */
-  public Output[] getOutputs();
-
-  /**
-   * Run the {@link Task}.
-   * 
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void run() throws IOException, InterruptedException;
-  
-  /**
-   * Stop the {@link Task}.
-   * 
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void close() throws IOException, InterruptedException;
-  
-  
-  public Configuration getConfiguration();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java b/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
deleted file mode 100644
index 7099299..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectLifeCycle.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.common.objectregistry;
-
-/**
- * Defines the valid lifecycle and scope of Objects stored in ObjectRegistry.
- * Objects are guaranteed to not be valid outside of their defined life-cycle
- * period. Objects are not guaranteed to be retained through the defined period
- * as they may be evicted for various reasons.
- */
-public enum ObjectLifeCycle {
-  /** Objects are valid for the lifetime of the Tez JVM/Session
-   */
-  SESSION,
-  /** Objects are valid for the lifetime of the DAG.
-   */
-  DAG,
-  /** Objects are valid for the lifetime of the Vertex.
-   */
-  VERTEX,
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java b/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
deleted file mode 100644
index a27903d..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistry.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.common.objectregistry;
-
-/**
- * Preliminary version of a simple shared object cache to re-use
- * objects across multiple tasks within the same container/JVM.
- */
-public interface ObjectRegistry {
-
-  /**
-   * Insert or update object into the registry. This will remove an object
-   * associated with the same key with a different life-cycle as there is only
-   * one instance of an Object stored for a given key irrespective of the
-   * life-cycle attached to the Object.
-   * @param lifeCycle What life-cycle is the Object valid for
-   * @param key Key to identify the Object
-   * @param value Object to be inserted
-   * @return Previous Object associated with the key attached if present
-   * else null. Could return the same object if the object was associated with
-   * the same key for a different life-cycle.
-   */
-  public Object add(ObjectLifeCycle lifeCycle, String key, Object value);
-
-  /**
-   * Return the object associated with the provided key
-   * @param key Key to find object
-   * @return Object if found else null
-   */
-  public Object get(String key);
-
-  /**
-   * Delete the object associated with the provided key
-   * @param lifeCycle What life-cycle is the Object valid for
-   * @param key Key to find object
-   * @return True if an object was found and removed
-   */
-  public boolean delete(String key);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java b/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
deleted file mode 100644
index 94352b3..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryFactory.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.common.objectregistry;
-
-import com.google.inject.Inject;
-
-public class ObjectRegistryFactory {
-
-  @Inject
-  private static ObjectRegistry objectRegistry;
-
-  public static ObjectRegistry getObjectRegistry() {
-    return objectRegistry;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Event.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Event.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Event.java
deleted file mode 100644
index 20157c7..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Event.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-/**
- * Base class for all events generated within the Tez execution engine.
- * Used as the primary mode of communication between the AM, Inputs, Processors
- * and Outputs.
- */
-public abstract class Event {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Input.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Input.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Input.java
deleted file mode 100644
index 5608a85..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Input.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-import java.util.List;
-
-/**
- * Represents an input through which a TezProcessor receives data on an edge.
- * </p>
- *
- * <code>Input</code> classes must have a 0 argument public constructor for Tez
- * to construct the <code>Input</code>. Tez will take care of initializing and
- * closing the Input after a {@link Processor} completes. </p>
- */
-public interface Input {
-
-  /**
-   * Initializes the <code>Input</code>.
-   *
-   * @param inputContext
-   *          the {@link TezInputContext}
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> initialize(TezInputContext inputContext)
-      throws Exception;
-
-  /**
-   * Gets an instance of the {@link Reader} for this <code>Output</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public Reader getReader() throws Exception;
-
-  /**
-   * Handles user and system generated {@link Events}s, which typically carry
-   * information such as an output being available on the previous vertex.
-   *
-   * @param inputEvents
-   *          the list of {@link Event}s
-   */
-  public void handleEvents(List<Event> inputEvents);
-
-  /**
-   * Closes the <code>Input</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> close() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalIOProcessor.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalIOProcessor.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalIOProcessor.java
deleted file mode 100644
index 53b5184..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalIOProcessor.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-import java.util.Map;
-
-/**
- * Represents a processor which consumes {@link LogicalInput}s and produces
- * {@link LogicalOutput}s
- */
-public interface LogicalIOProcessor extends Processor {
-
-  /**
-   * Runs the {@link LogicalProcessor}
-   * 
-   * @param inputs
-   *          a map of the source vertex name to {@link LogicalInput} - one per
-   *          incoming edge.
-   * @param outputs
-   *          a map of the destination vertex name to {@link LogicalOutput} -
-   *          one per outgoing edge
-   * @throws Exception TODO
-   */
-  public void run(Map<String, LogicalInput> inputs,
-      Map<String, LogicalOutput> outputs) throws Exception;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalInput.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalInput.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalInput.java
deleted file mode 100644
index 554172c..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalInput.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-/**
- * An @link {@link Input} which handles all incoming physical connections on an
- * edge. A {@link LogicalIOProcessor} sees a single Logical Input per incoming
- * edge.
- */
-public interface LogicalInput extends Input {
-
-  /**
-   * Sets the number of physical inputs that this <code>LogicalInput</code> will
-   * receive. This will be called by the Tez framework before initializing the
-   * <code>LogicalInput</code>
-   * 
-   * @param numInputs
-   *          the number of physical inputs.
-   */
-  public void setNumPhysicalInputs(int numInputs);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalOutput.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalOutput.java
deleted file mode 100644
index d88e043..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/LogicalOutput.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-/**
- * An @link {@link Output} which handles all outgoing physical connections on an
- * edge. A {@link LogicalIOProcessor} sees a single Logical Output per outgoing
- * edge.
- */
-public interface LogicalOutput extends Output {
-  /**
-   * Sets the number of physical ouputs that this <code>LogicalOutput</code>
-   * will receive. This will be called by the Tez framework before initializing
-   * the <code>LogicalOutput</code>
-   * 
-   * @param numOutputs
-   *          the number of physical outputs
-   */
-  public void setNumPhysicalOutputs(int numOutputs);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Output.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Output.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Output.java
deleted file mode 100644
index 5a6b5da..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Output.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-import java.util.List;
-
-/**
- * Represents an Output through which a TezProcessor writes information on an
- * edge. </p>
- *
- * <code>Output</code> implementations must have a 0 argument public constructor
- * for Tez to construct the <code>Output</code>. Tez will take care of
- * initializing and closing the Input after a {@link Processor} completes. </p>
- */
-public interface Output {
-
-  /**
-   * Initializes the <code>Output</code>
-   *
-   * @param outputContext
-   *          the {@link TezOutputContext}
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws Exception;
-
-  /**
-   * Gets an instance of the {@link Writer} in an <code>Output</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public Writer getWriter() throws Exception;
-
-  /**
-   * Handles user and system generated {@link Events}s, which typically carry
-   * information such as a downstream vertex being ready to consume input.
-   *
-   * @param outputEvents
-   *          the list of {@link Event}s
-   */
-  public void handleEvents(List<Event> outputEvents);
-
-  /**
-   * Closes the <code>Output</code>
-   *
-   * @return
-   * @throws Exception
-   *           if an error occurs
-   */
-  public List<Event> close() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Processor.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Processor.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Processor.java
deleted file mode 100644
index 3e71b58..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Processor.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Output;
-
-/**
- * {@link Processor} represents the <em>Tez</em> entity responsible for
- * consuming {@link Input} and producing {@link Output}.
- */
-public interface Processor {
-
-  /**
-   * Initializes the <code>Processor</code>
-   *
-   * @param processorContext
-   * @throws IOException
-   *           if an error occurs
-   */
-  public void initialize(TezProcessorContext processorContext)
-      throws Exception;
-
-  /**
-   * Handles user and system generated {@link Events}s.
-   *
-   * @param processorEvents
-   *          the list of {@link Event}s
-   */
-  public void handleEvents(List<Event> processorEvents);
-
-  /**
-   * Closes the <code>Processor</code>
-   *
-   * @throws IOException
-   *           if an error occurs
-   */
-  public void close() throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Reader.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Reader.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Reader.java
deleted file mode 100644
index 8b8750c..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Reader.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-/**
- * A <code>Reader</code> represents the data being read in an {@link Input}
- */
-public interface Reader {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezInputContext.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezInputContext.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezInputContext.java
deleted file mode 100644
index a56c9d4..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezInputContext.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-/**
- * Context handle for the Input to initialize itself.
- */
-public interface TezInputContext extends TezTaskContext {
-
-  /**
-   * Get the Vertex Name of the Source that generated data for this Input
-   * @return Name of the Source Vertex
-   */
-  public String getSourceVertexName();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezOutputContext.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezOutputContext.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezOutputContext.java
deleted file mode 100644
index 34c6028..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezOutputContext.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-/**
- * Context handle for the Output to initialize itself.
- */
-public interface TezOutputContext extends TezTaskContext {
-
-  /**
-   * Get the Vertex Name of the Destination that is the recipient of this
-   * Output's data
-   * @return Name of the Destination Vertex
-   */
-  public String getDestinationVertexName();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java
deleted file mode 100644
index 5b44f23..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezProcessorContext.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-import java.io.IOException;
-
-/**
- * Context handle for the Processor to initialize itself.
- */
-public interface TezProcessorContext extends TezTaskContext {
-
-  /**
-   * Set the overall progress of this Task Attempt
-   * @param progress Progress in the range from [0.0 - 1.0f]
-   */
-  public void setProgress(float progress);
-
-  /**
-   * Check whether this attempt can commit its output
-   * @return true if commit allowed
-   * @throws IOException
-   */
-  public boolean canCommit() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezTaskContext.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezTaskContext.java
deleted file mode 100644
index 80f2ca0..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/TezTaskContext.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.common.counters.TezCounters;
-
-/**
- * Base interface for Context classes used to initialize the Input, Output
- * and Processor instances.
- */
-public interface TezTaskContext {
-
-  // TODO NEWTEZ
-  // Scale the maximum events we fetch per RPC call to mitigate OOM issues
-  // on the ApplicationMaster when a thundering herd of reducers fetch events
-  // This should not be necessary after HADOOP-8942
-
-  /**
-   * Get the {@link ApplicationId} for the running app
-   * @return the {@link ApplicationId}
-   */
-  public ApplicationId getApplicationId();
-
-  /**
-   * Get the current DAG Attempt Number
-   * @return DAG Attempt Number
-   */
-  public int getDAGAttemptNumber();
-
-  /**
-   * Get the index of this Task
-   * @return Task Index
-   */
-  public int getTaskIndex();
-
-  /**
-   * Get the current Task Attempt Number
-   * @return Task Attempt Number
-   */
-  public int getTaskAttemptNumber();
-
-  /**
-   * Get the name of the DAG
-   * @return the DAG name
-   */
-  public String getDAGName();
-
-  /**
-   * Get the name of the Vertex in which the task is running
-   * @return Vertex Name
-   */
-  public String getTaskVertexName();
-
-  public TezCounters getCounters();
-
-  /**
-   * Send Events to the AM and/or dependent Vertices
-   * @param events Events to be sent
-   */
-  public void sendEvents(List<Event> events);
-
-  /**
-   * Get the User Payload for the Input/Output/Processor
-   * @return User Payload
-   */
-  public byte[] getUserPayload();
-
-  /**
-   * Get the work diectories for the Input/Output/Processor
-   * @return an array of work dirs
-   */
-  public String[] getWorkDirs();
-
-  /**
-   * Returns an identifier which is unique to the specific Input, Processor or
-   * Output
-   *
-   * @return
-   */
-  public String getUniqueIdentifier();
-
-  /**
-   * Report a fatal error to the framework. This will cause the entire task to
-   * fail and should not be used for reporting temporary or recoverable errors
-   *
-   * @param exception an exception representing the error
-   */
-  public void fatalError(Throwable exception, String message);
-
-  /**
-   * Returns meta-data for the specified service. As an example, when the MR
-   * ShuffleHandler is used - this would return the jobToken serialized as bytes
-   *
-   * @param serviceName
-   *          the name of the service for which meta-data is required
-   * @return a ByteBuffer representing the meta-data
-   */
-  public ByteBuffer getServiceConsumerMetaData(String serviceName);
-
-  /**
-   * Return Provider meta-data for the specified service As an example, when the
-   * MR ShuffleHandler is used - this would return the shuffle port serialized
-   * as bytes
-   *
-   * @param serviceName
-   *          the name of the service for which provider meta-data is required
-   * @return a ByteBuffer representing the meta-data
-   */
-  public ByteBuffer getServiceProviderMetaData(String serviceName);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Writer.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Writer.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Writer.java
deleted file mode 100644
index d8c0b8a..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/Writer.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-/**
- * A <code>Writer</code> represents the data being written by an {@link Output}
- */
-public interface Writer {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/DataMovementEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/DataMovementEvent.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/DataMovementEvent.java
deleted file mode 100644
index 92006b8..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/DataMovementEvent.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.events;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.newapi.Event;
-
-/**
- * Event used by user code to send information between tasks. An output can
- * generate an Event of this type to sending information regarding output data
- * ( such as URI for file-based output data, port info in case of
- * streaming-based data transfers ) to the Input on the destination vertex.
- */
-public final class DataMovementEvent extends Event {
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that generated an Event.
-   * For a Processor-generated event, this is ignored.
-   */
-  private final int sourceIndex;
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that is meant to receive
-   * this Event. For a Processor event, this is ignored.
-   */
-  private int targetIndex;
-
-  /**
-   * User Payload for this Event
-   */
-  private final byte[] userPayload;
-
-  /**
-   * Version number to indicate what attempt generated this Event
-   */
-  private int version;
-
-  /**
-   * User Event constructor
-   * @param sourceIndex Index to identify the physical edge of the input/output
-   * that generated the event
-   * @param userPayload User Payload of the User Event
-   */
-  public DataMovementEvent(int sourceIndex,
-      byte[] userPayload) {
-    this.userPayload = userPayload;
-    this.sourceIndex = sourceIndex;
-  }
-
-  @Private
-  public DataMovementEvent(int sourceIndex,
-      int targetIndex,
-      byte[] userPayload) {
-    this.userPayload = userPayload;
-    this.sourceIndex = sourceIndex;
-    this.targetIndex = targetIndex;
-  }
-
-  /**
-   * Constructor for Processor-generated User Events
-   * @param userPayload
-   */
-  public DataMovementEvent(byte[] userPayload) {
-    this(-1, userPayload);
-  }
-
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  public int getSourceIndex() {
-    return sourceIndex;
-  }
-
-  public int getTargetIndex() {
-    return targetIndex;
-  }
-
-  @Private
-  public void setTargetIndex(int targetIndex) {
-    this.targetIndex = targetIndex;
-  }
-
-  public int getVersion() {
-    return version;
-  }
-
-  @Private
-  public void setVersion(int version) {
-    this.version = version;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputFailedEvent.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputFailedEvent.java
deleted file mode 100644
index 0ca4a3e..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputFailedEvent.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.events;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.newapi.Event;
-
-/**
- * Event sent from the AM to an Input to indicate that one of it's sources has
- * failed - effectively the input is no longer available from the particular
- * source.
- * Users are not expected to send this event.
- */
-public class InputFailedEvent extends Event{
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that generated the data.
-   * For a Processor-generated event, this is ignored.
-   */
-  private final int sourceIndex;
-
-  /**
-   * Index(i) of the i-th (physical) Input or Output that is meant to receive
-   * this Event. For a Processor event, this is ignored.
-   */
-  private int targetIndex;
-
-  /**
-   * Version number to indicate what attempt generated this Event
-   */
-  private int version;
-
-  /**
-   * User Event constructor
-   * @param sourceIndex Index to identify the physical edge of the input/output
-   * that generated the event
-   * @param userPayload User Payload of the User Event
-   */
-  public InputFailedEvent(int sourceIndex) {
-    this.sourceIndex = sourceIndex;
-  }
-
-  @Private
-  public InputFailedEvent(int sourceIndex,
-      int targetIndex,
-      int version) {
-    this.sourceIndex = sourceIndex;
-    this.targetIndex = targetIndex;
-    this.version = version;
-  }
-
-  public int getSourceIndex() {
-    return sourceIndex;
-  }
-
-  public int getTargetIndex() {
-    return targetIndex;
-  }
-
-  @Private
-  public void setTargetIndex(int targetIndex) {
-    this.targetIndex = targetIndex;
-  }
-
-  public int getVersion() {
-    return version;
-  }
-
-  @Private
-  public void setVersion(int version) {
-    this.version = version;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputInformationEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputInformationEvent.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputInformationEvent.java
deleted file mode 100644
index a452a98..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputInformationEvent.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.events;
-
-import org.apache.tez.engine.newapi.Event;
-
-/**
- * Event used to send user specific data from the user 
- * code in the AM to the task input
- */
-public class InputInformationEvent extends Event {
-
-  /**
-   * User Payload for this Event
-   */
-  private final byte[] userPayload;
-  public InputInformationEvent(byte[] userPayload) {
-    this.userPayload = userPayload;
-  }
-
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputReadErrorEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputReadErrorEvent.java b/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputReadErrorEvent.java
deleted file mode 100644
index f2517eb..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/newapi/events/InputReadErrorEvent.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.events;
-
-import org.apache.tez.engine.newapi.Event;
-
-/**
- * Event generated by an Input to indicate error when trying to retrieve data.
- * This is not necessarily a fatal event - it's an indication to the AM to retry
- * source data generation.
- */
-public final class InputReadErrorEvent extends Event {
-
-  /**
-   * Diagnostics/trace of the error that occurred on the Input's edge.
-   */
-  private final String diagnostics;
-
-  /**
-   * Index of the physical edge on which the error occurred.
-   */
-  private final int index;
-
-  /**
-   * Version of the data on which the error occurred.
-   */
-  private final int version;
-
-  public InputReadErrorEvent(String diagnostics, int index,
-      int version) {
-    super();
-    this.diagnostics = diagnostics;
-    this.index = index;
-    this.version = version;
-  }
-
-  public String getDiagnostics() {
-    return diagnostics;
-  }
-
-  public int getIndex() {
-    return index;
-  }
-
-  public int getVersion() {
-    return version;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/records/OutputContext.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/records/OutputContext.java b/tez-engine-api/src/main/java/org/apache/tez/engine/records/OutputContext.java
deleted file mode 100644
index a1e9f03..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/records/OutputContext.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.tez.engine.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-
-//TODO EVENTUALLY Add more interfaces. Maybe make this an abstract class.
-/**
- * Contains context information for Output. For example, shuffle headers, size
- * of output, etc. TODO Ideally should be Output specific.
- */
-
-public class OutputContext implements Writable {
-
-  public OutputContext(int shufflePort) {
-    this.shufflePort = shufflePort;
-  }
-  
-  public OutputContext() {
-  }
-
-  public int shufflePort;
-
-  public int getShufflePort() {
-    return this.shufflePort;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(shufflePort);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    shufflePort = in.readInt();
-  }
-  
-  @Override
-  public String toString() {
-    return "shufflePort: " + shufflePort;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java b/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
deleted file mode 100644
index 7a4dd13..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
+++ /dev/null
@@ -1,228 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-/**
- * This is used to track task completion events on 
- * job tracker. 
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-// TODO TEZAM3 This needs to be more generic. Maybe some kind of a serialized
-// blob - which can be interpretted by the Input plugin.
-public class TezDependentTaskCompletionEvent implements Writable {
-  @InterfaceAudience.Public
-  @InterfaceStability.Evolving
-  // TODO EVENTUALLY - Remove TIPFAILED state ?
-  static public enum Status {FAILED, KILLED, SUCCEEDED, OBSOLETE, TIPFAILED};
-    
-  private int eventId;
-  private int taskRunTime; // using int since runtime is the time difference
-  private TezTaskAttemptID taskAttemptId;
-  private long dataSize;
-  Status status;
-  byte[] userPayload;
-  // TODO TEZAM2 Get rid of the isMap field. Job specific type information can be determined from TaskAttemptId.getTaskType
-//  boolean isMap = false;
-  public static final TezDependentTaskCompletionEvent[] EMPTY_ARRAY = 
-    new TezDependentTaskCompletionEvent[0];
-
-  public TezDependentTaskCompletionEvent() {
-    taskAttemptId = new TezTaskAttemptID();
-  }
-  
-  /**
-   * Constructor. eventId should be created externally and incremented
-   * per event for each job. 
-   * @param eventId event id, event id should be unique and assigned in
-   *  incrementally, starting from 0. 
-   * @param taskAttemptId task id
-   * @param status task's status 
-   * @param taskTrackerHttp task tracker's host:port for http. 
-   */
-  public TezDependentTaskCompletionEvent(int eventId, 
-                             TezTaskAttemptID taskAttemptId,
-//                             boolean isMap,
-                             Status status, 
-                             int runTime,
-                             long dataSize){
-      
-    this.taskAttemptId = taskAttemptId;
-//    this.isMap = isMap;
-    this.eventId = eventId; 
-    this.status =status; 
-    this.taskRunTime = runTime;
-    this.dataSize = dataSize;
-  }
-  
-  public TezDependentTaskCompletionEvent clone() {
-    TezDependentTaskCompletionEvent clone = new TezDependentTaskCompletionEvent(
-        this.eventId, this.taskAttemptId, this.status, 
-        this.taskRunTime, this.dataSize);
-    
-    return clone;
-  }
-  
-  /**
-   * Returns event Id. 
-   * @return event id
-   */
-  public int getEventId() {
-    return eventId;
-  }
-
-  /**
-   * Returns task id. 
-   * @return task id
-   */
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskAttemptId;
-  }
-  
-  /**
-   * Returns enum Status.SUCESS or Status.FAILURE.
-   * @return task tracker status
-   */
-  public Status getStatus() {
-    return status;
-  }
-  
-  /**
-   * Returns time (in millisec) the task took to complete. 
-   */
-  public int getTaskRunTime() {
-    return taskRunTime;
-  }
-  
-  /**
-   * Return size of output produced by the task
-   */
-  public long getDataSize() {
-    return dataSize;
-  }
-  
-  /**
-   * @return user payload. Maybe null
-   */
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  /**
-   * Set the task completion time
-   * @param taskCompletionTime time (in millisec) the task took to complete
-   */
-  protected void setTaskRunTime(int taskCompletionTime) {
-    this.taskRunTime = taskCompletionTime;
-  }
-
-  /**
-   * set event Id. should be assigned incrementally starting from 0. 
-   * @param eventId
-   */
-  public void setEventId(int eventId) {
-    this.eventId = eventId;
-  }
-
-  /**
-   * Sets task id. 
-   * @param taskId
-   */
-  public void setTaskAttemptID(TezTaskAttemptID taskId) {
-    this.taskAttemptId = taskId;
-  }
-  
-  /**
-   * Set task status. 
-   * @param status
-   */
-  public void setTaskStatus(Status status) {
-    this.status = status;
-  }
-  
-  /**
-   * Set the user payload
-   * @param userPayload
-   */
-  public void setUserPayload(byte[] userPayload) {
-    this.userPayload = userPayload;
-  }
-    
-  @Override
-  public String toString(){
-    StringBuffer buf = new StringBuffer(); 
-    buf.append("Task Id : "); 
-    buf.append(taskAttemptId); 
-    buf.append(", Status : ");  
-    buf.append(status.name());
-    return buf.toString();
-  }
-    
-  @Override
-  public boolean equals(Object o) {
-    // not counting userPayload as that is a piggyback mechanism
-    if(o == null)
-      return false;
-    if(o.getClass().equals(this.getClass())) {
-      TezDependentTaskCompletionEvent event = (TezDependentTaskCompletionEvent) o;
-      return this.eventId == event.getEventId()
-             && this.status.equals(event.getStatus())
-             && this.taskAttemptId.equals(event.getTaskAttemptID()) 
-             && this.taskRunTime == event.getTaskRunTime()
-             && this.dataSize == event.getDataSize();
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return toString().hashCode(); 
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    taskAttemptId.write(out);
-//    out.writeBoolean(isMap);
-    WritableUtils.writeEnum(out, status);
-    WritableUtils.writeVInt(out, taskRunTime);
-    WritableUtils.writeVInt(out, eventId);
-    WritableUtils.writeCompressedByteArray(out, userPayload);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    taskAttemptId.readFields(in);
-//    isMap = in.readBoolean();
-    status = WritableUtils.readEnum(in, Status.class);
-    taskRunTime = WritableUtils.readVInt(in);
-    eventId = WritableUtils.readVInt(in);
-    userPayload = WritableUtils.readCompressedByteArray(in);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java b/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
deleted file mode 100644
index 13c9088..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-
-
-public class TezTaskDependencyCompletionEventsUpdate implements Writable {
-  TezDependentTaskCompletionEvent[] events;
-  boolean reset;
-
-  public TezTaskDependencyCompletionEventsUpdate() { }
-
-  public TezTaskDependencyCompletionEventsUpdate(
-      TezDependentTaskCompletionEvent[] events, boolean reset) {
-    this.events = events;
-    this.reset = reset;
-  }
-
-  public boolean shouldReset() {
-    return reset;
-  }
-
-  public TezDependentTaskCompletionEvent[] getDependentTaskCompletionEvents() {
-    return events;
-  }
-  
-  public void write(DataOutput out) throws IOException {
-    out.writeBoolean(reset);
-    out.writeInt(events.length);
-    for (TezDependentTaskCompletionEvent event : events) {
-      event.write(out);
-    }
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    reset = in.readBoolean();
-    events = new TezDependentTaskCompletionEvent[in.readInt()];
-    for (int i = 0; i < events.length; ++i) {
-      events[i] = new TezDependentTaskCompletionEvent();
-      events[i].readFields(in);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/proto/Events.proto
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/proto/Events.proto b/tez-engine-api/src/main/proto/Events.proto
deleted file mode 100644
index 21cacf6..0000000
--- a/tez-engine-api/src/main/proto/Events.proto
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-option java_package = "org.apache.tez.engine.api.events";
-option java_outer_classname = "EventProtos";
-option java_generate_equals_and_hash = true;
-
-message DataMovementEventProto {
-  optional int32 source_index = 1;
-  optional int32 target_index = 2;
-  optional bytes user_payload = 3;
-  optional int32 version = 4;
-}
-
-message InputReadErrorEventProto {
-  optional int32 index = 1;
-  optional string diagnostics = 2;
-  optional int32 version = 3;
-}
-
-message InputFailedEventProto {
-  optional int32 source_index = 1;
-  optional int32 target_index = 2;
-  optional int32 version = 4;
-}
-
-message InputInformationEventProto {
-  optional bytes user_payload = 1;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/pom.xml
----------------------------------------------------------------------
diff --git a/tez-engine/pom.xml b/tez-engine/pom.xml
index 82eb48a..498f2f2 100644
--- a/tez-engine/pom.xml
+++ b/tez-engine/pom.xml
@@ -35,11 +35,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-dag-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-engine-api</artifactId>
+      <artifactId>tez-api</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/common/Constants.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/Constants.java b/tez-engine/src/main/java/org/apache/tez/common/Constants.java
new file mode 100644
index 0000000..8ea2909
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/common/Constants.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.tez.common;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+
+
+public class Constants {
+
+  // TODO NEWTEZ Check which of these constants are expecting specific pieces of information which are being removed - like taskAttemptId
+  
+  public static final String TEZ = "tez";
+
+  public static final String MAP_OUTPUT_FILENAME_STRING = "file.out";
+  public static final String MAP_OUTPUT_INDEX_SUFFIX_STRING = ".index";
+  public static final String REDUCE_INPUT_FILE_FORMAT_STRING = "%s/map_%d.out";
+
+  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
+  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_ENGINE_TASK_OUTPUT_FILENAME_STRING = "file.out";
+
+  public static final String TEZ_ENGINE_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_ENGINE_JOB_CREDENTIALS = 
+      "tez.engine.job.credentials";
+  
+  @Private
+  public static final String TEZ_ENGINE_TASK_MEMORY =  "tez.engine.task.memory";
+  
+  public static final String TASK_OUTPUT_DIR = "output";
+  
+  public static final String TEZ_ENGINE_TASK_OUTPUT_MANAGER = 
+      "tez.engine.task.local.output.manager";
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java b/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java
new file mode 100644
index 0000000..df92bdc
--- /dev/null
+++ b/tez-engine/src/main/java/org/apache/tez/common/ContainerContext.java
@@ -0,0 +1,64 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.common;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+
+// TODO EVENTUALLY move this over to PB. Fix package/module.
+// TODO EVENTUALLY unit tests for functionality.
+public class ContainerContext implements Writable {
+
+  String containerIdentifier;
+  String pid;
+
+  public ContainerContext() {
+    containerIdentifier = "";
+    pid = "";
+  }
+
+  public ContainerContext(String containerIdStr, String pid) {
+    this.containerIdentifier = containerIdStr;
+    this.pid = pid;
+  }
+
+  public String getContainerIdentifier() {
+    return containerIdentifier;
+  }
+
+  public String getPid() {
+    return pid;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    this.containerIdentifier = Text.readString(in);
+    this.pid = Text.readString(in);
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    Text.writeString(out, containerIdentifier);
+    Text.writeString(out, pid);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java b/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
index 7d81b4c..9e4129f 100644
--- a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
+++ b/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
@@ -29,8 +29,6 @@ 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.engine.records.OutputContext;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 
 /** 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
@@ -47,21 +45,6 @@ public interface TezTaskUmbilicalProtocol extends VersionedProtocol {
 
   boolean canCommit(TezTaskAttemptID taskid) throws IOException;
 
-  // TODO TEZAM5 Can commitPending and outputReady be collapsed into a single
-  // call.
-  // IAC outputReady followed by commit is a little confusing - since the output
-  // isn't really in place till a commit is called. Maybe rename to
-  // processingComplete or some such.
-
-  // TODO EVENTUALLY This is not the most useful API. Once there's some kind of
-  // support for the Task handing output over to the Container, this won't rally
-  // be required. i.e. InMemShuffle running as a service in the Container, or
-  // the second task in getTask(). ContainerUmbilical would include getTask and
-  // getServices...
-
-  void outputReady(TezTaskAttemptID taskAttemptId, OutputContext outputContext)
-      throws IOException;
-
   ProceedToCompletionResponse
       proceedToCompletion(TezTaskAttemptID taskAttemptId) throws IOException;
 
@@ -71,7 +54,4 @@ public interface TezTaskUmbilicalProtocol extends VersionedProtocol {
   public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
       throws IOException, TezException;
 
-  public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-      int fromEventIdx, int maxEventsToFetch,
-      TezTaskAttemptID taskAttemptId);
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java b/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
index 150b598..0178b3a 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
@@ -20,7 +20,7 @@ package org.apache.tez.engine.api;
 
 import java.io.IOException;
 
-import org.apache.tez.engine.newapi.Reader;
+import org.apache.tez.engine.api.Reader;
 
 /**
  * A key/value(s) pair based {@link Reader}.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
index 079d488..970831b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
@@ -20,7 +20,7 @@ package org.apache.tez.engine.api;
 
 import java.io.IOException;
 
-import org.apache.tez.engine.newapi.Writer;
+import org.apache.tez.engine.api.Writer;
 
 /**
  * A key/value(s) pair based {@link Writer}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
index 3a90f56..a9f2c98 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptCompletedEvent.java
@@ -18,7 +18,7 @@
 
 package org.apache.tez.engine.api.events;
 
-import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.api.Event;
 
 public class TaskAttemptCompletedEvent extends Event {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
index bd0bc04..fc67472 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskAttemptFailedEvent.java
@@ -18,7 +18,7 @@
 
 package org.apache.tez.engine.api.events;
 
-import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.api.Event;
 
 public class TaskAttemptFailedEvent extends Event {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
index c0d77da..c0d1ee6 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/events/TaskStatusUpdateEvent.java
@@ -24,7 +24,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.io.Writable;
 import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.api.Event;
 
 public class TaskStatusUpdateEvent extends Event implements Writable {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
index 9d0228d..6841d72 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezEvent.java
@@ -24,6 +24,11 @@ import java.io.IOException;
 
 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;
@@ -33,11 +38,6 @@ 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.engine.newapi.Event;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.InputInformationEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
 
 import com.google.protobuf.ByteString;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
index 2e10a93..245cd3b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezInputContextImpl.java
@@ -27,9 +27,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 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.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.engine.newruntime.RuntimeTask;
 
 public class TezInputContextImpl extends TezTaskContextImpl

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
index ef58de2..6b42e13 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezOutputContextImpl.java
@@ -27,9 +27,9 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
 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.newapi.Event;
-import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.newruntime.RuntimeTask;
 
 public class TezOutputContextImpl extends TezTaskContextImpl

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
index 3f20d5c..7ffcfd6 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezProcessorContextImpl.java
@@ -27,9 +27,9 @@ import java.util.Map;
 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.newapi.Event;
-import org.apache.tez.engine.newapi.TezProcessorContext;
 import org.apache.tez.engine.newruntime.RuntimeTask;
 
 public class TezProcessorContextImpl extends TezTaskContextImpl

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
index 2312c49..d5a4037 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/api/impl/TezTaskContextImpl.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
 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.newapi.TezTaskContext;
+import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.engine.newruntime.RuntimeTask;
 
 public abstract class TezTaskContextImpl implements TezTaskContext {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
index 927f0ad..1211598 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
@@ -25,9 +25,9 @@ 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.newapi.TezInputContext;
 import org.apache.tez.engine.shuffle.common.DiskFetchedInput;
 import org.apache.tez.engine.shuffle.common.FetchedInput;
 import org.apache.tez.engine.shuffle.common.FetchedInputAllocator;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
index 703e1d9..e89e892 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.java
@@ -24,13 +24,13 @@ 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.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.InvalidProtocolBufferException;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
index dace07c..7b205fa 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
@@ -49,14 +49,14 @@ import org.apache.hadoop.io.compress.Decompressor;
 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.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
 import org.apache.tez.engine.shuffle.common.FetchResult;
 import org.apache.tez.engine.shuffle.common.FetchedInput;
 import org.apache.tez.engine.shuffle.common.Fetcher;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
index 84ddd28..474d1cd 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
@@ -30,13 +30,13 @@ 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.engine.newapi.TezOutputContext;
 
 public class FileBasedKVWriter implements KVWriter {
 


[08/20] Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module - tez-engine module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldInMemorySortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldInMemorySortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldInMemorySortedOutput.java
deleted file mode 100644
index 9ac92ba..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldInMemorySortedOutput.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.lib.oldoutput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.RunningTaskContext;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.common.sort.SortingOutput;
-import org.apache.tez.engine.records.OutputContext;
-
-/**
- * {@link OldInMemorySortedOutput} is an {@link Output} which sorts key/value pairs 
- * written to it and persists it to a file.
- */
-public class OldInMemorySortedOutput implements SortingOutput {
-  
-  public OldInMemorySortedOutput(TezEngineTaskContext task) throws IOException {
-  }
-  
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException {
-  }
-
-  public void setTask(RunningTaskContext task) {
-  }
-  
-  public void write(Object key, Object value) throws IOException,
-      InterruptedException {
-  }
-
-  public void close() throws IOException, InterruptedException {
-  }
-
-  @Override
-  public OutputContext getOutputContext() {
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldLocalOnFileSorterOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldLocalOnFileSorterOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldLocalOnFileSorterOutput.java
deleted file mode 100644
index b7f913c..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldLocalOnFileSorterOutput.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.engine.lib.oldoutput;
-
-import java.io.IOException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tez.common.TezEngineTaskContext;
-
-public class OldLocalOnFileSorterOutput extends OldOnFileSortedOutput {
-
-  private static final Log LOG = LogFactory.getLog(OldLocalOnFileSorterOutput.class);
-
-  public OldLocalOnFileSorterOutput(TezEngineTaskContext task) throws IOException {
-    super(task);
-  }
-
-  @Override
-  public void close() throws IOException, InterruptedException {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldOnFileSortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldOnFileSortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldOnFileSortedOutput.java
deleted file mode 100644
index f259df9..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/oldoutput/OldOnFileSortedOutput.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.lib.oldoutput;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.RunningTaskContext;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.common.sort.SortingOutput;
-import org.apache.tez.engine.records.OutputContext;
-
-/**
- * {@link OldOnFileSortedOutput} is an {@link Output} which sorts key/value pairs 
- * written to it and persists it to a file.
- */
-public class OldOnFileSortedOutput implements SortingOutput {
-
-  public OldOnFileSortedOutput(TezEngineTaskContext task) throws IOException {
-  }
-  
-  @Override
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException {
-  }
-
-  @Override
-  public void setTask(RunningTaskContext task) {
-  }
-  
-  @Override
-  public void write(Object key, Object value) throws IOException,
-      InterruptedException {
-  }
-
-  @Override
-  public void close() throws IOException, InterruptedException {
-  }
-
-  @Override
-  public OutputContext getOutputContext() {
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
index 5d2a2ba..218aa21 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
@@ -22,9 +22,9 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.tez.common.TezUtils;
+import org.apache.tez.engine.api.KVWriter;
 import org.apache.tez.engine.common.sort.impl.dflt.InMemoryShuffleSorter;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalOutput;
 import org.apache.tez.engine.newapi.Output;
 import org.apache.tez.engine.newapi.TezOutputContext;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
index d23ac1e..963276d 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
@@ -25,7 +25,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.tez.engine.common.task.local.newoutput.TezTaskOutput;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
 import org.apache.tez.engine.newapi.Event;
 
 public class LocalOnFileSorterOutput extends OnFileSortedOutput {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
index ffb36c5..7e0ca37 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
@@ -26,11 +26,11 @@ import org.apache.hadoop.conf.Configuration;
 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.KVWriter;
 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.newapi.Event;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalOutput;
 import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.newapi.events.DataMovementEvent;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
index ec193c5..37edde8 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
@@ -24,10 +24,10 @@ import java.util.List;
 
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.engine.api.KVWriter;
 import org.apache.tez.engine.broadcast.output.FileBasedKVWriter;
 import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
 import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVWriter;
 import org.apache.tez.engine.newapi.LogicalOutput;
 import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.newapi.events.DataMovementEvent;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVReader.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVReader.java
deleted file mode 100644
index 79615ce..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVReader.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-import java.io.IOException;
-
-/**
- * A key/value(s) pair based {@link Reader}.
- * 
- * Example usage
- * <code>
- * while (kvReader.next()) {
- *   KVRecord kvRecord = getCurrentKV();
- *   Object key =  kvRecord.getKey();
- *   Iterable values = kvRecord.getValues();
- * </code>
- *
- */
-public interface KVReader extends Reader {
-
-  /**
-   * Moves to the next key/values(s) pair
-   * 
-   * @return true if another key/value(s) pair exists, false if there are no more.
-   * @throws IOException
-   *           if an error occurs
-   */
-  public boolean next() throws IOException;
-
-  /**
-   * Return the current key/value(s) pair. Use moveToNext() to advance.
-   * @return
-   * @throws IOException
-   */
-  public KVRecord getCurrentKV() throws IOException;
-  
-  // TODO NEWTEZ Move this to getCurrentKey and getCurrentValue independently. Otherwise usage pattern seems to be getCurrentKV.getKey, getCurrentKV.getValue
-  
-  // TODO NEWTEZ KVRecord which does not need to return a list!
-  // TODO NEWTEZ Parameterize this
-  /**
-   * Represents a key and an associated set of values
-   *
-   */
-  public static class KVRecord {
-
-    private Object key;
-    private Iterable<Object> values;
-
-    public KVRecord(Object key, Iterable<Object> values) {
-      this.key = key;
-      this.values = values;
-    }
-
-    public Object getKey() {
-      return this.key;
-    }
-
-    public Iterable<Object> getValues() {
-      return this.values;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVWriter.java
deleted file mode 100644
index ad48912..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/KVWriter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi;
-
-import java.io.IOException;
-
-/**
- * A key/value(s) pair based {@link Writer}
- */
-public interface KVWriter extends Writer {
-  /**
-   * Writes a key/value pair.
-   * 
-   * @param key
-   *          the key to write
-   * @param value
-   *          the value to write
-   * @throws IOException
-   *           if an error occurs
-   */
-  public void write(Object key, Object value) throws IOException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptCompletedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptCompletedEvent.java
deleted file mode 100644
index d3a582d..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptCompletedEvent.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.events;
-
-import org.apache.tez.engine.newapi.Event;
-
-public class TaskAttemptCompletedEvent extends Event {
-
-  public TaskAttemptCompletedEvent() {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptFailedEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptFailedEvent.java
deleted file mode 100644
index 772d7fe..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskAttemptFailedEvent.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.events;
-
-import org.apache.tez.engine.newapi.Event;
-
-public class TaskAttemptFailedEvent extends Event {
-
-  private final String diagnostics;
-
-  public TaskAttemptFailedEvent(String diagnostics) {
-    this.diagnostics = diagnostics;
-  }
-
-  public String getDiagnostics() {
-    return diagnostics;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskStatusUpdateEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskStatusUpdateEvent.java
deleted file mode 100644
index 0f09867..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/events/TaskStatusUpdateEvent.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.events;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.engine.newapi.Event;
-
-public class TaskStatusUpdateEvent extends Event implements Writable {
-
-  private TezCounters tezCounters;
-  private float progress;
-
-  public TaskStatusUpdateEvent() {
-  }
-
-  public TaskStatusUpdateEvent(TezCounters tezCounters, float progress) {
-    this.tezCounters = tezCounters;
-    this.progress = progress;
-  }
-
-  public TezCounters getCounters() {
-    return tezCounters;
-  }
-
-  public float getProgress() {
-    return progress;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeFloat(progress);
-    if (tezCounters != null) {
-      out.writeBoolean(true);
-      tezCounters.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    progress = in.readFloat();
-    if (in.readBoolean()) {
-      tezCounters = new TezCounters();
-      tezCounters.readFields(in);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventMetaData.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventMetaData.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventMetaData.java
deleted file mode 100644
index 9faafc5..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventMetaData.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-/**
- * Class that encapsulates all the information to identify the unique
- * object that either generated an Event or is the recipient of an Event.
- */
-public class EventMetaData implements Writable {
-
-  public static enum EventProducerConsumerType {
-    INPUT,
-    PROCESSOR,
-    OUTPUT,
-    SYSTEM
-  }
-
-  /**
-   * Producer Type ( one of Input/Output/Processor ) that generated the Event
-   * or Consumer Type that will consume the Event.
-   */
-  private EventProducerConsumerType producerConsumerType;
-
-  /**
-   * Name of the vertex where the event was generated.
-   */
-  private String taskVertexName;
-
-  /**
-   * Name of the vertex to which the Input or Output is connected to.
-   */
-  private String edgeVertexName;
-
-  /**
-   * i'th physical input/output that this event maps to.
-   */
-  private int index;
-
-  /**
-   * Task Attempt ID
-   */
-  private TezTaskAttemptID taskAttemptID;
-
-  public EventMetaData() {
-  }
-
-  public EventMetaData(EventProducerConsumerType generator,
-      String taskVertexName, String edgeVertexName,
-      TezTaskAttemptID taskAttemptID) {
-    this.producerConsumerType = generator;
-    this.taskVertexName = taskVertexName;
-    this.edgeVertexName = edgeVertexName;
-    this.taskAttemptID = taskAttemptID;
-  }
-
-  public EventProducerConsumerType getEventGenerator() {
-    return producerConsumerType;
-  }
-
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskAttemptID;
-  }
-
-  public String getTaskVertexName() {
-    return taskVertexName;
-  }
-
-  public String getEdgeVertexName() {
-    return edgeVertexName;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(producerConsumerType.ordinal());
-    if (taskVertexName != null) {
-      out.writeBoolean(true);
-      out.writeUTF(taskVertexName);
-    } else {
-      out.writeBoolean(false);
-    }
-    if (edgeVertexName != null) {
-      out.writeBoolean(true);
-      out.writeUTF(edgeVertexName);
-    } else {
-      out.writeBoolean(false);
-    }
-    if(taskAttemptID != null) {
-      out.writeBoolean(true);
-      taskAttemptID.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-    
-    out.writeInt(index);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    producerConsumerType = EventProducerConsumerType.values()[in.readInt()];
-    if (in.readBoolean()) {
-      taskVertexName = in.readUTF();
-    }
-    if (in.readBoolean()) {
-      edgeVertexName = in.readUTF();
-    }
-    if (in.readBoolean()) {
-      taskAttemptID = new TezTaskAttemptID();
-      taskAttemptID.readFields(in);
-    }
-    index = in.readInt();
-  }
-
-  public int getIndex() {
-    return index;
-  }
-
-  public void setIndex(int index) {
-    this.index = index;
-  }
-
-  @Override
-  public String toString() {
-    return "{ producerConsumerType=" + producerConsumerType
-        + ", taskVertexName=" + taskVertexName
-        + ", edgeVertexName=" + edgeVertexName
-        + ", taskAttemptId=" + taskAttemptID
-        + ", index=" + index + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventType.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventType.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventType.java
deleted file mode 100644
index 87d6665..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/EventType.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-public enum EventType {
-  TASK_ATTEMPT_COMPLETED_EVENT,
-  TASK_ATTEMPT_FAILED_EVENT,
-  DATA_MOVEMENT_EVENT,
-  INPUT_READ_ERROR_EVENT,
-  INPUT_FAILED_EVENT,
-  INTPUT_INFORMATION_EVENT,
-  TASK_STATUS_UPDATE_EVENT
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/InputSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/InputSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/InputSpec.java
deleted file mode 100644
index a2b8cc8..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/InputSpec.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.InputDescriptor;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-
-public class InputSpec implements Writable {
-
-  private String sourceVertexName;
-  private InputDescriptor inputDescriptor;
-  private int physicalEdgeCount;
-
-  public InputSpec() {
-  }
-
-  public InputSpec(String sourceVertexName, InputDescriptor inputDescriptor,
-      int physicalEdgeCount) {
-    this.sourceVertexName = sourceVertexName;
-    this.inputDescriptor = inputDescriptor;
-    this.physicalEdgeCount = physicalEdgeCount;
-  }
-
-  public String getSourceVertexName() {
-    return sourceVertexName;
-  }
-
-  public InputDescriptor getInputDescriptor() {
-    return inputDescriptor;
-  }
-
-  public int getPhysicalEdgeCount() {
-    return physicalEdgeCount;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    // TODONEWTEZ convert to PB
-    out.writeUTF(sourceVertexName);
-    out.writeInt(physicalEdgeCount);
-    byte[] inputDescBytes =
-        DagTypeConverters.convertToDAGPlan(inputDescriptor).toByteArray();
-    out.writeInt(inputDescBytes.length);
-    out.write(inputDescBytes);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    sourceVertexName = in.readUTF();
-    physicalEdgeCount = in.readInt();
-    int inputDescLen = in.readInt();
-    byte[] inputDescBytes = new byte[inputDescLen];
-    in.readFully(inputDescBytes);
-    inputDescriptor =
-        DagTypeConverters.convertInputDescriptorFromDAGPlan(
-            TezEntityDescriptorProto.parseFrom(inputDescBytes));
-  }
-
-  public String toString() {
-    return "{ sourceVertexName=" + sourceVertexName
-        + ", physicalEdgeCount" + physicalEdgeCount
-        + ", inputClassName=" + inputDescriptor.getClassName()
-        + " }";
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/OutputSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/OutputSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/OutputSpec.java
deleted file mode 100644
index 1b34ef0..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/OutputSpec.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.OutputDescriptor;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-
-public class OutputSpec implements Writable {
-
-  private String destinationVertexName;
-  private OutputDescriptor outputDescriptor;
-  private int physicalEdgeCount;
-
-  public OutputSpec() {
-  }
-
-  public OutputSpec(String destinationVertexName,
-      OutputDescriptor inputDescriptor, int physicalEdgeCount) {
-    this.destinationVertexName = destinationVertexName;
-    this.outputDescriptor = inputDescriptor;
-    this.physicalEdgeCount = physicalEdgeCount;
-  }
-
-  public String getDestinationVertexName() {
-    return destinationVertexName;
-  }
-
-  public OutputDescriptor getOutputDescriptor() {
-    return outputDescriptor;
-  }
-
-  public int getPhysicalEdgeCount() {
-    return physicalEdgeCount;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    // TODONEWTEZ convert to PB
-    out.writeUTF(destinationVertexName);
-    out.writeInt(physicalEdgeCount);
-    byte[] inputDescBytes =
-        DagTypeConverters.convertToDAGPlan(outputDescriptor).toByteArray();
-    out.writeInt(inputDescBytes.length);
-    out.write(inputDescBytes);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    destinationVertexName = in.readUTF();
-    physicalEdgeCount = in.readInt();
-    int inputDescLen = in.readInt();
-    byte[] inputDescBytes = new byte[inputDescLen];
-    in.readFully(inputDescBytes);
-    outputDescriptor =
-        DagTypeConverters.convertOutputDescriptorFromDAGPlan(
-            TezEntityDescriptorProto.parseFrom(inputDescBytes));
-  }
-
-  public String toString() {
-    return "{ destinationVertexName=" + destinationVertexName
-        + ", physicalEdgeCount" + physicalEdgeCount
-        + ", outputClassName=" + outputDescriptor.getClassName()
-        + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TaskSpec.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TaskSpec.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TaskSpec.java
deleted file mode 100644
index 8290e30..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TaskSpec.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public class TaskSpec implements Writable {
-
-  private TezTaskAttemptID taskAttemptId;
-  private String vertexName;
-  private String user;
-  private ProcessorDescriptor processorDescriptor;
-  private List<InputSpec> inputSpecList;
-  private List<OutputSpec> outputSpecList;
-
-  public TaskSpec() {
-  }
-
-  // TODO NEWTEZ Remove user
-  public TaskSpec(TezTaskAttemptID taskAttemptID, String user,
-      String vertexName, ProcessorDescriptor processorDescriptor,
-      List<InputSpec> inputSpecList, List<OutputSpec> outputSpecList) {
-    this.taskAttemptId = taskAttemptID;
-    this.vertexName = vertexName;
-    this.user = user;
-    this.processorDescriptor = processorDescriptor;
-    this.inputSpecList = inputSpecList;
-    this.outputSpecList = outputSpecList;
-  }
-
-  public String getVertexName() {
-    return vertexName;
-  }
-
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskAttemptId;
-  }
-
-  public String getUser() {
-    return user;
-  }
-
-  public ProcessorDescriptor getProcessorDescriptor() {
-    return processorDescriptor;
-  }
-
-  public List<InputSpec> getInputs() {
-    return inputSpecList;
-  }
-
-  public List<OutputSpec> getOutputs() {
-    return outputSpecList;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    taskAttemptId.write(out);
-    out.writeUTF(vertexName);
-    byte[] procDesc =
-        DagTypeConverters.convertToDAGPlan(processorDescriptor).toByteArray();
-    out.writeInt(procDesc.length);
-    out.write(procDesc);
-    out.writeInt(inputSpecList.size());
-    for (InputSpec inputSpec : inputSpecList) {
-      inputSpec.write(out);
-    }
-    out.writeInt(outputSpecList.size());
-    for (OutputSpec outputSpec : outputSpecList) {
-      outputSpec.write(out);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    taskAttemptId = new TezTaskAttemptID();
-    taskAttemptId.readFields(in);
-    vertexName = in.readUTF();
-    int procDescLength = in.readInt();
-    // TODO at least 3 buffer copies here. Need to convert this to full PB
-    // TEZ-305
-    byte[] procDescBytes = new byte[procDescLength];
-    in.readFully(procDescBytes);
-    processorDescriptor =
-        DagTypeConverters.convertProcessorDescriptorFromDAGPlan(
-            TezEntityDescriptorProto.parseFrom(procDescBytes));
-    int numInputSpecs = in.readInt();
-    inputSpecList = new ArrayList<InputSpec>(numInputSpecs);
-    for (int i = 0; i < numInputSpecs; i++) {
-      InputSpec inputSpec = new InputSpec();
-      inputSpec.readFields(in);
-      inputSpecList.add(inputSpec);
-    }
-    int numOutputSpecs = in.readInt();
-    outputSpecList = new ArrayList<OutputSpec>(numOutputSpecs);
-    for (int i = 0; i < numOutputSpecs; i++) {
-      OutputSpec outputSpec = new OutputSpec();
-      outputSpec.readFields(in);
-      outputSpecList.add(outputSpec);
-    }
-  }
-
-  @Override
-  public String toString() {
-    StringBuffer sb = new StringBuffer();
-    sb.append("TaskAttemptID:" + taskAttemptId);
-    sb.append("processorName=" + processorDescriptor.getClassName()
-        + ", inputSpecListSize=" + inputSpecList.size()
-        + ", outputSpecListSize=" + outputSpecList.size());
-    sb.append(", inputSpecList=[");
-    for (InputSpec i : inputSpecList) {
-      sb.append("{" + i.toString() + "}, ");
-    }
-    sb.append("], outputSpecList=[");
-    for (OutputSpec i : outputSpecList) {
-      sb.append("{" + i.toString() + "}, ");
-    }
-    sb.append("]");
-    return sb.toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezEvent.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezEvent.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezEvent.java
deleted file mode 100644
index 0f65750..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezEvent.java
+++ /dev/null
@@ -1,248 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.api.TezUncheckedException;
-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.engine.newapi.Event;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.InputInformationEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.events.TaskAttemptCompletedEvent;
-import org.apache.tez.engine.newapi.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.newapi.events.TaskStatusUpdateEvent;
-
-import com.google.protobuf.ByteString;
-
-public class TezEvent implements Writable {
-
-  private EventType eventType;
-
-  private Event event;
-
-  private EventMetaData sourceInfo;
-
-  private EventMetaData destinationInfo;
-
-  public TezEvent() {
-  }
-
-  public TezEvent(Event event, EventMetaData sourceInfo) {
-    this.event = event;
-    this.setSourceInfo(sourceInfo);
-    if (event instanceof DataMovementEvent) {
-      eventType = EventType.DATA_MOVEMENT_EVENT;
-    } else if (event instanceof InputReadErrorEvent) {
-      eventType = EventType.INPUT_READ_ERROR_EVENT;
-    } else if (event instanceof TaskAttemptFailedEvent) {
-      eventType = EventType.TASK_ATTEMPT_FAILED_EVENT;
-    } else if (event instanceof TaskAttemptCompletedEvent) {
-      eventType = EventType.TASK_ATTEMPT_COMPLETED_EVENT;
-    } else if (event instanceof InputInformationEvent) {
-      eventType = EventType.INTPUT_INFORMATION_EVENT;
-    } else if (event instanceof InputFailedEvent) {
-      eventType = EventType.INPUT_FAILED_EVENT;
-    } else if (event instanceof TaskStatusUpdateEvent) {
-      eventType = EventType.TASK_STATUS_UPDATE_EVENT;
-    } else {
-      throw new TezUncheckedException("Unknown event, event="
-          + event.getClass().getName());
-    }
-  }
-
-  public Event getEvent() {
-    return event;
-  }
-
-  public EventMetaData getSourceInfo() {
-    return sourceInfo;
-  }
-
-  public void setSourceInfo(EventMetaData sourceInfo) {
-    this.sourceInfo = sourceInfo;
-  }
-
-  public EventMetaData getDestinationInfo() {
-    return destinationInfo;
-  }
-
-  public void setDestinationInfo(EventMetaData destinationInfo) {
-    this.destinationInfo = destinationInfo;
-  }
-
-  public EventType getEventType() {
-    return eventType;
-  }
-
-  private void serializeEvent(DataOutput out) throws IOException {
-    if (event == null) {
-      out.writeBoolean(false);
-      return;
-    }
-    out.writeBoolean(true);
-    out.writeInt(eventType.ordinal());
-    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
-      // TODO NEWTEZ convert to PB
-      TaskStatusUpdateEvent sEvt = (TaskStatusUpdateEvent) event;
-      sEvt.write(out);
-    } else {
-      byte[] eventBytes = null;
-      switch (eventType) {
-      case DATA_MOVEMENT_EVENT:
-        DataMovementEvent dmEvt = (DataMovementEvent) event;
-        eventBytes = DataMovementEventProto.newBuilder()
-          .setSourceIndex(dmEvt.getSourceIndex())
-          .setTargetIndex(dmEvt.getTargetIndex())
-          .setUserPayload(ByteString.copyFrom(dmEvt.getUserPayload()))
-          .build().toByteArray();
-        break;
-      case INPUT_READ_ERROR_EVENT:
-        InputReadErrorEvent ideEvt = (InputReadErrorEvent) event;
-        eventBytes = InputReadErrorEventProto.newBuilder()
-            .setIndex(ideEvt.getIndex())
-            .setDiagnostics(ideEvt.getDiagnostics())
-            .build().toByteArray();
-        break;
-      case TASK_ATTEMPT_FAILED_EVENT:
-        TaskAttemptFailedEvent tfEvt = (TaskAttemptFailedEvent) event;
-        eventBytes = TaskAttemptFailedEventProto.newBuilder()
-            .setDiagnostics(tfEvt.getDiagnostics())
-            .build().toByteArray();
-        break;
-      case TASK_ATTEMPT_COMPLETED_EVENT:
-        eventBytes = TaskAttemptCompletedEventProto.newBuilder()
-            .build().toByteArray();
-        break;
-      case INPUT_FAILED_EVENT:
-        InputFailedEvent ifEvt = (InputFailedEvent) event;
-        eventBytes = InputFailedEventProto.newBuilder()
-            .setSourceIndex(ifEvt.getSourceIndex())
-            .setTargetIndex(ifEvt.getTargetIndex())
-            .setVersion(ifEvt.getVersion()).build().toByteArray();
-      case INTPUT_INFORMATION_EVENT:
-        InputInformationEvent iEvt = (InputInformationEvent) event;
-        eventBytes = InputInformationEventProto.newBuilder()
-            .setUserPayload(ByteString.copyFrom(iEvt.getUserPayload()))
-            .build().toByteArray();
-      default:
-        throw new TezUncheckedException("Unknown TezEvent"
-           + ", type=" + eventType);
-      }
-      out.writeInt(eventBytes.length);
-      out.write(eventBytes);
-    }
-  }
-
-  private void deserializeEvent(DataInput in) throws IOException {
-    if (!in.readBoolean()) {
-      event = null;
-      return;
-    }
-    eventType = EventType.values()[in.readInt()];
-    if (eventType.equals(EventType.TASK_STATUS_UPDATE_EVENT)) {
-      // TODO NEWTEZ convert to PB
-      event = new TaskStatusUpdateEvent();
-      ((TaskStatusUpdateEvent)event).readFields(in);
-    } else {
-      int eventBytesLen = in.readInt();
-      byte[] eventBytes = new byte[eventBytesLen];
-      in.readFully(eventBytes);
-      switch (eventType) {
-      case DATA_MOVEMENT_EVENT:
-        DataMovementEventProto dmProto =
-            DataMovementEventProto.parseFrom(eventBytes);
-        event = new DataMovementEvent(dmProto.getSourceIndex(),
-            dmProto.getTargetIndex(),
-            dmProto.getUserPayload().toByteArray());
-        break;
-      case INPUT_READ_ERROR_EVENT:
-        InputReadErrorEventProto ideProto =
-            InputReadErrorEventProto.parseFrom(eventBytes);
-        event = new InputReadErrorEvent(ideProto.getDiagnostics(),
-            ideProto.getIndex(), ideProto.getVersion());
-        break;
-      case TASK_ATTEMPT_FAILED_EVENT:
-        TaskAttemptFailedEventProto tfProto =
-            TaskAttemptFailedEventProto.parseFrom(eventBytes);
-        event = new TaskAttemptFailedEvent(tfProto.getDiagnostics());
-        break;
-      case TASK_ATTEMPT_COMPLETED_EVENT:
-        event = new TaskAttemptCompletedEvent();
-        break;
-      case INPUT_FAILED_EVENT:
-        InputFailedEventProto ifProto =
-            InputFailedEventProto.parseFrom(eventBytes);
-        event = new InputFailedEvent(ifProto.getSourceIndex(),
-            ifProto.getTargetIndex(), ifProto.getVersion());
-        break;
-      case INTPUT_INFORMATION_EVENT:
-        InputInformationEventProto infoProto =
-            InputInformationEventProto.parseFrom(eventBytes);
-        event = new InputInformationEvent(
-            infoProto.getUserPayload().toByteArray());
-        break;
-      default:
-        throw new TezUncheckedException("Unknown TezEvent"
-           + ", type=" + eventType);
-      }
-    }
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    serializeEvent(out);
-    if (sourceInfo != null) {
-      out.writeBoolean(true);
-      sourceInfo.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-    if (destinationInfo != null) {
-      out.writeBoolean(true);
-      destinationInfo.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    deserializeEvent(in);
-    if (in.readBoolean()) {
-      sourceInfo = new EventMetaData();
-      sourceInfo.readFields(in);
-    }
-    if (in.readBoolean()) {
-      destinationInfo = new EventMetaData();
-      destinationInfo.readFields(in);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatRequest.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatRequest.java
deleted file mode 100644
index 79a0968..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatRequest.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-
-public class TezHeartbeatRequest implements Writable {
-
-  private String containerIdentifier;
-  private List<TezEvent> events;
-  private TezTaskAttemptID currentTaskAttemptID;
-  private int startIndex;
-  private int maxEvents;
-  private long requestId;
-
-  public TezHeartbeatRequest() {
-  }
-
-  public TezHeartbeatRequest(long requestId, List<TezEvent> events,
-      String containerIdentifier, TezTaskAttemptID taskAttemptID,
-      int startIndex, int maxEvents) {
-    this.containerIdentifier = containerIdentifier;
-    this.requestId = requestId;
-    this.events = Collections.unmodifiableList(events);
-    this.startIndex = startIndex;
-    this.maxEvents = maxEvents;
-    this.currentTaskAttemptID = taskAttemptID;
-  }
-
-  public String getContainerIdentifier() {
-    return containerIdentifier;
-  }
-
-  public List<TezEvent> getEvents() {
-    return events;
-  }
-
-  public int getStartIndex() {
-    return startIndex;
-  }
-
-  public int getMaxEvents() {
-    return maxEvents;
-  }
-
-  public long getRequestId() {
-    return requestId;
-  }
-
-  public TezTaskAttemptID getCurrentTaskAttemptID() {
-    return currentTaskAttemptID;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    if (events != null) {
-      out.writeBoolean(true);
-      out.writeInt(events.size());
-      for (TezEvent e : events) {
-        e.write(out);
-      }
-    } else {
-      out.writeBoolean(false);
-    }
-    if (currentTaskAttemptID != null) {
-      out.writeBoolean(true);
-      currentTaskAttemptID.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-    out.writeInt(startIndex);
-    out.writeInt(maxEvents);
-    out.writeLong(requestId);
-    Text.writeString(out, containerIdentifier);
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    if (in.readBoolean()) {
-      int eventsCount = in.readInt();
-      events = new ArrayList<TezEvent>(eventsCount);
-      for (int i = 0; i < eventsCount; ++i) {
-        TezEvent e = new TezEvent();
-        e.readFields(in);
-        events.add(e);
-      }
-    }
-    if (in.readBoolean()) {
-      currentTaskAttemptID = new TezTaskAttemptID();
-      currentTaskAttemptID.readFields(in);
-    } else {
-      currentTaskAttemptID = null;
-    }
-    startIndex = in.readInt();
-    maxEvents = in.readInt();
-    requestId = in.readLong();
-    containerIdentifier = Text.readString(in);
-  }
-
-  @Override
-  public String toString() {
-    return "{ "
-        + " containerId=" + containerIdentifier
-        + ", requestId=" + requestId
-        + ", startIndex=" + startIndex
-        + ", maxEventsToGet=" + maxEvents
-        + ", taskAttemptId" + currentTaskAttemptID
-        + ", eventCount=" + (events != null ? events.size() : 0)
-        + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatResponse.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatResponse.java
deleted file mode 100644
index addd17f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezHeartbeatResponse.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.hadoop.io.Writable;
-
-public class TezHeartbeatResponse implements Writable {
-
-  private long lastRequestId;
-  private boolean shouldDie = false;
-  private List<TezEvent> events;
-
-  public TezHeartbeatResponse() {
-  }
-
-  public TezHeartbeatResponse(List<TezEvent> events) {
-    this.events = Collections.unmodifiableList(events);
-  }
-
-  public List<TezEvent> getEvents() {
-    return events;
-  }
-
-  public boolean shouldDie() {
-    return shouldDie;
-  }
-
-  public long getLastRequestId() {
-    return lastRequestId;
-  }
-
-  public void setEvents(List<TezEvent> events) {
-    this.events = Collections.unmodifiableList(events);
-  }
-
-  public void setLastRequestId(long lastRequestId ) {
-    this.lastRequestId = lastRequestId;
-  }
-
-  public void setShouldDie() {
-    this.shouldDie = true;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(lastRequestId);
-    out.writeBoolean(shouldDie);
-    if(events != null) {
-      out.writeBoolean(true);
-      out.writeInt(events.size());
-      for (TezEvent e : events) {
-        e.write(out);
-      }
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    lastRequestId = in.readLong();
-    shouldDie = in.readBoolean();
-    if(in.readBoolean()) {
-      int eventCount = in.readInt();
-      events = new ArrayList<TezEvent>(eventCount);
-      for (int i = 0; i < eventCount; ++i) {
-        TezEvent e = new TezEvent();
-        e.readFields(in);
-        events.add(e);
-      }
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "{ "
-        + " lastRequestId=" + lastRequestId
-        + ", shouldDie=" + shouldDie
-        + ", eventCount=" + (events != null ? events.size() : 0)
-        + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezInputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezInputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezInputContextImpl.java
deleted file mode 100644
index daafc5a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezInputContextImpl.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-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.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public class TezInputContextImpl extends TezTaskContextImpl
-    implements TezInputContext {
-
-  private final byte[] userPayload;
-  private final String sourceVertexName;
-  private final EventMetaData sourceInfo;
-
-  @Private
-  public TezInputContextImpl(Configuration conf, int appAttemptNumber,
-      TezUmbilical tezUmbilical, String taskVertexName,
-      String sourceVertexName, TezTaskAttemptID taskAttemptID,
-      TezCounters counters, byte[] userPayload,
-      RuntimeTask runtimeTask, Map<String, ByteBuffer> serviceConsumerMetadata) {
-    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
-        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
-    this.userPayload = userPayload;
-    this.sourceVertexName = sourceVertexName;
-    this.sourceInfo = new EventMetaData(
-        EventProducerConsumerType.INPUT, taskVertexName, sourceVertexName,
-        taskAttemptID);
-    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
-        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
-        getTaskIndex(), getTaskAttemptNumber(), sourceVertexName);
-  }
-
-  @Override
-  public void sendEvents(List<Event> events) {
-    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
-    for (Event e : events) {
-      TezEvent tEvt = new TezEvent(e, sourceInfo);
-      tezEvents.add(tEvt);
-    }
-    tezUmbilical.addEvents(tezEvents);
-  }
-
-  @Override
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  @Override
-  public String getSourceVertexName() {
-    return sourceVertexName;
-  }
-
-  @Override
-  public void fatalError(Throwable exception, String message) {
-    super.signalFatalError(exception, message, sourceInfo);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezOutputContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezOutputContextImpl.java
deleted file mode 100644
index 9de41ae..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezOutputContextImpl.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-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.newapi.Event;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public class TezOutputContextImpl extends TezTaskContextImpl
-    implements TezOutputContext {
-
-  private final byte[] userPayload;
-  private final String destinationVertexName;
-  private final EventMetaData sourceInfo;
-
-  @Private
-  public TezOutputContextImpl(Configuration conf, int appAttemptNumber,
-      TezUmbilical tezUmbilical, String taskVertexName,
-      String destinationVertexName,
-      TezTaskAttemptID taskAttemptID, TezCounters counters,
-      byte[] userPayload, RuntimeTask runtimeTask,
-      Map<String, ByteBuffer> serviceConsumerMetadata) {
-    super(conf, appAttemptNumber, taskVertexName, taskAttemptID,
-        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
-    this.userPayload = userPayload;
-    this.destinationVertexName = destinationVertexName;
-    this.sourceInfo = new EventMetaData(EventProducerConsumerType.OUTPUT,
-        taskVertexName, destinationVertexName, taskAttemptID);
-    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d_%s", taskAttemptID
-        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
-        getTaskIndex(), getTaskAttemptNumber(), destinationVertexName);
-  }
-
-  @Override
-  public void sendEvents(List<Event> events) {
-    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
-    for (Event e : events) {
-      TezEvent tEvt = new TezEvent(e, sourceInfo);
-      tezEvents.add(tEvt);
-    }
-    tezUmbilical.addEvents(tezEvents);
-  }
-
-  @Override
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  @Override
-  public String getDestinationVertexName() {
-    return destinationVertexName;
-  }
-
-  @Override
-  public void fatalError(Throwable exception, String message) {
-    super.signalFatalError(exception, message, sourceInfo);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java
deleted file mode 100644
index d710f7a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezProcessorContextImpl.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.nio.ByteBuffer;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-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.newapi.Event;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public class TezProcessorContextImpl extends TezTaskContextImpl
-  implements TezProcessorContext {
-
-  private final byte[] userPayload;
-  private final EventMetaData sourceInfo;
-
-  public TezProcessorContextImpl(Configuration conf, int appAttemptNumber,
-      TezUmbilical tezUmbilical, String vertexName,
-      TezTaskAttemptID taskAttemptID, TezCounters counters,
-      byte[] userPayload, RuntimeTask runtimeTask,
-      Map<String, ByteBuffer> serviceConsumerMetadata) {
-    super(conf, appAttemptNumber, vertexName, taskAttemptID,
-        counters, runtimeTask, tezUmbilical, serviceConsumerMetadata);
-    this.userPayload = userPayload;
-    this.sourceInfo = new EventMetaData(EventProducerConsumerType.PROCESSOR,
-        taskVertexName, "", taskAttemptID);
-    this.uniqueIdentifier = String.format("%s_%s_%06d_%02d", taskAttemptID
-        .getTaskID().getVertexID().getDAGId().toString(), taskVertexName,
-        getTaskIndex(), getTaskAttemptNumber());
-  }
-
-  @Override
-  public void sendEvents(List<Event> events) {
-    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
-    for (Event e : events) {
-      TezEvent tEvt = new TezEvent(e, sourceInfo);
-      tezEvents.add(tEvt);
-    }
-    tezUmbilical.addEvents(tezEvents);
-  }
-
-  @Override
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
-
-  @Override
-  public void setProgress(float progress) {
-    runtimeTask.setProgress(progress);
-  }
-
-  @Override
-  public void fatalError(Throwable exception, String message) {
-    super.signalFatalError(exception, message, sourceInfo);
-  }
-
-  @Override
-  public boolean canCommit() throws IOException {
-    return tezUmbilical.canCommit(this.taskAttemptID);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezTaskContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezTaskContextImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezTaskContextImpl.java
deleted file mode 100644
index 1d17158..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezTaskContextImpl.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
-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.newapi.TezTaskContext;
-import org.apache.tez.engine.newruntime.RuntimeTask;
-
-public abstract class TezTaskContextImpl implements TezTaskContext {
-
-  private final Configuration conf;
-  protected final String taskVertexName;
-  protected final TezTaskAttemptID taskAttemptID;
-  private final TezCounters counters;
-  private String[] workDirs;
-  protected String uniqueIdentifier;
-  protected final RuntimeTask runtimeTask;
-  protected final TezUmbilical tezUmbilical;
-  private final Map<String, ByteBuffer> serviceConsumerMetadata;
-  private final int appAttemptNumber;
-
-  @Private
-  public TezTaskContextImpl(Configuration conf, int appAttemptNumber,
-      String taskVertexName, TezTaskAttemptID taskAttemptID,
-      TezCounters counters, RuntimeTask runtimeTask,
-      TezUmbilical tezUmbilical, Map<String, ByteBuffer> serviceConsumerMetadata) {
-    this.conf = conf;
-    this.taskVertexName = taskVertexName;
-    this.taskAttemptID = taskAttemptID;
-    this.counters = counters;
-    // TODO Maybe change this to be task id specific at some point. For now
-    // Shuffle code relies on this being a path specified by YARN
-    this.workDirs = this.conf.getStrings(TezJobConfig.LOCAL_DIRS);
-    this.runtimeTask = runtimeTask;
-    this.tezUmbilical = tezUmbilical;
-    this.serviceConsumerMetadata = serviceConsumerMetadata;
-    // TODO NEWTEZ at some point dag attempt should not map to app attempt
-    this.appAttemptNumber = appAttemptNumber;
-  }
-
-  @Override
-  public ApplicationId getApplicationId() {
-    return taskAttemptID.getTaskID().getVertexID().getDAGId()
-        .getApplicationId();
-  }
-
-  @Override
-  public int getTaskIndex() {
-    return taskAttemptID.getTaskID().getId();
-  }
-
-  @Override
-  public int getDAGAttemptNumber() {
-    return appAttemptNumber;
-  }
-
-  @Override
-  public int getTaskAttemptNumber() {
-    return taskAttemptID.getId();
-  }
-
-  @Override
-  public String getDAGName() {
-    // TODO NEWTEZ Change to some form of the DAG name, for now using dagId as
-    // the unique identifier.
-    return taskAttemptID.getTaskID().getVertexID().getDAGId().toString();
-  }
-
-  @Override
-  public String getTaskVertexName() {
-    return taskVertexName;
-  }
-
-
-  @Override
-  public TezCounters getCounters() {
-    return counters;
-  }
-
-  @Override
-  public String[] getWorkDirs() {
-    return Arrays.copyOf(workDirs, workDirs.length);
-  }
-
-  @Override
-  public String getUniqueIdentifier() {
-    return uniqueIdentifier;
-  }
-
-  @Override
-  public ByteBuffer getServiceConsumerMetaData(String serviceName) {
-    return (ByteBuffer) serviceConsumerMetadata.get(serviceName)
-        .asReadOnlyBuffer().rewind();
-  }
-
-  @Override
-  public ByteBuffer getServiceProviderMetaData(String serviceName) {
-    return AuxiliaryServiceHelper.getServiceDataFromEnv(
-        serviceName, System.getenv());
-  }
-
-  protected void signalFatalError(Throwable t, String message,
-      EventMetaData sourceInfo) {
-    runtimeTask.setFatalError(t, message);
-    String diagnostics;
-    if (t != null && message != null) {
-      diagnostics = "exceptionThrown=" + StringUtils.stringifyException(t)
-          + ", errorMessage=" + message;
-    } else if (t == null && message == null) {
-      diagnostics = "Unknown error";
-    } else {
-      diagnostics = t != null ?
-          "exceptionThrown=" + StringUtils.stringifyException(t)
-          : " errorMessage=" + message;
-    }
-    tezUmbilical.signalFatalError(taskAttemptID, diagnostics, sourceInfo);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java b/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java
deleted file mode 100644
index 5889622..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newapi/impl/TezUmbilical.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.newapi.impl;
-
-import java.io.IOException;
-import java.util.Collection;
-
-import org.apache.tez.dag.records.TezTaskAttemptID;
-
-public interface TezUmbilical {
-
-  public void addEvents(Collection<TezEvent> events);
-
-  public void signalFatalError(TezTaskAttemptID taskAttemptID,
-      String diagnostics,
-      EventMetaData sourceInfo);
-
-  public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
index eb055b6..77299de 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
@@ -38,6 +38,16 @@ import org.apache.hadoop.util.StringUtils;
 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.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.newapi.Event;
 import org.apache.tez.engine.newapi.Input;
@@ -49,16 +59,6 @@ import org.apache.tez.engine.newapi.Processor;
 import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.engine.newapi.TezOutputContext;
 import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.impl.EventMetaData;
-import org.apache.tez.engine.newapi.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.newapi.impl.InputSpec;
-import org.apache.tez.engine.newapi.impl.OutputSpec;
-import org.apache.tez.engine.newapi.impl.TaskSpec;
-import org.apache.tez.engine.newapi.impl.TezEvent;
-import org.apache.tez.engine.newapi.impl.TezInputContextImpl;
-import org.apache.tez.engine.newapi.impl.TezOutputContextImpl;
-import org.apache.tez.engine.newapi.impl.TezProcessorContextImpl;
-import org.apache.tez.engine.newapi.impl.TezUmbilical;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
index ee6cde8..22cbc7c 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
@@ -25,9 +25,9 @@ import java.util.concurrent.atomic.AtomicInteger;
 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.newapi.impl.TaskSpec;
-import org.apache.tez.engine.newapi.impl.TezEvent;
-import org.apache.tez.engine.newapi.impl.TezUmbilical;
+import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.engine.api.impl.TezUmbilical;
 
 public abstract class RuntimeTask {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/runtime/RuntimeUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/runtime/RuntimeUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/runtime/RuntimeUtils.java
deleted file mode 100644
index c673d16..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/runtime/RuntimeUtils.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.engine.runtime;
-
-import java.lang.reflect.Constructor;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.OutputSpec;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
-import org.apache.tez.engine.task.RuntimeTask;
-
-public class RuntimeUtils {
-
-  private static final Log LOG = LogFactory.getLog(RuntimeUtils.class);
-
-  private static final Class<?>[] CONTEXT_ARRAY =
-      new Class[] { TezEngineTaskContext.class };
-  private static final Class<?>[] CONTEXT_INT_ARRAY =
-      new Class[] { TezEngineTaskContext.class, Integer.TYPE };
-  private static final Map<Class<?>, Constructor<?>> CONSTRUCTOR_CACHE =
-    new ConcurrentHashMap<Class<?>, Constructor<?>>();
-
-  @SuppressWarnings("unchecked")
-  public static <T> T getNewInstance(Class<T> theClass,
-      TezEngineTaskContext context) {
-    T result;
-    try {
-      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
-      if (meth == null) {
-        meth = theClass.getDeclaredConstructor(CONTEXT_ARRAY);
-        meth.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(theClass, meth);
-      }
-      result = meth.newInstance(context);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-    return result;
-  }
-  
-  @SuppressWarnings("unchecked")
-  public static <T> T getNewInputInstance(Class<T> theClass,
-      TezEngineTaskContext context, int index) {
-    T result;
-    try {
-      Constructor<T> meth = (Constructor<T>) CONSTRUCTOR_CACHE.get(theClass);
-      if (meth == null) {
-        meth = theClass.getDeclaredConstructor(CONTEXT_INT_ARRAY);
-        meth.setAccessible(true);
-        CONSTRUCTOR_CACHE.put(theClass, meth);
-      }
-      result = meth.newInstance(context, index);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-    return result;
-  }
-
-  public static RuntimeTask createRuntimeTask(
-      TezEngineTaskContext taskContext) {
-    LOG.info("Creating a runtime task from TaskContext"
-        + ", Processor: " + taskContext.getProcessorName()
-        + ", InputCount=" + taskContext.getInputSpecList().size()
-        + ", OutputCount=" + taskContext.getOutputSpecList().size());
-
-    RuntimeTask t = null;
-    try {
-      Class<?> processorClazz =
-          Class.forName(taskContext.getProcessorName());
-
-      Processor processor = (Processor) getNewInstance(
-          processorClazz, taskContext);
-
-      Input[] inputs;
-      Output[] outputs;
-      if (taskContext.getInputSpecList().isEmpty()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Initializing task with 0 inputs");
-        }
-        inputs = new Input[0];
-      } else {
-        int iSpecCount = taskContext.getInputSpecList().size();
-        inputs = new Input[iSpecCount];
-        for (int i = 0; i < iSpecCount; ++i) {
-          InputSpec inSpec = taskContext.getInputSpecList().get(i);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Using Input"
-                + ", index=" + i
-                + ", inputClass=" + inSpec.getInputClassName());
-          }
-          Class<?> inputClazz = Class.forName(inSpec.getInputClassName());
-          Input input = (Input) getNewInputInstance(inputClazz, taskContext, i);
-          inputs[i] = input;
-        }
-      }
-      if (taskContext.getOutputSpecList().isEmpty()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Initializing task with 0 outputs");
-        }
-        outputs = new Output[0];
-      } else {
-        int oSpecCount = taskContext.getOutputSpecList().size();
-        outputs = new Output[oSpecCount];
-        for (int i = 0; i < oSpecCount; ++i) {
-          OutputSpec outSpec = taskContext.getOutputSpecList().get(i);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Using Output"
-                + ", index=" + i
-                + ", output=" + outSpec.getOutputClassName());
-          }
-          Class<?> outputClazz = Class.forName(outSpec.getOutputClassName());
-          Output output = (Output) getNewInstance(outputClazz, taskContext);
-          outputs[i] = output;
-        }
-      }
-      t = createRuntime(taskContext, processor, inputs, outputs);
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException("Unable to initialize RuntimeTask, context="
-          + taskContext, e);
-    }
-    return t;
-  }
-
-  private static RuntimeTask createRuntime(TezEngineTaskContext taskContext,
-      Processor processor, Input[] inputs, Output[] outputs) {
-    try {
-      // TODO Change this to use getNewInstance
-      Class<?> runtimeClazz = Class.forName(taskContext.getRuntimeName());
-      Constructor<?> ctor = runtimeClazz.getConstructor(
-          TezEngineTaskContext.class, Processor.class, Input[].class,
-          Output[].class);
-      ctor.setAccessible(true);
-      return (RuntimeTask) ctor.newInstance(taskContext, processor, inputs, outputs);
-    } catch (ClassNotFoundException e) {
-      throw new RuntimeException("Unable to load runtimeClass: "
-          + taskContext.getRuntimeName(), e);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
index e9bfe36..531e460 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
@@ -29,7 +29,7 @@ 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.newoutput.TezTaskOutputFiles;
+import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
 
 import com.google.common.base.Preconditions;
 


[03/20] TEZ-444. Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java
deleted file mode 100644
index d71dba0..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTask.java
+++ /dev/null
@@ -1,731 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.mapreduce.newprocessor;
-
-import java.io.IOException;
-import java.net.URI;
-import java.text.NumberFormat;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import javax.crypto.SecretKey;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.FileSystem.Statistics;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapred.FileAlreadyExistsException;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.MapOutputFile;
-import org.apache.hadoop.mapred.RawKeyValueIterator;
-import org.apache.hadoop.mapred.TaskAttemptContext;
-import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.mapred.TaskID;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.hadoop.mapreduce.filecache.DistributedCache;
-import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
-import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
-import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.StringUtils;
-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;
-import org.apache.tez.common.counters.TaskCounter;
-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.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.records.OutputContext;
-import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
-import org.apache.tez.mapreduce.hadoop.IDConverter;
-import org.apache.tez.mapreduce.hadoop.MRConfig;
-import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.hadoop.newmapred.TaskAttemptContextImpl;
-import org.apache.tez.mapreduce.hadoop.mapreduce.JobContextImpl;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
-//import org.apache.tez.mapreduce.partition.MRPartitioner;
-import org.apache.tez.mapreduce.task.impl.YarnOutputFiles;
-
-@SuppressWarnings("deprecation")
-public abstract class MRTask {
-
-  static final Log LOG = LogFactory.getLog(MRTask.class);
-
-  protected JobConf jobConf;
-  protected JobContext jobContext;
-  protected TaskAttemptContext taskAttemptContext;
-  protected OutputCommitter committer;
-
-  // Current counters
-  transient TezCounters counters;
-  protected GcTimeUpdater gcUpdater;
-  private ResourceCalculatorProcessTree pTree;
-  private long initCpuCumulativeTime = 0;
-  protected TezProcessorContext processorContext;
-  protected TaskAttemptID taskAttemptId;
-  protected Progress progress = new Progress();
-  protected SecretKey jobTokenSecret;
-
-  boolean isMap;
-
-  /* flag to track whether task is done */
-  AtomicBoolean taskDone = new AtomicBoolean(false);
-
-  /** Construct output file names so that, when an output directory listing is
-   * sorted lexicographically, positions correspond to output partitions.*/
-  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
-  static {
-    NUMBER_FORMAT.setMinimumIntegerDigits(5);
-    NUMBER_FORMAT.setGroupingUsed(false);
-  }
-
-  protected MRTaskReporter mrReporter;
-  protected boolean useNewApi;
-
-  /**
-   * A Map where Key-> URIScheme and value->FileSystemStatisticUpdater
-   */
-  private Map<String, FileSystemStatisticUpdater> statisticUpdaters =
-     new HashMap<String, FileSystemStatisticUpdater>();
-
-  public MRTask(boolean isMap) {
-    this.isMap = isMap;
-  }
-
-  // TODO how to update progress
-  public void initialize(TezProcessorContext context) throws IOException,
-  InterruptedException {
-
-    DeprecatedKeys.init();
-
-    processorContext = context;
-    counters = context.getCounters();
-    this.taskAttemptId = new TaskAttemptID(
-        new TaskID(
-            Long.toString(context.getApplicationId().getClusterTimestamp()),
-            context.getApplicationId().getId(),
-            (isMap ? TaskType.MAP : TaskType.REDUCE),
-            context.getTaskIndex()),
-          context.getTaskAttemptNumber());
-    // TODO TEZAM4 Figure out initialization / run sequence of Input, Process,
-    // Output. Phase is MR specific.
-    gcUpdater = new GcTimeUpdater(counters);
-
-    byte[] userPayload = context.getUserPayload();
-    Configuration conf = TezUtils.createConfFromUserPayload(userPayload);
-    if (conf instanceof JobConf) {
-      this.jobConf = (JobConf)conf;
-    } else {
-      this.jobConf = new JobConf(conf);
-    }
-    jobConf.set(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID,
-        taskAttemptId.toString());
-    jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
-        context.getDAGAttemptNumber());
-
-    initResourceCalculatorPlugin();
-
-    LOG.info("MRTask.inited: taskAttemptId = " + taskAttemptId.toString());
-
-    // TODO Post MRR
-    // A single file per vertex will likely be a better solution. Does not
-    // require translation - client can take care of this. Will work independent
-    // of whether the configuration is for intermediate tasks or not. Has the
-    // overhead of localizing multiple files per job - i.e. the client would
-    // need to write these files to hdfs, add them as local resources per
-    // vertex. A solution like this may be more practical once it's possible to
-    // submit configuration parameters to the AM and effectively tasks via RPC.
-
-    jobConf.set(MRJobConfig.VERTEX_NAME, processorContext.getTaskVertexName());
-
-    if (LOG.isDebugEnabled() && userPayload != null) {
-      Iterator<Entry<String, String>> iter = jobConf.iterator();
-      String taskIdStr = taskAttemptId.getTaskID().toString();
-      while (iter.hasNext()) {
-        Entry<String, String> confEntry = iter.next();
-        LOG.debug("TaskConf Entry"
-            + ", taskId=" + taskIdStr
-            + ", key=" + confEntry.getKey()
-            + ", value=" + confEntry.getValue());
-      }
-    }
-
-    configureMRTask();
-  }
-
-  private void configureMRTask()
-      throws IOException, InterruptedException {
-
-    Credentials credentials = UserGroupInformation.getCurrentUser()
-        .getCredentials();
-    jobConf.setCredentials(credentials);
-    // TODO Can this be avoided all together. Have the MRTezOutputCommitter use
-    // the Tez parameter.
-    // TODO This could be fetched from the env if YARN is setting it for all
-    // Containers.
-    // Set it in conf, so as to be able to be used the the OutputCommitter.
-
-    jobConf.setClass(MRConfig.TASK_LOCAL_OUTPUT_CLASS, YarnOutputFiles.class,
-        MapOutputFile.class); // MR
-
-    // Not needed. This is probably being set via the source/consumer meta
-    Token<JobTokenIdentifier> jobToken = TokenCache.getJobToken(credentials);
-    if (jobToken != null) {
-      // Will MR ever run without a job token.
-      SecretKey sk = JobTokenSecretManager.createSecretKey(jobToken
-          .getPassword());
-      this.jobTokenSecret = sk;
-    } else {
-      LOG.warn("No job token set");
-    }
-
-    configureLocalDirs();
-
-    if (jobConf.get(TezJobConfig.DAG_CREDENTIALS_BINARY) != null) {
-      jobConf.set(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY,
-          jobConf.get(TezJobConfig.DAG_CREDENTIALS_BINARY));
-    }
-
-    // Set up the DistributedCache related configs
-    setupDistributedCacheConfig(jobConf);
-  }
-
-  private void configureLocalDirs() throws IOException {
-    // TODO NEWTEZ Is most of this functionality required ?
-    jobConf.setStrings(TezJobConfig.LOCAL_DIRS, processorContext.getWorkDirs());
-    jobConf.set(TezJobConfig.TASK_LOCAL_RESOURCE_DIR, System.getenv(Environment.PWD.name()));
-
-    jobConf.setStrings(MRConfig.LOCAL_DIR, processorContext.getWorkDirs());
-
-    LocalDirAllocator lDirAlloc = new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-    Path workDir = null;
-    // First, try to find the JOB_LOCAL_DIR on this host.
-    try {
-      workDir = lDirAlloc.getLocalPathToRead("work", jobConf);
-    } catch (DiskErrorException e) {
-      // DiskErrorException means dir not found. If not found, it will
-      // be created below.
-    }
-    if (workDir == null) {
-      // JOB_LOCAL_DIR doesn't exist on this host -- Create it.
-      workDir = lDirAlloc.getLocalPathForWrite("work", jobConf);
-      FileSystem lfs = FileSystem.getLocal(jobConf).getRaw();
-      boolean madeDir = false;
-      try {
-        madeDir = lfs.mkdirs(workDir);
-      } catch (FileAlreadyExistsException e) {
-        // Since all tasks will be running in their own JVM, the race condition
-        // exists where multiple tasks could be trying to create this directory
-        // at the same time. If this task loses the race, it's okay because
-        // the directory already exists.
-        madeDir = true;
-        workDir = lDirAlloc.getLocalPathToRead("work", jobConf);
-      }
-      if (!madeDir) {
-          throw new IOException("Mkdirs failed to create "
-              + workDir.toString());
-      }
-    }
-    // TODO NEWTEZ Is this required ?
-    jobConf.set(TezJobConfig.JOB_LOCAL_DIR, workDir.toString());
-    jobConf.set(MRJobConfig.JOB_LOCAL_DIR, workDir.toString());
-  }
-
-  /**
-   * Set up the DistributedCache related configs to make
-   * {@link DistributedCache#getLocalCacheFiles(Configuration)} and
-   * {@link DistributedCache#getLocalCacheArchives(Configuration)} working.
-   *
-   * @param job
-   * @throws IOException
-   */
-  private static void setupDistributedCacheConfig(final JobConf job)
-      throws IOException {
-
-    String localWorkDir = (job.get(TezJobConfig.TASK_LOCAL_RESOURCE_DIR));
-    // ^ ^ all symlinks are created in the current work-dir
-
-    // Update the configuration object with localized archives.
-    URI[] cacheArchives = DistributedCache.getCacheArchives(job);
-    if (cacheArchives != null) {
-      List<String> localArchives = new ArrayList<String>();
-      for (int i = 0; i < cacheArchives.length; ++i) {
-        URI u = cacheArchives[i];
-        Path p = new Path(u);
-        Path name = new Path((null == u.getFragment()) ? p.getName()
-            : u.getFragment());
-        String linkName = name.toUri().getPath();
-        localArchives.add(new Path(localWorkDir, linkName).toUri().getPath());
-      }
-      if (!localArchives.isEmpty()) {
-        job.set(MRJobConfig.CACHE_LOCALARCHIVES, StringUtils
-            .arrayToString(localArchives.toArray(new String[localArchives
-                .size()])));
-      }
-    }
-
-    // Update the configuration object with localized files.
-    URI[] cacheFiles = DistributedCache.getCacheFiles(job);
-    if (cacheFiles != null) {
-      List<String> localFiles = new ArrayList<String>();
-      for (int i = 0; i < cacheFiles.length; ++i) {
-        URI u = cacheFiles[i];
-        Path p = new Path(u);
-        Path name = new Path((null == u.getFragment()) ? p.getName()
-            : u.getFragment());
-        String linkName = name.toUri().getPath();
-        localFiles.add(new Path(localWorkDir, linkName).toUri().getPath());
-      }
-      if (!localFiles.isEmpty()) {
-        job.set(MRJobConfig.CACHE_LOCALFILES, StringUtils
-            .arrayToString(localFiles.toArray(new String[localFiles.size()])));
-      }
-    }
-  }
-
-
-  private void initResourceCalculatorPlugin() {
-    Class<? extends ResourceCalculatorProcessTree> clazz =
-        this.jobConf.getClass(MRConfig.RESOURCE_CALCULATOR_PROCESS_TREE,
-            null, ResourceCalculatorProcessTree.class);
-    pTree = ResourceCalculatorProcessTree
-        .getResourceCalculatorProcessTree(System.getenv().get("JVM_PID"), clazz, this.jobConf);
-    LOG.info(" Using ResourceCalculatorProcessTree : " + pTree);
-    if (pTree != null) {
-      pTree.updateProcessTree();
-      initCpuCumulativeTime = pTree.getCumulativeCpuTime();
-    }
-  }
-
-  public TezProcessorContext getUmbilical() {
-    return this.processorContext;
-  }
-
-  public void initTask() throws IOException,
-                                InterruptedException {
-    this.mrReporter = new MRTaskReporter(processorContext);
-    this.useNewApi = jobConf.getUseNewMapper();
-    TezDAGID dagId = IDConverter.fromMRTaskAttemptId(taskAttemptId).getTaskID()
-        .getVertexID().getDAGId();
-
-    this.jobContext = new JobContextImpl(jobConf, dagId, mrReporter);
-    this.taskAttemptContext =
-        new TaskAttemptContextImpl(jobConf, taskAttemptId, mrReporter);
-
-    if (getState() == State.UNASSIGNED) {
-      setState(State.RUNNING);
-    }
-
-//    combineProcessor = null;
-//    boolean useCombiner = false;
-//    if (useNewApi) {
-//      try {
-//        useCombiner = (taskAttemptContext.getCombinerClass() != null);
-//      } catch (ClassNotFoundException e) {
-//        throw new IOException("Could not find combiner class", e);
-//      }
-//    } else {
-//      useCombiner = (job.getCombinerClass() != null);
-//    }
-//    if (useCombiner) {
-//      combineProcessor = new MRCombiner(this);
-//      combineProcessor.initialize(job, getTaskReporter());
-//    } else {
-//    }
-
-    localizeConfiguration(jobConf);
-  }
-
-//  public void initPartitioner(JobConf job) throws IOException,
-//      InterruptedException {
-//    partitioner = new MRPartitioner(this);
-//    ((MRPartitioner) partitioner).initialize(job, getTaskReporter());
-//  }
-
-  public MRTaskReporter getMRReporter() {
-    return mrReporter;
-  }
-
-  public void setState(State state) {
-    // TODO Auto-generated method stub
-
-  }
-
-  public State getState() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  public OutputCommitter getCommitter() {
-    return committer;
-  }
-
-  public void setCommitter(OutputCommitter committer) {
-    this.committer = committer;
-  }
-
-  public TezCounters getCounters() { return counters; }
-
-  public void setConf(JobConf jobConf) {
-    this.jobConf = jobConf;
-  }
-
-  public JobConf getConf() {
-    return this.jobConf;
-  }
-
-  /**
-   * Gets a handle to the Statistics instance based on the scheme associated
-   * with path.
-   *
-   * @param path the path.
-   * @param conf the configuration to extract the scheme from if not part of
-   *   the path.
-   * @return a Statistics instance, or null if none is found for the scheme.
-   */
-  @Private
-  public static List<Statistics> getFsStatistics(Path path, Configuration conf) throws IOException {
-    List<Statistics> matchedStats = new ArrayList<FileSystem.Statistics>();
-    path = path.getFileSystem(conf).makeQualified(path);
-    String scheme = path.toUri().getScheme();
-    for (Statistics stats : FileSystem.getAllStatistics()) {
-      if (stats.getScheme().equals(scheme)) {
-        matchedStats.add(stats);
-      }
-    }
-    return matchedStats;
-  }
-
-  @Private
-  public synchronized String getOutputName() {
-    return "part-" + NUMBER_FORMAT.format(taskAttemptId.getTaskID().getId());
-  }
-
-  public void waitBeforeCompletion(MRTaskReporter reporter) throws IOException,
-      InterruptedException {
-  }
-
-  public void outputReady(MRTaskReporter reporter, OutputContext outputContext)
-      throws IOException,
-      InterruptedException {
-    LOG.info("Task: " + taskAttemptId + " reporting outputReady");
-    updateCounters();
-    statusUpdate();
-  }
-
-  public void done(LogicalOutput output) throws IOException, InterruptedException {
-    updateCounters();
-
-    LOG.info("Task:" + taskAttemptId + " is done."
-        + " And is in the process of committing");
-    // TODO change this to use the new context
-    // TODO TEZ Interaciton between Commit and OutputReady. Merge ?
-    if (output instanceof SimpleOutput) {
-      SimpleOutput sOut = (SimpleOutput)output;
-      if (sOut.isCommitRequired()) {
-        //wait for commit approval and commit
-        // TODO EVENTUALLY - Commit is not required for map tasks.
-        // skip a couple of RPCs before exiting.
-        commit(sOut);
-      }
-    }
-    taskDone.set(true);
-    // Make sure we send at least one set of counter increments. It's
-    // ok to call updateCounters() in this thread after comm thread stopped.
-    updateCounters();
-    sendLastUpdate();
-    //signal the tasktracker that we are done
-    //sendDone(umbilical);
-  }
-
-  /**
-   * Send a status update to the task tracker
-   * @param umbilical
-   * @throws IOException
-   */
-  public void statusUpdate() throws IOException, InterruptedException {
-    // TODO call progress update here if not being called within Map/Reduce
-  }
-
-  /**
-   * Sends last status update before sending umbilical.done();
-   */
-  private void sendLastUpdate()
-      throws IOException, InterruptedException {
-    statusUpdate();
-  }
-
-  private void commit(SimpleOutput output) throws IOException {
-    int retries = 3;
-    while (true) {
-      // This will loop till the AM asks for the task to be killed. As
-      // against, the AM sending a signal to the task to kill itself
-      // gracefully.
-      try {
-        if (processorContext.canCommit()) {
-          break;
-        }
-        Thread.sleep(1000);
-      } catch(InterruptedException ie) {
-        //ignore
-      } catch (IOException ie) {
-        LOG.warn("Failure sending canCommit: "
-            + StringUtils.stringifyException(ie));
-        if (--retries == 0) {
-          throw ie;
-        }
-      }
-    }
-
-    // task can Commit now
-    try {
-      LOG.info("Task " + taskAttemptId + " is allowed to commit now");
-      output.commit();
-      return;
-    } catch (IOException iee) {
-      LOG.warn("Failure committing: " +
-          StringUtils.stringifyException(iee));
-      //if it couldn't commit a successfully then delete the output
-      discardOutput(output);
-      throw iee;
-    }
-  }
-
-  private
-  void discardOutput(SimpleOutput output) {
-    try {
-      output.abort();
-    } catch (IOException ioe)  {
-      LOG.warn("Failure cleaning up: " +
-               StringUtils.stringifyException(ioe));
-    }
-  }
-
-
-  public void updateCounters() {
-    // TODO Auto-generated method stub
-    // TODO TEZAM Implement.
-    Map<String, List<FileSystem.Statistics>> map = new
-        HashMap<String, List<FileSystem.Statistics>>();
-    for(Statistics stat: FileSystem.getAllStatistics()) {
-      String uriScheme = stat.getScheme();
-      if (map.containsKey(uriScheme)) {
-        List<FileSystem.Statistics> list = map.get(uriScheme);
-        list.add(stat);
-      } else {
-        List<FileSystem.Statistics> list = new ArrayList<FileSystem.Statistics>();
-        list.add(stat);
-        map.put(uriScheme, list);
-      }
-    }
-    for (Map.Entry<String, List<FileSystem.Statistics>> entry: map.entrySet()) {
-      FileSystemStatisticUpdater updater = statisticUpdaters.get(entry.getKey());
-      if(updater==null) {//new FileSystem has been found in the cache
-        updater =
-            new FileSystemStatisticUpdater(counters, entry.getValue(),
-                entry.getKey());
-        statisticUpdaters.put(entry.getKey(), updater);
-      }
-      updater.updateCounters();
-    }
-
-    gcUpdater.incrementGcCounter();
-    updateResourceCounters();
-  }
-
-  /**
-   * Updates the {@link TaskCounter#COMMITTED_HEAP_BYTES} counter to reflect the
-   * current total committed heap space usage of this JVM.
-   */
-  private void updateHeapUsageCounter() {
-    long currentHeapUsage = Runtime.getRuntime().totalMemory();
-    counters.findCounter(TaskCounter.COMMITTED_HEAP_BYTES)
-            .setValue(currentHeapUsage);
-  }
-
-  /**
-   * Update resource information counters
-   */
-  void updateResourceCounters() {
-    // Update generic resource counters
-    updateHeapUsageCounter();
-
-    // Updating resources specified in ResourceCalculatorPlugin
-    if (pTree == null) {
-      return;
-    }
-    pTree.updateProcessTree();
-    long cpuTime = pTree.getCumulativeCpuTime();
-    long pMem = pTree.getCumulativeRssmem();
-    long vMem = pTree.getCumulativeVmem();
-    // Remove the CPU time consumed previously by JVM reuse
-    cpuTime -= initCpuCumulativeTime;
-    counters.findCounter(TaskCounter.CPU_MILLISECONDS).setValue(cpuTime);
-    counters.findCounter(TaskCounter.PHYSICAL_MEMORY_BYTES).setValue(pMem);
-    counters.findCounter(TaskCounter.VIRTUAL_MEMORY_BYTES).setValue(vMem);
-  }
-
-
-  public static String normalizeStatus(String status, Configuration conf) {
-    // Check to see if the status string is too long
-    // and truncate it if needed.
-    int progressStatusLength = conf.getInt(
-        MRConfig.PROGRESS_STATUS_LEN_LIMIT_KEY,
-        MRConfig.PROGRESS_STATUS_LEN_LIMIT_DEFAULT);
-    if (status.length() > progressStatusLength) {
-      LOG.warn("Task status: \"" + status + "\" truncated to max limit ("
-          + progressStatusLength + " characters)");
-      status = status.substring(0, progressStatusLength);
-    }
-    return status;
-  }
-
-  protected static <INKEY,INVALUE,OUTKEY,OUTVALUE>
-  org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context
-  createReduceContext(org.apache.hadoop.mapreduce.Reducer
-                        <INKEY,INVALUE,OUTKEY,OUTVALUE> reducer,
-                      Configuration job,
-                      TaskAttemptID taskId,
-                      final TezRawKeyValueIterator rIter,
-                      org.apache.hadoop.mapreduce.Counter inputKeyCounter,
-                      org.apache.hadoop.mapreduce.Counter inputValueCounter,
-                      org.apache.hadoop.mapreduce.RecordWriter<OUTKEY,OUTVALUE> output,
-                      org.apache.hadoop.mapreduce.OutputCommitter committer,
-                      org.apache.hadoop.mapreduce.StatusReporter reporter,
-                      RawComparator<INKEY> comparator,
-                      Class<INKEY> keyClass, Class<INVALUE> valueClass
-  ) throws IOException, InterruptedException {
-    RawKeyValueIterator r =
-        new RawKeyValueIterator() {
-
-          @Override
-          public boolean next() throws IOException {
-            return rIter.next();
-          }
-
-          @Override
-          public DataInputBuffer getValue() throws IOException {
-            return rIter.getValue();
-          }
-
-          @Override
-          public Progress getProgress() {
-            return rIter.getProgress();
-          }
-
-          @Override
-          public DataInputBuffer getKey() throws IOException {
-            return rIter.getKey();
-          }
-
-          @Override
-          public void close() throws IOException {
-            rIter.close();
-          }
-        };
-    org.apache.hadoop.mapreduce.ReduceContext<INKEY, INVALUE, OUTKEY, OUTVALUE>
-    reduceContext =
-      new ReduceContextImpl<INKEY, INVALUE, OUTKEY, OUTVALUE>(
-          job,
-          taskId,
-          r,
-          inputKeyCounter,
-          inputValueCounter,
-          output,
-          committer,
-          reporter,
-          comparator,
-          keyClass,
-          valueClass);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Using key class: " + keyClass
-          + ", valueClass: " + valueClass);
-    }
-
-    org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context
-        reducerContext =
-          new WrappedReducer<INKEY, INVALUE, OUTKEY, OUTVALUE>().getReducerContext(
-              reduceContext);
-
-    return reducerContext;
-  }
-
-  public void taskCleanup()
-      throws IOException, InterruptedException {
-    // set phase for this task
-    statusUpdate();
-    LOG.info("Runnning cleanup for the task");
-    // do the cleanup
-    committer.abortTask(taskAttemptContext);
-  }
-
-  public void localizeConfiguration(JobConf jobConf)
-      throws IOException, InterruptedException {
-    jobConf.set(JobContext.TASK_ID, taskAttemptId.getTaskID().toString());
-    jobConf.set(JobContext.TASK_ATTEMPT_ID, taskAttemptId.toString());
-    jobConf.setInt(JobContext.TASK_PARTITION,
-        taskAttemptId.getTaskID().getId());
-    jobConf.set(JobContext.ID, taskAttemptId.getJobID().toString());
-  }
-
-  public abstract TezCounter getOutputRecordsCounter();
-
-  public abstract TezCounter getInputRecordsCounter();
-
-  public org.apache.hadoop.mapreduce.TaskAttemptContext getTaskAttemptContext() {
-    return taskAttemptContext;
-  }
-
-  public JobContext getJobContext() {
-    return jobContext;
-  }
-
-  public TaskAttemptID getTaskAttemptId() {
-    return taskAttemptId;
-  }
-
-  public TezProcessorContext getTezEngineTaskContext() {
-    return processorContext;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java
deleted file mode 100644
index c7c9567..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/MRTaskReporter.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.mapreduce.newprocessor;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.mapred.Counters;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
-import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
-import org.apache.tez.mapreduce.hadoop.newmapred.MRReporter;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class MRTaskReporter
-    extends org.apache.hadoop.mapreduce.StatusReporter
-    implements Reporter {
-
-  private final TezTaskContext context;
-  private final boolean isProcessorContext;
-  private final Reporter reporter;
-
-  private InputSplit split = null;
-
-  public MRTaskReporter(TezProcessorContext context) {
-    this.context = context;
-    this.reporter = new MRReporter(context);
-    this.isProcessorContext = true;
-  }
-
-  public MRTaskReporter(TezOutputContext context) {
-    this.context = context;
-    this.reporter = new MRReporter(context);
-    this.isProcessorContext = false;
-  }
-  
-  public MRTaskReporter(TezInputContext context) {
-    this.context= context;
-    this.reporter = new MRReporter(context);
-    this.isProcessorContext = false;
-  }
-
-  public void setProgress(float progress) {
-    if (isProcessorContext) {
-      ((TezProcessorContext)context).setProgress(progress);
-    } else {
-      // TODO FIXME NEWTEZ - will simpleoutput's reporter use this api?
-    }
-  }
-
-  public void setStatus(String status) {
-    reporter.setStatus(status);
-  }
-
-  public float getProgress() {
-    return reporter.getProgress();
-  };
-
-  public void progress() {
-    reporter.progress();
-  }
-
-  public Counters.Counter getCounter(String group, String name) {
-    TezCounter counter = context.getCounters().findCounter(group, name);
-    MRCounters.MRCounter mrCounter = null;
-    if (counter != null) {
-      mrCounter = new MRCounters.MRCounter(counter);
-    }
-    return mrCounter;
-  }
-
-  public Counters.Counter getCounter(Enum<?> name) {
-    TezCounter counter = context.getCounters().findCounter(name);
-    MRCounters.MRCounter mrCounter = null;
-    if (counter != null) {
-      mrCounter = new MRCounters.MRCounter(counter);
-    }
-    return mrCounter;
-  }
-
-  public void incrCounter(Enum<?> key, long amount) {
-    reporter.incrCounter(key, amount);
-  }
-
-  public void incrCounter(String group, String counter, long amount) {
-    reporter.incrCounter(group, counter, amount);
-  }
-
-  public void setInputSplit(InputSplit split) {
-    this.split = split;
-  }
-
-  public InputSplit getInputSplit() throws UnsupportedOperationException {
-    if (split == null) {
-      throw new UnsupportedOperationException("Input only available on map");
-    } else {
-      return split;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/map/MapProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/map/MapProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/map/MapProcessor.java
deleted file mode 100644
index 21df743..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/map/MapProcessor.java
+++ /dev/null
@@ -1,341 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.mapreduce.newprocessor.map;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.MapRunnable;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
-import org.apache.hadoop.util.ReflectionUtils;
-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.lib.output.OnFileSortedOutput;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
-import org.apache.tez.engine.newapi.KVWriter;
-import org.apache.tez.engine.newapi.LogicalIOProcessor;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.mapreduce.hadoop.newmapreduce.MapContextImpl;
-import org.apache.tez.mapreduce.newinput.SimpleInput;
-import org.apache.tez.mapreduce.newinput.SimpleInputLegacy;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
-import org.apache.tez.mapreduce.newprocessor.MRTask;
-import org.apache.tez.mapreduce.newprocessor.MRTaskReporter;
-
-@SuppressWarnings({ "unchecked", "rawtypes" })
-public class MapProcessor extends MRTask implements LogicalIOProcessor {
-
-  private static final Log LOG = LogFactory.getLog(MapProcessor.class);
-
-  public MapProcessor(){
-    super(true);
-  }
-
-  @Override
-  public void initialize(TezProcessorContext processorContext)
-      throws IOException {
-    try {
-      super.initialize(processorContext);
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-  }
-
-
-  @Override
-  public void handleEvents(List<Event> processorEvents) {
-    // TODO Auto-generated method stub
-
-  }
-
-  public void close() throws IOException {
-    // TODO Auto-generated method stub
-
-  }
-
-  @Override
-  public void run(Map<String, LogicalInput> inputs,
-      Map<String, LogicalOutput> outputs) throws Exception {
-
-    LOG.info("Running map: " + processorContext.getUniqueIdentifier());
-
-    initTask();
-
-    if (inputs.size() != 1
-        || outputs.size() != 1) {
-      throw new IOException("Cannot handle multiple inputs or outputs"
-          + ", inputCount=" + inputs.size()
-          + ", outputCount=" + outputs.size());
-    }
-    LogicalInput in = inputs.values().iterator().next();
-    LogicalOutput out = outputs.values().iterator().next();
-
-    // Sanity check
-    if (!(in instanceof SimpleInputLegacy)) {
-      throw new IOException(new TezException(
-          "Only Simple Input supported. Input: " + in.getClass()));
-    }
-    SimpleInputLegacy input = (SimpleInputLegacy)in;
-
-    KVWriter kvWriter = null;
-    if (!(out instanceof OnFileSortedOutput)) {
-      kvWriter = ((SimpleOutput)out).getWriter();
-    } else {
-      kvWriter = ((OnFileSortedOutput)out).getWriter();
-    }
-
-    if (useNewApi) {
-      runNewMapper(jobConf, mrReporter, input, kvWriter);
-    } else {
-      runOldMapper(jobConf, mrReporter, input, kvWriter);
-    }
-
-    done(out);
-  }
-
-  void runOldMapper(
-      final JobConf job,
-      final MRTaskReporter reporter,
-      final SimpleInputLegacy input,
-      final KVWriter output
-      ) throws IOException, InterruptedException {
-
-    // Initialize input in-line since it sets parameters which may be used by the processor.
-    // Done only for SimpleInput.
-    // TODO use new method in SimpleInput to get required info
-    //input.initialize(job, master);
-
-    RecordReader in = new OldRecordReader(input);
-
-    OutputCollector collector = new OldOutputCollector(output);
-
-    MapRunnable runner =
-        (MapRunnable)ReflectionUtils.newInstance(job.getMapRunnerClass(), job);
-
-    runner.run(in, collector, (Reporter)reporter);
-    // start the sort phase only if there are reducers
-    this.statusUpdate();
-  }
-
-  private void runNewMapper(final JobConf job,
-      MRTaskReporter reporter,
-      final SimpleInputLegacy in,
-      KVWriter out
-      ) throws IOException, InterruptedException {
-
-    // Initialize input in-line since it sets parameters which may be used by the processor.
-    // Done only for SimpleInput.
-    // TODO use new method in SimpleInput to get required info
-    //in.initialize(job, master);
-
-    // make a task context so we can get the classes
-    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
-        getTaskAttemptContext();
-
-    // make a mapper
-    org.apache.hadoop.mapreduce.Mapper mapper;
-    try {
-      mapper = (org.apache.hadoop.mapreduce.Mapper)
-          ReflectionUtils.newInstance(taskContext.getMapperClass(), job);
-    } catch (ClassNotFoundException cnfe) {
-      throw new IOException(cnfe);
-    }
-
-    org.apache.hadoop.mapreduce.RecordReader input =
-        new NewRecordReader(in);
-
-    org.apache.hadoop.mapreduce.RecordWriter output =
-        new NewOutputCollector(out);
-
-    org.apache.hadoop.mapreduce.InputSplit split = in.getNewInputSplit();
-
-    org.apache.hadoop.mapreduce.MapContext
-    mapContext =
-    new MapContextImpl(
-        job, taskAttemptId,
-        input, output,
-        getCommitter(),
-        processorContext, split);
-
-    org.apache.hadoop.mapreduce.Mapper.Context mapperContext =
-        new WrappedMapper().getMapContext(mapContext);
-
-    input.initialize(split, mapperContext);
-    mapper.run(mapperContext);
-    this.statusUpdate();
-    input.close();
-    output.close(mapperContext);
-  }
-
-  private static class NewRecordReader extends
-      org.apache.hadoop.mapreduce.RecordReader {
-    private final SimpleInput in;
-    private KVReader reader;
-
-    private NewRecordReader(SimpleInput in) throws IOException {
-      this.in = in;
-      this.reader = in.getReader();
-    }
-
-    @Override
-    public void initialize(org.apache.hadoop.mapreduce.InputSplit split,
-        TaskAttemptContext context) throws IOException,
-        InterruptedException {
-      //in.initializeNewRecordReader(split, context);
-    }
-
-    @Override
-    public boolean nextKeyValue() throws IOException,
-        InterruptedException {
-      return reader.next();
-    }
-
-    @Override
-    public Object getCurrentKey() throws IOException,
-        InterruptedException {
-      return reader.getCurrentKV().getKey();
-    }
-
-    @Override
-    public Object getCurrentValue() throws IOException,
-        InterruptedException {
-      return reader.getCurrentKV().getValues().iterator().next();
-    }
-
-    @Override
-    public float getProgress() throws IOException, InterruptedException {
-      return in.getProgress();
-    }
-
-    @Override
-    public void close() throws IOException {
-    }
-  }
-
-  private static class OldRecordReader implements RecordReader {
-    private final SimpleInputLegacy simpleInput;
-
-    private OldRecordReader(SimpleInputLegacy simpleInput) {
-      this.simpleInput = simpleInput;
-    }
-
-    @Override
-    public boolean next(Object key, Object value) throws IOException {
-      // TODO broken
-//      simpleInput.setKey(key);
-//      simpleInput.setValue(value);
-//      try {
-//        return simpleInput.hasNext();
-//      } catch (InterruptedException ie) {
-//        throw new IOException(ie);
-//      }
-      return simpleInput.getOldRecordReader().next(key, value);
-    }
-
-    @Override
-    public Object createKey() {
-      return simpleInput.getOldRecordReader().createKey();
-    }
-
-    @Override
-    public Object createValue() {
-      return simpleInput.getOldRecordReader().createValue();
-    }
-
-    @Override
-    public long getPos() throws IOException {
-      return simpleInput.getOldRecordReader().getPos();
-    }
-
-    @Override
-    public void close() throws IOException {
-    }
-
-    @Override
-    public float getProgress() throws IOException {
-      try {
-        return simpleInput.getProgress();
-      } catch (InterruptedException ie) {
-        throw new IOException(ie);
-      }
-    }
-  }
-
-  private static class OldOutputCollector
-  implements OutputCollector {
-    private final KVWriter output;
-
-    OldOutputCollector(KVWriter output) {
-      this.output = output;
-    }
-
-    public void collect(Object key, Object value) throws IOException {
-        output.write(key, value);
-    }
-  }
-
-  private class NewOutputCollector
-    extends org.apache.hadoop.mapreduce.RecordWriter {
-    private final KVWriter out;
-
-    NewOutputCollector(KVWriter out) throws IOException {
-      this.out = out;
-    }
-
-    @Override
-    public void write(Object key, Object value) throws IOException, InterruptedException {
-      out.write(key, value);
-    }
-
-    @Override
-    public void close(TaskAttemptContext context
-                      ) throws IOException, InterruptedException {
-    }
-  }
-
-  @Override
-  public void localizeConfiguration(JobConf jobConf)
-      throws IOException, InterruptedException {
-    super.localizeConfiguration(jobConf);
-    jobConf.setBoolean(JobContext.TASK_ISMAP, true);
-  }
-
-  @Override
-  public TezCounter getOutputRecordsCounter() {
-    return processorContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS);
-  }
-
-  @Override
-  public TezCounter getInputRecordsCounter() {
-    return processorContext.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/reduce/ReduceProcessor.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/reduce/ReduceProcessor.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/reduce/ReduceProcessor.java
deleted file mode 100644
index cedcdd6..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/newprocessor/reduce/ReduceProcessor.java
+++ /dev/null
@@ -1,353 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.mapreduce.newprocessor.reduce;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapred.Counters.Counter;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.JobContext;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.Progressable;
-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.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.KVReader;
-import org.apache.tez.engine.newapi.KVWriter;
-import org.apache.tez.engine.newapi.LogicalIOProcessor;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.mapreduce.newinput.ShuffledMergedInputLegacy;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
-import org.apache.tez.mapreduce.newprocessor.MRTask;
-import org.apache.tez.mapreduce.newprocessor.MRTaskReporter;
-
-
-@SuppressWarnings({ "unchecked", "rawtypes" })
-public class ReduceProcessor
-extends MRTask
-implements LogicalIOProcessor {
-
-  private static final Log LOG = LogFactory.getLog(ReduceProcessor.class);
-
-  private Counter reduceInputKeyCounter;
-  private Counter reduceInputValueCounter;
-
-  public ReduceProcessor() {
-    super(false);
-  }
-
-  @Override
-  public void initialize(TezProcessorContext processorContext)
-      throws IOException {
-    try {
-      super.initialize(processorContext);
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-  }
-
-
-  @Override
-  public void handleEvents(List<Event> processorEvents) {
-    // TODO Auto-generated method stub
-
-  }
-
-  public void close() throws IOException {
-    // TODO Auto-generated method stub
-
-  }
-
-  @Override
-  public void run(Map<String, LogicalInput> inputs,
-      Map<String, LogicalOutput> outputs) throws Exception {
-
-    LOG.info("Running reduce: " + processorContext.getUniqueIdentifier());
-
-    initTask();
-
-    if (outputs.size() <= 0 || outputs.size() > 1) {
-      throw new IOException("Invalid number of outputs"
-          + ", outputCount=" + outputs.size());
-    }
-
-    if (inputs.size() <= 0 || inputs.size() > 1) {
-      throw new IOException("Invalid number of inputs"
-          + ", inputCount=" + inputs.size());
-    }
-
-    LogicalInput in = inputs.values().iterator().next();
-    LogicalOutput out = outputs.values().iterator().next();
-
-    this.statusUpdate();
-
-    Class keyClass = ConfigUtils.getIntermediateInputKeyClass(jobConf);
-    Class valueClass = ConfigUtils.getIntermediateInputValueClass(jobConf);
-    LOG.info("Using keyClass: " + keyClass);
-    LOG.info("Using valueClass: " + valueClass);
-    RawComparator comparator =
-        ConfigUtils.getInputKeySecondaryGroupingComparator(jobConf);
-    LOG.info("Using comparator: " + comparator);
-
-    reduceInputKeyCounter =
-        mrReporter.getCounter(TaskCounter.REDUCE_INPUT_GROUPS);
-    reduceInputValueCounter =
-        mrReporter.getCounter(TaskCounter.REDUCE_INPUT_RECORDS);
-
-    // Sanity check
-    if (!(in instanceof ShuffledMergedInputLegacy)) {
-      throw new IOException("Illegal input to reduce: " + in.getClass());
-    }
-    ShuffledMergedInputLegacy shuffleInput = (ShuffledMergedInputLegacy)in;
-    KVReader kvReader = shuffleInput.getReader();
-
-    KVWriter kvWriter = null;
-    if((out instanceof SimpleOutput)) {
-      kvWriter = ((SimpleOutput) out).getWriter();
-    } else if ((out instanceof OnFileSortedOutput)) {
-      kvWriter = ((OnFileSortedOutput) out).getWriter();
-    } else {
-      throw new IOException("Illegal input to reduce: " + in.getClass());
-    }
-
-    if (useNewApi) {
-      try {
-        runNewReducer(
-            jobConf,
-            mrReporter,
-            shuffleInput, comparator,  keyClass, valueClass,
-            kvWriter);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException(cnfe);
-      }
-    } else {
-      runOldReducer(
-          jobConf, mrReporter,
-          kvReader, comparator, keyClass, valueClass, kvWriter);
-    }
-
-    done(out);
-  }
-
-  void runOldReducer(JobConf job,
-      final MRTaskReporter reporter,
-      KVReader input,
-      RawComparator comparator,
-      Class keyClass,
-      Class valueClass,
-      final KVWriter output) throws IOException, InterruptedException {
-
-    Reducer reducer =
-        ReflectionUtils.newInstance(job.getReducerClass(), job);
-
-    // make output collector
-
-    OutputCollector collector =
-        new OutputCollector() {
-      public void collect(Object key, Object value)
-          throws IOException {
-        output.write(key, value);
-      }
-    };
-
-    // apply reduce function
-    try {
-      ReduceValuesIterator values =
-          new ReduceValuesIterator(
-              input, reporter, reduceInputValueCounter);
-
-      values.informReduceProgress();
-      while (values.more()) {
-        reduceInputKeyCounter.increment(1);
-        reducer.reduce(values.getKey(), values, collector, reporter);
-        values.informReduceProgress();
-      }
-
-      //Clean up: repeated in catch block below
-      reducer.close();
-      //End of clean up.
-    } catch (IOException ioe) {
-      try {
-        reducer.close();
-      } catch (IOException ignored) {
-      }
-
-      throw ioe;
-    }
-  }
-
-  private static class ReduceValuesIterator<KEY,VALUE>
-  implements Iterator<VALUE> {
-    private Counter reduceInputValueCounter;
-    private KVReader in;
-    private Progressable reporter;
-    private Object currentKey;
-    private Iterator<Object> currentValues;
-
-    public ReduceValuesIterator (KVReader in,
-        Progressable reporter,
-        Counter reduceInputValueCounter)
-            throws IOException {
-      this.reduceInputValueCounter = reduceInputValueCounter;
-      this.in = in;
-      this.reporter = reporter;
-    }
-
-    public boolean more() throws IOException {
-      boolean more = in.next();
-      if(more) {
-        currentKey = in.getCurrentKV().getKey();
-        currentValues = in.getCurrentKV().getValues().iterator();
-      } else {
-        currentKey = null;
-        currentValues = null;
-      }
-      return more;
-    }
-
-    public KEY getKey() throws IOException {
-      return (KEY) currentKey;
-    }
-
-    public void informReduceProgress() {
-      reporter.progress();
-    }
-
-    @Override
-    public boolean hasNext() {
-      return currentValues.hasNext();
-    }
-
-    @Override
-    public VALUE next() {
-      reduceInputValueCounter.increment(1);
-      return (VALUE) currentValues.next();
-    }
-
-    @Override
-    public void remove() {
-      throw new UnsupportedOperationException();
-    }
-
-  }
-
-  void runNewReducer(JobConf job,
-      final MRTaskReporter reporter,
-      ShuffledMergedInputLegacy input,
-      RawComparator comparator,
-      Class keyClass,
-      Class valueClass,
-      final KVWriter out
-      ) throws IOException,InterruptedException,
-      ClassNotFoundException {
-
-    // make a task context so we can get the classes
-    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext = getTaskAttemptContext();
-
-    // make a reducer
-    org.apache.hadoop.mapreduce.Reducer reducer =
-        (org.apache.hadoop.mapreduce.Reducer)
-        ReflectionUtils.newInstance(taskContext.getReducerClass(), job);
-
-    // wrap value iterator to report progress.
-    final TezRawKeyValueIterator rawIter = input.getIterator();
-    TezRawKeyValueIterator rIter = new TezRawKeyValueIterator() {
-      public void close() throws IOException {
-        rawIter.close();
-      }
-      public DataInputBuffer getKey() throws IOException {
-        return rawIter.getKey();
-      }
-      public Progress getProgress() {
-        return rawIter.getProgress();
-      }
-      public DataInputBuffer getValue() throws IOException {
-        return rawIter.getValue();
-      }
-      public boolean next() throws IOException {
-        boolean ret = rawIter.next();
-        reporter.setProgress(rawIter.getProgress().getProgress());
-        return ret;
-      }
-    };
-
-    org.apache.hadoop.mapreduce.RecordWriter trackedRW =
-        new org.apache.hadoop.mapreduce.RecordWriter() {
-
-      @Override
-      public void write(Object key, Object value) throws IOException,
-      InterruptedException {
-        out.write(key, value);
-      }
-
-      @Override
-      public void close(TaskAttemptContext context) throws IOException,
-      InterruptedException {
-      }
-    };
-
-    org.apache.hadoop.mapreduce.Reducer.Context reducerContext =
-        createReduceContext(
-            reducer, job, taskAttemptId,
-            rIter, reduceInputKeyCounter,
-            reduceInputValueCounter,
-            trackedRW,
-            committer,
-            reporter, comparator, keyClass,
-            valueClass);
-
-
-
-    reducer.run(reducerContext);
-    trackedRW.close(reducerContext);
-  }
-
-  @Override
-  public void localizeConfiguration(JobConf jobConf)
-      throws IOException, InterruptedException {
-    super.localizeConfiguration(jobConf);
-    jobConf.setBoolean(JobContext.TASK_ISMAP, false);
-  }
-
-  @Override
-  public TezCounter getOutputRecordsCounter() {
-    return processorContext.getCounters().findCounter(TaskCounter.REDUCE_OUTPUT_RECORDS);
-  }
-
-  @Override
-  public TezCounter getInputRecordsCounter() {
-    return processorContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_GROUPS);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
index 060e28c..91fb8cc 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
@@ -1,181 +1,226 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
 package org.apache.tez.mapreduce.output;
 
 import java.io.IOException;
+import java.text.NumberFormat;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem.Statistics;
+import org.apache.hadoop.mapred.FileOutputCommitter;
+import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskID;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezEngineTaskContext;
+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.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.records.OutputContext;
-import org.apache.tez.mapreduce.processor.MRTask;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.KVWriter;
+import org.apache.tez.engine.newapi.LogicalOutput;
+import org.apache.tez.engine.newapi.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;
 
-/**
- * {@link SimpleOutput} is an {@link Output} which persists key/value pairs
- * written to it. 
- * 
- * It is compatible with all standard Apache Hadoop MapReduce 
- * {@link OutputFormat} implementations. 
- */
-@SuppressWarnings({ "unchecked", "rawtypes" })
-public class SimpleOutput implements Output {
-
-  private MRTask task;
-  
+public class SimpleOutput implements LogicalOutput {
+
+  private static final Log LOG = LogFactory.getLog(SimpleOutput.class);
+
+  private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance();
+  static {
+    NUMBER_FORMAT.setMinimumIntegerDigits(5);
+    NUMBER_FORMAT.setGroupingUsed(false);
+  }
+
+  private TezOutputContext outputContext;
+  private JobConf jobConf;
   boolean useNewApi;
-  JobConf jobConf;
-  
-  org.apache.hadoop.mapreduce.TaskAttemptContext taskAttemptContext;
+  private AtomicBoolean closed = new AtomicBoolean(false);
+
+  @SuppressWarnings("rawtypes")
   org.apache.hadoop.mapreduce.OutputFormat newOutputFormat;
+  @SuppressWarnings("rawtypes")
   org.apache.hadoop.mapreduce.RecordWriter newRecordWriter;
-  
+
+  @SuppressWarnings("rawtypes")
   org.apache.hadoop.mapred.OutputFormat oldOutputFormat;
+  @SuppressWarnings("rawtypes")
   org.apache.hadoop.mapred.RecordWriter oldRecordWriter;
-  
+
   private TezCounter outputRecordCounter;
-  private TezCounter fileOutputByteCounter; 
+  private TezCounter fileOutputByteCounter;
   private List<Statistics> fsStats;
-  private MRTaskReporter reporter;
-  
-  public SimpleOutput(TezEngineTaskContext task)
-  {}
-  
-  public void setTask(MRTask task) {
-    this.task = task;
-  }
-  
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
 
-    if (task == null) {
-      return;
-    }
-    
-    if (conf instanceof JobConf) {
-      jobConf = (JobConf)conf;
-    } else {
-      jobConf = new JobConf(conf);
-    }
-    
-    useNewApi = jobConf.getUseNewMapper();
-    taskAttemptContext = task.getTaskAttemptContext();
-    
-    outputRecordCounter = task.getOutputRecordsCounter();
-    fileOutputByteCounter = task.getFileOutputBytesCounter();
-
-    reporter = task.getMRReporter();
-    
+  private TaskAttemptContext newApiTaskAttemptContext;
+  private org.apache.hadoop.mapred.TaskAttemptContext oldApiTaskAttemptContext;
+
+  private boolean isMapperOutput;
+
+  private OutputCommitter committer;
+
+  @Override
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws IOException, InterruptedException {
+    LOG.info("Initializing Simple Output");
+    this.outputContext = outputContext;
+    Configuration conf = TezUtils.createConfFromUserPayload(
+        outputContext.getUserPayload());
+    this.jobConf = new JobConf(conf);
+    this.useNewApi = this.jobConf.getUseNewMapper();
+    this.isMapperOutput = jobConf.getBoolean(MRConfig.IS_MAP_PROCESSOR,
+        false);
+    jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
+        outputContext.getDAGAttemptNumber());
+
+    outputRecordCounter = outputContext.getCounters().findCounter(
+        TaskCounter.MAP_OUTPUT_RECORDS);
+    fileOutputByteCounter = outputContext.getCounters().findCounter(
+        FileOutputFormatCounter.BYTES_WRITTEN);
+
     if (useNewApi) {
+      newApiTaskAttemptContext = createTaskAttemptContext();
       try {
         newOutputFormat =
             ReflectionUtils.newInstance(
-                taskAttemptContext.getOutputFormatClass(), jobConf);
+                newApiTaskAttemptContext.getOutputFormatClass(), jobConf);
       } catch (ClassNotFoundException cnfe) {
         throw new IOException(cnfe);
       }
-      
+
       List<Statistics> matchedStats = null;
-      if (newOutputFormat instanceof 
+      if (newOutputFormat instanceof
           org.apache.hadoop.mapreduce.lib.output.FileOutputFormat) {
-        matchedStats = 
-            MRTask.getFsStatistics(
+        matchedStats =
+            Utils.getFsStatistics(
                 org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
-                    .getOutputPath(taskAttemptContext), 
+                    .getOutputPath(newApiTaskAttemptContext),
                 jobConf);
       }
       fsStats = matchedStats;
 
       long bytesOutPrev = getOutputBytes();
-      newRecordWriter = 
-          newOutputFormat.getRecordWriter(this.taskAttemptContext);
+      try {
+        newRecordWriter =
+            newOutputFormat.getRecordWriter(newApiTaskAttemptContext);
+      } catch (InterruptedException e) {
+        throw new IOException("Interrupted while creating record writer", e);
+      }
       long bytesOutCurr = getOutputBytes();
       fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
     } else {
+      TaskAttemptID taskAttemptId = new TaskAttemptID(new TaskID(Long.toString(
+          outputContext.getApplicationId().getClusterTimestamp()),
+          outputContext.getApplicationId().getId(),
+          (isMapperOutput ? TaskType.MAP : TaskType.REDUCE),
+          outputContext.getTaskIndex()),
+          outputContext.getTaskAttemptNumber());
+      jobConf.set(JobContext.TASK_ATTEMPT_ID, taskAttemptId.toString());
+      jobConf.set(JobContext.TASK_ID, taskAttemptId.getTaskID().toString());
+      jobConf.setBoolean(JobContext.TASK_ISMAP, isMapperOutput);
+      jobConf.setInt(JobContext.TASK_PARTITION,
+          taskAttemptId.getTaskID().getId());
+      jobConf.set(JobContext.ID, taskAttemptId.getJobID().toString());
+
+      oldApiTaskAttemptContext =
+          new org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl(
+              jobConf, taskAttemptId,
+              new MRTaskReporter(outputContext));
       oldOutputFormat = jobConf.getOutputFormat();
-      
+
       List<Statistics> matchedStats = null;
-      if (oldOutputFormat instanceof org.apache.hadoop.mapred.FileOutputFormat) {
-        matchedStats = 
-            MRTask.getFsStatistics(
+      if (oldOutputFormat
+          instanceof org.apache.hadoop.mapred.FileOutputFormat) {
+        matchedStats =
+            Utils.getFsStatistics(
                 org.apache.hadoop.mapred.FileOutputFormat.getOutputPath(
-                    jobConf), 
+                    jobConf),
                 jobConf);
       }
       fsStats = matchedStats;
 
       FileSystem fs = FileSystem.get(jobConf);
-      String finalName = task.getOutputName();
+      String finalName = getOutputName();
 
       long bytesOutPrev = getOutputBytes();
-      oldRecordWriter = 
+      oldRecordWriter =
           oldOutputFormat.getRecordWriter(
-              fs, jobConf, finalName, reporter);
+              fs, jobConf, finalName, new MRReporter(outputContext));
       long bytesOutCurr = getOutputBytes();
       fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
     }
+    initCommitter(jobConf, useNewApi);
+
+    LOG.info("Initialized Simple Output"
+        + ", using_new_api: " + useNewApi);
+    return null;
   }
-  
-  public void write(Object key, Object value) 
+
+  public void initCommitter(JobConf job, boolean useNewApi)
       throws IOException, InterruptedException {
 
-    reporter.progress();
-    long bytesOutPrev = getOutputBytes();
-  
     if (useNewApi) {
-      newRecordWriter.write(key, value);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("using new api for output committer");
+      }
+
+      OutputFormat<?, ?> outputFormat = null;
+      try {
+        outputFormat = ReflectionUtils.newInstance(
+            newApiTaskAttemptContext.getOutputFormatClass(), job);
+      } catch (ClassNotFoundException cnfe) {
+        throw new IOException("Unknown OutputFormat", cnfe);
+      }
+      this.committer = outputFormat.getOutputCommitter(
+          newApiTaskAttemptContext);
     } else {
-      oldRecordWriter.write(key, value);
+      this.committer = job.getOutputCommitter();
     }
-    
-    long bytesOutCurr = getOutputBytes();
-    fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
-    outputRecordCounter.increment(1);
 
+    Path outputPath = FileOutputFormat.getOutputPath(job);
+    if (outputPath != null) {
+      if ((this.committer instanceof FileOutputCommitter)) {
+        FileOutputFormat.setWorkOutputPath(job,
+            ((FileOutputCommitter) this.committer).getTaskAttemptPath(
+                oldApiTaskAttemptContext));
+      } else {
+        FileOutputFormat.setWorkOutputPath(job, outputPath);
+      }
+    }
+    if (useNewApi) {
+      this.committer.setupTask(newApiTaskAttemptContext);
+    } else {
+      this.committer.setupTask(oldApiTaskAttemptContext);
+    }
   }
 
-  public void close() throws IOException, InterruptedException {
-    reporter.progress();
-    long bytesOutPrev = getOutputBytes();
+  public boolean isCommitRequired() throws IOException {
     if (useNewApi) {
-      newRecordWriter.close(taskAttemptContext);
+      return committer.needsTaskCommit(newApiTaskAttemptContext);
     } else {
-      oldRecordWriter.close(null);
+      return committer.needsTaskCommit(oldApiTaskAttemptContext);
     }
-    long bytesOutCurr = getOutputBytes();
-    fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
   }
 
-  public org.apache.hadoop.mapreduce.OutputFormat getNewOutputFormat() {
-    return newOutputFormat;
-  }
-  
-  public org.apache.hadoop.mapred.OutputFormat getOldOutputFormat() {
-    return oldOutputFormat;
+  private TaskAttemptContext createTaskAttemptContext() {
+    return new TaskAttemptContextImpl(this.jobConf, outputContext,
+        isMapperOutput);
   }
-  
+
   private long getOutputBytes() {
     if (fsStats == null) return 0;
     long bytesWritten = 0;
@@ -185,9 +230,97 @@ public class SimpleOutput implements Output {
     return bytesWritten;
   }
 
+  private String getOutputName() {
+    return "part-" + NUMBER_FORMAT.format(outputContext.getTaskIndex());
+  }
+
+  @Override
+  public KVWriter getWriter() throws IOException {
+    return new KVWriter() {
+      private final boolean useNewWriter = useNewApi;
+
+      @SuppressWarnings("unchecked")
+      @Override
+      public void write(Object key, Object value) throws IOException {
+        long bytesOutPrev = getOutputBytes();
+        if (useNewWriter) {
+          try {
+            newRecordWriter.write(key, value);
+          } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new IOException("Interrupted while writing next key-value",e);
+          }
+        } else {
+          oldRecordWriter.write(key, value);
+        }
+
+        long bytesOutCurr = getOutputBytes();
+        fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
+        outputRecordCounter.increment(1);
+      }
+    };
+  }
+
+  @Override
+  public void handleEvents(List<Event> outputEvents) {
+    // Not expecting any events at the moment.
+  }
+
   @Override
-  public OutputContext getOutputContext() {
+  public synchronized List<Event> close() throws IOException {
+    if (closed.getAndSet(true)) {
+      return null;
+    }
+
+    LOG.info("Closing Simple Output");
+    long bytesOutPrev = getOutputBytes();
+    if (useNewApi) {
+      try {
+        newRecordWriter.close(newApiTaskAttemptContext);
+      } catch (InterruptedException e) {
+        throw new IOException("Interrupted while closing record writer", e);
+      }
+    } else {
+      oldRecordWriter.close(null);
+    }
+    long bytesOutCurr = getOutputBytes();
+    fileOutputByteCounter.increment(bytesOutCurr - bytesOutPrev);
+    LOG.info("Closed Simple Output");
     return null;
   }
 
+  @Override
+  public void setNumPhysicalOutputs(int numOutputs) {
+    // Nothing to do for now
+  }
+
+  /**
+   * SimpleOutput expects that a Processor call commit prior to the
+   * Processor's completion
+   * @throws IOException
+   */
+  public void commit() throws IOException {
+    close();
+    if (useNewApi) {
+      committer.commitTask(newApiTaskAttemptContext);
+    } else {
+      committer.commitTask(oldApiTaskAttemptContext);
+    }
+  }
+
+
+  /**
+   * SimpleOutput expects that a Processor call abort in case of any error
+   * ( including an error during commit ) prior to the Processor's completion
+   * @throws IOException
+   */
+  public void abort() throws IOException {
+    close();
+    if (useNewApi) {
+      committer.abortTask(newApiTaskAttemptContext);
+    } else {
+      committer.abortTask(oldApiTaskAttemptContext);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
----------------------------------------------------------------------
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 61dfcd1..d061ad5 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
@@ -1,70 +1,54 @@
 /**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 
 package org.apache.tez.mapreduce.partition;
 
-import java.io.IOException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Partitioner;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.mapreduce.processor.MRTask;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.engine.common.ConfigUtils;
+import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 
-@SuppressWarnings({"rawtypes", "unchecked"})
+@SuppressWarnings({ "rawtypes", "unchecked" })
 public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
 
   static final Log LOG = LogFactory.getLog(MRPartitioner.class);
-  private final MRTask task;
-  
-  JobConf jobConf;
-  boolean useNewApi;
-  
-  org.apache.hadoop.mapred.Partitioner oldPartitioner;
-  org.apache.hadoop.mapreduce.Partitioner newPartitioner;
 
-  public MRPartitioner(MRTask task) {
-    this.task = task;
-  }
-  
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException {
-    if (conf instanceof JobConf) {
-      jobConf = (JobConf)conf;
-    } else {
-      jobConf = new JobConf(conf);
-    }
-    
-    useNewApi = jobConf.getUseNewMapper();
-    final int partitions = this.task.getTezEngineTaskContext()
-        .getOutputSpecList().get(0).getNumOutputs();
+  private final boolean useNewApi;
+  private int partitions = 1;
+
+  private org.apache.hadoop.mapreduce.Partitioner newPartitioner;
+  private org.apache.hadoop.mapred.Partitioner oldPartitioner;
+
+  public MRPartitioner(Configuration conf) {
+    this.useNewApi = ConfigUtils.useNewApi(conf);
+    this.partitions = conf.getInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, 1);
+
     if (useNewApi) {
       if (partitions > 1) {
-        try {
-          newPartitioner = (org.apache.hadoop.mapreduce.Partitioner)
-            ReflectionUtils.newInstance(
-                task.getJobContext().getPartitionerClass(), jobConf);
-        } catch (ClassNotFoundException cnfe) {
-          throw new IOException(cnfe);
-        }
+        newPartitioner = (org.apache.hadoop.mapreduce.Partitioner) ReflectionUtils
+            .newInstance(
+                (Class<? extends org.apache.hadoop.mapreduce.Partitioner<?, ?>>) conf
+                    .getClass(MRJobConfig.PARTITIONER_CLASS_ATTR,
+                        org.apache.hadoop.mapreduce.lib.partition.HashPartitioner.class), conf);
       } else {
         newPartitioner = new org.apache.hadoop.mapreduce.Partitioner() {
           @Override
@@ -75,24 +59,24 @@ public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
       }
     } else {
       if (partitions > 1) {
-        oldPartitioner = (Partitioner)
-          ReflectionUtils.newInstance(jobConf.getPartitionerClass(), jobConf);
+        oldPartitioner = (org.apache.hadoop.mapred.Partitioner) ReflectionUtils.newInstance(
+            (Class<? extends org.apache.hadoop.mapred.Partitioner>) conf.getClass(
+                "mapred.partitioner.class", org.apache.hadoop.mapred.lib.HashPartitioner.class), conf);
       } else {
-        oldPartitioner = new Partitioner() {
+        oldPartitioner = new org.apache.hadoop.mapred.Partitioner() {
           @Override
-          public void configure(JobConf job) {}
-          
+          public void configure(JobConf job) {
+          }
+
           @Override
           public int getPartition(Object key, Object value, int numPartitions) {
             return numPartitions - 1;
           }
         };
       }
-
     }
-
   }
-  
+
   @Override
   public int getPartition(Object key, Object value, int numPartitions) {
     if (useNewApi) {
@@ -101,5 +85,4 @@ public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
       return oldPartitioner.getPartition(key, value, numPartitions);
     }
   }
-
-}
+}
\ No newline at end of file


[13/20] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
deleted file mode 100644
index dae5625..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
+++ /dev/null
@@ -1,291 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api.client.rpc;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.Collections;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.client.api.YarnClient;
-import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezException;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.api.client.DAGClient;
-import org.apache.tez.dag.api.client.DAGStatus;
-import org.apache.tez.dag.api.client.VertexStatus;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetDAGStatusRequestProto;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.GetVertexStatusRequestProto;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TryKillDAGRequestProto;
-import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProto;
-import org.apache.tez.dag.api.records.DAGProtos.DAGStatusStateProto;
-
-import com.google.protobuf.ServiceException;
-
-public class DAGClientRPCImpl implements DAGClient {
-  private static final Log LOG = LogFactory.getLog(DAGClientRPCImpl.class);
-
-  private final ApplicationId appId;
-  private final String dagId;
-  private final TezConfiguration conf;
-  private ApplicationReport appReport;
-  private YarnClient yarnClient;
-  private DAGClientAMProtocolBlockingPB proxy = null;
-
-  public DAGClientRPCImpl(ApplicationId appId, String dagId,
-      TezConfiguration conf) {
-    this.appId = appId;
-    this.dagId = dagId;
-    this.conf = conf;
-    yarnClient = new YarnClientImpl();
-    yarnClient.init(new YarnConfiguration(conf));
-    yarnClient.start();
-    appReport = null;
-  }
-
-  @Override
-  public ApplicationId getApplicationId() {
-    return appId;
-  }
-
-  @Override
-  public DAGStatus getDAGStatus() throws IOException, TezException {
-    if(createAMProxyIfNeeded()) {
-      try {
-        return getDAGStatusViaAM();
-      } catch (TezException e) {
-        resetProxy(e); // create proxy again
-      }
-    }
-
-    // Later maybe from History
-    return getDAGStatusViaRM();
-  }
-
-  @Override
-  public VertexStatus getVertexStatus(String vertexName)
-                                    throws IOException, TezException {
-    if(createAMProxyIfNeeded()) {
-      try {
-        return getVertexStatusViaAM(vertexName);
-      } catch (TezException e) {
-        resetProxy(e); // create proxy again
-      }
-    }
-
-    // need AM for this. Later maybe from History
-    return null;
-  }
-
-  @Override
-  public void tryKillDAG() throws TezException, IOException {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("TryKill for app: " + appId + " dag:" + dagId);
-    }
-    if(createAMProxyIfNeeded()) {
-      TryKillDAGRequestProto requestProto =
-          TryKillDAGRequestProto.newBuilder().setDagId(dagId).build();
-      try {
-        proxy.tryKillDAG(null, requestProto);
-      } catch (ServiceException e) {
-        resetProxy(e);
-      }
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (this.proxy != null) {
-      RPC.stopProxy(this.proxy);
-    }
-    if(yarnClient != null) {
-      yarnClient.stop();
-    }
-  }
-
-  @Override
-  public ApplicationReport getApplicationReport() {
-    return appReport;
-  }
-
-  void resetProxy(Exception e) {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Resetting AM proxy for app: " + appId + " dag:" + dagId +
-          " due to exception :", e);
-    }
-    proxy = null;
-  }
-
-  DAGStatus getDAGStatusViaAM() throws IOException, TezException {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId);
-    }
-    GetDAGStatusRequestProto requestProto =
-        GetDAGStatusRequestProto.newBuilder().setDagId(dagId).build();
-    try {
-      return new DAGStatus(
-                 proxy.getDAGStatus(null, requestProto).getDagStatus());
-    } catch (ServiceException e) {
-      // TEZ-151 retrieve wrapped TezException
-      throw new TezException(e);
-    }
-  }
-
-
-
-  DAGStatus getDAGStatusViaRM() throws TezException, IOException {
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("GetDAGStatus via AM for app: " + appId + " dag:" + dagId);
-    }
-    ApplicationReport appReport;
-    try {
-      appReport = yarnClient.getApplicationReport(appId);
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-
-    if(appReport == null) {
-      throw new TezException("Unknown/Invalid appId: " + appId);
-    }
-
-    DAGStatusProto.Builder builder = DAGStatusProto.newBuilder();
-    DAGStatus dagStatus = new DAGStatus(builder);
-    DAGStatusStateProto dagState = null;
-    switch (appReport.getYarnApplicationState()) {
-    case NEW:
-    case NEW_SAVING:
-    case SUBMITTED:
-    case ACCEPTED:
-      dagState = DAGStatusStateProto.DAG_SUBMITTED;
-      break;
-    case RUNNING:
-      dagState = DAGStatusStateProto.DAG_RUNNING;
-      break;
-    case FAILED:
-      dagState = DAGStatusStateProto.DAG_FAILED;
-      break;
-    case KILLED:
-      dagState = DAGStatusStateProto.DAG_KILLED;
-      break;
-    case FINISHED:
-      switch(appReport.getFinalApplicationStatus()) {
-      case UNDEFINED:
-      case FAILED:
-        dagState = DAGStatusStateProto.DAG_FAILED;
-        break;
-      case KILLED:
-        dagState = DAGStatusStateProto.DAG_KILLED;
-        break;
-      case SUCCEEDED:
-        dagState = DAGStatusStateProto.DAG_SUCCEEDED;
-        break;
-      default:
-        throw new TezUncheckedException("Encountered unknown final application"
-          + " status from YARN"
-          + ", appState=" + appReport.getYarnApplicationState()
-          + ", finalStatus=" + appReport.getFinalApplicationStatus());
-      }
-      break;
-    default:
-      throw new TezUncheckedException("Encountered unknown application state"
-          + " from YARN, appState=" + appReport.getYarnApplicationState());
-    }
-
-    builder.setState(dagState);
-    if(appReport.getDiagnostics() != null) {
-      builder.addAllDiagnostics(Collections.singleton(appReport.getDiagnostics()));
-    }
-
-    return dagStatus;
-  }
-
-  VertexStatus getVertexStatusViaAM(String vertexName) throws TezException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("GetVertexStatus via AM for app: " + appId + " dag: " + dagId
-          + " vertex: " + vertexName);
-    }
-    GetVertexStatusRequestProto requestProto =
-        GetVertexStatusRequestProto.newBuilder().
-                        setDagId(dagId).setVertexName(vertexName).build();
-
-    try {
-      return new VertexStatus(
-                 proxy.getVertexStatus(null, requestProto).getVertexStatus());
-    } catch (ServiceException e) {
-      // TEZ-151 retrieve wrapped TezException
-      throw new TezException(e);
-    }
-  }
-
-  ApplicationReport getAppReport() throws IOException, TezException {
-    try {
-      ApplicationReport appReport = yarnClient.getApplicationReport(appId);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("App: " + appId + " in state: "
-            + appReport.getYarnApplicationState());
-      }
-      return appReport;
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-  }
-
-  boolean createAMProxyIfNeeded() throws IOException, TezException {
-    if(proxy != null) {
-      // if proxy exist optimistically use it assuming there is no retry
-      return true;
-    }
-    appReport = getAppReport();
-
-    if(appReport == null) {
-      return false;
-    }
-    YarnApplicationState appState = appReport.getYarnApplicationState();
-    if(appState != YarnApplicationState.RUNNING) {
-      return false;
-    }
-
-    // YARN-808. Cannot ascertain if AM is ready until we connect to it.
-    // workaround check the default string set by YARN
-    if(appReport.getHost() == null || appReport.getHost().equals("N/A") ||
-        appReport.getRpcPort() == 0){
-      // attempt not running
-      return false;
-    }
-
-    InetSocketAddress addr = new InetSocketAddress(appReport.getHost(),
-        appReport.getRpcPort());
-
-    RPC.setProtocolEngine(conf, DAGClientAMProtocolBlockingPB.class,
-        ProtobufRpcEngine.class);
-    proxy = (DAGClientAMProtocolBlockingPB) RPC.getProxy(
-        DAGClientAMProtocolBlockingPB.class, 0, addr, conf);
-    return true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/proto/DAGApiRecords.proto
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/proto/DAGApiRecords.proto b/tez-dag-api/src/main/proto/DAGApiRecords.proto
deleted file mode 100644
index 4385749..0000000
--- a/tez-dag-api/src/main/proto/DAGApiRecords.proto
+++ /dev/null
@@ -1,183 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-option java_package = "org.apache.tez.dag.api.records";
-option java_outer_classname = "DAGProtos";
-option java_generate_equals_and_hash = true;
-
-// DAG plan messages
-
-// Many of these types have a dual in the Tez-api.  To reduce confusion, these types have prefix or suffix 
-// of "Plan" to indicate they are to be used in the dag-plan.
-// The big types use a suffix:  JobPlan, VertexPlan, EdgePlan 
-//   --> these get more direct use in the runtime and the naming is natural.
-// The enums and utility types use prefix: PlanVertexType, PlanEdgeConnectionPaatern, etc
-//   --> there is not great naming choice for these that avoids ambiguity, but this one seems acceptable. 
-
-enum PlanVertexType {
-  INPUT = 0;
-  NORMAL = 1;
-  OUTPUT = 2;
-}
-
-enum PlanEdgeDataMovementType {
-  ONE_TO_ONE = 0;
-  BROADCAST = 1;
-  SCATTER_GATHER = 2;
-}
-
-enum PlanEdgeDataSourceType {
-  PERSISTED = 0;
-  PERSISTED_RELIABLE = 1;
-  EPHEMERAL = 2;
-}
-
-enum PlanEdgeSchedulingType {
-  SEQUENTIAL = 0;
-  CONCURRENT = 1;
-}
-
-message PlanKeyValuePair {
-  required string key = 1;
-  required string value = 2;
-}
-
-enum PlanLocalResourceType {
-  FILE = 0;
-  ARCHIVE = 1;
-  PATTERN = 2;
-}
-
-enum PlanLocalResourceVisibility {
-  PUBLIC = 0;
-  PRIVATE = 1;
-  APPLICATION = 2;
-}
-
-message PlanLocalResource {
-  required string name = 1;
-  required string uri = 2;
-  required int64 size = 3;
-  required int64 timeStamp = 4;
-  required PlanLocalResourceType type = 5;
-  required PlanLocalResourceVisibility visibility = 6;
-  optional string pattern = 7; // only used if type=PATTERN
-}
-
-// Each taskLocationHint represents a single split in in the input.
-// It is the list of [{rack,machines}] that host a replica of each particular split.
-// For now it is represented as pair-of-arrays rather than array-of-pairs.
-message PlanTaskLocationHint {
-  repeated string rack = 1;
-  repeated string host = 2;
-}
-
-message PlanTaskConfiguration {
-  required int32 numTasks = 1;
-  required int32 memoryMb = 2;
-  required int32 virtualCores = 3;
-  required string javaOpts = 4;
-  required string taskModule = 5;
-  repeated PlanLocalResource localResource = 6;
-  repeated PlanKeyValuePair environmentSetting = 8;  
-}
-
-message TezEntityDescriptorProto {
-  optional string class_name = 1;
-  optional bytes user_payload = 2;
-}
-
-message VertexPlan {
-  required string name = 1;
-  required PlanVertexType type = 2;
-  optional TezEntityDescriptorProto processor_descriptor = 3;
-  required PlanTaskConfiguration taskConfig = 4;
-  repeated PlanTaskLocationHint taskLocationHint = 7;
-  repeated string inEdgeId = 8;
-  repeated string outEdgeId = 9;
-}
-
-message EdgePlan {
-  required string id = 1;
-  required string inputVertexName = 2;
-  required string outputVertexName = 3;
-  required PlanEdgeDataMovementType dataMovementType = 4;
-  required PlanEdgeDataSourceType dataSourceType = 5;
-  required PlanEdgeSchedulingType schedulingType = 6;
-  optional TezEntityDescriptorProto edge_source = 7;
-  optional TezEntityDescriptorProto edge_destination = 8;
-}
-
-message ConfigurationProto {
-  repeated PlanKeyValuePair confKeyValues = 1;
-}
-
-message DAGPlan {
-  required string name = 1;
-  repeated VertexPlan vertex = 2;
-  repeated EdgePlan edge = 3;
-  optional ConfigurationProto dagKeyValues = 4;
-}
-
-// DAG monitoring messages
-message ProgressProto {
-  optional int32 totalTaskCount = 1;
-  optional int32 succeededTaskCount = 2;
-  optional int32 runningTaskCount = 3;
-  optional int32 failedTaskCount = 4;
-  optional int32 killedTaskCount = 5;
-}
-
-enum VertexStatusStateProto {
-  VERTEX_INITED = 0;
-  VERTEX_RUNNING = 1;
-  VERTEX_SUCCEEDED = 2;
-  VERTEX_KILLED = 3;
-  VERTEX_FAILED = 4;
-  VERTEX_ERROR = 5;
-  VERTEX_TERMINATING = 6;
-}
-
-message VertexStatusProto {
-  optional VertexStatusStateProto state = 1;
-  repeated string diagnostics = 2;
-  optional ProgressProto progress = 3;
-}
-
-enum DAGStatusStateProto {
-  DAG_SUBMITTED = 0;
-  DAG_INITING = 1;
-  DAG_RUNNING = 2;
-  DAG_SUCCEEDED = 3;
-  DAG_KILLED = 4;
-  DAG_FAILED = 5;
-  DAG_ERROR = 6;
-  DAG_TERMINATING = 7;
-}
-
-message StringProgressPairProto {
-  required string key = 1;
-  required ProgressProto progress = 2;
-}
-
-message DAGStatusProto {
-  optional DAGStatusStateProto state = 1;
-  repeated string diagnostics = 2;
-  optional ProgressProto DAGProgress = 3;
-  repeated StringProgressPairProto vertexProgress = 4;  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/proto/DAGClientAMProtocol.proto
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/proto/DAGClientAMProtocol.proto b/tez-dag-api/src/main/proto/DAGClientAMProtocol.proto
deleted file mode 100644
index 6fcd1f8..0000000
--- a/tez-dag-api/src/main/proto/DAGClientAMProtocol.proto
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-option java_package = "org.apache.tez.dag.api.client.rpc";
-option java_outer_classname = "DAGClientAMProtocolRPC";
-option java_generic_services = true;
-option java_generate_equals_and_hash = true;
-
-//import "DAGClientAMProtocolRecords.proto";
-
-import "DAGApiRecords.proto";
-
-message GetAllDAGsRequestProto {
-}
-
-message GetAllDAGsResponseProto {
-  repeated string dagId = 1;
-}
-
-message GetDAGStatusRequestProto {
-  optional string dagId = 1;
-}
-
-message GetDAGStatusResponseProto {
-  optional DAGStatusProto dagStatus = 1;
-}
-
-message GetVertexStatusRequestProto {
-  optional string dagId = 1;
-  optional string vertexName = 2;
-}
-
-message GetVertexStatusResponseProto {
-  optional VertexStatusProto vertexStatus = 1;
-}
-
-message TryKillDAGRequestProto {
-  optional string dagId = 1;
-}
-
-message TryKillDAGResponseProto {
-  //nothing yet
-}
-
-message SubmitDAGRequestProto {
-  optional DAGPlan d_a_g_plan = 1;
-}
-
-message SubmitDAGResponseProto {
-  optional string dagId = 1;
-}
-
-message ShutdownSessionRequestProto {
-}
-
-message ShutdownSessionResponseProto {
-}
-
-service DAGClientAMProtocol {
-  rpc getAllDAGs (GetAllDAGsRequestProto) returns (GetAllDAGsResponseProto);
-  rpc getDAGStatus (GetDAGStatusRequestProto) returns (GetDAGStatusResponseProto);
-  rpc getVertexStatus (GetVertexStatusRequestProto) returns (GetVertexStatusResponseProto);
-  rpc tryKillDAG (TryKillDAGRequestProto) returns (TryKillDAGResponseProto);
-  rpc submitDAG (SubmitDAGRequestProto) returns (SubmitDAGResponseProto);
-  rpc shutdownSession (ShutdownSessionRequestProto) returns (ShutdownSessionResponseProto);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java b/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
deleted file mode 100644
index 53ec357..0000000
--- a/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGPlan.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-import static org.junit.Assert.*;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.HashMap;
-
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
-import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
-import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
-import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
-import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
-import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
-import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
-import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
-import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-// based on TestDAGLocationHint
-public class TestDAGPlan {
-  @Rule
-  public TemporaryFolder tempFolder = new TemporaryFolder(); //TODO: doesn't seem to be deleting this folder automatically as expected.
-
-  @Test
-  public void testBasicJobPlanSerde() throws IOException {
-
-    DAGPlan job = DAGPlan.newBuilder()
-       .setName("test")
-       .addVertex(
-           VertexPlan.newBuilder()
-             .setName("vertex1")
-             .setType(PlanVertexType.NORMAL)
-             .addTaskLocationHint(PlanTaskLocationHint.newBuilder().addHost("machineName").addRack("rack1").build())
-             .setTaskConfig(
-                 PlanTaskConfiguration.newBuilder()
-                   .setNumTasks(2)
-                   .setVirtualCores(4)
-                   .setMemoryMb(1024)
-                   .setJavaOpts("")
-                   .setTaskModule("x.y")
-                   .build())
-             .build())
-        .build();
-   File file = tempFolder.newFile("jobPlan");
-   FileOutputStream outStream = null;
-   try {
-     outStream = new FileOutputStream(file);
-     job.writeTo(outStream);
-   }
-   finally {
-     if(outStream != null){
-       outStream.close();
-     }
-   }
-
-   DAGPlan inJob;
-   FileInputStream inputStream;
-   try {
-     inputStream = new FileInputStream(file);
-     inJob = DAGPlan.newBuilder().mergeFrom(inputStream).build();
-   }
-   finally {
-     outStream.close();
-   }
-
-   Assert.assertEquals(job, inJob);
-  }
-
-  @Test
-  public void testUserPayloadSerde() {
-    DAG dag = new DAG("testDag");
-    ProcessorDescriptor pd1 = new ProcessorDescriptor("processor1").
-        setUserPayload("processor1Bytes".getBytes());
-    ProcessorDescriptor pd2 = new ProcessorDescriptor("processor2").
-        setUserPayload("processor2Bytes".getBytes());
-    Vertex v1 = new Vertex("v1", pd1, 10, Resource.newInstance(1024, 1));
-    Vertex v2 = new Vertex("v2", pd2, 1, Resource.newInstance(1024, 1));
-    v1.setJavaOpts("").setTaskEnvironment(new HashMap<String, String>())
-        .setTaskLocalResources(new HashMap<String, LocalResource>());
-    v2.setJavaOpts("").setTaskEnvironment(new HashMap<String, String>())
-        .setTaskLocalResources(new HashMap<String, LocalResource>());
-
-    InputDescriptor inputDescriptor = new InputDescriptor("input").
-        setUserPayload("inputBytes".getBytes());
-    OutputDescriptor outputDescriptor = new OutputDescriptor("output").
-        setUserPayload("outputBytes".getBytes());
-    Edge edge = new Edge(v1, v2, new EdgeProperty(
-        DataMovementType.SCATTER_GATHER, DataSourceType.PERSISTED,
-        SchedulingType.SEQUENTIAL, outputDescriptor, inputDescriptor));
-
-    dag.addVertex(v1).addVertex(v2).addEdge(edge);
-
-    DAGPlan dagProto = dag.createDag(new TezConfiguration());
-
-    assertEquals(2, dagProto.getVertexCount());
-    assertEquals(1, dagProto.getEdgeCount());
-
-    VertexPlan v1Proto = dagProto.getVertex(0);
-    VertexPlan v2Proto = dagProto.getVertex(1);
-    EdgePlan edgeProto = dagProto.getEdge(0);
-
-    assertEquals("processor1Bytes", new String(v1Proto.getProcessorDescriptor()
-        .getUserPayload().toByteArray()));
-    assertEquals("processor1", v1Proto.getProcessorDescriptor().getClassName());
-
-    assertEquals("processor2Bytes", new String(v2Proto.getProcessorDescriptor()
-        .getUserPayload().toByteArray()));
-    assertEquals("processor2", v2Proto.getProcessorDescriptor().getClassName());
-
-    assertEquals("inputBytes", new String(edgeProto.getEdgeDestination()
-        .getUserPayload().toByteArray()));
-    assertEquals("input", edgeProto.getEdgeDestination().getClassName());
-
-    assertEquals("outputBytes", new String(edgeProto.getEdgeSource()
-        .getUserPayload().toByteArray()));
-    assertEquals("output", edgeProto.getEdgeSource().getClassName());
-
-    EdgeProperty edgeProperty = DagTypeConverters
-        .createEdgePropertyMapFromDAGPlan(dagProto.getEdgeList().get(0));
-
-    byte[] ib = edgeProperty.getEdgeDestination().getUserPayload();
-    assertEquals("inputBytes", new String(ib));
-    assertEquals("input", edgeProperty.getEdgeDestination().getClassName());
-
-    byte[] ob = edgeProperty.getEdgeSource().getUserPayload();
-    assertEquals("outputBytes", new String(ob));
-    assertEquals("output", edgeProperty.getEdgeSource().getClassName());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java b/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
deleted file mode 100644
index b33f3a6..0000000
--- a/tez-dag-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
+++ /dev/null
@@ -1,417 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.dag.api;
-
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
-import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
-import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class TestDAGVerify {
-
-  private final String dummyProcessorClassName = TestDAGVerify.class.getName();
-  private final String dummyInputClassName = TestDAGVerify.class.getName();
-  private final String dummyOutputClassName = TestDAGVerify.class.getName();
-  private final int dummyTaskCount = 2;
-  private final Resource dummyTaskResource = Resource.newInstance(1, 1);
-
-  //    v1
-  //    |
-  //    v2
-  @Test
-  public void testVerify1() {
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor(dummyProcessorClassName),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor(dummyOutputClassName),
-            new InputDescriptor(dummyInputClassName)));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addEdge(e1);
-    dag.verify();
-  }
-
-  @Test(expected = IllegalStateException.class)  
-  public void testVerify2() {
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor(dummyProcessorClassName),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.ONE_TO_ONE, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor(dummyOutputClassName),
-            new InputDescriptor(dummyInputClassName)));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addEdge(e1);
-    dag.verify();
-  }
-
-  @Test(expected = IllegalStateException.class)  
-  public void testVerify3() {
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor(dummyProcessorClassName),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.EPHEMERAL, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor(dummyOutputClassName),
-            new InputDescriptor(dummyInputClassName)));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addEdge(e1);
-    dag.verify();
-  }
-
-  @Test(expected = IllegalStateException.class)  
-  public void testVerify4() {
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor(dummyProcessorClassName),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.EPHEMERAL, SchedulingType.CONCURRENT, 
-            new OutputDescriptor(dummyOutputClassName),
-            new InputDescriptor(dummyInputClassName)));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addEdge(e1);
-    dag.verify();
-  }
-
-  //    v1 <----
-  //      |     ^
-  //       v2   ^
-  //      |  |  ^
-  //    v3    v4
-  @Test
-  public void testCycle1() {
-    IllegalStateException ex=null;
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v3 = new Vertex("v3",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v4 = new Vertex("v4",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e2 = new Edge(v2, v3,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e3 = new Edge(v2, v4,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e4 = new Edge(v4, v1,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addVertex(v3);
-    dag.addVertex(v4);
-    dag.addEdge(e1);
-    dag.addEdge(e2);
-    dag.addEdge(e3);
-    dag.addEdge(e4);
-    try{
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage().startsWith("DAG contains a cycle"));
-  }
-
-  //     v1
-  //      |
-  //    -> v2
-  //    ^  | |
-  //    v3    v4
-  @Test
-  public void testCycle2() {
-    IllegalStateException ex=null;
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v3 = new Vertex("v3",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v4 = new Vertex("v4",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e2 = new Edge(v2, v3,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e3 = new Edge(v2, v4,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e4 = new Edge(v3, v2,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addVertex(v3);
-    dag.addVertex(v4);
-    dag.addEdge(e1);
-    dag.addEdge(e2);
-    dag.addEdge(e3);
-    dag.addEdge(e4);
-    try{
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage().startsWith("DAG contains a cycle"));
-  }
-
-  @Test
-  public void repeatedVertexName() {
-    IllegalStateException ex=null;
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v1repeat = new Vertex("v1",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v1repeat);
-    try {
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage().startsWith("DAG contains multiple vertices with name"));
-  }
-
-  //  v1  v2
-  //   |  |
-  //    v3
-  @Test
-  public void BinaryInputDisallowed() {
-    IllegalStateException ex=null;
-    try {
-      Vertex v1 = new Vertex("v1",
-          new ProcessorDescriptor("MapProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Vertex v2 = new Vertex("v2",
-          new ProcessorDescriptor("MapProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Vertex v3 = new Vertex("v3",
-          new ProcessorDescriptor("ReduceProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Edge e1 = new Edge(v1, v3,
-          new EdgeProperty(DataMovementType.ONE_TO_ONE, 
-              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-              new OutputDescriptor("dummy output class"),
-              new InputDescriptor("dummy input class")));
-      Edge e2 = new Edge(v2, v3,
-          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-              new OutputDescriptor("dummy output class"),
-              new InputDescriptor("dummy input class")));
-      DAG dag = new DAG("testDag");
-      dag.addVertex(v1);
-      dag.addVertex(v2);
-      dag.addVertex(v3);
-      dag.addEdge(e1);
-      dag.addEdge(e2);
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage().startsWith(
-        "Unsupported connection pattern on edge"));
-  }
-
-  //  v1  v2
-  //   |  |
-  //    v3
-  @Test
-  public void BinaryInputAllowed() {
-    Vertex v1 = new Vertex("v1",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v2 = new Vertex("v2",
-        new ProcessorDescriptor("MapProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Vertex v3 = new Vertex("v3",
-        new ProcessorDescriptor("ReduceProcessor"),
-        dummyTaskCount, dummyTaskResource);
-    Edge e1 = new Edge(v1, v3,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL,
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    Edge e2 = new Edge(v2, v3,
-        new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-            DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-            new OutputDescriptor("dummy output class"),
-            new InputDescriptor("dummy input class")));
-    DAG dag = new DAG("testDag");
-    dag.addVertex(v1);
-    dag.addVertex(v2);
-    dag.addVertex(v3);
-    dag.addEdge(e1);
-    dag.addEdge(e2);
-    dag.verify();
-  }
-
-  //   v1
-  //  |  |
-  //  v2  v3
-  @Test
-  public void BinaryOutput() {
-    IllegalStateException ex=null;
-    try {
-      Vertex v1 = new Vertex("v1",
-          new ProcessorDescriptor("MapProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Vertex v2 = new Vertex("v2",
-          new ProcessorDescriptor("MapProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Vertex v3 = new Vertex("v3",
-          new ProcessorDescriptor("MapProcessor"),
-          dummyTaskCount, dummyTaskResource);
-      Edge e1 = new Edge(v1, v2,
-          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-              new OutputDescriptor("dummy output class"),
-              new InputDescriptor("dummy input class")));
-      Edge e2 = new Edge(v1, v2,
-          new EdgeProperty(DataMovementType.SCATTER_GATHER, 
-              DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-              new OutputDescriptor("dummy output class"),
-              new InputDescriptor("dummy input class")));
-      DAG dag = new DAG("testDag");
-      dag.addVertex(v1);
-      dag.addVertex(v2);
-      dag.addVertex(v3);
-      dag.addEdge(e1);
-      dag.addEdge(e2);
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage().startsWith("Vertex has outDegree>1"));
-  }
-
-  @Test
-  public void testDagWithNoVertices() {
-    IllegalStateException ex=null;
-    try {
-      DAG dag = new DAG("testDag");
-      dag.verify();
-    }
-    catch (IllegalStateException e){
-      ex = e;
-    }
-    Assert.assertNotNull(ex);
-    System.out.println(ex.getMessage());
-    Assert.assertTrue(ex.getMessage()
-        .startsWith("Invalid dag containing 0 vertices"));
-  }
-
-  @SuppressWarnings("unused")
-  @Test
-  public void testInvalidVertexConstruction() {
-    try {
-      Vertex v1 = new Vertex("v1",
-          new ProcessorDescriptor("MapProcessor"),
-          0, dummyTaskResource);
-      Assert.fail("Expected exception for 0 parallelism");
-    } catch (IllegalArgumentException e) {
-      Assert.assertTrue(e.getMessage().startsWith("Parallelism cannot be 0"));
-    }
-    try {
-      Vertex v1 = new Vertex("v1",
-          new ProcessorDescriptor("MapProcessor"),
-          1, null);
-      Assert.fail("Expected exception for 0 parallelism");
-    } catch (IllegalArgumentException e) {
-      Assert.assertTrue(e.getMessage().startsWith("Resource cannot be null"));
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml
index 875a196..bc6aeef 100644
--- a/tez-dag/pom.xml
+++ b/tez-dag/pom.xml
@@ -29,6 +29,10 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.tez</groupId>
+      <artifactId>tez-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
       <artifactId>tez-common</artifactId>
     </dependency>
     <dependency>
@@ -55,14 +59,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-dag-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-engine-api</artifactId>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
----------------------------------------------------------------------
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 36486c9..7a143a5 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
@@ -44,7 +44,6 @@ import org.apache.tez.common.records.ProceedToCompletionResponse;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.Task;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventOutputConsumable;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
 import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.app.rm.container.AMContainerImpl;
@@ -56,9 +55,6 @@ 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.engine.records.OutputContext;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 
 @SuppressWarnings("unchecked")
 public class TaskAttemptListenerImpTezDag extends AbstractService implements
@@ -176,29 +172,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
-  public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-      int fromEventIdx, int maxEvents,
-      TezTaskAttemptID taskAttemptID) {
-
-    LOG.info("Dependency Completion Events request from " + taskAttemptID
-        + ". fromEventID " + fromEventIdx + " maxEvents " + maxEvents);
-
-    // TODO: shouldReset is never used. See TT. Ask for Removal.
-    boolean shouldReset = false;
-    TezDependentTaskCompletionEvent[] events =
-        context.getCurrentDAG().
-            getVertex(taskAttemptID.getTaskID().getVertexID()).
-                getTaskAttemptCompletionEvents(taskAttemptID, fromEventIdx, maxEvents);
-
-    taskHeartbeatHandler.progressing(taskAttemptID);
-    pingContainerHeartbeatHandler(taskAttemptID);
-
-    // No filters for now. Only required events stored in a vertex.
-
-    return new TezTaskDependencyCompletionEventsUpdate(events,shouldReset);
-  }
-
-  @Override
   public ContainerTask getTask(ContainerContext containerContext)
       throws IOException {
 
@@ -370,17 +343,6 @@ public class TaskAttemptListenerImpTezDag extends AbstractService implements
   }
 
   @Override
-  public void outputReady(TezTaskAttemptID taskAttemptId,
-      OutputContext outputContext) throws IOException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("AttemptId: " + taskAttemptId + " reported output context: "
-          + outputContext);
-    }
-    context.getEventHandler().handle(
-        new TaskAttemptEventOutputConsumable(taskAttemptId, outputContext));
-  }
-
-  @Override
   public ProceedToCompletionResponse
       proceedToCompletion(TezTaskAttemptID taskAttemptId) throws IOException {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
----------------------------------------------------------------------
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 c476966..2779faf 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 @@ package org.apache.tez.dag.app.dag;
 
 import java.util.List;
 
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 
 public abstract class EdgeManager {
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputConsumable.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputConsumable.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputConsumable.java
deleted file mode 100644
index f10209f..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventOutputConsumable.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.tez.dag.app.dag.event;
-
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.OutputContext;
-
-public class TaskAttemptEventOutputConsumable extends TaskAttemptEvent {
-
-  private final OutputContext outputContext;
-
-  public TaskAttemptEventOutputConsumable(TezTaskAttemptID id,
-      OutputContext outputContext) {
-    super(id, TaskAttemptEventType.TA_OUTPUT_CONSUMABLE);
-    this.outputContext = outputContext;
-  }
-
-  public OutputContext getOutputContext() {
-    return this.outputContext;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
----------------------------------------------------------------------
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 71f17ac..b05a6f1 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 java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 
 public class BroadcastEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
----------------------------------------------------------------------
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 3605857..060a112 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.Vertex;
 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.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
 
 public class Edge {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
----------------------------------------------------------------------
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 7c4743e..a916ad2 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 java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 
 public class OneToOneEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
----------------------------------------------------------------------
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 380b6b6..1d4df5b 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 java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.DataMovementEvent;
+import org.apache.tez.engine.api.events.InputFailedEvent;
+import org.apache.tez.engine.api.events.InputReadErrorEvent;
 
 public class ScatterGatherEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
----------------------------------------------------------------------
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 b854a43..a0ed329 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,9 +39,9 @@ import org.apache.tez.dag.app.dag.VertexScheduler;
 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.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
----------------------------------------------------------------------
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 1ec1225..74005b7 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,14 +106,14 @@ import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.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.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java b/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
new file mode 100644
index 0000000..7a4dd13
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/engine/records/TezDependentTaskCompletionEvent.java
@@ -0,0 +1,228 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.engine.records;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+/**
+ * This is used to track task completion events on 
+ * job tracker. 
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+// TODO TEZAM3 This needs to be more generic. Maybe some kind of a serialized
+// blob - which can be interpretted by the Input plugin.
+public class TezDependentTaskCompletionEvent implements Writable {
+  @InterfaceAudience.Public
+  @InterfaceStability.Evolving
+  // TODO EVENTUALLY - Remove TIPFAILED state ?
+  static public enum Status {FAILED, KILLED, SUCCEEDED, OBSOLETE, TIPFAILED};
+    
+  private int eventId;
+  private int taskRunTime; // using int since runtime is the time difference
+  private TezTaskAttemptID taskAttemptId;
+  private long dataSize;
+  Status status;
+  byte[] userPayload;
+  // TODO TEZAM2 Get rid of the isMap field. Job specific type information can be determined from TaskAttemptId.getTaskType
+//  boolean isMap = false;
+  public static final TezDependentTaskCompletionEvent[] EMPTY_ARRAY = 
+    new TezDependentTaskCompletionEvent[0];
+
+  public TezDependentTaskCompletionEvent() {
+    taskAttemptId = new TezTaskAttemptID();
+  }
+  
+  /**
+   * Constructor. eventId should be created externally and incremented
+   * per event for each job. 
+   * @param eventId event id, event id should be unique and assigned in
+   *  incrementally, starting from 0. 
+   * @param taskAttemptId task id
+   * @param status task's status 
+   * @param taskTrackerHttp task tracker's host:port for http. 
+   */
+  public TezDependentTaskCompletionEvent(int eventId, 
+                             TezTaskAttemptID taskAttemptId,
+//                             boolean isMap,
+                             Status status, 
+                             int runTime,
+                             long dataSize){
+      
+    this.taskAttemptId = taskAttemptId;
+//    this.isMap = isMap;
+    this.eventId = eventId; 
+    this.status =status; 
+    this.taskRunTime = runTime;
+    this.dataSize = dataSize;
+  }
+  
+  public TezDependentTaskCompletionEvent clone() {
+    TezDependentTaskCompletionEvent clone = new TezDependentTaskCompletionEvent(
+        this.eventId, this.taskAttemptId, this.status, 
+        this.taskRunTime, this.dataSize);
+    
+    return clone;
+  }
+  
+  /**
+   * Returns event Id. 
+   * @return event id
+   */
+  public int getEventId() {
+    return eventId;
+  }
+
+  /**
+   * Returns task id. 
+   * @return task id
+   */
+  public TezTaskAttemptID getTaskAttemptID() {
+    return taskAttemptId;
+  }
+  
+  /**
+   * Returns enum Status.SUCESS or Status.FAILURE.
+   * @return task tracker status
+   */
+  public Status getStatus() {
+    return status;
+  }
+  
+  /**
+   * Returns time (in millisec) the task took to complete. 
+   */
+  public int getTaskRunTime() {
+    return taskRunTime;
+  }
+  
+  /**
+   * Return size of output produced by the task
+   */
+  public long getDataSize() {
+    return dataSize;
+  }
+  
+  /**
+   * @return user payload. Maybe null
+   */
+  public byte[] getUserPayload() {
+    return userPayload;
+  }
+
+  /**
+   * Set the task completion time
+   * @param taskCompletionTime time (in millisec) the task took to complete
+   */
+  protected void setTaskRunTime(int taskCompletionTime) {
+    this.taskRunTime = taskCompletionTime;
+  }
+
+  /**
+   * set event Id. should be assigned incrementally starting from 0. 
+   * @param eventId
+   */
+  public void setEventId(int eventId) {
+    this.eventId = eventId;
+  }
+
+  /**
+   * Sets task id. 
+   * @param taskId
+   */
+  public void setTaskAttemptID(TezTaskAttemptID taskId) {
+    this.taskAttemptId = taskId;
+  }
+  
+  /**
+   * Set task status. 
+   * @param status
+   */
+  public void setTaskStatus(Status status) {
+    this.status = status;
+  }
+  
+  /**
+   * Set the user payload
+   * @param userPayload
+   */
+  public void setUserPayload(byte[] userPayload) {
+    this.userPayload = userPayload;
+  }
+    
+  @Override
+  public String toString(){
+    StringBuffer buf = new StringBuffer(); 
+    buf.append("Task Id : "); 
+    buf.append(taskAttemptId); 
+    buf.append(", Status : ");  
+    buf.append(status.name());
+    return buf.toString();
+  }
+    
+  @Override
+  public boolean equals(Object o) {
+    // not counting userPayload as that is a piggyback mechanism
+    if(o == null)
+      return false;
+    if(o.getClass().equals(this.getClass())) {
+      TezDependentTaskCompletionEvent event = (TezDependentTaskCompletionEvent) o;
+      return this.eventId == event.getEventId()
+             && this.status.equals(event.getStatus())
+             && this.taskAttemptId.equals(event.getTaskAttemptID()) 
+             && this.taskRunTime == event.getTaskRunTime()
+             && this.dataSize == event.getDataSize();
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return toString().hashCode(); 
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    taskAttemptId.write(out);
+//    out.writeBoolean(isMap);
+    WritableUtils.writeEnum(out, status);
+    WritableUtils.writeVInt(out, taskRunTime);
+    WritableUtils.writeVInt(out, eventId);
+    WritableUtils.writeCompressedByteArray(out, userPayload);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    taskAttemptId.readFields(in);
+//    isMap = in.readBoolean();
+    status = WritableUtils.readEnum(in, Status.class);
+    taskRunTime = WritableUtils.readVInt(in);
+    eventId = WritableUtils.readVInt(in);
+    userPayload = WritableUtils.readCompressedByteArray(in);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java b/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
new file mode 100644
index 0000000..13c9088
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/engine/records/TezTaskDependencyCompletionEventsUpdate.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.engine.records;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+
+
+public class TezTaskDependencyCompletionEventsUpdate implements Writable {
+  TezDependentTaskCompletionEvent[] events;
+  boolean reset;
+
+  public TezTaskDependencyCompletionEventsUpdate() { }
+
+  public TezTaskDependencyCompletionEventsUpdate(
+      TezDependentTaskCompletionEvent[] events, boolean reset) {
+    this.events = events;
+    this.reset = reset;
+  }
+
+  public boolean shouldReset() {
+    return reset;
+  }
+
+  public TezDependentTaskCompletionEvent[] getDependentTaskCompletionEvents() {
+    return events;
+  }
+  
+  public void write(DataOutput out) throws IOException {
+    out.writeBoolean(reset);
+    out.writeInt(events.length);
+    for (TezDependentTaskCompletionEvent event : events) {
+      event.write(out);
+    }
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    reset = in.readBoolean();
+    events = new TezDependentTaskCompletionEvent[in.readInt()];
+    for (int i = 0; i < events.length; ++i) {
+      events[i] = new TezDependentTaskCompletionEvent();
+      events[i].readFields(in);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
----------------------------------------------------------------------
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 d4eae9d..c2457e1 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
@@ -52,7 +52,6 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.tez.common.TezTaskContext;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerHeartbeatHandler;
 import org.apache.tez.dag.app.ContainerContext;
@@ -66,6 +65,7 @@ import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.engine.api.impl.TaskSpec;
 import org.apache.tez.engine.common.security.JobTokenIdentifier;
 import org.apache.tez.engine.common.security.TokenCache;
 import org.junit.Test;
@@ -109,7 +109,7 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.RUNNING);
     wc.verifyNoOutgoingEvents();
     assertFalse(pulledTask.shouldDie());
-    assertEquals(wc.tezTaskContext.getTaskAttemptId(), pulledTask.getTask()
+    assertEquals(wc.taskSpec.getTaskAttemptID(), pulledTask.getTask()
         .getTaskAttemptID());
     assertEquals(wc.taskAttemptID, wc.amContainer.getRunningTaskAttempt());
     assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
@@ -165,7 +165,7 @@ public class TestAMContainer {
     wc.verifyState(AMContainerState.RUNNING);
     wc.verifyNoOutgoingEvents();
     assertFalse(pulledTask.shouldDie());
-    assertEquals(wc.tezTaskContext.getTaskAttemptId(), pulledTask.getTask()
+    assertEquals(wc.taskSpec.getTaskAttemptID(), pulledTask.getTask()
         .getTaskAttemptID());
     assertEquals(wc.taskAttemptID, wc.amContainer.getRunningTaskAttempt());
     assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
@@ -824,7 +824,7 @@ public class TestAMContainer {
     TezTaskID taskID;
     TezTaskAttemptID taskAttemptID;
 
-    TezTaskContext tezTaskContext;
+    TaskSpec taskSpec;
 
     public AMContainerImpl amContainer;
 
@@ -859,8 +859,8 @@ public class TestAMContainer {
       taskID = new TezTaskID(vertexID, 1);
       taskAttemptID = new TezTaskAttemptID(taskID, 1);
 
-      tezTaskContext = mock(TezTaskContext.class);
-      doReturn(taskAttemptID).when(tezTaskContext).getTaskAttemptId();
+      taskSpec = mock(TaskSpec.class);
+      doReturn(taskAttemptID).when(taskSpec).getTaskAttemptID();
 
       amContainer = new AMContainerImpl(container, chh, tal,
           appContext);
@@ -904,7 +904,7 @@ public class TestAMContainer {
     public void assignTaskAttempt(TezTaskAttemptID taID) {
       reset(eventHandler);
       amContainer.handle(new AMContainerEventAssignTA(containerID, taID,
-          tezTaskContext));
+          taskSpec));
     }
 
     public AMContainerTask pullTaskToRun() {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dist/src/main/assembly/tez-dist-full.xml
----------------------------------------------------------------------
diff --git a/tez-dist/src/main/assembly/tez-dist-full.xml b/tez-dist/src/main/assembly/tez-dist-full.xml
index 3176dd1..383eb43 100644
--- a/tez-dist/src/main/assembly/tez-dist-full.xml
+++ b/tez-dist/src/main/assembly/tez-dist-full.xml
@@ -24,8 +24,7 @@
     <moduleSet>
       <useAllReactorProjects>true</useAllReactorProjects>
       <includes>
-        <include>org.apache.tez:tez-dag-api</include>
-        <include>org.apache.tez:tez-engine-api</include>
+        <include>org.apache.tez:tez-api</include>
       </includes>
       <binaries>
         <outputDirectory>/</outputDirectory>
@@ -40,8 +39,7 @@
       <outputDirectory>/lib</outputDirectory>
       <!-- Exclude hadoop artifacts. They will be found via HADOOP* env -->
       <excludes>
-        <exclude>org.apache.tez:tez-dag-api</exclude>
-        <exclude>org.apache.tez:tez-engine-api</exclude>
+        <exclude>org.apache.tez:tez-api</exclude>
         <exclude>*:*:test-jar</exclude>
         <exclude>org.apache.hadoop:hadoop-common</exclude>
         <exclude>org.apache.hadoop:hadoop-auth</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dist/src/main/assembly/tez-dist.xml
----------------------------------------------------------------------
diff --git a/tez-dist/src/main/assembly/tez-dist.xml b/tez-dist/src/main/assembly/tez-dist.xml
index 01d0d40..aefb8a2 100644
--- a/tez-dist/src/main/assembly/tez-dist.xml
+++ b/tez-dist/src/main/assembly/tez-dist.xml
@@ -24,8 +24,7 @@
     <moduleSet>
       <useAllReactorProjects>true</useAllReactorProjects>
      <includes>
-        <include>org.apache.tez:tez-dag-api</include>
-        <include>org.apache.tez:tez-engine-api</include>
+       <include>org.apache.tez:tez-api</include>
       </includes>
       <binaries>
         <outputDirectory>/</outputDirectory>
@@ -42,8 +41,7 @@
       <useTransitiveFiltering>true</useTransitiveFiltering>
       <!-- Exclude hadoop artifacts. They will be found via HADOOP* env -->
       <excludes>
-        <exclude>org.apache.tez:tez-dag-api</exclude>
-        <exclude>org.apache.tez:tez-engine-api</exclude>
+        <exclude>org.apache.tez:tez-api</exclude>
         <exclude>*:*:test-jar</exclude>
         <exclude>org.apache.hadoop:hadoop-common</exclude>
         <exclude>org.apache.hadoop:hadoop-auth</exclude>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-engine-api/findbugs-exclude.xml b/tez-engine-api/findbugs-exclude.xml
deleted file mode 100644
index 5b11308..0000000
--- a/tez-engine-api/findbugs-exclude.xml
+++ /dev/null
@@ -1,16 +0,0 @@
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-<FindBugsFilter>
-
-</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/pom.xml
----------------------------------------------------------------------
diff --git a/tez-engine-api/pom.xml b/tez-engine-api/pom.xml
deleted file mode 100644
index b19e96b..0000000
--- a/tez-engine-api/pom.xml
+++ /dev/null
@@ -1,91 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.tez</groupId>
-    <artifactId>tez</artifactId>
-    <version>0.2.0-SNAPSHOT</version>
-  </parent>
-  <artifactId>tez-engine-api</artifactId>
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-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>
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-        <configuration>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-maven-plugins</artifactId>
-        <executions>
-          <execution>
-            <id>compile-protoc</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>protoc</goal>
-            </goals>
-            <configuration>
-              <protocVersion>${protobuf.version}</protocVersion>
-              <protocCommand>${protoc.path}</protocCommand>
-              <imports>
-                <param>${basedir}/src/main/proto</param>
-              </imports>
-              <source>
-                <directory>${basedir}/src/main/proto</directory>
-                <includes>
-                  <include>Events.proto</include>
-                </includes>
-              </source>
-              <output>${project.build.directory}/generated-sources/java</output>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/api/Input.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Input.java b/tez-engine-api/src/main/java/org/apache/tez/engine/api/Input.java
deleted file mode 100644
index 64c3834..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Input.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.api;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * {@link Input} represents a pipe through which an <em>tez</em> task
- * can get input key/value pairs.
- */
-public interface Input {
-  
-  /**
-   * Initialize <code>Input</code>.
-   * 
-   * @param conf job configuration
-   * @param master master process controlling the task
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException;
-  
-  /**
-   * Check if there is another key/value pair.
-   * 
-   * @return true if a key/value pair was read
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public boolean hasNext() throws IOException, InterruptedException;
-
-  /**
-   * Get the next key.
-   * 
-   * @return the current key or null if there is no current key
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public Object getNextKey() throws IOException, InterruptedException;
-  
-  /**
-   * Get the next values.
-   * 
-   * @return the object that was read
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public Iterable<Object> getNextValues() 
-      throws IOException, InterruptedException;
-  
-  /**
-   * The current progress of the {@link Input} through its data.
-   * 
-   * @return a number between 0.0 and 1.0 that is the fraction of the data read
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public float getProgress() throws IOException, InterruptedException;
-  
-  /**
-   * Close this <code>Input</code> for future operations.
-   */
-  public void close() throws IOException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/api/Master.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Master.java b/tez-engine-api/src/main/java/org/apache/tez/engine/api/Master.java
deleted file mode 100644
index f3add9a..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Master.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.api;
-
-import org.apache.hadoop.ipc.ProtocolInfo;
-import org.apache.hadoop.ipc.VersionedProtocol;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
-
-/**
- * {@link Master} represents the master controlling the {@link Task}. 
- */
-@ProtocolInfo(protocolName = "Master", protocolVersion = 1)
-public interface Master extends VersionedProtocol {
-
-  // TODO TEZAM3 This likely needs to change to be a little more generic.
-  // Many output / input relationships cannot be captured via this. The current
-  // form works primarily works for the existing MR
-
-  TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-      int fromEventIdx, int maxEventsToFetch,
-      TezTaskAttemptID taskAttemptId);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/api/Output.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Output.java b/tez-engine-api/src/main/java/org/apache/tez/engine/api/Output.java
deleted file mode 100644
index daa80d0..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Output.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.api;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.engine.records.OutputContext;
-
-/**
- * {@link Output} represents a pipe through which an <em>tez</em> task
- * can send out outputs.
- */
-public interface Output {
-
-  /**
-   * Initialize <code>Output</code>.
-   * 
-   * @param conf job configuration
-   * @param master master process controlling the task
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException;
-
-  /** 
-   * Writes a key/value pair.
-   *
-   * @param key the key to write.
-   * @param value the value to write.
-   * @throws IOException
-   */      
-  public void write(Object key, Object value
-                             ) throws IOException, InterruptedException;
-
-  /**
-   * Returns the OutputContext for the particular <code>Output</code>. 
-   * 
-   * @return the OutputContext for this Output if it exists, otherwise null.
-   */
-  public OutputContext getOutputContext();
-  
-  /** 
-   * Close this <code>Output</code> for future operations.
-   * 
-   * @throws IOException
-   */ 
-  public void close() throws IOException, InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine-api/src/main/java/org/apache/tez/engine/api/Processor.java
----------------------------------------------------------------------
diff --git a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Processor.java b/tez-engine-api/src/main/java/org/apache/tez/engine/api/Processor.java
deleted file mode 100644
index 550ee73..0000000
--- a/tez-engine-api/src/main/java/org/apache/tez/engine/api/Processor.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.engine.api;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-
-/**
- * {@link Processor} represents the <em>tez</em> entity responsible for
- * consuming {@link Input} and producing {@link Output}. 
- */
-public interface Processor {
-  
-  /**
-   * Initialize the <code>Processor</code>.
-   * 
-   * @param conf job-configuration
-   * @param master master process controlling the task
-   * @throws IOException 
-   * @throws InterruptedException
-   */
-  public void initialize(Configuration conf, Master master) 
-      throws IOException, InterruptedException;
-  
-  /**
-   * Process input data from <code>input</code> and 
-   * send it to <code>output</code>.
-   * 
-   * @param in input
-   * @param out output
-   * @param master master process controlling the task
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void process(Input[] in, Output[]  out)
-      throws IOException, InterruptedException;
-
-  /**
-   * Close the {@link Processor}.
-   * 
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void close() throws IOException, InterruptedException;
-
-}


[02/20] TEZ-444. Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
----------------------------------------------------------------------
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 0653cc8..af087c9 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
@@ -19,69 +19,75 @@
 package org.apache.tez.mapreduce.processor;
 
 import java.io.IOException;
+import java.net.URI;
 import java.text.NumberFormat;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.FutureTask;
+import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import javax.crypto.SecretKey;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.FileSystem.Statistics;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapred.FileOutputCommitter;
-import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobContext;
+import org.apache.hadoop.mapred.MapOutputFile;
 import org.apache.hadoop.mapred.RawKeyValueIterator;
 import org.apache.hadoop.mapred.TaskAttemptContext;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskID;
 import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
+import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
 import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
+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.RunningTaskContext;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.common.TezTaskStatus;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.common.TezTaskStatus.Phase;
+import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezTaskStatus.State;
+import org.apache.tez.common.TezUtils;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.common.records.ProceedToCompletionResponse;
 import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
+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.engine.newapi.LogicalOutput;
+import org.apache.tez.engine.newapi.TezProcessorContext;
 import org.apache.tez.engine.records.OutputContext;
-import org.apache.tez.mapreduce.combine.MRCombiner;
+import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
-import org.apache.tez.mapreduce.hadoop.MRTaskStatus;
+import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
 import org.apache.tez.mapreduce.hadoop.mapreduce.JobContextImpl;
-import org.apache.tez.mapreduce.hadoop.mapreduce.TezNullOutputCommitter;
-import org.apache.tez.mapreduce.partition.MRPartitioner;
+import org.apache.tez.mapreduce.output.SimpleOutput;
+import org.apache.tez.mapreduce.task.impl.YarnOutputFiles;
 
-public abstract class MRTask extends RunningTaskContext {
+@SuppressWarnings("deprecation")
+public abstract class MRTask {
 
   static final Log LOG = LogFactory.getLog(MRTask.class);
 
@@ -91,12 +97,16 @@ public abstract class MRTask extends RunningTaskContext {
   protected OutputCommitter committer;
 
   // Current counters
-  transient TezCounters counters = new TezCounters();
+  transient TezCounters counters;
   protected GcTimeUpdater gcUpdater;
   private ResourceCalculatorProcessTree pTree;
   private long initCpuCumulativeTime = 0;
-  protected TezEngineTaskContext tezEngineTaskContext;
-  protected TezTaskAttemptID taskAttemptId;
+  protected TezProcessorContext processorContext;
+  protected TaskAttemptID taskAttemptId;
+  protected Progress progress = new Progress();
+  protected SecretKey jobTokenSecret;
+
+  boolean isMap;
 
   /* flag to track whether task is done */
   AtomicBoolean taskDone = new AtomicBoolean(false);
@@ -109,12 +119,8 @@ public abstract class MRTask extends RunningTaskContext {
     NUMBER_FORMAT.setGroupingUsed(false);
   }
 
-  private final static int MAX_RETRIES = 10;
-
-  /** The number of milliseconds between progress reports. */
-  public static final int PROGRESS_INTERVAL = 3000;
-
-  private MRTaskReporter mrReporter;
+  protected MRTaskReporter mrReporter;
+  protected boolean useNewApi;
 
   /**
    * A Map where Key-> URIScheme and value->FileSystemStatisticUpdater
@@ -122,41 +128,202 @@ public abstract class MRTask extends RunningTaskContext {
   private Map<String, FileSystemStatisticUpdater> statisticUpdaters =
      new HashMap<String, FileSystemStatisticUpdater>();
 
-  public MRTask(TezEngineTaskContext context) {
-    tezEngineTaskContext = context;
-    this.taskAttemptId = context.getTaskAttemptId();
-    // TODO TEZAM4 Figure out initialization / run sequence of Input, Process,
-    // Output. Phase is MR specific.
-    status =
-        new MRTaskStatus(
-            taskAttemptId,
-            counters,
-            (taskAttemptId.getTaskID().getVertexID().getId() == 0 ?
-                Phase.MAP : Phase.SHUFFLE)
-        );
-    gcUpdater = new GcTimeUpdater(counters);
+  public MRTask(boolean isMap) {
+    this.isMap = isMap;
   }
 
-  public void initialize(Configuration conf, Master master) throws IOException,
+  // TODO how to update progress
+  public void initialize(TezProcessorContext context) throws IOException,
   InterruptedException {
 
+    DeprecatedKeys.init();
+
+    processorContext = context;
+    counters = context.getCounters();
+    this.taskAttemptId = new TaskAttemptID(
+        new TaskID(
+            Long.toString(context.getApplicationId().getClusterTimestamp()),
+            context.getApplicationId().getId(),
+            (isMap ? TaskType.MAP : TaskType.REDUCE),
+            context.getTaskIndex()),
+          context.getTaskAttemptNumber());
+    // TODO TEZAM4 Figure out initialization / run sequence of Input, Process,
+    // Output. Phase is MR specific.
+    gcUpdater = new GcTimeUpdater(counters);
+
+    byte[] userPayload = context.getUserPayload();
+    Configuration conf = TezUtils.createConfFromUserPayload(userPayload);
     if (conf instanceof JobConf) {
       this.jobConf = (JobConf)conf;
     } else {
       this.jobConf = new JobConf(conf);
     }
-    reporter =
-        new TezTaskReporterImpl(this, (TezTaskUmbilicalProtocol)master);
-    ((TezTaskReporterImpl)reporter).startCommunicationThread();
-
     jobConf.set(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID,
         taskAttemptId.toString());
+    jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
+        context.getDAGAttemptNumber());
 
     initResourceCalculatorPlugin();
 
     LOG.info("MRTask.inited: taskAttemptId = " + taskAttemptId.toString());
+
+    // TODO Post MRR
+    // A single file per vertex will likely be a better solution. Does not
+    // require translation - client can take care of this. Will work independent
+    // of whether the configuration is for intermediate tasks or not. Has the
+    // overhead of localizing multiple files per job - i.e. the client would
+    // need to write these files to hdfs, add them as local resources per
+    // vertex. A solution like this may be more practical once it's possible to
+    // submit configuration parameters to the AM and effectively tasks via RPC.
+
+    jobConf.set(MRJobConfig.VERTEX_NAME, processorContext.getTaskVertexName());
+
+    if (LOG.isDebugEnabled() && userPayload != null) {
+      Iterator<Entry<String, String>> iter = jobConf.iterator();
+      String taskIdStr = taskAttemptId.getTaskID().toString();
+      while (iter.hasNext()) {
+        Entry<String, String> confEntry = iter.next();
+        LOG.debug("TaskConf Entry"
+            + ", taskId=" + taskIdStr
+            + ", key=" + confEntry.getKey()
+            + ", value=" + confEntry.getValue());
+      }
+    }
+
+    configureMRTask();
   }
 
+  private void configureMRTask()
+      throws IOException, InterruptedException {
+
+    Credentials credentials = UserGroupInformation.getCurrentUser()
+        .getCredentials();
+    jobConf.setCredentials(credentials);
+    // TODO Can this be avoided all together. Have the MRTezOutputCommitter use
+    // the Tez parameter.
+    // TODO This could be fetched from the env if YARN is setting it for all
+    // Containers.
+    // Set it in conf, so as to be able to be used the the OutputCommitter.
+
+    jobConf.setClass(MRConfig.TASK_LOCAL_OUTPUT_CLASS, YarnOutputFiles.class,
+        MapOutputFile.class); // MR
+
+    // Not needed. This is probably being set via the source/consumer meta
+    Token<JobTokenIdentifier> jobToken = TokenCache.getJobToken(credentials);
+    if (jobToken != null) {
+      // Will MR ever run without a job token.
+      SecretKey sk = JobTokenSecretManager.createSecretKey(jobToken
+          .getPassword());
+      this.jobTokenSecret = sk;
+    } else {
+      LOG.warn("No job token set");
+    }
+
+    configureLocalDirs();
+
+    if (jobConf.get(TezJobConfig.DAG_CREDENTIALS_BINARY) != null) {
+      jobConf.set(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY,
+          jobConf.get(TezJobConfig.DAG_CREDENTIALS_BINARY));
+    }
+
+    // Set up the DistributedCache related configs
+    setupDistributedCacheConfig(jobConf);
+  }
+
+  private void configureLocalDirs() throws IOException {
+    // TODO NEWTEZ Is most of this functionality required ?
+    jobConf.setStrings(TezJobConfig.LOCAL_DIRS, processorContext.getWorkDirs());
+    jobConf.set(TezJobConfig.TASK_LOCAL_RESOURCE_DIR, System.getenv(Environment.PWD.name()));
+
+    jobConf.setStrings(MRConfig.LOCAL_DIR, processorContext.getWorkDirs());
+
+    LocalDirAllocator lDirAlloc = new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+    Path workDir = null;
+    // First, try to find the JOB_LOCAL_DIR on this host.
+    try {
+      workDir = lDirAlloc.getLocalPathToRead("work", jobConf);
+    } catch (DiskErrorException e) {
+      // DiskErrorException means dir not found. If not found, it will
+      // be created below.
+    }
+    if (workDir == null) {
+      // JOB_LOCAL_DIR doesn't exist on this host -- Create it.
+      workDir = lDirAlloc.getLocalPathForWrite("work", jobConf);
+      FileSystem lfs = FileSystem.getLocal(jobConf).getRaw();
+      boolean madeDir = false;
+      try {
+        madeDir = lfs.mkdirs(workDir);
+      } catch (FileAlreadyExistsException e) {
+        // Since all tasks will be running in their own JVM, the race condition
+        // exists where multiple tasks could be trying to create this directory
+        // at the same time. If this task loses the race, it's okay because
+        // the directory already exists.
+        madeDir = true;
+        workDir = lDirAlloc.getLocalPathToRead("work", jobConf);
+      }
+      if (!madeDir) {
+          throw new IOException("Mkdirs failed to create "
+              + workDir.toString());
+      }
+    }
+    // TODO NEWTEZ Is this required ?
+    jobConf.set(TezJobConfig.JOB_LOCAL_DIR, workDir.toString());
+    jobConf.set(MRJobConfig.JOB_LOCAL_DIR, workDir.toString());
+  }
+
+  /**
+   * Set up the DistributedCache related configs to make
+   * {@link DistributedCache#getLocalCacheFiles(Configuration)} and
+   * {@link DistributedCache#getLocalCacheArchives(Configuration)} working.
+   *
+   * @param job
+   * @throws IOException
+   */
+  private static void setupDistributedCacheConfig(final JobConf job)
+      throws IOException {
+
+    String localWorkDir = (job.get(TezJobConfig.TASK_LOCAL_RESOURCE_DIR));
+    // ^ ^ all symlinks are created in the current work-dir
+
+    // Update the configuration object with localized archives.
+    URI[] cacheArchives = DistributedCache.getCacheArchives(job);
+    if (cacheArchives != null) {
+      List<String> localArchives = new ArrayList<String>();
+      for (int i = 0; i < cacheArchives.length; ++i) {
+        URI u = cacheArchives[i];
+        Path p = new Path(u);
+        Path name = new Path((null == u.getFragment()) ? p.getName()
+            : u.getFragment());
+        String linkName = name.toUri().getPath();
+        localArchives.add(new Path(localWorkDir, linkName).toUri().getPath());
+      }
+      if (!localArchives.isEmpty()) {
+        job.set(MRJobConfig.CACHE_LOCALARCHIVES, StringUtils
+            .arrayToString(localArchives.toArray(new String[localArchives
+                .size()])));
+      }
+    }
+
+    // Update the configuration object with localized files.
+    URI[] cacheFiles = DistributedCache.getCacheFiles(job);
+    if (cacheFiles != null) {
+      List<String> localFiles = new ArrayList<String>();
+      for (int i = 0; i < cacheFiles.length; ++i) {
+        URI u = cacheFiles[i];
+        Path p = new Path(u);
+        Path name = new Path((null == u.getFragment()) ? p.getName()
+            : u.getFragment());
+        String linkName = name.toUri().getPath();
+        localFiles.add(new Path(localWorkDir, linkName).toUri().getPath());
+      }
+      if (!localFiles.isEmpty()) {
+        job.set(MRJobConfig.CACHE_LOCALFILES, StringUtils
+            .arrayToString(localFiles.toArray(new String[localFiles.size()])));
+      }
+    }
+  }
+
+
   private void initResourceCalculatorPlugin() {
     Class<? extends ResourceCalculatorProcessTree> clazz =
         this.jobConf.getClass(MRConfig.RESOURCE_CALCULATOR_PROCESS_TREE,
@@ -170,85 +337,28 @@ public abstract class MRTask extends RunningTaskContext {
     }
   }
 
-  public TezTaskUmbilicalProtocol getUmbilical() {
-    return ((TezTaskReporterImpl)reporter).getUmbilical();
+  public TezProcessorContext getUmbilical() {
+    return this.processorContext;
   }
 
-  public void initTask(JobConf job, TezDAGID dagId,
-      MRTaskReporter mrReporter,
-      boolean useNewApi) throws IOException,
+  public void initTask() throws IOException,
                                 InterruptedException {
-    this.jobConf = job;
-    this.jobContext = new JobContextImpl(job, dagId, mrReporter);
+    this.mrReporter = new MRTaskReporter(processorContext);
+    this.useNewApi = jobConf.getUseNewMapper();
+    TezDAGID dagId = IDConverter.fromMRTaskAttemptId(taskAttemptId).getTaskID()
+        .getVertexID().getDAGId();
+
+    this.jobContext = new JobContextImpl(jobConf, dagId, mrReporter);
     this.taskAttemptContext =
-        new TaskAttemptContextImpl(job, taskAttemptId, mrReporter);
-    this.mrReporter = mrReporter;
+        new TaskAttemptContextImpl(jobConf, taskAttemptId, mrReporter);
 
     if (getState() == State.UNASSIGNED) {
       setState(State.RUNNING);
     }
 
-    boolean useCombiner = false;
-    combineProcessor = null;
-    if (useNewApi) {
-      try {
-        useCombiner = (taskAttemptContext.getCombinerClass() != null);
-      } catch (ClassNotFoundException e) {
-        throw new IOException("Could not find combiner class", e);
-      }
-    } else {
-      useCombiner = (job.getCombinerClass() != null);
-    }
-    if (useCombiner) {
-      combineProcessor = new MRCombiner(this);
-      combineProcessor.initialize(job, getTaskReporter());
-    } else {
-    }
-
     localizeConfiguration(jobConf);
   }
 
-  public void initPartitioner(JobConf job) throws IOException,
-      InterruptedException {
-    partitioner = new MRPartitioner(this);
-    ((MRPartitioner) partitioner).initialize(job, getTaskReporter());
-  }
-
-  public void initCommitter(JobConf job, boolean useNewApi,
-      boolean useNullCommitter) throws IOException, InterruptedException {
-    if (useNullCommitter) {
-      setCommitter(new TezNullOutputCommitter());
-      return;
-    }
-    if (useNewApi) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("using new api for output committer");
-      }
-      OutputFormat<?, ?> outputFormat = null;
-      try {
-        outputFormat = ReflectionUtils.newInstance(
-            taskAttemptContext.getOutputFormatClass(), job);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException("Unknown OutputFormat", cnfe);
-      }
-      setCommitter(outputFormat.getOutputCommitter(taskAttemptContext));
-    } else {
-      setCommitter(job.getOutputCommitter());
-    }
-
-    Path outputPath = FileOutputFormat.getOutputPath(job);
-    if (outputPath != null) {
-      if ((getCommitter() instanceof FileOutputCommitter)) {
-        FileOutputFormat.setWorkOutputPath(job,
-            ((FileOutputCommitter) getCommitter())
-                .getTaskAttemptPath(taskAttemptContext));
-      } else {
-        FileOutputFormat.setWorkOutputPath(job, outputPath);
-      }
-    }
-    getCommitter().setupTask(taskAttemptContext);
-  }
-
   public MRTaskReporter getMRReporter() {
     return mrReporter;
   }
@@ -273,23 +383,6 @@ public abstract class MRTask extends RunningTaskContext {
 
   public TezCounters getCounters() { return counters; }
 
-  /**
-   * Return current phase of the task.
-   * needs to be synchronized as communication thread sends the phase every second
-   * @return the curent phase of the task
-   */
-  public synchronized TezTaskStatus.Phase getPhase(){
-    return status.getPhase();
-  }
-
-  /**
-   * Set current phase of the task.
-   * @param phase task phase
-   */
-  protected synchronized void setPhase(TezTaskStatus.Phase phase){
-    status.setPhase(phase);
-  }
-
   public void setConf(JobConf jobConf) {
     this.jobConf = jobConf;
   }
@@ -327,33 +420,6 @@ public abstract class MRTask extends RunningTaskContext {
 
   public void waitBeforeCompletion(MRTaskReporter reporter) throws IOException,
       InterruptedException {
-    TezTaskUmbilicalProtocol umbilical = getUmbilical();
-    int retries = MAX_RETRIES;
-    boolean readyToProceed = false;
-    while (!readyToProceed) {
-      try {
-        ProceedToCompletionResponse response =
-            umbilical.proceedToCompletion(taskAttemptId);
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Got readyToProceed: " + response);
-        }
-        if (response.shouldDie()) {
-          throw new IOException("Task was asked to die by the AM");
-          // TODO EVENTUALLY Figure out a good way for a graceful exit, instead
-          // of an exit via an Exception. This isn' necessarily an error.
-        }
-        readyToProceed = response.readyToProceed();
-      } catch (IOException ie) {
-        LOG.warn("Failure waiting for exit signal: " +
-            StringUtils.stringifyException(ie));
-        if (--retries == 0) {
-          throw ie;
-        }
-      }
-      synchronized(this) {
-        wait(1000l); // Check if ready to exit every second.
-      }
-    }
   }
 
   public void outputReady(MRTaskReporter reporter, OutputContext outputContext)
@@ -362,61 +428,31 @@ public abstract class MRTask extends RunningTaskContext {
     LOG.info("Task: " + taskAttemptId + " reporting outputReady");
     updateCounters();
     statusUpdate();
-
-    TezTaskUmbilicalProtocol umbilical = getUmbilical();
-    int retries = MAX_RETRIES;
-    while (true) {
-      try {
-        umbilical.outputReady(taskAttemptId, outputContext);
-        LOG.info("Task '" + taskAttemptId + "' reported outputReady.");
-        return;
-      } catch (IOException ie) {
-        LOG.warn("Failure signalling outputReady: " +
-            StringUtils.stringifyException(ie));
-        if (--retries == 0) {
-          throw ie;
-        }
-      }
-    }
   }
 
-  public void done(
-      OutputContext outputContext,
-      MRTaskReporter reporter
-      ) throws IOException, InterruptedException {
+  public void done(LogicalOutput output) throws IOException, InterruptedException {
     updateCounters();
-    if (outputContext != null) {
-      LOG.info("Task: "
-          + taskAttemptId
-          + " is done."
-          + " And is in the process of sending output-context with shuffle port: "
-          + outputContext.getShufflePort());
-      outputReady(reporter, outputContext);
-      waitBeforeCompletion(reporter);
-    }
 
     LOG.info("Task:" + taskAttemptId + " is done."
         + " And is in the process of committing");
-    TezTaskUmbilicalProtocol umbilical = getUmbilical();
+    // TODO change this to use the new context
     // TODO TEZ Interaciton between Commit and OutputReady. Merge ?
-    if (isCommitRequired()) {
-      //wait for commit approval and commit
-      // TODO EVENTUALLY - Commit is not required for map tasks. skip a couple of RPCs before exiting.
-      commit(umbilical, reporter, committer);
+    if (output instanceof SimpleOutput) {
+      SimpleOutput sOut = (SimpleOutput)output;
+      if (sOut.isCommitRequired()) {
+        //wait for commit approval and commit
+        // TODO EVENTUALLY - Commit is not required for map tasks.
+        // skip a couple of RPCs before exiting.
+        commit(sOut);
+      }
     }
     taskDone.set(true);
-    reporter.stopCommunicationThread();
     // Make sure we send at least one set of counter increments. It's
     // ok to call updateCounters() in this thread after comm thread stopped.
     updateCounters();
     sendLastUpdate();
     //signal the tasktracker that we are done
-    sendDone(umbilical);
-  }
-
-
-  private boolean isCommitRequired() throws IOException {
-    return committer.needsTaskCommit(taskAttemptContext);
+    //sendDone(umbilical);
   }
 
   /**
@@ -425,27 +461,7 @@ public abstract class MRTask extends RunningTaskContext {
    * @throws IOException
    */
   public void statusUpdate() throws IOException, InterruptedException {
-    int retries = MAX_RETRIES;
-    /* broken code due to engine re-factor
-    while (true) {
-      try {
-        if (!getUmbilical().statusUpdate(taskAttemptId, status)) {
-          LOG.warn("Parent died.  Exiting " + taskAttemptId);
-          System.exit(66);
-        }
-        status.clearStatus();
-        return;
-      } catch (InterruptedException ie) {
-        Thread.currentThread().interrupt(); // interrupt ourself
-      } catch (IOException ie) {
-        LOG.warn("Failure sending status update: " +
-            StringUtils.stringifyException(ie));
-        if (--retries == 0) {
-          throw ie;
-        }
-      }
-    }
-     */
+    // TODO call progress update here if not being called within Map/Reduce
   }
 
   /**
@@ -453,39 +469,27 @@ public abstract class MRTask extends RunningTaskContext {
    */
   private void sendLastUpdate()
       throws IOException, InterruptedException {
-    status.setOutputSize(-1l);
-    // send a final status report
-    status.statusUpdate(
-        getProgress().get(), getProgress().toString(), counters);
     statusUpdate();
   }
 
-  private void commit(TezTaskUmbilicalProtocol umbilical,
-      MRTaskReporter reporter,
-      org.apache.hadoop.mapreduce.OutputCommitter committer
-      ) throws IOException {
-    int retries = MAX_RETRIES;
+  private void commit(SimpleOutput output) throws IOException {
+    int retries = 3;
     while (true) {
+      // This will loop till the AM asks for the task to be killed. As
+      // against, the AM sending a signal to the task to kill itself
+      // gracefully.
       try {
-        while (!umbilical.canCommit(taskAttemptId)) {
-          // This will loop till the AM asks for the task to be killed. As
-          // against, the AM sending a signal to the task to kill itself
-          // gracefully.
-          try {
-            Thread.sleep(1000);
-          } catch(InterruptedException ie) {
-            //ignore
-          }
-          reporter.setProgressFlag();
+        if (processorContext.canCommit()) {
+          break;
         }
-        break;
+        Thread.sleep(1000);
+      } catch(InterruptedException ie) {
+        //ignore
       } catch (IOException ie) {
-        LOG.warn("Failure asking whether task can commit: " +
-            StringUtils.stringifyException(ie));
+        LOG.warn("Failure sending canCommit: "
+            + StringUtils.stringifyException(ie));
         if (--retries == 0) {
-          //if it couldn't query successfully then delete the output
-          discardOutput(taskAttemptContext);
-          System.exit(68);
+          throw ie;
         }
       }
     }
@@ -493,21 +497,21 @@ public abstract class MRTask extends RunningTaskContext {
     // task can Commit now
     try {
       LOG.info("Task " + taskAttemptId + " is allowed to commit now");
-      committer.commitTask(taskAttemptContext);
+      output.commit();
       return;
     } catch (IOException iee) {
       LOG.warn("Failure committing: " +
           StringUtils.stringifyException(iee));
       //if it couldn't commit a successfully then delete the output
-      discardOutput(taskAttemptContext);
+      discardOutput(output);
       throw iee;
     }
   }
 
   private
-  void discardOutput(TaskAttemptContext taskContext) {
+  void discardOutput(SimpleOutput output) {
     try {
-      committer.abortTask(taskContext);
+      output.abort();
     } catch (IOException ioe)  {
       LOG.warn("Failure cleaning up: " +
                StringUtils.stringifyException(ioe));
@@ -515,25 +519,6 @@ public abstract class MRTask extends RunningTaskContext {
   }
 
 
-  private void sendDone(TezTaskUmbilicalProtocol umbilical) throws IOException {
-    int retries = MAX_RETRIES;
-    /* broken code due to engine re-factor
-    while (true) {
-      try {
-        umbilical.done(taskAttemptId);
-        LOG.info("Task '" + taskAttemptId + "' done.");
-        return;
-      } catch (IOException ie) {
-        LOG.warn("Failure signalling completion: " +
-                 StringUtils.stringifyException(ie));
-        if (--retries == 0) {
-          throw ie;
-        }
-      }
-    }
-    */
-  }
-
   public void updateCounters() {
     // TODO Auto-generated method stub
     // TODO TEZAM Implement.
@@ -617,7 +602,7 @@ public abstract class MRTask extends RunningTaskContext {
   createReduceContext(org.apache.hadoop.mapreduce.Reducer
                         <INKEY,INVALUE,OUTKEY,OUTVALUE> reducer,
                       Configuration job,
-                      TezTaskAttemptID taskId,
+                      TaskAttemptID taskId,
                       final TezRawKeyValueIterator rIter,
                       org.apache.hadoop.mapreduce.Counter inputKeyCounter,
                       org.apache.hadoop.mapreduce.Counter inputValueCounter,
@@ -659,7 +644,7 @@ public abstract class MRTask extends RunningTaskContext {
     reduceContext =
       new ReduceContextImpl<INKEY, INVALUE, OUTKEY, OUTVALUE>(
           job,
-          IDConverter.toMRTaskAttemptId(taskId),
+          taskId,
           r,
           inputKeyCounter,
           inputValueCounter,
@@ -682,11 +667,9 @@ public abstract class MRTask extends RunningTaskContext {
     return reducerContext;
   }
 
-  public void taskCleanup(TezTaskUmbilicalProtocol umbilical)
+  public void taskCleanup()
       throws IOException, InterruptedException {
     // set phase for this task
-    setPhase(TezTaskStatus.Phase.CLEANUP);
-    getProgress().setStatus("cleanup");
     statusUpdate();
     LOG.info("Runnning cleanup for the task");
     // do the cleanup
@@ -695,106 +678,30 @@ public abstract class MRTask extends RunningTaskContext {
 
   public void localizeConfiguration(JobConf jobConf)
       throws IOException, InterruptedException {
-    jobConf.set(JobContext.TASK_ID, IDConverter
-        .toMRTaskAttemptId(taskAttemptId).toString());
-    jobConf.set(JobContext.TASK_ATTEMPT_ID,
-        IDConverter.toMRTaskAttemptId(taskAttemptId).toString());
+    jobConf.set(JobContext.TASK_ID, taskAttemptId.getTaskID().toString());
+    jobConf.set(JobContext.TASK_ATTEMPT_ID, taskAttemptId.toString());
     jobConf.setInt(JobContext.TASK_PARTITION,
         taskAttemptId.getTaskID().getId());
-    jobConf.set(JobContext.ID, taskAttemptId.getTaskID().getVertexID().getDAGId().toString());
+    jobConf.set(JobContext.ID, taskAttemptId.getJobID().toString());
   }
 
   public abstract TezCounter getOutputRecordsCounter();
 
   public abstract TezCounter getInputRecordsCounter();
 
-  public TezCounter getFileOutputBytesCounter() {
-    return reporter.getCounter(FileOutputFormatCounter.BYTES_WRITTEN);
-  }
-
   public org.apache.hadoop.mapreduce.TaskAttemptContext getTaskAttemptContext() {
     return taskAttemptContext;
   }
 
-  public TezCounter getFileInputBytesCounter() {
-    return reporter.getCounter(FileInputFormatCounter.BYTES_READ);
-  }
-
   public JobContext getJobContext() {
     return jobContext;
   }
 
-  public TezTaskAttemptID getTaskAttemptId() {
+  public TaskAttemptID getTaskAttemptId() {
     return taskAttemptId;
   }
 
-  public TezEngineTaskContext getTezEngineTaskContext() {
-    return tezEngineTaskContext;
-  }
-
-  protected FutureTask<Void> initInputAsync(Input input) {
-    FutureTask<Void> initInputFuture = new FutureTask<Void>(
-        new InitInputCallable(input));
-    new Thread(initInputFuture, "InitInputThread").start();
-    return initInputFuture;
-  }
-
-  protected FutureTask<Void> initOutputAsync(Output output) {
-    FutureTask<Void> initOutputFuture = new FutureTask<Void>(
-        new InitOutputCallable(output));
-    new Thread(initOutputFuture, "InitOutputThread").start();
-    return initOutputFuture;
-  }
-
-  protected class InitInputCallable implements Callable<Void> {
-    Input input;
-    InitInputCallable(Input input) {
-      this.input = input;
-    }
-    @Override
-    public Void call() throws IOException, InterruptedException {
-      input.initialize(jobConf, getTaskReporter());
-      LOG.info("Input initialized");
-      return null;
-    }
-  }
-
-  protected class InitOutputCallable implements Callable<Void> {
-    Output output;
-    InitOutputCallable(Output output) {
-      this.output = output;
-    }
-    @Override
-    public Void call() throws IOException, InterruptedException {
-      output.initialize(jobConf, getTaskReporter());
-      LOG.info("Output initialized");
-      return null;
-    }
-  }
-
-  private void waitForIOInitialization(FutureTask<Void> future)
-      throws InterruptedException, IOException {
-    try {
-      future.get();
-    } catch (ExecutionException e) {
-      if (e.getCause() instanceof InterruptedException) {
-        throw (InterruptedException) e.getCause();
-      } else if (e.getCause() instanceof IOException) {
-        throw (IOException) e.getCause();
-      } else {
-        throw new RuntimeException("UnknownException from I/O initialization",
-            e.getCause());
-      }
-    }
-  }
-
-  protected void waitForInputInitialization(FutureTask<Void> future)
-      throws InterruptedException, IOException {
-    waitForIOInitialization(future);
-  }
-
-  protected void waitForOutputInitialization(FutureTask<Void> future)
-      throws InterruptedException, IOException {
-    waitForIOInitialization(future);
+  public TezProcessorContext getTezEngineTaskContext() {
+    return processorContext;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
----------------------------------------------------------------------
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 1639bf2..d32b4c0 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
@@ -23,89 +23,100 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.Counters;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.tez.common.TezTaskReporter;
 import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.engine.newapi.TezInputContext;
+import org.apache.tez.engine.newapi.TezOutputContext;
+import org.apache.tez.engine.newapi.TezProcessorContext;
+import org.apache.tez.engine.newapi.TezTaskContext;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
+import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
 
 @InterfaceAudience.Private
-@InterfaceStability.Unstable 
-public class MRTaskReporter 
+@InterfaceStability.Unstable
+public class MRTaskReporter
     extends org.apache.hadoop.mapreduce.StatusReporter
     implements Reporter {
 
-  private final TezTaskReporterImpl reporter;
-  
+  private final TezTaskContext context;
+  private final boolean isProcessorContext;
+  private final Reporter reporter;
+
   private InputSplit split = null;
 
-  public MRTaskReporter(TezTaskReporter reporter) {
-    this.reporter =  (TezTaskReporterImpl)reporter;
+  public MRTaskReporter(TezProcessorContext context) {
+    this.context = context;
+    this.reporter = new MRReporter(context);
+    this.isProcessorContext = true;
   }
 
-  // getters and setters for flag
-  void setProgressFlag() {
-    reporter.setProgressFlag();
+  public MRTaskReporter(TezOutputContext context) {
+    this.context = context;
+    this.reporter = new MRReporter(context);
+    this.isProcessorContext = false;
+  }
+  
+  public MRTaskReporter(TezInputContext context) {
+    this.context= context;
+    this.reporter = new MRReporter(context);
+    this.isProcessorContext = false;
   }
-  boolean resetProgressFlag() {
-    return reporter.resetProgressFlag();
+
+  public void setProgress(float progress) {
+    if (isProcessorContext) {
+      ((TezProcessorContext)context).setProgress(progress);
+    } else {
+      // TODO FIXME NEWTEZ - will simpleoutput's reporter use this api?
+    }
   }
+
   public void setStatus(String status) {
     reporter.setStatus(status);
   }
-  public void setProgress(float progress) {
-    reporter.setProgress(progress);
-  }
-  
+
   public float getProgress() {
     return reporter.getProgress();
   };
-  
+
   public void progress() {
     reporter.progress();
   }
-  
+
   public Counters.Counter getCounter(String group, String name) {
-    TezCounter counter = reporter.getCounter(group, name);
+    TezCounter counter = context.getCounters().findCounter(group, name);
     MRCounters.MRCounter mrCounter = null;
     if (counter != null) {
       mrCounter = new MRCounters.MRCounter(counter);
     }
     return mrCounter;
   }
-  
+
   public Counters.Counter getCounter(Enum<?> name) {
-    TezCounter counter = reporter.getCounter(name);
+    TezCounter counter = context.getCounters().findCounter(name);
     MRCounters.MRCounter mrCounter = null;
     if (counter != null) {
       mrCounter = new MRCounters.MRCounter(counter);
     }
     return mrCounter;
   }
-  
+
   public void incrCounter(Enum<?> key, long amount) {
     reporter.incrCounter(key, amount);
   }
-  
+
   public void incrCounter(String group, String counter, long amount) {
     reporter.incrCounter(group, counter, amount);
   }
-  
+
   public void setInputSplit(InputSplit split) {
     this.split = split;
   }
-  
+
   public InputSplit getInputSplit() throws UnsupportedOperationException {
     if (split == null) {
       throw new UnsupportedOperationException("Input only available on map");
     } else {
       return split;
     }
-  }  
-  
-  public void startCommunicationThread() {
-    reporter.startCommunicationThread();
-  }
-  
-  public void stopCommunicationThread() throws InterruptedException {
-    reporter.stopCommunicationThread();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/TezTaskReporterImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/TezTaskReporterImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/TezTaskReporterImpl.java
deleted file mode 100644
index 6323fc9..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/TezTaskReporterImpl.java
+++ /dev/null
@@ -1,279 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.mapreduce.processor;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.ipc.ProtocolSignature;
-import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable 
-class TezTaskReporterImpl 
-    implements org.apache.tez.common.TezTaskReporter, Runnable {
-
-  private static final Log LOG = LogFactory.getLog(TezTaskReporterImpl.class);
-  
-  private final MRTask mrTask;
-  private final TezTaskUmbilicalProtocol umbilical;
-  private final Progress taskProgress;
-  
-  private Thread pingThread = null;
-  private boolean done = true;
-  private Object lock = new Object();
-
-  /**
-   * flag that indicates whether progress update needs to be sent to parent.
-   * If true, it has been set. If false, it has been reset. 
-   * Using AtomicBoolean since we need an atomic read & reset method. 
-   */  
-  private AtomicBoolean progressFlag = new AtomicBoolean(false);
-  
-  TezTaskReporterImpl(MRTask mrTask, TezTaskUmbilicalProtocol umbilical) {
-    this.mrTask = mrTask;
-    this.umbilical = umbilical;
-    this.taskProgress = mrTask.getProgress();
-  }
-
-  // getters and setters for flag
-  void setProgressFlag() {
-    progressFlag.set(true);
-  }
-  
-  boolean resetProgressFlag() {
-    return progressFlag.getAndSet(false);
-  }
-  
-  public void setStatus(String status) {
-    // FIXME - BADLY
-    if (true) {
-      return;
-    }
-    taskProgress.setStatus(
-        MRTask.normalizeStatus(status, this.mrTask.jobConf));
-    // indicate that progress update needs to be sent
-    setProgressFlag();
-  }
-  
-  public void setProgress(float progress) {
-    // set current phase progress.
-    // This method assumes that task has phases.
-    taskProgress.phase().set(progress);
-    // indicate that progress update needs to be sent
-    setProgressFlag();
-  }
-  
-  public float getProgress() {
-    return taskProgress.getProgress();
-  };
-  
-  public void progress() {
-    // indicate that progress update needs to be sent
-    setProgressFlag();
-  }
-  
-  public TezCounter getCounter(String group, String name) {
-    return this.mrTask.counters == null ? 
-        null : 
-        this.mrTask.counters.findCounter(group, name);
-  }
-  
-  public TezCounter getCounter(Enum<?> name) {
-    return this.mrTask.counters == null ? 
-        null : 
-        this.mrTask.counters.findCounter(name);
-  }
-  
-  public void incrCounter(Enum<?> key, long amount) {
-    if (this.mrTask.counters != null) {
-      this.mrTask.counters.findCounter(key).increment(amount);
-    }
-    setProgressFlag();
-  }
-  
-  public void incrCounter(String group, String counter, long amount) {
-    if (this.mrTask.counters != null) {
-      this.mrTask.counters.findCounter(group, counter).increment(amount);
-    }
-    setProgressFlag();
-  }
-  
-  /** 
-   * The communication thread handles communication with the parent (Task Tracker). 
-   * It sends progress updates if progress has been made or if the task needs to 
-   * let the parent know that it's alive. It also pings the parent to see if it's alive. 
-   */
-  public void run() {
-    final int MAX_RETRIES = 3;
-    int remainingRetries = MAX_RETRIES;
-    // get current flag value and reset it as well
-    boolean sendProgress = resetProgressFlag();
-    while (!this.mrTask.taskDone.get()) {
-      synchronized (lock) {
-        done = false;
-      }
-      try {
-        boolean taskFound = true; // whether TT knows about this task
-        // sleep for a bit
-        synchronized(lock) {
-          if (this.mrTask.taskDone.get()) {
-            break;
-          }
-          lock.wait(MRTask.PROGRESS_INTERVAL);
-        }
-        if (this.mrTask.taskDone.get()) {
-          break;
-        }
-
-        if (sendProgress) {
-          // we need to send progress update
-          this.mrTask.updateCounters();
-          this.mrTask.getStatus().statusUpdate(
-              taskProgress.get(),
-              taskProgress.toString(), 
-              this.mrTask.counters);
-
-          // broken code now due to tez engine changes
-          taskFound = false;
-          /*
-          taskFound = 
-              umbilical.statusUpdate(
-                  this.mrTask.getTaskAttemptId(), this.mrTask.getStatus());
-           */
-          this.mrTask.getStatus().clearStatus();
-        }
-        else {
-          // send ping 
-          taskFound = false;
-          // broken code now due to tez engine changes
-          //umbilical.ping(this.mrTask.getTaskAttemptId());
-        }
-
-        // if Task Tracker is not aware of our task ID (probably because it died and 
-        // came back up), kill ourselves
-        if (!taskFound) {
-          MRTask.LOG.warn("Parent died.  Exiting " + this.mrTask.getTaskAttemptId());
-          resetDoneFlag();
-          System.exit(66);
-        }
-
-        sendProgress = resetProgressFlag(); 
-        remainingRetries = MAX_RETRIES;
-      } 
-      catch (Throwable t) {
-        MRTask.LOG.info("Communication exception: " + StringUtils.stringifyException(t));
-        remainingRetries -=1;
-        if (remainingRetries == 0) {
-          ReflectionUtils.logThreadInfo(MRTask.LOG, "Communication exception", 0);
-          MRTask.LOG.warn("Last retry, killing " + this.mrTask.getTaskAttemptId());
-          resetDoneFlag();
-          System.exit(65);
-        }
-      }
-    }
-    //Notify that we are done with the work
-    resetDoneFlag();
-  }
-  void resetDoneFlag() {
-    synchronized (lock) {
-      done = true;
-      lock.notify();
-    }
-  }
-  public void startCommunicationThread() {
-    if (pingThread == null) {
-      pingThread = new Thread(this, "communication thread");
-      pingThread.setDaemon(true);
-      pingThread.start();
-    }
-  }
-  public void stopCommunicationThread() throws InterruptedException {
-    if (pingThread != null) {
-      // Intent of the lock is to not send an interupt in the middle of an
-      // umbilical.ping or umbilical.statusUpdate
-      synchronized(lock) {
-      //Interrupt if sleeping. Otherwise wait for the RPC call to return.
-        lock.notify(); 
-      }
-
-      synchronized (lock) { 
-        while (!done) {
-          lock.wait();
-        }
-      }
-      pingThread.interrupt();
-      pingThread.join();
-    }
-  }
-
-  @Override
-  public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-      int fromEventIdx, int maxEventsToFetch,
-      TezTaskAttemptID reduce) {
-    return umbilical.getDependentTasksCompletionEvents(
-        fromEventIdx, maxEventsToFetch, reduce);
-  }
-
-  @Override
-  public void reportFatalError(TezTaskAttemptID taskAttemptId,
-      Throwable throwable, String logMsg) {
-    LOG.fatal(logMsg);
-    Throwable tCause = throwable.getCause();
-    String cause = tCause == null 
-                   ? StringUtils.stringifyException(throwable)
-                   : StringUtils.stringifyException(tCause);
-/*
-                   try {
-      umbilical.fatalError(mrTask.getTaskAttemptId(), cause);
-    } catch (IOException ioe) {
-      LOG.fatal("Failed to contact the tasktracker", ioe);
-      System.exit(-1);
-    }
-    */
-  }
-
-  public TezTaskUmbilicalProtocol getUmbilical() {
-    return umbilical;
-  }
-
-  @Override
-  public long getProtocolVersion(String protocol, long clientVersion)
-      throws IOException {
-    // TODO TEZAM3
-    return 1;
-  }
-
-  @Override
-  public ProtocolSignature getProtocolSignature(String protocol,
-      long clientVersion, int clientMethodsHash) throws IOException {
-    return ProtocolSignature.getProtocolSignature(this, protocol,
-        clientVersion, clientMethodsHash);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
----------------------------------------------------------------------
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 67794e7..80a2337 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
@@ -18,11 +18,11 @@
 package org.apache.tez.mapreduce.processor.map;
 
 import java.io.IOException;
-import java.util.concurrent.FutureTask;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobContext;
 import org.apache.hadoop.mapred.MapRunnable;
@@ -31,162 +31,134 @@ import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.map.WrappedMapper;
-import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.common.TezTaskStatus;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
-import org.apache.tez.engine.common.sort.SortingOutput;
-import org.apache.tez.mapreduce.hadoop.IDConverter;
-import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.engine.lib.output.OnFileSortedOutput;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.KVReader;
+import org.apache.tez.engine.newapi.KVWriter;
+import org.apache.tez.engine.newapi.LogicalIOProcessor;
+import org.apache.tez.engine.newapi.LogicalInput;
+import org.apache.tez.engine.newapi.LogicalOutput;
+import org.apache.tez.engine.newapi.TezProcessorContext;
+import org.apache.tez.mapreduce.hadoop.mapreduce.MapContextImpl;
 import org.apache.tez.mapreduce.input.SimpleInput;
+import org.apache.tez.mapreduce.input.SimpleInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
 
 @SuppressWarnings({ "unchecked", "rawtypes" })
-public class MapProcessor extends MRTask implements Processor {
+public class MapProcessor extends MRTask implements LogicalIOProcessor {
 
   private static final Log LOG = LogFactory.getLog(MapProcessor.class);
 
-  private Progress mapPhase;
+  public MapProcessor(){
+    super(true);
+  }
 
-  public MapProcessor(TezEngineTaskContext context) throws IOException {
-    super(context);
+  @Override
+  public void initialize(TezProcessorContext processorContext)
+      throws IOException {
+    try {
+      super.initialize(processorContext);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
   }
-  
 
 
   @Override
-  public void initialize(Configuration conf, Master master) throws IOException,
-  InterruptedException {
-    super.initialize(conf, master);
+  public void handleEvents(List<Event> processorEvents) {
+    // TODO Auto-generated method stub
+
+  }
+
+  public void close() throws IOException {
+    // TODO Auto-generated method stub
+
   }
 
   @Override
-  public void process(
-      final Input[] ins,
-      final Output[] outs)
-          throws IOException, InterruptedException {
-
-    MRTaskReporter reporter = new MRTaskReporter(getTaskReporter());
-    boolean useNewApi = jobConf.getUseNewMapper();
-    initTask(jobConf, taskAttemptId.getTaskID().getVertexID().getDAGId(),
-        reporter, useNewApi);
-
-    if (ins.length != 1
-        || outs.length != 1) {
+  public void run(Map<String, LogicalInput> inputs,
+      Map<String, LogicalOutput> outputs) throws Exception {
+
+    LOG.info("Running map: " + processorContext.getUniqueIdentifier());
+
+    initTask();
+
+    if (inputs.size() != 1
+        || outputs.size() != 1) {
       throw new IOException("Cannot handle multiple inputs or outputs"
-          + ", inputCount=" + ins.length
-          + ", outputCount=" + outs.length);
+          + ", inputCount=" + inputs.size()
+          + ", outputCount=" + outputs.size());
     }
-    Input in = ins[0];
-    Output out = outs[0];
-    
+    LogicalInput in = inputs.values().iterator().next();
+    LogicalOutput out = outputs.values().iterator().next();
+
     // Sanity check
-    if (!(in instanceof SimpleInput)) {
-      throw new IOException("Unknown input! - " + in.getClass());
-    }
-    SimpleInput input = (SimpleInput)in;
-    input.setTask(this);
-
-    if (out instanceof SimpleOutput) {
-      initCommitter(jobConf, useNewApi, false);
-      ((SimpleOutput)out).setTask(this);
-    } else if (out instanceof SortingOutput) {
-      initCommitter(jobConf, useNewApi, true);
-      initPartitioner(jobConf);
-      ((SortingOutput)out).setTask(this);
+    if (!(in instanceof SimpleInputLegacy)) {
+      throw new IOException(new TezException(
+          "Only Simple Input supported. Input: " + in.getClass()));
     }
+    SimpleInputLegacy input = (SimpleInputLegacy)in;
 
-    // If there are no reducers then there won't be any sort. Hence the map 
-    // phase will govern the entire attempt's progress.
-    if (tezEngineTaskContext.getOutputSpecList().get(0).getNumOutputs() == 0) {
-      mapPhase = getProgress().addPhase("map");
+    KVWriter kvWriter = null;
+    if (!(out instanceof OnFileSortedOutput)) {
+      kvWriter = ((SimpleOutput)out).getWriter();
     } else {
-      // If there are reducers then the entire attempt's progress will be 
-      // split between the map phase (67%) and the sort phase (33%).
-      mapPhase = getProgress().addPhase("map");
+      kvWriter = ((OnFileSortedOutput)out).getWriter();
     }
 
-
     if (useNewApi) {
-      runNewMapper(jobConf, reporter, input, out, getTaskReporter());
+      runNewMapper(jobConf, mrReporter, input, kvWriter);
     } else {
-      runOldMapper(jobConf, reporter, input, out, getTaskReporter());
+      runOldMapper(jobConf, mrReporter, input, kvWriter);
     }
 
-    done(out.getOutputContext(), reporter);
+    done(out);
   }
 
-  public void close() throws IOException, InterruptedException {
-    // TODO Auto-generated method stub
-
-  }
-  
   void runOldMapper(
       final JobConf job,
       final MRTaskReporter reporter,
-      final SimpleInput input,
-      final Output output,
-      final Master master
+      final SimpleInputLegacy input,
+      final KVWriter output
       ) throws IOException, InterruptedException {
-    
+
     // Initialize input in-line since it sets parameters which may be used by the processor.
     // Done only for SimpleInput.
-    input.initialize(job, master);
-    FutureTask<Void> initOutputFuture = initOutputAsync(output);
-    
+    // TODO use new method in SimpleInput to get required info
+    //input.initialize(job, master);
+
     RecordReader in = new OldRecordReader(input);
-        
-    int numReduceTasks = tezEngineTaskContext.getOutputSpecList().get(0)
-        .getNumOutputs();
-    LOG.info("numReduceTasks: " + numReduceTasks);
 
     OutputCollector collector = new OldOutputCollector(output);
 
     MapRunnable runner =
         (MapRunnable)ReflectionUtils.newInstance(job.getMapRunnerClass(), job);
 
-    // Wait for output to be initialized before starting processing.
-    LOG.info("Waiting on output initialization");
-    waitForOutputInitialization(initOutputFuture);
-
-    try {
-      runner.run(in, collector, (Reporter)reporter);
-      mapPhase.complete();
-      // start the sort phase only if there are reducers
-      if (numReduceTasks > 0) {
-        setPhase(TezTaskStatus.Phase.SORT);
-      }
-      this.statusUpdate();
-    } finally {
-      //close
-      in.close();                               // close input
-      output.close();
-    }
+    runner.run(in, collector, (Reporter)reporter);
+    // start the sort phase only if there are reducers
+    this.statusUpdate();
   }
 
   private void runNewMapper(final JobConf job,
       MRTaskReporter reporter,
-      final SimpleInput in,
-      Output out,
-      final Master master
+      final SimpleInputLegacy in,
+      KVWriter out
       ) throws IOException, InterruptedException {
 
     // Initialize input in-line since it sets parameters which may be used by the processor.
     // Done only for SimpleInput.
-    in.initialize(job, master);
-    FutureTask<Void> initOutputFuture = initOutputAsync(out);
-    
+    // TODO use new method in SimpleInput to get required info
+    //in.initialize(job, master);
+
     // make a task context so we can get the classes
     org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
-        new TaskAttemptContextImpl(job, taskAttemptId, reporter);
+        getTaskAttemptContext();
 
     // make a mapper
     org.apache.hadoop.mapreduce.Mapper mapper;
@@ -197,37 +169,27 @@ public class MapProcessor extends MRTask implements Processor {
       throw new IOException(cnfe);
     }
 
-    if (!(in instanceof SimpleInput)) {
-      throw new IOException("Unknown input! - " + in.getClass());
-    }
-
     org.apache.hadoop.mapreduce.RecordReader input =
         new NewRecordReader(in);
 
-    org.apache.hadoop.mapreduce.RecordWriter output = 
+    org.apache.hadoop.mapreduce.RecordWriter output =
         new NewOutputCollector(out);
 
-    // Wait for output to be initialized before starting processing.
-    LOG.info("Waiting on output initialization");
-    waitForOutputInitialization(initOutputFuture);
-
     org.apache.hadoop.mapreduce.InputSplit split = in.getNewInputSplit();
-    
-    org.apache.hadoop.mapreduce.MapContext 
-    mapContext = 
-    new org.apache.tez.mapreduce.hadoop.mapreduce.MapContextImpl(
-        job, IDConverter.toMRTaskAttemptId(taskAttemptId), 
-        input, output, 
-        getCommitter(), 
-        reporter, split);
-
-    org.apache.hadoop.mapreduce.Mapper.Context mapperContext = 
+
+    org.apache.hadoop.mapreduce.MapContext
+    mapContext =
+    new MapContextImpl(
+        job, taskAttemptId,
+        input, output,
+        getCommitter(),
+        processorContext, split);
+
+    org.apache.hadoop.mapreduce.Mapper.Context mapperContext =
         new WrappedMapper().getMapContext(mapContext);
 
     input.initialize(split, mapperContext);
     mapper.run(mapperContext);
-    mapPhase.complete();
-    setPhase(TezTaskStatus.Phase.SORT);
     this.statusUpdate();
     input.close();
     output.close(mapperContext);
@@ -236,34 +198,36 @@ public class MapProcessor extends MRTask implements Processor {
   private static class NewRecordReader extends
       org.apache.hadoop.mapreduce.RecordReader {
     private final SimpleInput in;
+    private KVReader reader;
 
-    private NewRecordReader(SimpleInput in) {
+    private NewRecordReader(SimpleInput in) throws IOException {
       this.in = in;
+      this.reader = in.getReader();
     }
 
     @Override
     public void initialize(org.apache.hadoop.mapreduce.InputSplit split,
         TaskAttemptContext context) throws IOException,
         InterruptedException {
-      in.initializeNewRecordReader(split, context);
+      //in.initializeNewRecordReader(split, context);
     }
 
     @Override
     public boolean nextKeyValue() throws IOException,
         InterruptedException {
-      return in.hasNext();
+      return reader.next();
     }
 
     @Override
     public Object getCurrentKey() throws IOException,
         InterruptedException {
-      return in.getNextKey();
+      return reader.getCurrentKV().getKey();
     }
 
     @Override
     public Object getCurrentValue() throws IOException,
         InterruptedException {
-      return in.getNextValues().iterator().next();
+      return reader.getCurrentKV().getValues().iterator().next();
     }
 
     @Override
@@ -273,26 +237,27 @@ public class MapProcessor extends MRTask implements Processor {
 
     @Override
     public void close() throws IOException {
-      in.close();
     }
   }
 
   private static class OldRecordReader implements RecordReader {
-    private final SimpleInput simpleInput;
+    private final SimpleInputLegacy simpleInput;
 
-    private OldRecordReader(SimpleInput simpleInput) {
+    private OldRecordReader(SimpleInputLegacy simpleInput) {
       this.simpleInput = simpleInput;
     }
 
     @Override
     public boolean next(Object key, Object value) throws IOException {
-      simpleInput.setKey(key);
-      simpleInput.setValue(value);
-      try {
-        return simpleInput.hasNext();
-      } catch (InterruptedException ie) {
-        throw new IOException(ie);
-      }
+      // TODO broken
+//      simpleInput.setKey(key);
+//      simpleInput.setValue(value);
+//      try {
+//        return simpleInput.hasNext();
+//      } catch (InterruptedException ie) {
+//        throw new IOException(ie);
+//      }
+      return simpleInput.getOldRecordReader().next(key, value);
     }
 
     @Override
@@ -312,7 +277,6 @@ public class MapProcessor extends MRTask implements Processor {
 
     @Override
     public void close() throws IOException {
-      simpleInput.close();
     }
 
     @Override
@@ -325,29 +289,24 @@ public class MapProcessor extends MRTask implements Processor {
     }
   }
 
-  private static class OldOutputCollector 
+  private static class OldOutputCollector
   implements OutputCollector {
-    private final Output output;
-    
-    OldOutputCollector(Output output) {
+    private final KVWriter output;
+
+    OldOutputCollector(KVWriter output) {
       this.output = output;
     }
 
     public void collect(Object key, Object value) throws IOException {
-      try {
         output.write(key, value);
-      } catch (InterruptedException ie) {
-        Thread.currentThread().interrupt();
-        throw new IOException("interrupt exception", ie);
-      }
     }
   }
 
   private class NewOutputCollector
     extends org.apache.hadoop.mapreduce.RecordWriter {
-    private final Output out;
+    private final KVWriter out;
 
-    NewOutputCollector(Output out) throws IOException {
+    NewOutputCollector(KVWriter out) throws IOException {
       this.out = out;
     }
 
@@ -359,25 +318,24 @@ public class MapProcessor extends MRTask implements Processor {
     @Override
     public void close(TaskAttemptContext context
                       ) throws IOException, InterruptedException {
-      out.close();
     }
   }
 
   @Override
-  public void localizeConfiguration(JobConf jobConf) 
+  public void localizeConfiguration(JobConf jobConf)
       throws IOException, InterruptedException {
     super.localizeConfiguration(jobConf);
     jobConf.setBoolean(JobContext.TASK_ISMAP, true);
   }
-  
+
   @Override
   public TezCounter getOutputRecordsCounter() {
-    return reporter.getCounter(TaskCounter.MAP_OUTPUT_RECORDS);
+    return processorContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS);
   }
 
   @Override
   public TezCounter getInputRecordsCounter() {
-    return reporter.getCounter(TaskCounter.MAP_INPUT_RECORDS);
-
+    return processorContext.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
----------------------------------------------------------------------
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 4fb1876..88ab754 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
@@ -18,12 +18,12 @@
 package org.apache.tez.mapreduce.processor.reduce;
 
 import java.io.IOException;
+import java.util.Iterator;
 import java.util.List;
-import java.util.concurrent.FutureTask;
+import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.mapred.Counters.Counter;
@@ -35,295 +35,251 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.TezEngineTaskContext;
-import org.apache.tez.common.TezTaskStatus;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
 import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.sort.SortingOutput;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.lib.oldinput.OldShuffledMergedInput;
-import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
-import org.apache.tez.mapreduce.input.SimpleInput;
+import org.apache.tez.engine.lib.output.OnFileSortedOutput;
+import org.apache.tez.engine.newapi.Event;
+import org.apache.tez.engine.newapi.KVReader;
+import org.apache.tez.engine.newapi.KVWriter;
+import org.apache.tez.engine.newapi.LogicalIOProcessor;
+import org.apache.tez.engine.newapi.LogicalInput;
+import org.apache.tez.engine.newapi.LogicalOutput;
+import org.apache.tez.engine.newapi.TezProcessorContext;
+import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
 
-import com.google.common.base.Preconditions;
 
 @SuppressWarnings({ "unchecked", "rawtypes" })
 public class ReduceProcessor
 extends MRTask
-implements Processor {
+implements LogicalIOProcessor {
 
   private static final Log LOG = LogFactory.getLog(ReduceProcessor.class);
-  
-  private Progress sortPhase;
-  private Progress reducePhase;
 
   private Counter reduceInputKeyCounter;
   private Counter reduceInputValueCounter;
 
-  public ReduceProcessor(TezEngineTaskContext context) {
-    super(context);
-    TezEngineTaskContext tezEngineContext = (TezEngineTaskContext) context;
-    Preconditions.checkNotNull(tezEngineContext.getInputSpecList(),
-        "InputSpecList should not be null");
+  public ReduceProcessor() {
+    super(false);
   }
-  
+
   @Override
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-    super.initialize(conf, master);
+  public void initialize(TezProcessorContext processorContext)
+      throws IOException {
+    try {
+      super.initialize(processorContext);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
   }
 
+
   @Override
-  public void process(Input[] ins, Output[] outs)
-      throws IOException, InterruptedException {
-    MRTaskReporter reporter = new MRTaskReporter(getTaskReporter());
-    boolean useNewApi = jobConf.getUseNewReducer();
-    initTask(jobConf, taskAttemptId.getTaskID().getVertexID().getDAGId(),
-        reporter, useNewApi);
+  public void handleEvents(List<Event> processorEvents) {
+    // TODO Auto-generated method stub
 
-    if (outs.length <= 0 || outs.length > 1) {
-      throw new IOException("Invalid number of outputs"
-          + ", outputCount=" + outs.length);
-    }
-    
-    if (ins.length <= 0) {
-      throw new IOException("Invalid number of inputs"
-          + ", inputCount=" + ins.length);
-    }
+  }
 
-    Input in = ins[0];
-    Output out = outs[0];
+  public void close() throws IOException {
+    // TODO Auto-generated method stub
 
-    List<InputSpec> inputs = getTezEngineTaskContext().getInputSpecList();
+  }
 
-    if (in instanceof SimpleInput) {
-      ((SimpleInput)in).setTask(this);
-    } else if (in instanceof OldShuffledMergedInput) {
-      ((OldShuffledMergedInput)in).setTask(this);
-    }
-    
-    if(ins.length > 1) {
-      if (!(in instanceof OldShuffledMergedInput)) {
-        throw new IOException(
-            "Only ShuffledMergedInput can support multiple inputs"
-                + ". inputCount=" + ins.length);
-      }      
-      if(ins.length != inputs.size()) {
-        throw new IOException(
-            "Mismatch in input size passed and context inputspec size. Passed: "
-                + ins.length + " From contex:" + inputs.size());
-      }
-      // initialize and merge the remaining
-      OldShuffledMergedInput s0 = ((OldShuffledMergedInput)in);
-      for(int i=1; i<ins.length; ++i) {
-        Input inputi = ins[i];
-        if (!(inputi instanceof OldShuffledMergedInput)) {
-          throw new IOException(
-              "Only ShuffledMergedInput can support multiple inputs"
-                  + ". inputCount=" + ins.length);
-        }      
-        OldShuffledMergedInput si = ((OldShuffledMergedInput)inputi);
-        s0.mergeWith(si);
-      }
+  @Override
+  public void run(Map<String, LogicalInput> inputs,
+      Map<String, LogicalOutput> outputs) throws Exception {
+
+    LOG.info("Running reduce: " + processorContext.getUniqueIdentifier());
+
+    initTask();
+
+    if (outputs.size() <= 0 || outputs.size() > 1) {
+      throw new IOException("Invalid number of outputs"
+          + ", outputCount=" + outputs.size());
     }
-    
-    if (out instanceof SimpleOutput) {
-      initCommitter(jobConf, useNewApi, false);
-      ((SimpleOutput)out).setTask(this);
-    } else if (out instanceof SortingOutput) {
-      initCommitter(jobConf, useNewApi, true);
-      initPartitioner(jobConf);
-      ((SortingOutput)out).setTask(this);
+
+    if (inputs.size() <= 0 || inputs.size() > 1) {
+      throw new IOException("Invalid number of inputs"
+          + ", inputCount=" + inputs.size());
     }
 
+    LogicalInput in = inputs.values().iterator().next();
+    LogicalOutput out = outputs.values().iterator().next();
+
     this.statusUpdate();
-    
+
     Class keyClass = ConfigUtils.getIntermediateInputKeyClass(jobConf);
     Class valueClass = ConfigUtils.getIntermediateInputValueClass(jobConf);
     LOG.info("Using keyClass: " + keyClass);
     LOG.info("Using valueClass: " + valueClass);
-    RawComparator comparator = 
+    RawComparator comparator =
         ConfigUtils.getInputKeySecondaryGroupingComparator(jobConf);
     LOG.info("Using comparator: " + comparator);
 
-    reduceInputKeyCounter = 
-        reporter.getCounter(TaskCounter.REDUCE_INPUT_GROUPS);
-    reduceInputValueCounter = 
-        reporter.getCounter(TaskCounter.REDUCE_INPUT_RECORDS);
-        
+    reduceInputKeyCounter =
+        mrReporter.getCounter(TaskCounter.REDUCE_INPUT_GROUPS);
+    reduceInputValueCounter =
+        mrReporter.getCounter(TaskCounter.REDUCE_INPUT_RECORDS);
+
     // Sanity check
-    if (!(in instanceof OldShuffledMergedInput)) {
+    if (!(in instanceof ShuffledMergedInputLegacy)) {
+      throw new IOException("Illegal input to reduce: " + in.getClass());
+    }
+    ShuffledMergedInputLegacy shuffleInput = (ShuffledMergedInputLegacy)in;
+    KVReader kvReader = shuffleInput.getReader();
+
+    KVWriter kvWriter = null;
+    if((out instanceof SimpleOutput)) {
+      kvWriter = ((SimpleOutput) out).getWriter();
+    } else if ((out instanceof OnFileSortedOutput)) {
+      kvWriter = ((OnFileSortedOutput) out).getWriter();
+    } else {
       throw new IOException("Illegal input to reduce: " + in.getClass());
     }
-    OldShuffledMergedInput shuffleInput = (OldShuffledMergedInput)in;
 
     if (useNewApi) {
       try {
         runNewReducer(
-            jobConf, 
-            (TezTaskUmbilicalProtocol)getUmbilical(), reporter, 
-            shuffleInput, comparator,  keyClass, valueClass, 
-            out);
+            jobConf,
+            mrReporter,
+            shuffleInput, comparator,  keyClass, valueClass,
+            kvWriter);
       } catch (ClassNotFoundException cnfe) {
         throw new IOException(cnfe);
       }
     } else {
       runOldReducer(
-          jobConf, (TezTaskUmbilicalProtocol)getUmbilical(), reporter, 
-          shuffleInput, comparator, keyClass, valueClass, out);
+          jobConf, mrReporter,
+          kvReader, comparator, keyClass, valueClass, kvWriter);
     }
-    
-    done(out.getOutputContext(), reporter);
-  }
 
-  public void close() throws IOException, InterruptedException {
-    // TODO Auto-generated method stub
-    
+    done(out);
   }
 
   void runOldReducer(JobConf job,
-      TezTaskUmbilicalProtocol umbilical,
       final MRTaskReporter reporter,
-      OldShuffledMergedInput input,
+      KVReader input,
       RawComparator comparator,
       Class keyClass,
       Class valueClass,
-      final Output output) throws IOException, InterruptedException {
-    
-    FutureTask<Void> initInputFuture = initInputAsync(input);
-    FutureTask<Void> initOutputFuture = initOutputAsync(output);
+      final KVWriter output) throws IOException, InterruptedException {
 
-    Reducer reducer = 
+    Reducer reducer =
         ReflectionUtils.newInstance(job.getReducerClass(), job);
 
     // make output collector
 
-    OutputCollector collector = 
+    OutputCollector collector =
         new OutputCollector() {
       public void collect(Object key, Object value)
           throws IOException {
-        try {
-          output.write(key, value);
-        } catch (InterruptedException ie) {
-          throw new IOException(ie);
-        }
+        output.write(key, value);
       }
     };
 
-    // Wait for input/output to be initialized before starting processing.
-    LOG.info("Waiting on input initialization");
-    waitForInputInitialization(initInputFuture);
-
-    LOG.info("Waiting on output initialization");
-    waitForOutputInitialization(initOutputFuture);
-
     // apply reduce function
     try {
-      ReduceValuesIterator values = 
+      ReduceValuesIterator values =
           new ReduceValuesIterator(
-              input, 
-              job.getOutputValueGroupingComparator(), keyClass, valueClass, 
-              job, reporter, reduceInputValueCounter, reducePhase);
-      
+              input, reporter, reduceInputValueCounter);
+
       values.informReduceProgress();
       while (values.more()) {
         reduceInputKeyCounter.increment(1);
         reducer.reduce(values.getKey(), values, collector, reporter);
-        values.nextKey();
         values.informReduceProgress();
       }
 
       //Clean up: repeated in catch block below
       reducer.close();
-      output.close();
       //End of clean up.
     } catch (IOException ioe) {
       try {
         reducer.close();
-      } catch (IOException ignored) {}
-
-      try {
-        output.close();
-      } catch (IOException ignored) {}
+      } catch (IOException ignored) {
+      }
 
       throw ioe;
     }
   }
-  
-  private static class ReduceValuesIterator<KEY,VALUE> 
-  extends org.apache.tez.engine.common.task.impl.ValuesIterator<KEY,VALUE> {
+
+  private static class ReduceValuesIterator<KEY,VALUE>
+  implements Iterator<VALUE> {
     private Counter reduceInputValueCounter;
-    private Progress reducePhase;
-
-    public ReduceValuesIterator (OldShuffledMergedInput in,
-        RawComparator<KEY> comparator, 
-        Class<KEY> keyClass,
-        Class<VALUE> valClass,
-        Configuration conf, Progressable reporter,
-        Counter reduceInputValueCounter,
-        Progress reducePhase)
+    private KVReader in;
+    private Progressable reporter;
+    private Object currentKey;
+    private Iterator<Object> currentValues;
+
+    public ReduceValuesIterator (KVReader in,
+        Progressable reporter,
+        Counter reduceInputValueCounter)
             throws IOException {
-      super(in.getIterator(), comparator, keyClass, valClass, conf, reporter);
       this.reduceInputValueCounter = reduceInputValueCounter;
-      this.reducePhase = reducePhase;
+      this.in = in;
+      this.reporter = reporter;
     }
 
-    @Override
-    public VALUE next() {
-      reduceInputValueCounter.increment(1);
-      return moveToNext();
+    public boolean more() throws IOException {
+      boolean more = in.next();
+      if(more) {
+        currentKey = in.getCurrentKV().getKey();
+        currentValues = in.getCurrentKV().getValues().iterator();
+      } else {
+        currentKey = null;
+        currentValues = null;
+      }
+      return more;
     }
 
-    protected VALUE moveToNext() {
-      return super.next();
+    public KEY getKey() throws IOException {
+      return (KEY) currentKey;
     }
 
     public void informReduceProgress() {
-      reducePhase.set(super.in.getProgress().getProgress()); // update progress
       reporter.progress();
     }
+
+    @Override
+    public boolean hasNext() {
+      return currentValues.hasNext();
+    }
+
+    @Override
+    public VALUE next() {
+      reduceInputValueCounter.increment(1);
+      return (VALUE) currentValues.next();
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+
   }
 
   void runNewReducer(JobConf job,
-      final TezTaskUmbilicalProtocol umbilical,
       final MRTaskReporter reporter,
-      OldShuffledMergedInput input,
+      ShuffledMergedInputLegacy input,
       RawComparator comparator,
       Class keyClass,
       Class valueClass,
-      final Output out
-      ) throws IOException,InterruptedException, 
+      final KVWriter out
+      ) throws IOException,InterruptedException,
       ClassNotFoundException {
-    
-    FutureTask<Void> initInputFuture = initInputAsync(input);
-    FutureTask<Void> initOutputFuture = initOutputAsync(out);
-    
+
     // make a task context so we can get the classes
-    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
-        new TaskAttemptContextImpl(job, taskAttemptId, reporter);
-    
+    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext = getTaskAttemptContext();
+
     // make a reducer
     org.apache.hadoop.mapreduce.Reducer reducer =
         (org.apache.hadoop.mapreduce.Reducer)
         ReflectionUtils.newInstance(taskContext.getReducerClass(), job);
 
-    // Wait for input/output to be initialized before starting processing.
-    LOG.info("Waiting on input initialization");
-    waitForInputInitialization(initInputFuture);
-
-    LOG.info("Waiting on output initialization");
-    waitForOutputInitialization(initOutputFuture);
-
     // wrap value iterator to report progress.
     final TezRawKeyValueIterator rawIter = input.getIterator();
     TezRawKeyValueIterator rIter = new TezRawKeyValueIterator() {
@@ -346,7 +302,7 @@ implements Processor {
       }
     };
 
-    org.apache.hadoop.mapreduce.RecordWriter trackedRW = 
+    org.apache.hadoop.mapreduce.RecordWriter trackedRW =
         new org.apache.hadoop.mapreduce.RecordWriter() {
 
       @Override
@@ -358,28 +314,27 @@ implements Processor {
       @Override
       public void close(TaskAttemptContext context) throws IOException,
       InterruptedException {
-        out.close();
       }
     };
 
-    org.apache.hadoop.mapreduce.Reducer.Context reducerContext = 
+    org.apache.hadoop.mapreduce.Reducer.Context reducerContext =
         createReduceContext(
             reducer, job, taskAttemptId,
-            rIter, reduceInputKeyCounter, 
-            reduceInputValueCounter, 
+            rIter, reduceInputKeyCounter,
+            reduceInputValueCounter,
             trackedRW,
             committer,
             reporter, comparator, keyClass,
             valueClass);
-    
-    
-    
+
+
+
     reducer.run(reducerContext);
     trackedRW.close(reducerContext);
   }
 
   @Override
-  public void localizeConfiguration(JobConf jobConf) 
+  public void localizeConfiguration(JobConf jobConf)
       throws IOException, InterruptedException {
     super.localizeConfiguration(jobConf);
     jobConf.setBoolean(JobContext.TASK_ISMAP, false);
@@ -387,21 +342,12 @@ implements Processor {
 
   @Override
   public TezCounter getOutputRecordsCounter() {
-    return reporter.getCounter(TaskCounter.REDUCE_OUTPUT_RECORDS);
+    return processorContext.getCounters().findCounter(TaskCounter.REDUCE_OUTPUT_RECORDS);
   }
 
   @Override
   public TezCounter getInputRecordsCounter() {
-    return reporter.getCounter(TaskCounter.REDUCE_INPUT_GROUPS);
+    return processorContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_GROUPS);
   }
 
-  @Override
-  protected void waitForInputInitialization(FutureTask<Void> future)
-      throws InterruptedException, IOException {
-    super.waitForInputInitialization(future);
-    sortPhase = getProgress().addPhase("sort");
-    reducePhase = getProgress().addPhase("reduce");
-    sortPhase.complete(); // sort is complete
-    setPhase(TezTaskStatus.Phase.REDUCE);
-  }
 }


[05/20] git commit: TEZ-444. Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
TEZ-444.  Rename *.new* packages back to what they should be, remove
dead code from the old packages - mapreduce module (part of TEZ-398).
(sseth)


Project: http://git-wip-us.apache.org/repos/asf/incubator-tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tez/commit/3d609458
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tez/tree/3d609458
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tez/diff/3d609458

Branch: refs/heads/TEZ-398
Commit: 3d6094588cc43f8140b2360bd9032b2ed7249e1a
Parents: 5eb0c86
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Sep 23 10:43:39 2013 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Sep 23 10:43:39 2013 -0700

----------------------------------------------------------------------
 .../apache/hadoop/mapred/YarnTezDagChild.java   |   4 +-
 .../tez/mapreduce/examples/MRRSleepJob.java     |   6 +-
 .../mapreduce/examples/OrderedWordCount.java    |   6 +-
 .../apache/tez/mapreduce/TestMRRJobsDAGApi.java |   6 +-
 .../apache/hadoop/mapred/LocalJobRunnerTez.java |   8 +-
 .../tez/mapreduce/combine/MRCombiner.java       | 440 +++++------
 .../apache/tez/mapreduce/hadoop/MRHelpers.java  |   4 +-
 .../hadoop/MultiStageMRConfToTezTranslator.java |   4 +-
 .../tez/mapreduce/hadoop/mapred/MRReporter.java |  76 ++
 .../hadoop/mapred/TaskAttemptContextImpl.java   |  21 +-
 .../hadoop/mapreduce/MapContextImpl.java        |   6 +-
 .../hadoop/mapreduce/ReduceContextImpl.java     | 359 ---------
 .../mapreduce/TaskAttemptContextImpl.java       | 102 +--
 .../mapreduce/TaskInputOutputContextImpl.java   |   6 +-
 .../mapreduce/hadoop/newmapred/MRReporter.java  |  76 --
 .../newmapred/TaskAttemptContextImpl.java       |  93 ---
 .../hadoop/newmapreduce/MapContextImpl.java     |  84 ---
 .../newmapreduce/TaskAttemptContextImpl.java    |  90 ---
 .../TaskInputOutputContextImpl.java             |  95 ---
 .../input/ShuffledMergedInputLegacy.java        |  29 +
 .../apache/tez/mapreduce/input/SimpleInput.java | 359 +++++----
 .../tez/mapreduce/input/SimpleInputLegacy.java  |  36 +
 .../tez/mapreduce/newcombine/MRCombiner.java    | 242 ------
 .../newinput/ShuffledMergedInputLegacy.java     |  29 -
 .../tez/mapreduce/newinput/SimpleInput.java     | 438 -----------
 .../mapreduce/newinput/SimpleInputLegacy.java   |  36 -
 .../tez/mapreduce/newoutput/SimpleOutput.java   | 326 ---------
 .../mapreduce/newpartition/MRPartitioner.java   |  88 ---
 .../FileSystemStatisticsUpdater.java            |  84 ---
 .../mapreduce/newprocessor/GcTimeUpdater.java   |  71 --
 .../tez/mapreduce/newprocessor/MRTask.java      | 731 -------------------
 .../mapreduce/newprocessor/MRTaskReporter.java  | 122 ----
 .../newprocessor/map/MapProcessor.java          | 341 ---------
 .../newprocessor/reduce/ReduceProcessor.java    | 353 ---------
 .../tez/mapreduce/output/SimpleOutput.java      | 355 ++++++---
 .../tez/mapreduce/partition/MRPartitioner.java  | 105 ++-
 .../apache/tez/mapreduce/processor/MRTask.java  | 625 +++++++---------
 .../tez/mapreduce/processor/MRTaskReporter.java |  77 +-
 .../processor/TezTaskReporterImpl.java          | 279 -------
 .../mapreduce/processor/map/MapProcessor.java   | 268 +++----
 .../processor/reduce/ReduceProcessor.java       | 336 ++++-----
 .../tez/mapreduce/task/MRRuntimeTask.java       | 272 -------
 .../tez/mapreduce/processor/MapUtils.java       |   2 +-
 .../processor/map/TestMapProcessor.java         |   2 +-
 .../processor/reduce/TestReduceProcessor.java   |   8 +-
 .../org/apache/tez/mapreduce/YARNRunner.java    |   6 +-
 46 files changed, 1439 insertions(+), 5667 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
----------------------------------------------------------------------
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 2d3e4d1..31898a3 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
@@ -85,8 +85,8 @@ import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
 import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
 import org.apache.tez.engine.newapi.impl.TezUmbilical;
 import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
-import org.apache.tez.mapreduce.newinput.SimpleInputLegacy;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
+import org.apache.tez.mapreduce.input.SimpleInputLegacy;
+import org.apache.tez.mapreduce.output.SimpleOutput;
 
 import com.google.inject.Guice;
 import com.google.inject.Injector;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
----------------------------------------------------------------------
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 3e79949..429d458 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
@@ -85,9 +85,9 @@ 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.newinput.ShuffledMergedInputLegacy;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
-import org.apache.tez.mapreduce.newprocessor.reduce.ReduceProcessor;
+import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
----------------------------------------------------------------------
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 adfc8cd..07fe58a 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
@@ -75,9 +75,9 @@ 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.newinput.ShuffledMergedInputLegacy;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
-import org.apache.tez.mapreduce.newprocessor.reduce.ReduceProcessor;
+import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 
 /**
  * An MRR job built on top of word count to return words sorted by

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
----------------------------------------------------------------------
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 eb20876..aca5b8e 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
@@ -79,9 +79,9 @@ 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.newinput.ShuffledMergedInputLegacy;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
-import org.apache.tez.mapreduce.newprocessor.reduce.ReduceProcessor;
+import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
----------------------------------------------------------------------
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 1362396..ac6d5dd 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
@@ -84,10 +84,10 @@ import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 import org.apache.tez.engine.runtime.RuntimeUtils;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
-import org.apache.tez.mapreduce.newinput.SimpleInput;
-import org.apache.tez.mapreduce.newoutput.SimpleOutput;
-import org.apache.tez.mapreduce.newprocessor.map.MapProcessor;
-import org.apache.tez.mapreduce.newprocessor.reduce.ReduceProcessor;
+import org.apache.tez.mapreduce.input.SimpleInput;
+import org.apache.tez.mapreduce.output.SimpleOutput;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
----------------------------------------------------------------------
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 dc7c53d..56e88c7 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
@@ -25,324 +25,218 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.RawKeyValueIterator;
 import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapreduce.Counter;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.ReduceContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskCounter;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
 import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
 import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
-import org.apache.tez.common.counters.TaskCounter;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.Input;
-import org.apache.tez.engine.api.Master;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.Processor;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.combine.CombineInput;
-import org.apache.tez.engine.common.combine.CombineOutput;
+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.IDConverter;
-import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
-import org.apache.tez.mapreduce.processor.MRTask;
+import org.apache.tez.engine.newapi.TezInputContext;
+import org.apache.tez.engine.newapi.TezOutputContext;
+import org.apache.tez.engine.newapi.TezTaskContext;
+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;
 
-public class MRCombiner implements Processor {
+@SuppressWarnings({"rawtypes", "unchecked"})
+public class MRCombiner implements Combiner {
 
   private static Log LOG = LogFactory.getLog(MRCombiner.class);
-
-  JobConf jobConf;
-  boolean useNewApi;
-
-  private final MRTask task;
-
-  private Counter combinerInputKeyCounter;
-  private Counter combinerInputValueCounter;
-  private Progress combinePhase;
-
-  public MRCombiner(MRTask task) {
-    this.task = task;
-  }
-
-  @Override
-  public void initialize(Configuration conf, Master master) throws IOException,
-      InterruptedException {
-    if (conf instanceof JobConf) {
-      jobConf = (JobConf)conf;
+  
+  private final Configuration conf;
+  private final Class<?> keyClass;
+  private final Class<?> valClass;
+  private final RawComparator<?> comparator;
+  private final boolean useNewApi;
+  
+  private final TezCounter combineInputKeyCounter;
+  private final TezCounter combineInputValueCounter;
+  
+  private final MRTaskReporter reporter;
+  private final TaskAttemptID mrTaskAttemptID;
+
+  public MRCombiner(TezTaskContext taskContext) throws IOException {
+    this.conf = TezUtils.createConfFromUserPayload(taskContext.getUserPayload());
+
+    assert(taskContext instanceof TezInputContext || taskContext instanceof TezOutputContext);
+    if (taskContext instanceof TezOutputContext) {
+      this.keyClass = ConfigUtils.getIntermediateOutputKeyClass(conf);
+      this.valClass = ConfigUtils.getIntermediateOutputValueClass(conf);
+      this.comparator = ConfigUtils.getIntermediateOutputKeyComparator(conf);
+      this.reporter = new MRTaskReporter((TezOutputContext)taskContext);
     } else {
-      jobConf = new JobConf(conf);
+      this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
+      this.valClass = ConfigUtils.getIntermediateInputValueClass(conf);
+      this.comparator = ConfigUtils.getIntermediateInputKeyComparator(conf);
+      this.reporter = new MRTaskReporter((TezInputContext)taskContext);
     }
-    useNewApi = jobConf.getUseNewMapper();
+
+    this.useNewApi = ConfigUtils.useNewApi(conf);
+    
+    combineInputKeyCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
+    combineInputValueCounter = taskContext.getCounters().findCounter(TaskCounter.COMBINE_OUTPUT_RECORDS);
+    
+    boolean isMap = conf.getBoolean(MRConfig.IS_MAP_PROCESSOR,false);
+    this.mrTaskAttemptID = new TaskAttemptID(
+        new TaskID(String.valueOf(taskContext.getApplicationId()
+            .getClusterTimestamp()), taskContext.getApplicationId().getId(),
+            isMap ? TaskType.MAP : TaskType.REDUCE,
+            taskContext.getTaskIndex()), taskContext.getTaskAttemptNumber());
+    
+    LOG.info("Using combineKeyClass: " + keyClass + ", combineValueClass: " + valClass + ", combineComparator: " +comparator + ", useNewApi: " + useNewApi);
   }
 
   @Override
-  public void process(Input[] in, Output[] out) throws IOException,
-      InterruptedException {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Running MRCombiner, usingNewAPI=" + useNewApi);
-    }
-
-    CombineInput input = (CombineInput)in[0];
-    CombineOutput output = (CombineOutput)out[0];
-
-    combinePhase  = task.getProgress().addPhase("combine");
-
-    Class<?> keyClass = ConfigUtils.getIntermediateOutputKeyClass(jobConf);
-    Class<?> valueClass = ConfigUtils.getIntermediateOutputValueClass(jobConf);
-    LOG.info("Using combineKeyClass: " + keyClass);
-    LOG.info("Using combineValueClass: " + valueClass);
-    RawComparator<?> comparator =
-        ConfigUtils.getIntermediateOutputKeyComparator(jobConf);
-    LOG.info("Using combineComparator: " + comparator);
-
-    combinerInputKeyCounter =
-        task.getMRReporter().getCounter(TaskCounter.COMBINE_INPUT_RECORDS);
-    combinerInputValueCounter =
-        task.getMRReporter().getCounter(TaskCounter.COMBINE_OUTPUT_RECORDS);
-
+  public void combine(TezRawKeyValueIterator rawIter, Writer writer)
+      throws InterruptedException, IOException {
     if (useNewApi) {
-      try {
-        runNewCombiner(this.jobConf,
-            task.getUmbilical(),
-            task.getMRReporter(),
-            input, comparator, keyClass, valueClass, output);
-      } catch (ClassNotFoundException cnfe) {
-        throw new IOException(cnfe);
-      }
+      runNewCombiner(rawIter, writer);
     } else {
-      runOldCombiner(this.jobConf,
-          task.getUmbilical(),
-          task.getMRReporter(),
-          input,
-          comparator, keyClass, valueClass,
-          output);
+      runOldCombiner(rawIter, writer);
     }
+    
   }
 
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  private void runOldCombiner(JobConf job,
-        TezTaskUmbilicalProtocol umbilical,
-        final MRTaskReporter reporter,
-        CombineInput input,
-        RawComparator comparator,
-        Class keyClass,
-        Class valueClass,
-        final Output output) throws IOException, InterruptedException {
-
-    Reducer combiner =
-        ReflectionUtils.newInstance(job.getCombinerClass(), job);
-
-    // make output collector
-
-    OutputCollector collector =
-        new OutputCollector() {
-      public void collect(Object key, Object value)
-          throws IOException {
-        try {
-          output.write(key, value);
-        } catch (InterruptedException ie) {
-          throw new IOException(ie);
-        }
+  ///////////////// Methods for old API //////////////////////
+  
+  private void runOldCombiner(final TezRawKeyValueIterator rawIter, final Writer writer) throws IOException {
+    Class<? extends Reducer> reducerClazz = (Class<? extends Reducer>) conf.getClass("mapred.combiner.class", null, Reducer.class);
+    
+    Reducer combiner = ReflectionUtils.newInstance(reducerClazz, conf);
+    
+    OutputCollector collector = new OutputCollector() {
+      @Override
+      public void collect(Object key, Object value) throws IOException {
+        writer.append(key, value);
       }
     };
-
-    // apply combiner function
-    CombinerValuesIterator values =
-        new CombinerValuesIterator(input,
-            comparator, keyClass, valueClass, job, reporter,
-            combinerInputValueCounter, combinePhase);
-
-    values.informReduceProgress();
-    while (values.more()) {
-      combinerInputKeyCounter.increment(1);
-      combiner.reduce(values.getKey(), values, collector, reporter);
-      values.nextKey();
-      values.informReduceProgress();
+    
+    CombinerValuesIterator values = new CombinerValuesIterator(rawIter, keyClass, valClass, comparator);
+    
+    while (values.moveToNext()) {
+      combiner.reduce(values.getKey(), values.getValues().iterator(), collector, reporter);
     }
   }
-
-  private static final class CombinerValuesIterator<KEY,VALUE>
-  extends org.apache.tez.engine.common.task.impl.ValuesIterator<KEY,VALUE> {
-    private Counter combineInputValueCounter;
-    private Progress combinePhase;
-
-    public CombinerValuesIterator (CombineInput in,
-        RawComparator<KEY> comparator,
-        Class<KEY> keyClass,
-        Class<VALUE> valClass,
-        Configuration conf, Progressable reporter,
-        Counter combineInputValueCounter,
-        Progress combinePhase)
-            throws IOException {
-      super(in.getIterator(), comparator, keyClass, valClass, conf, reporter);
-      this.combineInputValueCounter = combineInputValueCounter;
-      this.combinePhase = combinePhase;
-    }
-
-    @Override
-    public VALUE next() {
-      combineInputValueCounter.increment(1);
-      return moveToNext();
-    }
-
-    protected VALUE moveToNext() {
-      return super.next();
-    }
-
-    public void informReduceProgress() {
-      // FIXME implement correct progress updates for combiner TEZ-184
-      // combinePhase.set(super.in.getProgress().getProgress()); // update progress
-      reporter.progress();
+  
+  private final class CombinerValuesIterator<KEY,VALUE> extends ValuesIterator<KEY, VALUE> {
+    public CombinerValuesIterator(TezRawKeyValueIterator rawIter,
+        Class<KEY> keyClass, Class<VALUE> valClass,
+        RawComparator<KEY> comparator) throws IOException {
+      super(rawIter, comparator, keyClass, valClass, conf,
+          combineInputKeyCounter, combineInputValueCounter);
     }
   }
-
-
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  private void runNewCombiner(JobConf job,
-      final TezTaskUmbilicalProtocol umbilical,
-      final MRTaskReporter reporter,
-      CombineInput input,
-      RawComparator comparator,
-      Class keyClass,
-      Class valueClass,
-      final Output out
-      ) throws IOException,InterruptedException,
-      ClassNotFoundException {
-    // wrap value iterator to report progress.
-    final TezRawKeyValueIterator rawIter = input.getIterator();
-    TezRawKeyValueIterator rIter = new TezRawKeyValueIterator() {
-      public void close() throws IOException {
-        rawIter.close();
-      }
-      public DataInputBuffer getKey() throws IOException {
-        return rawIter.getKey();
+  
+  ///////////////// End of methods for old API //////////////////////
+  
+  ///////////////// Methods for new API //////////////////////
+  
+  private void runNewCombiner(final TezRawKeyValueIterator rawIter, final Writer writer) throws InterruptedException, IOException {
+    
+    RecordWriter recordWriter = new RecordWriter() {
+
+      @Override
+      public void write(Object key, Object value) throws IOException,
+          InterruptedException {
+        writer.append(key, value);
       }
-      public Progress getProgress() {
-        return rawIter.getProgress();
-      }
-      public DataInputBuffer getValue() throws IOException {
-        return rawIter.getValue();
-      }
-      public boolean next() throws IOException {
-        boolean ret = rawIter.next();
-        // FIXME progress updates for combiner
-        // reporter.setProgress(rawIter.getProgress().getProgress());
-        return ret;
+
+      @Override
+      public void close(TaskAttemptContext context) throws IOException,
+          InterruptedException {
+        // Will be closed by whoever invokes the combiner.
       }
     };
-
-    // make a task context so we can get the classes
-    org.apache.hadoop.mapreduce.TaskAttemptContext taskContext =
-        new TaskAttemptContextImpl(job, task.getTaskAttemptId(), reporter);
-
-    // make a reducer
-    org.apache.hadoop.mapreduce.Reducer reducer =
-        (org.apache.hadoop.mapreduce.Reducer)
-        ReflectionUtils.newInstance(taskContext.getCombinerClass(), job);
-
-    org.apache.hadoop.mapreduce.RecordWriter trackedRW =
-        new org.apache.hadoop.mapreduce.RecordWriter() {
-
-          @Override
-          public void write(Object key, Object value) throws IOException,
-              InterruptedException {
-            out.write(key, value);
-          }
-
-          @Override
-          public void close(TaskAttemptContext context) throws IOException,
-              InterruptedException {
-            // Should not close this here as the sorter will close the
-            // combine output
-          }
-        };
-
+    
+    Class<? extends org.apache.hadoop.mapreduce.Reducer> reducerClazz = (Class<? extends org.apache.hadoop.mapreduce.Reducer>) conf
+        .getClass(MRJobConfig.COMBINE_CLASS_ATTR, null,
+            org.apache.hadoop.mapreduce.Reducer.class);
+    org.apache.hadoop.mapreduce.Reducer reducer = ReflectionUtils.newInstance(reducerClazz, conf);
+    
     org.apache.hadoop.mapreduce.Reducer.Context reducerContext =
         createReduceContext(
-            reducer, job, task.getTaskAttemptId(),
-            rIter, combinerInputKeyCounter,
-            combinerInputValueCounter,
-            trackedRW,
-            null,
-            reporter, comparator, keyClass,
-            valueClass);
+            conf,
+            mrTaskAttemptID,
+            rawIter,
+            new MRCounters.MRCounter(combineInputKeyCounter),
+            new MRCounters.MRCounter(combineInputValueCounter),
+            recordWriter,
+            reporter,
+            (RawComparator)comparator,
+            keyClass,
+            valClass);
+    
     reducer.run(reducerContext);
-    trackedRW.close(reducerContext);
-  }
-
-  @Override
-  public void close() throws IOException, InterruptedException {
+    recordWriter.close(reducerContext);
   }
 
-  protected static <INKEY,INVALUE,OUTKEY,OUTVALUE>
-  org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context
-  createReduceContext(org.apache.hadoop.mapreduce.Reducer
-                        <INKEY,INVALUE,OUTKEY,OUTVALUE> reducer,
-                      Configuration job,
-                      TezTaskAttemptID taskId,
-                      final TezRawKeyValueIterator rIter,
-                      org.apache.hadoop.mapreduce.Counter inputKeyCounter,
-                      org.apache.hadoop.mapreduce.Counter inputValueCounter,
-                      org.apache.hadoop.mapreduce.RecordWriter<OUTKEY,OUTVALUE> output,
-                      org.apache.hadoop.mapreduce.OutputCommitter committer,
-                      org.apache.hadoop.mapreduce.StatusReporter reporter,
-                      RawComparator<INKEY> comparator,
-                      Class<INKEY> keyClass, Class<INVALUE> valueClass
-  ) throws IOException, InterruptedException {
-    RawKeyValueIterator r =
-        new RawKeyValueIterator() {
-
-          @Override
-          public boolean next() throws IOException {
-            return rIter.next();
-          }
+  private static <KEYIN, VALUEIN, KEYOUT, VALUEOUT> org.apache.hadoop.mapreduce.Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context createReduceContext(
+      Configuration conf,
+      TaskAttemptID mrTaskAttemptID,
+      final TezRawKeyValueIterator rawIter,
+      Counter combineInputKeyCounter,
+      Counter combineInputValueCounter,
+      RecordWriter<KEYOUT, VALUEOUT> recordWriter,
+      MRTaskReporter reporter,
+      RawComparator<KEYIN> comparator,
+      Class<KEYIN> keyClass,
+      Class<VALUEIN> valClass) throws InterruptedException, IOException {
+
+    RawKeyValueIterator r = new RawKeyValueIterator() {
+
+      @Override
+      public boolean next() throws IOException {
+        return rawIter.next();
+      }
 
-          @Override
-          public DataInputBuffer getValue() throws IOException {
-            return rIter.getValue();
-          }
+      @Override
+      public DataInputBuffer getValue() throws IOException {
+        return rawIter.getValue();
+      }
 
-          @Override
-          public Progress getProgress() {
-            return rIter.getProgress();
-          }
+      @Override
+      public Progress getProgress() {
+        return rawIter.getProgress();
+      }
 
-          @Override
-          public DataInputBuffer getKey() throws IOException {
-            return rIter.getKey();
-          }
+      @Override
+      public DataInputBuffer getKey() throws IOException {
+        return rawIter.getKey();
+      }
 
-          @Override
-          public void close() throws IOException {
-            rIter.close();
-          }
-        };
-    org.apache.hadoop.mapreduce.ReduceContext<INKEY, INVALUE, OUTKEY, OUTVALUE>
-    reduceContext =
-      new ReduceContextImpl<INKEY, INVALUE, OUTKEY, OUTVALUE>(
-          job,
-          IDConverter.toMRTaskAttemptId(taskId),
-          r,
-          inputKeyCounter,
-          inputValueCounter,
-          output,
-          committer,
-          reporter,
-          comparator,
-          keyClass,
-          valueClass);
+      @Override
+      public void close() throws IOException {
+        rawIter.close();
+      }
+    };
 
-    org.apache.hadoop.mapreduce.Reducer<INKEY,INVALUE,OUTKEY,OUTVALUE>.Context
-        reducerContext = new
-          WrappedReducer<INKEY, INVALUE, OUTKEY, OUTVALUE>().getReducerContext(
-              reduceContext);
+    ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> rContext = new ReduceContextImpl<KEYIN, VALUEIN, KEYOUT, VALUEOUT>(
+        conf, mrTaskAttemptID, r, combineInputKeyCounter,
+        combineInputValueCounter, recordWriter, null, reporter, comparator,
+        keyClass, valClass);
 
+    org.apache.hadoop.mapreduce.Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context reducerContext = new WrappedReducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>()
+        .getReducerContext(rContext);
     return reducerContext;
   }
 
+  
+ 
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
----------------------------------------------------------------------
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 7a9b7e0..b0ed6ab 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
@@ -59,8 +59,8 @@ import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
-import org.apache.tez.mapreduce.newcombine.MRCombiner;
-import org.apache.tez.mapreduce.newpartition.MRPartitioner;
+import org.apache.tez.mapreduce.combine.MRCombiner;
+import org.apache.tez.mapreduce.partition.MRPartitioner;
 
 
 public class MRHelpers {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
----------------------------------------------------------------------
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 d768312..d888c42 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
@@ -29,9 +29,9 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.mapreduce.combine.MRCombiner;
 import org.apache.tez.mapreduce.hadoop.DeprecatedKeys.MultiStageKeys;
-import org.apache.tez.mapreduce.newcombine.MRCombiner;
-import org.apache.tez.mapreduce.newpartition.MRPartitioner;
+import org.apache.tez.mapreduce.partition.MRPartitioner;
 
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f5e08dc
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
@@ -0,0 +1,76 @@
+package org.apache.tez.mapreduce.hadoop.mapred;
+
+import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.tez.engine.newapi.TezProcessorContext;
+import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.mapreduce.common.Utils;
+
+public class MRReporter implements Reporter {
+
+  private TezTaskContext tezTaskContext;
+  private InputSplit split;
+  private boolean isProcessorContext = false;
+  
+  public MRReporter(TezProcessorContext tezProcContext) {
+    this(tezProcContext, null);
+    isProcessorContext = true;
+  }
+  public MRReporter(TezTaskContext tezTaskContext) {
+    this(tezTaskContext, null);
+  }
+
+  public MRReporter(TezTaskContext tezTaskContext, InputSplit split) {
+    this.tezTaskContext = tezTaskContext;
+    this.split = split;
+  }
+  
+  @Override
+  public void progress() {
+    //TODO NEWTEZ
+  }
+
+  @Override
+  public void setStatus(String status) {
+    // Not setting status string in Tez.
+
+  }
+
+  @Override
+  public Counter getCounter(Enum<?> name) {
+    return Utils.getMRCounter(tezTaskContext.getCounters().findCounter(name));
+  }
+
+  @Override
+  public Counter getCounter(String group, String name) {
+    return Utils.getMRCounter(tezTaskContext.getCounters().findCounter(group,
+        name));
+  }
+
+  @Override
+  public void incrCounter(Enum<?> key, long amount) {
+    getCounter(key).increment(amount);
+  }
+
+  @Override
+  public void incrCounter(String group, String counter, long amount) {
+    getCounter(group, counter).increment(amount);
+  }
+
+  @Override
+  public InputSplit getInputSplit() throws UnsupportedOperationException {
+    if (split == null) {
+      throw new UnsupportedOperationException("Input only available on map");
+    } else {
+      return split;
+    }
+  }
+
+  @Override
+  public float getProgress() {
+    // TOOD NEWTEZ Does this make a difference to anything ?
+    return 0.0f;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/TaskAttemptContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/TaskAttemptContextImpl.java
index d953891..fc32825 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/TaskAttemptContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/TaskAttemptContextImpl.java
@@ -22,11 +22,8 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.TaskAttemptContext;
 import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.mapred.TaskID;
 import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.util.Progressable;
-import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
 
 @InterfaceAudience.Private
@@ -36,24 +33,10 @@ public class TaskAttemptContextImpl
        implements TaskAttemptContext {
   private MRTaskReporter reporter;
 
-  public TaskAttemptContextImpl(JobConf conf, TezTaskAttemptID taskid) {
-    this(conf, taskid, null);
-  }
-  
   // FIXME we need to use DAG Id but we are using App Id
-  public TaskAttemptContextImpl(JobConf conf, TezTaskAttemptID taskAttemptId,
+  public TaskAttemptContextImpl(JobConf conf, TaskAttemptID taskAttemptId,
                          MRTaskReporter reporter) {
-    super(conf, 
-        new TaskAttemptID(
-            new TaskID(
-                Long.toString(taskAttemptId.getTaskID().getVertexID().
-                    getDAGId().getApplicationId().getClusterTimestamp()),
-                taskAttemptId.getTaskID().getVertexID().getDAGId().
-                    getApplicationId().getId(),
-                (taskAttemptId.getTaskID().getVertexID().getId() == 0 ?
-                    TaskType.MAP : TaskType.REDUCE),
-                taskAttemptId.getTaskID().getId()),
-              taskAttemptId.getId()));
+    super(conf, taskAttemptId);
     this.reporter = reporter;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
----------------------------------------------------------------------
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 4552397..b0348c9 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.OutputCommitter;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.engine.newapi.TezTaskContext;
 
 /**
  * The context that is given to the {@link Mapper}.
@@ -51,9 +51,9 @@ public class MapContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
                         RecordReader<KEYIN,VALUEIN> reader,
                         RecordWriter<KEYOUT,VALUEOUT> writer,
                         OutputCommitter committer,
-                        MRTaskReporter reporter,
+                        TezTaskContext context,
                         InputSplit split) {
-    super(conf, taskid, writer, committer, reporter);
+    super(conf, taskid, writer, committer, context);
     this.reader = reader;
     this.split = split;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/ReduceContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/ReduceContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/ReduceContextImpl.java
deleted file mode 100644
index e775b7e..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/ReduceContextImpl.java
+++ /dev/null
@@ -1,359 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.hadoop.mapreduce;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
-import org.apache.hadoop.mapred.BackupStore;
-import org.apache.hadoop.mapred.RawKeyValueIterator;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.ReduceContext;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.mapreduce.processor.MRTaskReporter;
-
-/**
- * The context passed to the {@link Reducer}.
- * @param <KEYIN> the class of the input keys
- * @param <VALUEIN> the class of the input values
- * @param <KEYOUT> the class of the output keys
- * @param <VALUEOUT> the class of the output values
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class ReduceContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
-    extends TaskInputOutputContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
-    implements ReduceContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
-  private RawKeyValueIterator input;
-  private Counter inputValueCounter;
-  private Counter inputKeyCounter;
-  private RawComparator<KEYIN> comparator;
-  private KEYIN key;                                  // current key
-  private VALUEIN value;                              // current value
-  private boolean firstValue = false;                 // first value in key
-  private boolean nextKeyIsSame = false;              // more w/ this key
-  private boolean hasMore;                            // more in file
-  protected Progressable reporter;
-  private Deserializer<KEYIN> keyDeserializer;
-  private Deserializer<VALUEIN> valueDeserializer;
-  private DataInputBuffer buffer = new DataInputBuffer();
-  private BytesWritable currentRawKey = new BytesWritable();
-  private ValueIterable iterable = new ValueIterable();
-  private boolean isMarked = false;
-  private BackupStore<KEYIN,VALUEIN> backupStore;
-  private final SerializationFactory serializationFactory;
-  private final Class<KEYIN> keyClass;
-  private final Class<VALUEIN> valueClass;
-  private final Configuration conf;
-  private final TaskAttemptID taskid;
-  private int currentKeyLength = -1;
-  private int currentValueLength = -1;
-  
-  public ReduceContextImpl(Configuration conf, TaskAttemptID taskid,
-                           RawKeyValueIterator input, 
-                           Counter inputKeyCounter,
-                           Counter inputValueCounter,
-                           RecordWriter<KEYOUT,VALUEOUT> output,
-                           OutputCommitter committer,
-                           MRTaskReporter reporter,
-                           RawComparator<KEYIN> comparator,
-                           Class<KEYIN> keyClass,
-                           Class<VALUEIN> valueClass
-                          ) throws InterruptedException, IOException{
-    super(conf, taskid, output, committer, reporter);
-    this.input = input;
-    this.inputKeyCounter = inputKeyCounter;
-    this.inputValueCounter = inputValueCounter;
-    this.comparator = comparator;
-    this.serializationFactory = new SerializationFactory(conf);
-    this.keyDeserializer = serializationFactory.getDeserializer(keyClass);
-    this.keyDeserializer.open(buffer);
-    this.valueDeserializer = serializationFactory.getDeserializer(valueClass);
-    this.valueDeserializer.open(buffer);
-    hasMore = input.next();
-    this.keyClass = keyClass;
-    this.valueClass = valueClass;
-    this.conf = conf;
-    this.taskid = taskid;
-  }
-
-  /** Start processing next unique key. */
-  public boolean nextKey() throws IOException,InterruptedException {
-    while (hasMore && nextKeyIsSame) {
-      nextKeyValue();
-    }
-    if (hasMore) {
-      if (inputKeyCounter != null) {
-        inputKeyCounter.increment(1);
-      }
-      return nextKeyValue();
-    } else {
-      return false;
-    }
-  }
-
-  /**
-   * Advance to the next key/value pair.
-   */
-  @Override
-  public boolean nextKeyValue() throws IOException, InterruptedException {
-    if (!hasMore) {
-      key = null;
-      value = null;
-      return false;
-    }
-    firstValue = !nextKeyIsSame;
-    DataInputBuffer nextKey = input.getKey();
-    currentRawKey.set(nextKey.getData(), nextKey.getPosition(), 
-                      nextKey.getLength() - nextKey.getPosition());
-    buffer.reset(currentRawKey.getBytes(), 0, currentRawKey.getLength());
-    key = keyDeserializer.deserialize(key);
-    DataInputBuffer nextVal = input.getValue();
-    buffer.reset(nextVal.getData(), nextVal.getPosition(), nextVal.getLength());
-    value = valueDeserializer.deserialize(value);
-
-    currentKeyLength = nextKey.getLength() - nextKey.getPosition();
-    currentValueLength = nextVal.getLength() - nextVal.getPosition();
-
-    if (isMarked) {
-      backupStore.write(nextKey, nextVal);
-    }
-
-    hasMore = input.next();
-    if (hasMore) {
-      nextKey = input.getKey();
-      nextKeyIsSame = comparator.compare(currentRawKey.getBytes(), 0, 
-                                     currentRawKey.getLength(),
-                                     nextKey.getData(),
-                                     nextKey.getPosition(),
-                                     nextKey.getLength() - nextKey.getPosition()
-                                         ) == 0;
-    } else {
-      nextKeyIsSame = false;
-    }
-    inputValueCounter.increment(1);
-    return true;
-  }
-
-  public KEYIN getCurrentKey() {
-    return key;
-  }
-
-  @Override
-  public VALUEIN getCurrentValue() {
-    return value;
-  }
-  
-  BackupStore<KEYIN,VALUEIN> getBackupStore() {
-    return backupStore;
-  }
-  
-  protected class ValueIterator implements ReduceContext.ValueIterator<VALUEIN> {
-
-    private boolean inReset = false;
-    private boolean clearMarkFlag = false;
-
-    public boolean hasNext() {
-      try {
-        if (inReset && backupStore.hasNext()) {
-          return true;
-        } 
-      } catch (Exception e) {
-        e.printStackTrace();
-        throw new RuntimeException("hasNext failed", e);
-      }
-      return firstValue || nextKeyIsSame;
-    }
-
-    public VALUEIN next() {
-      if (inReset) {
-        try {
-          if (backupStore.hasNext()) {
-            backupStore.next();
-            DataInputBuffer next = backupStore.nextValue();
-            buffer.reset(next.getData(), next.getPosition(), next.getLength());
-            value = valueDeserializer.deserialize(value);
-            return value;
-          } else {
-            inReset = false;
-            backupStore.exitResetMode();
-            if (clearMarkFlag) {
-              clearMarkFlag = false;
-              isMarked = false;
-            }
-          }
-        } catch (IOException e) {
-          e.printStackTrace();
-          throw new RuntimeException("next value iterator failed", e);
-        }
-      } 
-
-      // if this is the first record, we don't need to advance
-      if (firstValue) {
-        firstValue = false;
-        return value;
-      }
-      // if this isn't the first record and the next key is different, they
-      // can't advance it here.
-      if (!nextKeyIsSame) {
-        throw new NoSuchElementException("iterate past last value");
-      }
-      // otherwise, go to the next key/value pair
-      try {
-        nextKeyValue();
-        return value;
-      } catch (IOException ie) {
-        throw new RuntimeException("next value iterator failed", ie);
-      } catch (InterruptedException ie) {
-        // this is bad, but we can't modify the exception list of java.util
-        throw new RuntimeException("next value iterator interrupted", ie);        
-      }
-    }
-
-    public void remove() {
-      throw new UnsupportedOperationException("remove not implemented");
-    }
-
-    public void mark() throws IOException {
-      if (getBackupStore() == null) {
-        backupStore = new BackupStore<KEYIN,VALUEIN>(conf, taskid);
-      }
-      isMarked = true;
-      if (!inReset) {
-        backupStore.reinitialize();
-        if (currentKeyLength == -1) {
-          // The user has not called next() for this iterator yet, so
-          // there is no current record to mark and copy to backup store.
-          return;
-        }
-        assert (currentValueLength != -1);
-        int requestedSize = currentKeyLength + currentValueLength + 
-          WritableUtils.getVIntSize(currentKeyLength) +
-          WritableUtils.getVIntSize(currentValueLength);
-        DataOutputStream out = backupStore.getOutputStream(requestedSize);
-        writeFirstKeyValueBytes(out);
-        backupStore.updateCounters(requestedSize);
-      } else {
-        backupStore.mark();
-      }
-    }
-
-    public void reset() throws IOException {
-      // We reached the end of an iteration and user calls a 
-      // reset, but a clearMark was called before, just throw
-      // an exception
-      if (clearMarkFlag) {
-        clearMarkFlag = false;
-        backupStore.clearMark();
-        throw new IOException("Reset called without a previous mark");
-      }
-      
-      if (!isMarked) {
-        throw new IOException("Reset called without a previous mark");
-      }
-      inReset = true;
-      backupStore.reset();
-    }
-
-    public void clearMark() throws IOException {
-      if (getBackupStore() == null) {
-        return;
-      }
-      if (inReset) {
-        clearMarkFlag = true;
-        backupStore.clearMark();
-      } else {
-        inReset = isMarked = false;
-        backupStore.reinitialize();
-      }
-    }
-    
-    /**
-     * This method is called when the reducer moves from one key to 
-     * another.
-     * @throws IOException
-     */
-    public void resetBackupStore() throws IOException {
-      if (getBackupStore() == null) {
-        return;
-      }
-      inReset = isMarked = false;
-      backupStore.reinitialize();
-      currentKeyLength = -1;
-    }
-
-    /**
-     * This method is called to write the record that was most recently
-     * served (before a call to the mark). Since the framework reads one
-     * record in advance, to get this record, we serialize the current key
-     * and value
-     * @param out
-     * @throws IOException
-     */
-    private void writeFirstKeyValueBytes(DataOutputStream out) 
-    throws IOException {
-      assert (getCurrentKey() != null && getCurrentValue() != null);
-      WritableUtils.writeVInt(out, currentKeyLength);
-      WritableUtils.writeVInt(out, currentValueLength);
-      Serializer<KEYIN> keySerializer = 
-        serializationFactory.getSerializer(keyClass);
-      keySerializer.open(out);
-      keySerializer.serialize(getCurrentKey());
-
-      Serializer<VALUEIN> valueSerializer = 
-        serializationFactory.getSerializer(valueClass);
-      valueSerializer.open(out);
-      valueSerializer.serialize(getCurrentValue());
-    }
-  }
-
-  protected class ValueIterable implements Iterable<VALUEIN> {
-    private ValueIterator iterator = new ValueIterator();
-    public Iterator<VALUEIN> iterator() {
-      return iterator;
-    } 
-  }
-  
-  /**
-   * Iterate through the values for the current key, reusing the same value 
-   * object, which is stored in the context.
-   * @return the series of values associated with the current key. All of the 
-   * objects returned directly and indirectly from this method are reused.
-   */
-  public 
-  Iterable<VALUEIN> getValues() throws IOException, InterruptedException {
-    return iterable;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
----------------------------------------------------------------------
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 7b69872..d8548a4 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
@@ -1,5 +1,4 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
+/* Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
  * regarding copyright ownership.  The ASF licenses this file
@@ -21,98 +20,71 @@ package org.apache.tez.mapreduce.hadoop.mapreduce;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapred.Task;
 import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.Counters;
-import org.apache.hadoop.mapreduce.StatusReporter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.tez.mapreduce.hadoop.IDConverter;
-import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.mapreduce.common.Utils;
+
+// 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
 
-/**
- * The context for task attempts.
- */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class TaskAttemptContextImpl extends JobContextImpl 
-    implements TaskAttemptContext {
-  private final TaskAttemptID taskId;
-  private String status = "";
-  private MRTaskReporter reporter;
+public class TaskAttemptContextImpl
+       extends org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl {
 
-  public TaskAttemptContextImpl(Configuration conf, 
-                                TaskAttemptID taskId) {
-    this(conf, taskId, null);
-  }
+  private TezTaskContext taskContext;
 
+  // FIXME we need to use DAG Id but we are using App Id
   public TaskAttemptContextImpl(Configuration conf,
-      TaskAttemptID taskId, MRTaskReporter reporter) {
-    super(conf, IDConverter.fromMRJobId(taskId.getJobID()));
-    this.taskId = taskId;
-    this.reporter = reporter;
+      TezTaskContext taskContext, boolean isMap) {
+    // TODO NEWTEZ Can the jt Identifier string be taskContext.getUniqueId ?
+    this(conf, new TaskAttemptID(
+        new TaskID(String.valueOf(taskContext.getApplicationId()
+            .getClusterTimestamp()), taskContext.getApplicationId().getId(),
+            isMap ? TaskType.MAP : TaskType.REDUCE,
+            taskContext.getTaskIndex()),
+            taskContext.getTaskAttemptNumber()), taskContext);
   }
 
-  /**
-   * Get the unique name for this task attempt.
-   */
-  public TaskAttemptID getTaskAttemptID() {
-    return taskId;
+  public TaskAttemptContextImpl(Configuration conf, TaskAttemptID taId, TezTaskContext context) {
+    super(conf, taId);
+    this.taskContext = context;
   }
 
-  /**
-   * Get the last set status message.
-   * @return the current status message
-   */
-  public String getStatus() {
-    return status;
+  @Override
+  public float getProgress() {
+    // TODO NEWTEZ Will this break anything ?
+    return 0.0f;
   }
 
+  @Override
   public Counter getCounter(Enum<?> counterName) {
-    return (Counter) reporter.getCounter(counterName);
+    return Utils.getMRCounter(taskContext.getCounters().findCounter(counterName));
   }
 
+  @Override
   public Counter getCounter(String groupName, String counterName) {
-    return (Counter) reporter.getCounter(groupName, counterName);
+    return Utils.getMRCounter(taskContext.getCounters().findCounter(groupName, counterName));
   }
 
   /**
    * Report progress.
    */
+  @Override
   public void progress() {
-    reporter.progress();
-  }
-
-  protected void setStatusString(String status) {
-    this.status = status;
+    // Nothing to do.
   }
 
   /**
    * Set the current status of the task to the given string.
    */
+  @Override
   public void setStatus(String status) {
-    String normalizedStatus = Task.normalizeStatus(status, conf);
-    setStatusString(normalizedStatus);
-    reporter.setStatus(normalizedStatus);
-  }
-
-  public static class DummyReporter extends StatusReporter {
-    public void setStatus(String s) {
-    }
-    public void progress() {
-    }
-    public Counter getCounter(Enum<?> name) {
-      return new Counters().findCounter(name);
-    }
-    public Counter getCounter(String group, String name) {
-      return new Counters().findCounter(group, name);
-    }
-    public float getProgress() {
-      return 0f;
-    }
-  }
-  
-  public float getProgress() {
-    return reporter.getProgress();
+    setStatusString(status);
+    // Nothing to do until InputContext supports some kind of custom string
+    // diagnostics.
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
----------------------------------------------------------------------
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 ff4e18f..c2920dc 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.RecordWriter;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskInputOutputContext;
-import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.engine.newapi.TezTaskContext;
 
 /**
  * A context object that allows input and output from the task. It is only
@@ -50,8 +50,8 @@ public abstract class TaskInputOutputContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
   public TaskInputOutputContextImpl(Configuration conf, TaskAttemptID taskid,
                                     RecordWriter<KEYOUT,VALUEOUT> output,
                                     OutputCommitter committer,
-                                    MRTaskReporter reporter) {
-    super(conf, taskid, reporter);
+                                    TezTaskContext context) {
+    super(conf, taskid, context);
     this.output = output;
     this.committer = committer;
   }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/MRReporter.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/MRReporter.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/MRReporter.java
deleted file mode 100644
index df23b9f..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/MRReporter.java
+++ /dev/null
@@ -1,76 +0,0 @@
-package org.apache.tez.mapreduce.hadoop.newmapred;
-
-import org.apache.hadoop.mapred.Counters.Counter;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
-import org.apache.tez.mapreduce.common.Utils;
-
-public class MRReporter implements Reporter {
-
-  private TezTaskContext tezTaskContext;
-  private InputSplit split;
-  private boolean isProcessorContext = false;
-  
-  public MRReporter(TezProcessorContext tezProcContext) {
-    this(tezProcContext, null);
-    isProcessorContext = true;
-  }
-  public MRReporter(TezTaskContext tezTaskContext) {
-    this(tezTaskContext, null);
-  }
-
-  public MRReporter(TezTaskContext tezTaskContext, InputSplit split) {
-    this.tezTaskContext = tezTaskContext;
-    this.split = split;
-  }
-  
-  @Override
-  public void progress() {
-    //TODO NEWTEZ
-  }
-
-  @Override
-  public void setStatus(String status) {
-    // Not setting status string in Tez.
-
-  }
-
-  @Override
-  public Counter getCounter(Enum<?> name) {
-    return Utils.getMRCounter(tezTaskContext.getCounters().findCounter(name));
-  }
-
-  @Override
-  public Counter getCounter(String group, String name) {
-    return Utils.getMRCounter(tezTaskContext.getCounters().findCounter(group,
-        name));
-  }
-
-  @Override
-  public void incrCounter(Enum<?> key, long amount) {
-    getCounter(key).increment(amount);
-  }
-
-  @Override
-  public void incrCounter(String group, String counter, long amount) {
-    getCounter(group, counter).increment(amount);
-  }
-
-  @Override
-  public InputSplit getInputSplit() throws UnsupportedOperationException {
-    if (split == null) {
-      throw new UnsupportedOperationException("Input only available on map");
-    } else {
-      return split;
-    }
-  }
-
-  @Override
-  public float getProgress() {
-    // TOOD NEWTEZ Does this make a difference to anything ?
-    return 0.0f;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/TaskAttemptContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/TaskAttemptContextImpl.java
deleted file mode 100644
index 956fcc2..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapred/TaskAttemptContextImpl.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/* Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.hadoop.newmapred;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.TaskAttemptContext;
-import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.mapreduce.newprocessor.MRTaskReporter;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TaskAttemptContextImpl
-       extends org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl 
-       implements TaskAttemptContext {
-  private MRTaskReporter reporter;
-
-  // FIXME we need to use DAG Id but we are using App Id
-  public TaskAttemptContextImpl(JobConf conf, TaskAttemptID taskAttemptId,
-                         MRTaskReporter reporter) {
-    super(conf, taskAttemptId);
-    this.reporter = reporter;
-  }
-  
-  /**
-   * Get the taskAttemptID.
-   *  
-   * @return TaskAttemptID
-   */
-  public TaskAttemptID getTaskAttemptID() {
-    return (TaskAttemptID) super.getTaskAttemptID();
-  }
-  
-  public Progressable getProgressible() {
-    return reporter;
-  }
-  
-  public JobConf getJobConf() {
-    return (JobConf) getConfiguration();
-  }
-  
-  @Override
-  public float getProgress() {
-    return reporter.getProgress();
-  }
-
-  @Override
-  public Counter getCounter(Enum<?> counterName) {
-    return (Counter) reporter.getCounter(counterName);
-  }
-
-  @Override
-  public Counter getCounter(String groupName, String counterName) {
-    return (Counter) reporter.getCounter(groupName, counterName);
-  }
-
-  /**
-   * Report progress.
-   */
-  @Override
-  public void progress() {
-    reporter.progress();
-  }
-
-  /**
-   * Set the current status of the task to the given string.
-   */
-  @Override
-  public void setStatus(String status) {
-    setStatusString(status);
-    reporter.setStatus(status);
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/MapContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/MapContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/MapContextImpl.java
deleted file mode 100644
index 39c7194..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/MapContextImpl.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.hadoop.newmapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.MapContext;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.tez.engine.newapi.TezTaskContext;
-
-/**
- * The context that is given to the {@link Mapper}.
- * @param <KEYIN> the key input type to the Mapper
- * @param <VALUEIN> the value input type to the Mapper
- * @param <KEYOUT> the key output type from the Mapper
- * @param <VALUEOUT> the value output type from the Mapper
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class MapContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
-    extends TaskInputOutputContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
-    implements MapContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
-  private RecordReader<KEYIN,VALUEIN> reader;
-  private InputSplit split;
-
-  public MapContextImpl(Configuration conf, TaskAttemptID taskid,
-                        RecordReader<KEYIN,VALUEIN> reader,
-                        RecordWriter<KEYOUT,VALUEOUT> writer,
-                        OutputCommitter committer,
-                        TezTaskContext context,
-                        InputSplit split) {
-    super(conf, taskid, writer, committer, context);
-    this.reader = reader;
-    this.split = split;
-  }
-
-  /**
-   * Get the input split for this map.
-   */
-  public InputSplit getInputSplit() {
-    return split;
-  }
-
-  @Override
-  public KEYIN getCurrentKey() throws IOException, InterruptedException {
-    return reader.getCurrentKey();
-  }
-
-  @Override
-  public VALUEIN getCurrentValue() throws IOException, InterruptedException {
-    return reader.getCurrentValue();
-  }
-
-  @Override
-  public boolean nextKeyValue() throws IOException, InterruptedException {
-    return reader.nextKeyValue();
-  }
-
-}
-     

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskAttemptContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskAttemptContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskAttemptContextImpl.java
deleted file mode 100644
index f2057e9..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskAttemptContextImpl.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/* Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.hadoop.newmapreduce;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskID;
-import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.tez.engine.newapi.TezTaskContext;
-import org.apache.tez.mapreduce.common.Utils;
-
-// 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
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TaskAttemptContextImpl
-       extends org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl {
-
-  private TezTaskContext taskContext;
-
-  // FIXME we need to use DAG Id but we are using App Id
-  public TaskAttemptContextImpl(Configuration conf,
-      TezTaskContext taskContext, boolean isMap) {
-    // TODO NEWTEZ Can the jt Identifier string be taskContext.getUniqueId ?
-    this(conf, new TaskAttemptID(
-        new TaskID(String.valueOf(taskContext.getApplicationId()
-            .getClusterTimestamp()), taskContext.getApplicationId().getId(),
-            isMap ? TaskType.MAP : TaskType.REDUCE,
-            taskContext.getTaskIndex()),
-            taskContext.getTaskAttemptNumber()), taskContext);
-  }
-
-  public TaskAttemptContextImpl(Configuration conf, TaskAttemptID taId, TezTaskContext context) {
-    super(conf, taId);
-    this.taskContext = context;
-  }
-
-  @Override
-  public float getProgress() {
-    // TODO NEWTEZ Will this break anything ?
-    return 0.0f;
-  }
-
-  @Override
-  public Counter getCounter(Enum<?> counterName) {
-    return Utils.getMRCounter(taskContext.getCounters().findCounter(counterName));
-  }
-
-  @Override
-  public Counter getCounter(String groupName, String counterName) {
-    return Utils.getMRCounter(taskContext.getCounters().findCounter(groupName, counterName));
-  }
-
-  /**
-   * Report progress.
-   */
-  @Override
-  public void progress() {
-    // Nothing to do.
-  }
-
-  /**
-   * Set the current status of the task to the given string.
-   */
-  @Override
-  public void setStatus(String status) {
-    setStatusString(status);
-    // Nothing to do until InputContext supports some kind of custom string
-    // diagnostics.
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskInputOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskInputOutputContextImpl.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskInputOutputContextImpl.java
deleted file mode 100644
index a34cf8d..0000000
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/newmapreduce/TaskInputOutputContextImpl.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.mapreduce.hadoop.newmapreduce;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.TaskInputOutputContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
-
-/**
- * A context object that allows input and output from the task. It is only
- * supplied to the {@link Mapper} or {@link Reducer}.
- * @param <KEYIN> the input key type for the task
- * @param <VALUEIN> the input value type for the task
- * @param <KEYOUT> the output key type for the task
- * @param <VALUEOUT> the output value type for the task
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public abstract class TaskInputOutputContextImpl<KEYIN,VALUEIN,KEYOUT,VALUEOUT> 
-       extends TaskAttemptContextImpl 
-       implements TaskInputOutputContext<KEYIN, VALUEIN, KEYOUT, VALUEOUT> {
-  private RecordWriter<KEYOUT,VALUEOUT> output;
-  private OutputCommitter committer;
-
-  public TaskInputOutputContextImpl(Configuration conf, TaskAttemptID taskid,
-                                    RecordWriter<KEYOUT,VALUEOUT> output,
-                                    OutputCommitter committer,
-                                    TezTaskContext context) {
-    super(conf, taskid, context);
-    this.output = output;
-    this.committer = committer;
-  }
-
-  /**
-   * Advance to the next key, value pair, returning null if at end.
-   * @return the key object that was read into, or null if no more
-   */
-  public abstract 
-  boolean nextKeyValue() throws IOException, InterruptedException;
- 
-  /**
-   * Get the current key.
-   * @return the current key object or null if there isn't one
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public abstract 
-  KEYIN getCurrentKey() throws IOException, InterruptedException;
-
-  /**
-   * Get the current value.
-   * @return the value object that was read into
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public abstract VALUEIN getCurrentValue() throws IOException, 
-                                                   InterruptedException;
-
-  /**
-   * Generate an output key/value pair.
-   */
-  public void write(KEYOUT key, VALUEOUT value
-                    ) throws IOException, InterruptedException {
-    output.write(key, value);
-  }
-
-  public OutputCommitter getOutputCommitter() {
-    return committer;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/3d609458/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java
new file mode 100644
index 0000000..2d230d6
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/ShuffledMergedInputLegacy.java
@@ -0,0 +1,29 @@
+/**
+ * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
+ * intermediate sorted data, merges them and provides key/<values> to the
+ * consumer.
+ * 
+ * The Copy and Merge will be triggered by the initialization - which is handled
+ * by the Tez framework. Input is not consumable until the Copy and Merge are
+ * complete. Methods are provided to check for this, as well as to wait for
+ * completion. Attempting to get a reader on a non-complete input will block.
+ * 
+ */
+
+package org.apache.tez.mapreduce.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.engine.lib.input.ShuffledMergedInput;
+
+public class ShuffledMergedInputLegacy extends ShuffledMergedInput {
+
+  @Private
+  public TezRawKeyValueIterator getIterator() throws IOException, InterruptedException {
+    // wait for input so that iterator is available
+    waitForInputReady();
+    return rawIter;
+  }
+}


[15/20] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/GenericCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/GenericCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/GenericCounter.java
deleted file mode 100644
index 5477606..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/GenericCounter.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * A generic counter implementation
- */
-@InterfaceAudience.Private
-public class GenericCounter extends AbstractCounter {
-
-  private String name;
-  private String displayName;
-  private long value = 0;
-
-  public GenericCounter() {
-    // mostly for readFields
-  }
-
-  public GenericCounter(String name, String displayName) {
-    this.name = name;
-    this.displayName = displayName;
-  }
-
-  public GenericCounter(String name, String displayName, long value) {
-    this.name = name;
-    this.displayName = displayName;
-    this.value = value;
-  }
-
-  @Override @Deprecated
-  public synchronized void setDisplayName(String displayName) {
-    this.displayName = displayName;
-  }
-
-  @Override
-  public synchronized void readFields(DataInput in) throws IOException {
-    name = Text.readString(in);
-    displayName = in.readBoolean() ? Text.readString(in) : name;
-    value = WritableUtils.readVLong(in);
-  }
-
-  /**
-   * GenericCounter ::= keyName isDistinctDisplayName [displayName] value
-   */
-  @Override
-  public synchronized void write(DataOutput out) throws IOException {
-    Text.writeString(out, name);
-    boolean distinctDisplayName = ! name.equals(displayName);
-    out.writeBoolean(distinctDisplayName);
-    if (distinctDisplayName) {
-      Text.writeString(out, displayName);
-    }
-    WritableUtils.writeVLong(out, value);
-  }
-
-  @Override
-  public synchronized String getName() {
-    return name;
-  }
-
-  @Override
-  public synchronized String getDisplayName() {
-    return displayName;
-  }
-
-  @Override
-  public synchronized long getValue() {
-    return value;
-  }
-
-  @Override
-  public synchronized void setValue(long value) {
-    this.value = value;
-  }
-
-  @Override
-  public synchronized void increment(long incr) {
-    value += incr;
-  }
-
-  @Override
-  public TezCounter getUnderlyingCounter() {
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/JobCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/JobCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/JobCounter.java
deleted file mode 100644
index 1eb2be8..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/JobCounter.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-// Per-job counters
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public enum JobCounter {
-  NUM_FAILED_MAPS, 
-  NUM_FAILED_REDUCES,
-  NUM_KILLED_MAPS,
-  NUM_KILLED_REDUCES,
-  TOTAL_LAUNCHED_MAPS,
-  TOTAL_LAUNCHED_REDUCES,
-  OTHER_LOCAL_MAPS,
-  DATA_LOCAL_MAPS,
-  RACK_LOCAL_MAPS,
-  SLOTS_MILLIS_MAPS,
-  SLOTS_MILLIS_REDUCES,
-  FALLOW_SLOTS_MILLIS_MAPS,
-  FALLOW_SLOTS_MILLIS_REDUCES,
-  TOTAL_LAUNCHED_UBERTASKS,
-  NUM_UBER_SUBMAPS,
-  NUM_UBER_SUBREDUCES,
-  NUM_FAILED_UBERTASKS
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/LimitExceededException.java b/tez-common/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
deleted file mode 100644
index e50bd81..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/LimitExceededException.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-@InterfaceAudience.Private
-public class LimitExceededException extends RuntimeException {
-
-  private static final long serialVersionUID = 1L;
-
-  public LimitExceededException(String msg) {
-    super(msg);
-  }
-
-  // Only allows chaining of related exceptions
-  public LimitExceededException(LimitExceededException cause) {
-    super(cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/Limits.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/Limits.java b/tez-common/src/main/java/org/apache/tez/common/counters/Limits.java
deleted file mode 100644
index aacce87..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/Limits.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.TezJobConfig;
-
-@InterfaceAudience.Private
-public class Limits {
-
-  private static Configuration conf = null;
-  private static int GROUP_NAME_MAX;
-  private static int COUNTER_NAME_MAX;
-  private static int GROUPS_MAX;
-  private static int COUNTERS_MAX;
-  private static boolean initialized = false;
-
-  private static synchronized void ensureInitialized() {
-    if (initialized) {
-      return;
-    }
-    if (conf == null) {
-      conf = new Configuration();
-    }
-    GROUP_NAME_MAX =
-        conf.getInt(TezJobConfig.COUNTER_GROUP_NAME_MAX_KEY,
-            TezJobConfig.COUNTER_GROUP_NAME_MAX_DEFAULT);
-    COUNTER_NAME_MAX =
-        conf.getInt(TezJobConfig.COUNTER_NAME_MAX_KEY,
-            TezJobConfig.COUNTER_NAME_MAX_DEFAULT);
-    GROUPS_MAX =
-        conf.getInt(TezJobConfig.COUNTER_GROUPS_MAX_KEY,
-            TezJobConfig.COUNTER_GROUPS_MAX_DEFAULT);
-    COUNTERS_MAX =
-        conf.getInt(TezJobConfig.COUNTERS_MAX_KEY, TezJobConfig.
-            COUNTERS_MAX_DEFAULT);
-    initialized = true;
-  }
-
-  private int totalCounters;
-  private LimitExceededException firstViolation;
-
-  public static String filterName(String name, int maxLen) {
-    return name.length() > maxLen ? name.substring(0, maxLen - 1) : name;
-  }
-
-  public static String filterCounterName(String name) {
-    ensureInitialized();
-    return filterName(name, COUNTER_NAME_MAX);
-  }
-
-  public static String filterGroupName(String name) {
-    ensureInitialized();
-    return filterName(name, GROUP_NAME_MAX);
-  }
-
-  public synchronized void checkCounters(int size) {
-    ensureInitialized();
-    if (firstViolation != null) {
-      throw new LimitExceededException(firstViolation);
-    }
-    if (size > COUNTERS_MAX) {
-      firstViolation = new LimitExceededException("Too many counters: "+ size +
-                                                  " max="+ COUNTERS_MAX);
-      throw firstViolation;
-    }
-  }
-
-  public synchronized void incrCounters() {
-    checkCounters(totalCounters + 1);
-    ++totalCounters;
-  }
-
-  public synchronized void checkGroups(int size) {
-    ensureInitialized();
-    if (firstViolation != null) {
-      throw new LimitExceededException(firstViolation);
-    }
-    if (size > GROUPS_MAX) {
-      firstViolation = new LimitExceededException("Too many counter groups: "+
-                                                  size +" max="+ GROUPS_MAX);
-    }
-  }
-
-  public synchronized LimitExceededException violation() {
-    return firstViolation;
-  }
-
-  public synchronized static void setConfiguration(Configuration conf) {
-    if (Limits.conf == null && conf != null) {
-      Limits.conf = conf;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/ResourceBundles.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/ResourceBundles.java b/tez-common/src/main/java/org/apache/tez/common/counters/ResourceBundles.java
deleted file mode 100644
index 8113cab..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/ResourceBundles.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import java.util.Locale;
-import java.util.ResourceBundle;
-import java.util.MissingResourceException;
-
-/**
- * Helper class to handle resource bundles in a saner way
- */
-public class ResourceBundles {
-
-  /**
-   * Get a resource bundle
-   * @param bundleName of the resource
-   * @return the resource bundle
-   * @throws MissingResourceException
-   */
-  public static ResourceBundle getBundle(String bundleName) {
-    return ResourceBundle.getBundle(bundleName.replace('$', '_'),
-        Locale.getDefault(), Thread.currentThread().getContextClassLoader());
-  }
-
-  /**
-   * Get a resource given bundle name and key
-   * @param <T> type of the resource
-   * @param bundleName name of the resource bundle
-   * @param key to lookup the resource
-   * @param suffix for the key to lookup
-   * @param defaultValue of the resource
-   * @return the resource or the defaultValue
-   * @throws ClassCastException if the resource found doesn't match T
-   */
-  @SuppressWarnings("unchecked")
-  public static synchronized <T> T getValue(String bundleName, String key,
-                                            String suffix, T defaultValue) {
-    T value;
-    try {
-      ResourceBundle bundle = getBundle(bundleName);
-      value = (T) bundle.getObject(getLookupKey(key, suffix));
-      if (value != null) {
-        return value;
-      }
-    }
-    catch (Exception e) {
-      // Ignore
-    }
-    return defaultValue;
-  }
-
-  private static String getLookupKey(String key, String suffix) {
-    if (suffix == null || suffix.isEmpty()) return key;
-    return key + suffix;
-  }
-
-  /**
-   * Get the counter group display name
-   * @param group the group name to lookup
-   * @param defaultValue of the group
-   * @return the group display name
-   */
-  public static String getCounterGroupName(String group, String defaultValue) {
-    return getValue(group, "CounterGroupName", "", defaultValue);
-  }
-
-  /**
-   * Get the counter display name
-   * @param group the counter group name for the counter
-   * @param counter the counter name to lookup
-   * @param defaultValue of the counter
-   * @return the counter display name
-   */
-  public static String getCounterName(String group, String counter,
-                                      String defaultValue) {
-    return getValue(group, counter, ".name", defaultValue);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/TaskCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/TaskCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/TaskCounter.java
deleted file mode 100644
index b6fca27..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/TaskCounter.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-// TODO TEZAM5 For MR compatibility, a conversion from tez.TaskCounters to
-// mapreduce.TaskCounters will likely be required somewhere.
-// Similarly for FileSystemCounters and others.
-
-// Counters used by Task classes
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public enum TaskCounter {
-  // TODO Eventually, rename counters to be non-MR specific and map them to MR equivalent.
-  MAP_INPUT_RECORDS, 
-  MAP_OUTPUT_RECORDS,
-  MAP_SKIPPED_RECORDS,
-  MAP_OUTPUT_BYTES,
-  MAP_OUTPUT_MATERIALIZED_BYTES,
-  SPLIT_RAW_BYTES,
-  COMBINE_INPUT_RECORDS,
-  COMBINE_OUTPUT_RECORDS,
-  REDUCE_INPUT_GROUPS,
-  REDUCE_SHUFFLE_BYTES,
-  REDUCE_INPUT_RECORDS,
-  REDUCE_OUTPUT_RECORDS,
-  REDUCE_SKIPPED_GROUPS,
-  REDUCE_SKIPPED_RECORDS,
-  SPILLED_RECORDS,
-  SHUFFLED_MAPS, 
-  FAILED_SHUFFLE,
-  MERGED_MAP_OUTPUTS,
-  GC_TIME_MILLIS,
-  CPU_MILLISECONDS,
-  PHYSICAL_MEMORY_BYTES,
-  VIRTUAL_MEMORY_BYTES,
-  COMMITTED_HEAP_BYTES,
-  
-  INPUT_RECORDS, 
-  OUTPUT_RECORDS,
-  SKIPPED_RECORDS,
-  OUTPUT_BYTES,
-  OUTPUT_MATERIALIZED_BYTES,
-  INPUT_GROUPS,
-  SHUFFLE_BYTES,
-  SHUFFLED_TASKS, 
-  MERGED_TASK_OUTPUTS,
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/TezCounter.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/TezCounter.java b/tez-common/src/main/java/org/apache/tez/common/counters/TezCounter.java
deleted file mode 100644
index 394c820..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/TezCounter.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.io.Writable;
-
-/**
- * A named counter that tracks the progress of a map/reduce job.
- *
- * <p><code>Counters</code> represent global counters, defined either by the
- * Map-Reduce framework or applications. Each <code>Counter</code> is named by
- * an {@link Enum} and has a long for the value.</p>
- *
- * <p><code>Counters</code> are bunched into Groups, each comprising of
- * counters from a particular <code>Enum</code> class.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public interface TezCounter extends Writable {
-
-  /**
-   * Set the display name of the counter
-   * @param displayName of the counter
-   * @deprecated (and no-op by default)
-   */
-  @Deprecated
-  void setDisplayName(String displayName);
-
-  /**
-   * @return the name of the counter
-   */
-  String getName();
-
-  /**
-   * Get the display name of the counter.
-   * @return the user facing name of the counter
-   */
-  String getDisplayName();
-
-  /**
-   * What is the current value of this counter?
-   * @return the current value
-   */
-  long getValue();
-
-  /**
-   * Set this counter by the given value
-   * @param value the value to set
-   */
-  void setValue(long value);
-
-  /**
-   * Increment this counter by the given value
-   * @param incr the value to increase this counter by
-   */
-  void increment(long incr);
- 
-  /**
-   * Return the underlying object if this is a facade.
-   * @return the undelying object.
-   */
-  @Private
-  TezCounter getUnderlyingCounter();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/common/counters/TezCounters.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/common/counters/TezCounters.java b/tez-common/src/main/java/org/apache/tez/common/counters/TezCounters.java
deleted file mode 100644
index 1c9521a..0000000
--- a/tez-common/src/main/java/org/apache/tez/common/counters/TezCounters.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.tez.common.counters;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * <p><code>Counters</code> holds per job/task counters, defined either by the
- * Map-Reduce framework or applications. Each <code>Counter</code> can be of
- * any {@link Enum} type.</p>
- *
- * <p><code>Counters</code> are bunched into {@link CounterGroup}s, each
- * comprising of counters from a particular <code>Enum</code> class.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class TezCounters extends AbstractCounters<TezCounter, CounterGroup> {
-
-  // Mix framework group implementation into CounterGroup interface
-  private static class FrameworkGroupImpl<T extends Enum<T>>
-      extends FrameworkCounterGroup<T, TezCounter> implements CounterGroup {
-
-    FrameworkGroupImpl(Class<T> cls) {
-      super(cls);
-    }
-
-    @Override
-    protected FrameworkCounter<T> newCounter(T key) {
-      return new FrameworkCounter<T>(key, getName());
-    }
-
-    @Override
-    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
-      return this;
-    }
-  }
-
-  // Mix generic group implementation into CounterGroup interface
-  // and provide some mandatory group factory methods.
-  private static class GenericGroup extends AbstractCounterGroup<TezCounter>
-      implements CounterGroup {
-
-    GenericGroup(String name, String displayName, Limits limits) {
-      super(name, displayName, limits);
-    }
-
-    @Override
-    protected TezCounter newCounter(String name, String displayName, long value) {
-      return new GenericCounter(name, displayName, value);
-    }
-
-    @Override
-    protected TezCounter newCounter() {
-      return new GenericCounter();
-    }
-
-    @Override
-    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
-      return this;
-    }
-  }
-
-  // Mix file system group implementation into the CounterGroup interface
-  private static class FileSystemGroup extends FileSystemCounterGroup<TezCounter>
-      implements CounterGroup {
-
-    @Override
-    protected TezCounter newCounter(String scheme, FileSystemCounter key) {
-      return new FSCounter(scheme, key);
-    }
-
-    @Override
-    public CounterGroupBase<TezCounter> getUnderlyingGroup() {
-      return this;
-    }
-  }
-
-  /**
-   * Provide factory methods for counter group factory implementation.
-   * See also the GroupFactory in
-   *  {@link org.apache.hadoop.TezCounters.Counters mapred.Counters}
-   */
-  private static class GroupFactory
-      extends CounterGroupFactory<TezCounter, CounterGroup> {
-
-    @Override
-    protected <T extends Enum<T>>
-    FrameworkGroupFactory<CounterGroup>
-        newFrameworkGroupFactory(final Class<T> cls) {
-      return new FrameworkGroupFactory<CounterGroup>() {
-        @Override public CounterGroup newGroup(String name) {
-          return new FrameworkGroupImpl<T>(cls); // impl in this package
-        }
-      };
-    }
-
-    @Override
-    protected CounterGroup newGenericGroup(String name, String displayName,
-                                           Limits limits) {
-      return new GenericGroup(name, displayName, limits);
-    }
-
-    @Override
-    protected CounterGroup newFileSystemGroup() {
-      return new FileSystemGroup();
-    }
-  }
-
-  private static final GroupFactory groupFactory = new GroupFactory();
-
-  /**
-   * Default constructor
-   */
-  public TezCounters() {
-    super(groupFactory);
-  }
-
-  /**
-   * Construct the Counters object from the another counters object
-   * @param <C> the type of counter
-   * @param <G> the type of counter group
-   * @param counters the old counters object
-   */
-  public <C extends TezCounter, G extends CounterGroupBase<C>>
-  TezCounters(AbstractCounters<C, G> counters) {
-    super(counters, groupFactory);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-common/src/main/java/org/apache/tez/records/TezContainerId.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/records/TezContainerId.java b/tez-common/src/main/java/org/apache/tez/records/TezContainerId.java
deleted file mode 100644
index 193b979..0000000
--- a/tez-common/src/main/java/org/apache/tez/records/TezContainerId.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with this
- * work for additional information regarding copyright ownership. The ASF
- * licenses this file to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
- * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
- */
-
-package org.apache.tez.records;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-
-//TODO EVENTUALLY Once everything is on PB, get rid of this.
-//Alternately have the PB interfaces implement Writable.
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class TezContainerId implements Writable {
-
-  private ContainerId containerId;
-
-  public TezContainerId() {
-  }
-  
-  public TezContainerId(ContainerId containerId) {
-    this.containerId = containerId;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(containerId.getApplicationAttemptId().getApplicationId()
-        .getClusterTimestamp());
-    out.writeInt(containerId.getApplicationAttemptId().getApplicationId()
-        .getId());
-    out.writeInt(containerId.getApplicationAttemptId().getAttemptId());
-    out.writeInt(containerId.getId());
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    long timestamp = in.readLong();
-    int appId = in.readInt();
-    int appAttemptId = in.readInt();
-    int id = in.readInt();
-
-    ApplicationId applicationId = ApplicationId.newInstance(timestamp, appId);
-    ApplicationAttemptId applicationAttemptId = ApplicationAttemptId
-        .newInstance(applicationId, appAttemptId);
-
-    this.containerId = ContainerId.newInstance(applicationAttemptId, id);
-  }
-
-  @Override
-  public String toString() {
-    return containerId.toString();
-  }
-  
-  public ContainerId getContainerId() {
-    return this.containerId;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-dag-api/findbugs-exclude.xml b/tez-dag-api/findbugs-exclude.xml
deleted file mode 100644
index 5b11308..0000000
--- a/tez-dag-api/findbugs-exclude.xml
+++ /dev/null
@@ -1,16 +0,0 @@
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-<FindBugsFilter>
-
-</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dag-api/pom.xml b/tez-dag-api/pom.xml
deleted file mode 100644
index 51d6c73..0000000
--- a/tez-dag-api/pom.xml
+++ /dev/null
@@ -1,88 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.tez</groupId>
-    <artifactId>tez</artifactId>
-    <version>0.2.0-SNAPSHOT</version>
-  </parent>
-  <artifactId>tez-dag-api</artifactId>
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-client</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-common</artifactId>
-    </dependency>
-    <dependency>
-     <groupId>com.google.protobuf</groupId>
-     <artifactId>protobuf-java</artifactId>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-        <configuration>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.hadoop</groupId>
-        <artifactId>hadoop-maven-plugins</artifactId>
-        <executions>
-          <execution>
-            <id>compile-protoc</id>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>protoc</goal>
-            </goals>
-            <configuration>
-              <protocVersion>${protobuf.version}</protocVersion>
-              <protocCommand>${protoc.path}</protocCommand>
-              <imports>
-                <param>${basedir}/src/main/proto</param>
-              </imports>
-              <source>
-                <directory>${basedir}/src/main/proto</directory>
-                <includes>
-                  <include>DAGApiRecords.proto</include>
-                  <include>DAGClientAMProtocol.proto</include>
-                </includes>
-              </source>
-              <output>${project.build.directory}/generated-sources/java</output>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-
-</project>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/client/AMConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/client/AMConfiguration.java b/tez-dag-api/src/main/java/org/apache/tez/client/AMConfiguration.java
deleted file mode 100644
index f452c74..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/client/AMConfiguration.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.client;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezUncheckedException;
-
-public class AMConfiguration {
-
-  private final Path stagingDir;
-  private final String queueName;
-  private final Map<String, String> env;
-  private final Map<String, LocalResource> localResources;
-  private final TezConfiguration amConf;
-  private final Credentials credentials;
-
-  public AMConfiguration(String queueName, Map<String, String> env,
-      Map<String, LocalResource> localResources,
-      TezConfiguration conf, Credentials credentials) {
-    this.queueName = queueName;
-    if (conf != null) {
-      this.amConf = conf;
-    } else {
-      this.amConf = new TezConfiguration();
-    }
-
-    if (env != null) {
-      this.env = env;
-    } else {
-      this.env = new HashMap<String, String>(0);
-    }
-    this.localResources = localResources;
-    String stagingDirStr = amConf.get(TezConfiguration.TEZ_AM_STAGING_DIR);
-    if (stagingDirStr == null || stagingDirStr.isEmpty()) {
-      throw new TezUncheckedException("Staging directory for AM resources"
-          + " not specified in config"
-          + ", property=" + TezConfiguration.TEZ_AM_STAGING_DIR);
-    }
-    try {
-      FileSystem fs = FileSystem.get(amConf);
-      this.stagingDir = fs.resolvePath(new Path(stagingDirStr));
-    } catch (IOException e) {
-      throw new TezUncheckedException(e);
-    }
-    this.credentials = credentials;
-  }
-
-  public Path getStagingDir() {
-    return stagingDir;
-  }
-
-  public String getQueueName() {
-    return queueName;
-  }
-
-  public Map<String, String> getEnv() {
-    return env;
-  }
-
-  public Map<String, LocalResource> getLocalResources() {
-    return localResources;
-  }
-
-  public TezConfiguration getAMConf() {
-    return amConf;
-  }
-
-  public Credentials getCredentials() {
-    return credentials;
-  }
-
-  public void isCompatible(AMConfiguration other) {
-    // TODO implement
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/client/TezClient.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-dag-api/src/main/java/org/apache/tez/client/TezClient.java
deleted file mode 100644
index df260ec..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/client/TezClient.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.tez.client;
-
-import java.io.IOException;
-import java.text.NumberFormat;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.client.api.YarnClient;
-import org.apache.hadoop.yarn.client.api.impl.YarnClientImpl;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.tez.dag.api.DAG;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezException;
-import org.apache.tez.dag.api.client.DAGClient;
-import org.apache.tez.dag.api.client.rpc.DAGClientRPCImpl;
-
-public class TezClient {
-  private static final Log LOG = LogFactory.getLog(TezClient.class);
-
-  private final TezConfiguration conf;
-  private final YarnConfiguration yarnConf;
-  private YarnClient yarnClient;
-  Map<String, LocalResource> tezJarResources = null;
-
-  /**
-   * <p>
-   * Create an instance of the TezClient which will be used to communicate with
-   * a specific instance of YARN, or TezService when that exists.
-   * </p>
-   * <p>
-   * Separate instances of TezClient should be created to communicate with
-   * different instances of YARN
-   * </p>
-   *
-   * @param conf
-   *          the configuration which will be used to establish which YARN or
-   *          Tez service instance this client is associated with.
-   */
-  public TezClient(TezConfiguration conf) {
-    this.conf = conf;
-    this.yarnConf = new YarnConfiguration(conf);
-    yarnClient = new YarnClientImpl();
-    yarnClient.init(yarnConf);
-    yarnClient.start();
-  }
-
-
-  public DAGClient submitDAGApplication(DAG dag, AMConfiguration amConfig)
-      throws TezException, IOException {
-    ApplicationId appId = createApplication();
-    return submitDAGApplication(appId, dag, amConfig);
-  }
-
-  @Private
-  // To be used only by YarnRunner
-  public DAGClient submitDAGApplication(ApplicationId appId,
-      DAG dag, AMConfiguration amConfig)
-          throws TezException, IOException {
-    try {
-      ApplicationSubmissionContext appContext =
-          TezClientUtils.createApplicationSubmissionContext(conf, appId, dag,
-              dag.getName(), amConfig, getTezJarResources());
-      LOG.info("Submitting DAG to YARN"
-          + ", applicationId=" + appId);
-      yarnClient.submitApplication(appContext);
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-    return getDAGClient(appId);
-  }
-
-  /**
-   * Create a new YARN application
-   * @return <code>ApplicationId</code> for the new YARN application
-   * @throws YarnException
-   * @throws IOException
-   */
-  public ApplicationId createApplication() throws TezException, IOException {
-    try {
-      return yarnClient.createApplication().
-          getNewApplicationResponse().getApplicationId();
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-  }
-
-  private synchronized Map<String, LocalResource> getTezJarResources()
-      throws IOException {
-    if (tezJarResources == null) {
-      tezJarResources = TezClientUtils.setupTezJarsLocalResources(conf);
-    }
-    return tezJarResources;
-  }
-
-  @Private
-  public DAGClient getDAGClient(ApplicationId appId)
-      throws IOException, TezException {
-      return new DAGClientRPCImpl(appId, getDefaultTezDAGID(appId),
-                                   conf);
-  }
-
-  // DO NOT CHANGE THIS. This code is replicated from TezDAGID.java
-  private static final char SEPARATOR = '_';
-  private static final String DAG = "dag";
-  private static final NumberFormat idFormat = NumberFormat.getInstance();
-  static {
-    idFormat.setGroupingUsed(false);
-    idFormat.setMinimumIntegerDigits(6);
-  }
-
-  String getDefaultTezDAGID(ApplicationId appId) {
-     return (new StringBuilder(DAG)).append(SEPARATOR).
-                   append(appId.getClusterTimestamp()).
-                   append(SEPARATOR).
-                   append(appId.getId()).
-                   append(SEPARATOR).
-                   append(idFormat.format(1)).toString();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-dag-api/src/main/java/org/apache/tez/client/TezClientUtils.java
deleted file mode 100644
index 7c6a5ed..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ /dev/null
@@ -1,560 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.client;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.Vector;
-import java.util.Map.Entry;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.client.api.YarnClient;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.util.Apps;
-import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.hadoop.yarn.util.Records;
-import org.apache.log4j.Level;
-import org.apache.tez.dag.api.DAG;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezConstants;
-import org.apache.tez.dag.api.TezException;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.api.Vertex;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB;
-import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
-import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
-import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
-
-import com.google.common.annotations.VisibleForTesting;
-
-public class TezClientUtils {
-
-  private static Log LOG = LogFactory.getLog(TezClientUtils.class);
-
-  public static final FsPermission TEZ_AM_DIR_PERMISSION =
-      FsPermission.createImmutable((short) 0700); // rwx--------
-  public static final FsPermission TEZ_AM_FILE_PERMISSION =
-      FsPermission.createImmutable((short) 0644); // rw-r--r--
-
-  private static final int UTF8_CHUNK_SIZE = 16 * 1024;
-
-  /**
-   * Setup LocalResource map for Tez jars based on provided Configuration
-   * @param conf Configuration to use to access Tez jars' locations
-   * @return Map of LocalResources to use when launching Tez AM
-   * @throws IOException
-   */
-  static Map<String, LocalResource> setupTezJarsLocalResources(
-      TezConfiguration conf)
-      throws IOException {
-    Map<String, LocalResource> tezJarResources =
-        new TreeMap<String, LocalResource>();
-    if (conf.getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false)) {
-      return tezJarResources;
-    }
-
-    // Add tez jars to local resource
-    String[] tezJarUris = conf.getStrings(
-        TezConfiguration.TEZ_LIB_URIS);
-    if (tezJarUris == null
-        || tezJarUris.length == 0) {
-      throw new TezUncheckedException("Invalid configuration of tez jars"
-          + ", " + TezConfiguration.TEZ_LIB_URIS
-          + " is not defined in the configurartion");
-    }
-
-    for (String tezJarUri : tezJarUris) {
-      URI uri;
-      try {
-        uri = new URI(tezJarUri.trim());
-      } catch (URISyntaxException e) {
-        String message = "Invalid URI defined in configuration for"
-            + " location of TEZ jars. providedURI=" + tezJarUri;
-        LOG.error(message);
-        throw new TezUncheckedException(message, e);
-      }
-      if (!uri.isAbsolute()) {
-        String message = "Non-absolute URI defined in configuration for"
-            + " location of TEZ jars. providedURI=" + tezJarUri;
-        LOG.error(message);
-        throw new TezUncheckedException(message);
-      }
-      Path p = new Path(uri);
-      FileSystem pathfs = p.getFileSystem(conf);
-      RemoteIterator<LocatedFileStatus> iter = pathfs.listFiles(p, false);
-      while (iter.hasNext()) {
-        LocatedFileStatus fStatus = iter.next();
-        String rsrcName = fStatus.getPath().getName();
-        // FIXME currently not checking for duplicates due to quirks
-        // in assembly generation
-        if (tezJarResources.containsKey(rsrcName)) {
-          String message = "Duplicate resource found"
-              + ", resourceName=" + rsrcName
-              + ", existingPath=" +
-              tezJarResources.get(rsrcName).getResource().toString()
-              + ", newPath=" + fStatus.getPath();
-          LOG.warn(message);
-          // throw new TezUncheckedException(message);
-        }
-        tezJarResources.put(rsrcName,
-            LocalResource.newInstance(
-                ConverterUtils.getYarnUrlFromPath(fStatus.getPath()),
-                LocalResourceType.FILE,
-                LocalResourceVisibility.PUBLIC,
-                fStatus.getLen(),
-                fStatus.getModificationTime()));
-      }
-    }
-    if (tezJarResources.isEmpty()) {
-      LOG.warn("No tez jars found in configured locations"
-          + ". Ignoring for now. Errors may occur");
-    }
-    return tezJarResources;
-  }
-
-  /**
-   * Verify or create the Staging area directory on the configured Filesystem
-   * @param stagingArea Staging area directory path
-   * @return
-   * @throws IOException
-   */
-  public static FileSystem ensureStagingDirExists(Configuration conf,
-      Path stagingArea)
-      throws IOException {
-    FileSystem fs = stagingArea.getFileSystem(conf);
-    String realUser;
-    String currentUser;
-    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
-    realUser = ugi.getShortUserName();
-    currentUser = UserGroupInformation.getCurrentUser().getShortUserName();
-    if (fs.exists(stagingArea)) {
-      FileStatus fsStatus = fs.getFileStatus(stagingArea);
-      String owner = fsStatus.getOwner();
-      if (!(owner.equals(currentUser) || owner.equals(realUser))) {
-        throw new IOException("The ownership on the staging directory "
-            + stagingArea + " is not as expected. " + "It is owned by " + owner
-            + ". The directory must " + "be owned by the submitter "
-            + currentUser + " or " + "by " + realUser);
-      }
-      if (!fsStatus.getPermission().equals(TEZ_AM_DIR_PERMISSION)) {
-        LOG.info("Permissions on staging directory " + stagingArea + " are "
-            + "incorrect: " + fsStatus.getPermission()
-            + ". Fixing permissions " + "to correct value "
-            + TEZ_AM_DIR_PERMISSION);
-        fs.setPermission(stagingArea, TEZ_AM_DIR_PERMISSION);
-      }
-    } else {
-      fs.mkdirs(stagingArea, new FsPermission(TEZ_AM_DIR_PERMISSION));
-    }
-    return fs;
-  }
-
-  /**
-   * Create an ApplicationSubmissionContext to launch a Tez AM
-   * @param conf
-   * @param appId
-   * @param dag
-   * @param appStagingDir
-   * @param ts
-   * @param amQueueName
-   * @param amName
-   * @param amArgs
-   * @param amEnv
-   * @param amLocalResources
-   * @param appConf
-   * @return
-   * @throws IOException
-   * @throws YarnException
-   */
-  static ApplicationSubmissionContext createApplicationSubmissionContext(
-      Configuration conf, ApplicationId appId, DAG dag, String amName,
-      AMConfiguration amConfig,
-      Map<String, LocalResource> tezJarResources)
-          throws IOException, YarnException{
-
-    FileSystem fs = TezClientUtils.ensureStagingDirExists(conf,
-        amConfig.getStagingDir());
-
-    // Setup resource requirements
-    Resource capability = Records.newRecord(Resource.class);
-    capability.setMemory(
-        amConfig.getAMConf().getInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB,
-            TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB_DEFAULT));
-    capability.setVirtualCores(
-        amConfig.getAMConf().getInt(TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES,
-            TezConfiguration.TEZ_AM_RESOURCE_CPU_VCORES_DEFAULT));
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("AppMaster capability = " + capability);
-    }
-
-    ByteBuffer securityTokens = null;
-    // Setup security tokens
-    if (amConfig.getCredentials() != null) {
-      DataOutputBuffer dob = new DataOutputBuffer();
-      amConfig.getCredentials().writeTokenStorageToStream(dob);
-      securityTokens = ByteBuffer.wrap(dob.getData(), 0,
-          dob.getLength());
-    }
-
-    // Setup the command to run the AM
-    List<String> vargs = new ArrayList<String>(8);
-    vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
-
-    String amLogLevel = amConfig.getAMConf().get(
-        TezConfiguration.TEZ_AM_LOG_LEVEL,
-        TezConfiguration.TEZ_AM_LOG_LEVEL_DEFAULT);
-    addLog4jSystemProperties(amLogLevel, vargs);
-
-    vargs.add(amConfig.getAMConf().get(TezConfiguration.TEZ_AM_JAVA_OPTS,
-        TezConfiguration.DEFAULT_TEZ_AM_JAVA_OPTS));
-
-    vargs.add(TezConfiguration.TEZ_APPLICATION_MASTER_CLASS);
-    vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
-        File.separator + ApplicationConstants.STDOUT);
-    vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
-        File.separator + ApplicationConstants.STDERR);
-
-
-    Vector<String> vargsFinal = new Vector<String>(8);
-    // Final command
-    StringBuilder mergedCommand = new StringBuilder();
-    for (CharSequence str : vargs) {
-      mergedCommand.append(str).append(" ");
-    }
-    vargsFinal.add(mergedCommand.toString());
-
-    LOG.debug("Command to launch container for ApplicationMaster is : "
-        + mergedCommand);
-
-    // Setup the CLASSPATH in environment
-    // i.e. add { Hadoop jars, job jar, CWD } to classpath.
-    Map<String, String> environment = new HashMap<String, String>();
-
-    boolean isMiniCluster =
-        conf.getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false);
-    if (isMiniCluster) {
-      Apps.addToEnvironment(environment, Environment.CLASSPATH.name(),
-          System.getProperty("java.class.path"));
-    }
-
-    Apps.addToEnvironment(environment,
-        Environment.CLASSPATH.name(),
-        Environment.PWD.$());
-
-    Apps.addToEnvironment(environment,
-        Environment.CLASSPATH.name(),
-        Environment.PWD.$() + File.separator + "*");
-
-    // Add YARN/COMMON/HDFS jars to path
-    if (!isMiniCluster) {
-      for (String c : conf.getStrings(
-          YarnConfiguration.YARN_APPLICATION_CLASSPATH,
-          YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH)) {
-        Apps.addToEnvironment(environment, Environment.CLASSPATH.name(),
-            c.trim());
-      }
-    }
-
-    if (amConfig.getEnv() != null) {
-      for (Map.Entry<String, String> entry : amConfig.getEnv().entrySet()) {
-        Apps.addToEnvironment(environment, entry.getKey(), entry.getValue());
-      }
-    }
-
-    Map<String, LocalResource> localResources =
-        new TreeMap<String, LocalResource>();
-
-    if (amConfig.getLocalResources() != null) {
-      localResources.putAll(amConfig.getLocalResources());
-    }
-    localResources.putAll(tezJarResources);
-
-    // emit conf as PB file
-    Configuration finalTezConf = createFinalTezConfForApp(amConfig.getAMConf());
-    Path binaryConfPath =  new Path(amConfig.getStagingDir(),
-        TezConfiguration.TEZ_PB_BINARY_CONF_NAME + "." + appId.toString());
-    FSDataOutputStream amConfPBOutBinaryStream = null;
-    try {
-      ConfigurationProto.Builder confProtoBuilder =
-          ConfigurationProto.newBuilder();
-      Iterator<Entry<String, String>> iter = finalTezConf.iterator();
-      while (iter.hasNext()) {
-        Entry<String, String> entry = iter.next();
-        PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder();
-        kvp.setKey(entry.getKey());
-        kvp.setValue(entry.getValue());
-        confProtoBuilder.addConfKeyValues(kvp);
-      }
-      //binary output
-      amConfPBOutBinaryStream = FileSystem.create(fs, binaryConfPath,
-          new FsPermission(TEZ_AM_FILE_PERMISSION));
-      confProtoBuilder.build().writeTo(amConfPBOutBinaryStream);
-    } finally {
-      if(amConfPBOutBinaryStream != null){
-        amConfPBOutBinaryStream.close();
-      }
-    }
-
-    LocalResource binaryConfLRsrc =
-        TezClientUtils.createLocalResource(fs,
-            binaryConfPath, LocalResourceType.FILE,
-            LocalResourceVisibility.APPLICATION);
-    localResources.put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
-        binaryConfLRsrc);
-
-    if(dag != null) {
-      // Add tez jars to vertices too
-      for (Vertex v : dag.getVertices()) {
-        v.getTaskLocalResources().putAll(tezJarResources);
-        v.getTaskLocalResources().put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
-            binaryConfLRsrc);
-      }
-
-      // emit protobuf DAG file style
-      Path binaryPath =  new Path(amConfig.getStagingDir(),
-          TezConfiguration.TEZ_PB_PLAN_BINARY_NAME + "." + appId.toString());
-      amConfig.getAMConf().set(TezConfiguration.TEZ_AM_PLAN_REMOTE_PATH,
-          binaryPath.toUri().toString());
-
-      DAGPlan dagPB = dag.createDag(null);
-
-      FSDataOutputStream dagPBOutBinaryStream = null;
-
-      try {
-        //binary output
-        dagPBOutBinaryStream = FileSystem.create(fs, binaryPath,
-            new FsPermission(TEZ_AM_FILE_PERMISSION));
-        dagPB.writeTo(dagPBOutBinaryStream);
-      } finally {
-        if(dagPBOutBinaryStream != null){
-          dagPBOutBinaryStream.close();
-        }
-      }
-
-      localResources.put(TezConfiguration.TEZ_PB_PLAN_BINARY_NAME,
-          TezClientUtils.createLocalResource(fs,
-              binaryPath, LocalResourceType.FILE,
-              LocalResourceVisibility.APPLICATION));
-
-      if (Level.DEBUG.isGreaterOrEqual(Level.toLevel(amLogLevel))) {
-        Path textPath = localizeDagPlanAsText(dagPB, fs,
-            amConfig.getStagingDir(), appId);
-        localResources.put(TezConfiguration.TEZ_PB_PLAN_TEXT_NAME,
-            TezClientUtils.createLocalResource(fs,
-                textPath, LocalResourceType.FILE,
-                LocalResourceVisibility.APPLICATION));
-      }
-    } else {
-      Apps.addToEnvironment(environment,
-          TezConstants.TEZ_AM_IS_SESSION_ENV, "set");
-    }
-
-    Map<ApplicationAccessType, String> acls
-        = new HashMap<ApplicationAccessType, String>();
-
-    // Setup ContainerLaunchContext for AM container
-    ContainerLaunchContext amContainer =
-        ContainerLaunchContext.newInstance(localResources, environment,
-            vargsFinal, null, securityTokens, acls);
-
-    // Set up the ApplicationSubmissionContext
-    ApplicationSubmissionContext appContext = Records
-        .newRecord(ApplicationSubmissionContext.class);
-
-    appContext.setApplicationType(TezConfiguration.TEZ_APPLICATION_TYPE);
-    appContext.setApplicationId(appId);
-    appContext.setResource(capability);
-    appContext.setQueue(amConfig.getQueueName());
-    appContext.setApplicationName(amName);
-    appContext.setCancelTokensWhenComplete(amConfig.getAMConf().getBoolean(
-        TezConfiguration.TEZ_AM_CANCEL_DELEGATION_TOKEN,
-        TezConfiguration.TEZ_AM_CANCEL_DELEGATION_TOKEN_DEFAULT));
-    appContext.setAMContainerSpec(amContainer);
-
-    return appContext;
-
-  }
-
-  @VisibleForTesting
-  static void addLog4jSystemProperties(String logLevel,
-      List<String> vargs) {
-    vargs.add("-Dlog4j.configuration="
-        + TezConfiguration.TEZ_CONTAINER_LOG4J_PROPERTIES_FILE);
-    vargs.add("-D" + YarnConfiguration.YARN_APP_CONTAINER_LOG_DIR + "="
-        + ApplicationConstants.LOG_DIR_EXPANSION_VAR);
-    vargs.add("-D" + TezConfiguration.TEZ_ROOT_LOGGER_NAME + "=" + logLevel
-        + "," + TezConfiguration.TEZ_CONTAINER_LOGGER_NAME);
-  }
-
-  static Configuration createFinalTezConfForApp(TezConfiguration amConf) {
-    Configuration conf = new Configuration(false);
-    conf.setQuietMode(true);
-
-    assert amConf != null;
-    Iterator<Entry<String, String>> iter = amConf.iterator();
-    while (iter.hasNext()) {
-      Entry<String, String> entry = iter.next();
-      // Copy all tez config parameters.
-      if (entry.getKey().startsWith(TezConfiguration.TEZ_PREFIX)) {
-        conf.set(entry.getKey(), entry.getValue());
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Adding tez dag am parameter: " + entry.getKey()
-              + ", with value: " + entry.getValue());
-        }
-      }
-    }
-    return conf;
-  }
-
-  /**
-   * Helper function to create a YARN LocalResource
-   * @param fs FileSystem object
-   * @param p Path of resource to localize
-   * @param type LocalResource Type
-   * @return
-   * @throws IOException
-   */
-  static LocalResource createLocalResource(FileSystem fs, Path p,
-      LocalResourceType type,
-      LocalResourceVisibility visibility) throws IOException {
-    LocalResource rsrc = Records.newRecord(LocalResource.class);
-    FileStatus rsrcStat = fs.getFileStatus(p);
-    rsrc.setResource(ConverterUtils.getYarnUrlFromPath(fs.resolvePath(rsrcStat
-        .getPath())));
-    rsrc.setSize(rsrcStat.getLen());
-    rsrc.setTimestamp(rsrcStat.getModificationTime());
-    rsrc.setType(type);
-    rsrc.setVisibility(visibility);
-    return rsrc;
-  }
-
-  private static Path localizeDagPlanAsText(DAGPlan dagPB, FileSystem fs,
-      Path appStagingDir, ApplicationId appId) throws IOException {
-    Path textPath = new Path(appStagingDir,
-        TezConfiguration.TEZ_PB_PLAN_TEXT_NAME + "." + appId.toString());
-    FSDataOutputStream dagPBOutTextStream = null;
-    try {
-      dagPBOutTextStream = FileSystem.create(fs, textPath, new FsPermission(
-          TEZ_AM_FILE_PERMISSION));
-      String dagPBStr = dagPB.toString();
-      int dagPBStrLen = dagPBStr.length();
-      if (dagPBStrLen <= UTF8_CHUNK_SIZE) {
-        dagPBOutTextStream.writeUTF(dagPBStr);
-      } else {
-        int startIndex = 0;
-        while (startIndex < dagPBStrLen) {
-          int endIndex = startIndex + UTF8_CHUNK_SIZE;
-          if (endIndex > dagPBStrLen) {
-            endIndex = dagPBStrLen;
-          }
-          dagPBOutTextStream.writeUTF(dagPBStr.substring(startIndex, endIndex));
-          startIndex += UTF8_CHUNK_SIZE;
-        }
-      }
-    } finally {
-      if (dagPBOutTextStream != null) {
-        dagPBOutTextStream.close();
-      }
-    }
-    return textPath;
-  }
-
-  static DAGClientAMProtocolBlockingPB getAMProxy(YarnClient yarnClient,
-      Configuration conf,
-      ApplicationId applicationId) throws TezException, IOException {
-    ApplicationReport appReport;
-    try {
-      appReport = yarnClient.getApplicationReport(
-          applicationId);
-
-      if(appReport == null) {
-        throw new TezUncheckedException("Could not retrieve application report"
-            + " from YARN, applicationId=" + applicationId);
-      }
-      YarnApplicationState appState = appReport.getYarnApplicationState();
-      if(appState != YarnApplicationState.RUNNING) {
-        if (appState == YarnApplicationState.FINISHED
-            || appState == YarnApplicationState.KILLED
-            || appState == YarnApplicationState.FAILED) {
-          throw new TezUncheckedException("Application not running"
-              + ", applicationId=" + applicationId
-              + ", yarnApplicationState=" + appReport.getYarnApplicationState()
-              + ", finalApplicationStatus="
-              + appReport.getFinalApplicationStatus()
-              + ", trackingUrl=" + appReport.getTrackingUrl());
-        }
-        return null;
-      }
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-    return getAMProxy(conf, appReport.getHost(), appReport.getRpcPort());
-  }
-
-  static DAGClientAMProtocolBlockingPB getAMProxy(Configuration conf,
-      String amHost, int amRpcPort) throws IOException {
-    InetSocketAddress addr = new InetSocketAddress(amHost,
-        amRpcPort);
-
-    RPC.setProtocolEngine(conf, DAGClientAMProtocolBlockingPB.class,
-        ProtobufRpcEngine.class);
-    DAGClientAMProtocolBlockingPB proxy =
-        (DAGClientAMProtocolBlockingPB) RPC.getProxy(
-            DAGClientAMProtocolBlockingPB.class, 0, addr, conf);
-    return proxy;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/client/TezSession.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/client/TezSession.java b/tez-dag-api/src/main/java/org/apache/tez/client/TezSession.java
deleted file mode 100644
index acf523d..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/client/TezSession.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.client;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.client.api.YarnClient;
-import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.tez.dag.api.DAG;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezException;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.api.Vertex;
-import org.apache.tez.dag.api.client.DAGClient;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB;
-import org.apache.tez.dag.api.client.rpc.DAGClientRPCImpl;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.ShutdownSessionRequestProto;
-import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.SubmitDAGRequestProto;
-import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ServiceException;
-
-public class TezSession {
-
-  private static final Log LOG = LogFactory.getLog(TezSession.class);
-
-  private final String sessionName;
-  private ApplicationId applicationId;
-  private LocalResource tezConfPBLRsrc = null;
-  private final TezSessionConfiguration sessionConfig;
-  private YarnClient yarnClient;
-  private Map<String, LocalResource> tezJarResources;
-  private boolean sessionStarted = false;
-
-  public TezSession(String sessionName,
-      ApplicationId applicationId,
-      TezSessionConfiguration sessionConfig) {
-    this.sessionName = sessionName;
-    this.sessionConfig = sessionConfig;
-    this.applicationId = applicationId;
-  }
-
-  public TezSession(String sessionName,
-      TezSessionConfiguration sessionConfig) {
-    this(sessionName, null, sessionConfig);
-  }
-
-  public synchronized void start() throws TezException, IOException {
-    yarnClient = YarnClient.createYarnClient();
-    yarnClient.init(sessionConfig.getYarnConfiguration());
-    yarnClient.start();
-
-    tezJarResources = TezClientUtils.setupTezJarsLocalResources(
-        sessionConfig.getTezConfiguration());
-
-    try {
-      if (applicationId == null) {
-        applicationId = yarnClient.createApplication().
-            getNewApplicationResponse().getApplicationId();
-      }
-
-      ApplicationSubmissionContext appContext =
-          TezClientUtils.createApplicationSubmissionContext(
-              sessionConfig.getTezConfiguration(), applicationId,
-              null, sessionName, sessionConfig.getAMConfiguration(),
-              tezJarResources);
-      tezConfPBLRsrc = appContext.getAMContainerSpec().getLocalResources().get(
-          TezConfiguration.TEZ_PB_BINARY_CONF_NAME);
-      yarnClient.submitApplication(appContext);
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-    sessionStarted = true;
-  }
-
-  public synchronized DAGClient submitDAG(DAG dag)
-      throws TezException, IOException {
-    if (!sessionStarted) {
-      throw new TezUncheckedException("Session not started");
-    }
-
-    String dagId = null;
-    LOG.info("Submitting dag to TezSession"
-        + ", sessionName=" + sessionName
-        + ", applicationId=" + applicationId);
-    // Add tez jars to vertices too
-    for (Vertex v : dag.getVertices()) {
-      v.getTaskLocalResources().putAll(tezJarResources);
-      if (null != tezConfPBLRsrc) {
-        v.getTaskLocalResources().put(TezConfiguration.TEZ_PB_BINARY_CONF_NAME,
-            tezConfPBLRsrc);
-      }
-    }
-    DAGPlan dagPlan = dag.createDag(sessionConfig.getTezConfiguration());
-    SubmitDAGRequestProto requestProto =
-        SubmitDAGRequestProto.newBuilder().setDAGPlan(dagPlan).build();
-
-    DAGClientAMProtocolBlockingPB proxy;
-    while (true) {
-      proxy = TezClientUtils.getAMProxy(yarnClient,
-          sessionConfig.getYarnConfiguration(), applicationId);
-      if (proxy != null) {
-        break;
-      }
-      try {
-        Thread.sleep(100l);
-      } catch (InterruptedException e) {
-        // Ignore
-      }
-    }
-
-    try {
-      dagId = proxy.submitDAG(null, requestProto).getDagId();
-    } catch (ServiceException e) {
-      throw new TezException(e);
-    }
-    LOG.info("Submitted dag to TezSession"
-        + ", sessionName=" + sessionName
-        + ", applicationId=" + applicationId
-        + ", dagId=" + dagId);
-    return new DAGClientRPCImpl(applicationId, dagId,
-        sessionConfig.getTezConfiguration());
-  }
-
-  public synchronized void stop() throws TezException, IOException {
-    LOG.info("Shutting down Tez Session"
-        + ", sessionName=" + sessionName
-        + ", applicationId=" + applicationId);
-    DAGClientAMProtocolBlockingPB proxy = TezClientUtils.getAMProxy(yarnClient,
-        sessionConfig.getYarnConfiguration(), applicationId);
-    if (proxy != null) {
-      try {
-        ShutdownSessionRequestProto request =
-            ShutdownSessionRequestProto.newBuilder().build();
-        proxy.shutdownSession(null, request);
-        return;
-      } catch (ServiceException e) {
-        LOG.info("Failed to shutdown Tez Session via proxy", e);
-      }
-    }
-    LOG.info("Could not connect to AM, killing session via YARN"
-        + ", sessionName=" + sessionName
-        + ", applicationId=" + applicationId);
-    try {
-      yarnClient.killApplication(applicationId);
-    } catch (YarnException e) {
-      throw new TezException(e);
-    }
-  }
-
-  public String getSessionName() {
-    return sessionName;
-  }
-
-  @Private
-  @VisibleForTesting
-  public synchronized ApplicationId getApplicationId() {
-    return applicationId;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-dag-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-dag-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java b/tez-dag-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
deleted file mode 100644
index 61ca60b..0000000
--- a/tez-dag-api/src/main/java/org/apache/tez/client/TezSessionConfiguration.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.tez.client;
-
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.tez.dag.api.TezConfiguration;
-
-public class TezSessionConfiguration {
-
-  private final AMConfiguration amConfiguration;
-  private final YarnConfiguration yarnConfig;
-  private final TezConfiguration tezConfig;
-
-  public TezSessionConfiguration(AMConfiguration amConfiguration,
-      TezConfiguration tezConfig) {
-    this.amConfiguration = amConfiguration;
-    this.tezConfig = tezConfig;
-    this.yarnConfig = new YarnConfiguration(tezConfig);
-  }
-
-  TezSessionConfiguration(AMConfiguration amConfiguration,
-      TezConfiguration tezConfig,
-      YarnConfiguration yarnConf) {
-    this.amConfiguration = amConfiguration;
-    this.tezConfig = tezConfig;
-    this.yarnConfig = yarnConf;
-  }
-
-  public AMConfiguration getAMConfiguration() {
-    return amConfiguration;
-  }
-
-  public YarnConfiguration getYarnConfiguration() {
-    return yarnConfig;
-  }
-
-  public TezConfiguration getTezConfiguration() {
-    return tezConfig;
-  }
-
-}


[18/20] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
new file mode 100644
index 0000000..9cb602c
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
@@ -0,0 +1,377 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.dag.api;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Stack;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
+import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
+import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
+import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
+import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
+import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
+import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResource;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
+import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
+import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
+
+
+public class DAG { // FIXME rename to Topology
+  final List<Vertex> vertices;
+  final List<Edge> edges;
+  final String name;
+
+  public DAG(String name) {
+    this.vertices = new ArrayList<Vertex>();
+    this.edges = new ArrayList<Edge>();
+    this.name = name;
+  }
+
+  public synchronized DAG addVertex(Vertex vertex) {
+    if (vertices.contains(vertex)) {
+      throw new IllegalArgumentException(
+          "Vertex " + vertex + " already defined!");
+    }
+    vertices.add(vertex);
+    return this;
+  }
+
+  @Private
+  public synchronized List<Vertex> getVertices() {
+    return Collections.unmodifiableList(this.vertices);
+  }
+
+  public synchronized DAG addEdge(Edge edge) {
+    // Sanity checks
+    if (!vertices.contains(edge.getInputVertex())) {
+      throw new IllegalArgumentException(
+          "Input vertex " + edge.getInputVertex() + " doesn't exist!");
+    }
+    if (!vertices.contains(edge.getOutputVertex())) {
+      throw new IllegalArgumentException(
+          "Output vertex " + edge.getOutputVertex() + " doesn't exist!");
+    }
+    if (edges.contains(edge)) {
+      throw new IllegalArgumentException(
+          "Edge " + edge + " already defined!");
+    }
+
+    // Inform the vertices
+    edge.getInputVertex().addOutputVertex(edge.getOutputVertex(), edge.getId());
+    edge.getOutputVertex().addInputVertex(edge.getInputVertex(), edge.getId());
+
+    edges.add(edge);
+    return this;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  // AnnotatedVertex is used by verify()
+  private static class AnnotatedVertex {
+    Vertex v;
+
+    int index; //for Tarjan's algorithm
+    int lowlink; //for Tarjan's algorithm
+    boolean onstack; //for Tarjan's algorithm
+
+    int outDegree;
+
+    private AnnotatedVertex(Vertex v){
+       this.v = v;
+       index = -1;
+       lowlink = -1;
+       outDegree = 0;
+    }
+  }
+
+  // verify()
+  //
+  // Default rules
+  //   Illegal:
+  //     - duplicate vertex id
+  //     - cycles
+  //
+  //   Ok:
+  //     - orphaned vertex.  Occurs in map-only
+  //     - islands.  Occurs if job has unrelated workflows.
+  //
+  //   Not yet categorized:
+  //     - orphaned vertex in DAG of >1 vertex.  Could be unrelated map-only job.
+  //     - v1->v2 via two edges.  perhaps some self-join job would use this?
+  //
+  // "restricted" mode:
+  //   In short term, the supported DAGs are limited. Call with restricted=true for these verifications.
+  //   Illegal:
+  //     - any vertex with more than one input or output edge. (n-ary input, n-ary merge)
+  public void verify() throws IllegalStateException {
+    verify(true);
+  }
+
+  public void verify(boolean restricted) throws IllegalStateException  {
+    if (vertices.isEmpty()) {
+      throw new IllegalStateException("Invalid dag containing 0 vertices");
+    }
+
+    Map<Vertex, List<Edge>> edgeMap = new HashMap<Vertex, List<Edge>>();
+    for(Edge e : edges){
+      Vertex inputVertex = e.getInputVertex();
+      List<Edge> edgeList = edgeMap.get(inputVertex);
+      if(edgeList == null){
+        edgeList = new ArrayList<Edge>();
+        edgeMap.put(inputVertex, edgeList);
+      }
+      edgeList.add(e);
+    }
+
+    // check for valid vertices, duplicate vertex names,
+    // and prepare for cycle detection
+    Map<String, AnnotatedVertex> vertexMap = new HashMap<String, AnnotatedVertex>();
+    for(Vertex v : vertices){
+      if(vertexMap.containsKey(v.getVertexName())){
+         throw new IllegalStateException("DAG contains multiple vertices"
+             + " with name: " + v.getVertexName());
+      }
+      vertexMap.put(v.getVertexName(), new AnnotatedVertex(v));
+    }
+
+    detectCycles(edgeMap, vertexMap);
+
+    if(restricted){
+      for(Edge e : edges){
+        vertexMap.get(e.getInputVertex().getVertexName()).outDegree++;
+        if (e.getEdgeProperty().getDataMovementType() !=
+            DataMovementType.SCATTER_GATHER) {
+          throw new IllegalStateException(
+              "Unsupported connection pattern on edge. " + e);
+        }
+        if (e.getEdgeProperty().getDataSourceType() !=
+            DataSourceType.PERSISTED) {
+          throw new IllegalStateException(
+              "Unsupported source type on edge. " + e);
+        }
+        if (e.getEdgeProperty().getSchedulingType() !=
+            SchedulingType.SEQUENTIAL) {
+          throw new IllegalStateException(
+              "Unsupported scheduling type on edge. " + e);
+        }
+      }
+      for(AnnotatedVertex av: vertexMap.values()){
+        if (av.outDegree > 1) {
+          throw new IllegalStateException("Vertex has outDegree>1: "
+              + av.v.getVertexName());
+        }
+      }
+    }
+  }
+
+  // Adaptation of Tarjan's algorithm for connected components.
+  // http://en.wikipedia.org/wiki/Tarjan%27s_strongly_connected_components_algorithm
+  private void detectCycles(Map<Vertex, List<Edge>> edgeMap, Map<String, AnnotatedVertex> vertexMap)
+      throws IllegalStateException{
+    Integer nextIndex = 0; // boxed integer so it is passed by reference.
+    Stack<AnnotatedVertex> stack = new Stack<DAG.AnnotatedVertex>();
+    for(AnnotatedVertex av: vertexMap.values()){
+      if(av.index == -1){
+        assert stack.empty();
+        strongConnect(av, vertexMap, edgeMap, stack, nextIndex);
+      }
+    }
+  }
+
+  // part of Tarjan's algorithm for connected components.
+  // http://en.wikipedia.org/wiki/Tarjan%27s_strongly_connected_components_algorithm
+  private void strongConnect(
+          AnnotatedVertex av,
+          Map<String, AnnotatedVertex> vertexMap,
+          Map<Vertex, List<Edge>> edgeMap,
+          Stack<AnnotatedVertex> stack, Integer nextIndex) throws IllegalStateException{
+    av.index = nextIndex;
+    av.lowlink = nextIndex;
+    nextIndex++;
+    stack.push(av);
+    av.onstack = true;
+
+    List<Edge> edges = edgeMap.get(av.v);
+    if(edges != null){
+      for(Edge e : edgeMap.get(av.v)){
+        AnnotatedVertex outVertex = vertexMap.get(e.getOutputVertex().getVertexName());
+        if(outVertex.index == -1){
+          strongConnect(outVertex, vertexMap, edgeMap, stack, nextIndex);
+          av.lowlink = Math.min(av.lowlink, outVertex.lowlink);
+        }
+        else if(outVertex.onstack){
+          // strongly connected component detected, but we will wait till later so that the full cycle can be displayed.
+          // update lowlink in case outputVertex should be considered the root of this component.
+          av.lowlink = Math.min(av.lowlink, outVertex.index);
+        }
+      }
+    }
+
+    if(av.lowlink == av.index ){
+       AnnotatedVertex pop = stack.pop();
+       pop.onstack = false;
+       if(pop != av){
+         // there was something on the stack other than this "av".
+         // this indicates there is a scc/cycle. It comprises all nodes from top of stack to "av"
+         StringBuilder message = new StringBuilder();
+         message.append(av.v.getVertexName() + " <- ");
+         for( ; pop != av; pop = stack.pop()){
+           message.append(pop.v.getVertexName() + " <- ");
+           pop.onstack = false;
+         }
+         message.append(av.v.getVertexName());
+         throw new IllegalStateException("DAG contains a cycle: " + message);
+       }
+    }
+  }
+
+
+  // create protobuf message describing DAG
+  @Private
+  public DAGPlan createDag(Configuration dagConf) {
+    verify(true);
+
+    DAGPlan.Builder dagBuilder = DAGPlan.newBuilder();
+
+    dagBuilder.setName(this.name);
+
+    for(Vertex vertex : vertices){
+      VertexPlan.Builder vertexBuilder = VertexPlan.newBuilder();
+      vertexBuilder.setName(vertex.getVertexName());
+      vertexBuilder.setType(PlanVertexType.NORMAL); // vertex type is implicitly NORMAL until  TEZ-46.
+      vertexBuilder.setProcessorDescriptor(DagTypeConverters
+          .convertToDAGPlan(vertex.getProcessorDescriptor()));
+
+      //task config
+      PlanTaskConfiguration.Builder taskConfigBuilder = PlanTaskConfiguration.newBuilder();
+      Resource resource = vertex.getTaskResource();
+      taskConfigBuilder.setNumTasks(vertex.getParallelism());
+      taskConfigBuilder.setMemoryMb(resource.getMemory());
+      taskConfigBuilder.setVirtualCores(resource.getVirtualCores());
+      taskConfigBuilder.setJavaOpts(vertex.getJavaOpts());
+
+      taskConfigBuilder.setTaskModule(vertex.getVertexName());
+      PlanLocalResource.Builder localResourcesBuilder = PlanLocalResource.newBuilder();
+      Map<String,LocalResource> lrs = vertex.getTaskLocalResources();
+      for(Entry<String, LocalResource> entry : lrs.entrySet()){
+        String key = entry.getKey();
+        LocalResource lr = entry.getValue();
+        localResourcesBuilder.setName(key);
+        localResourcesBuilder.setUri(
+            DagTypeConverters.convertToDAGPlan(lr.getResource()));
+        localResourcesBuilder.setSize(lr.getSize());
+        localResourcesBuilder.setTimeStamp(lr.getTimestamp());
+        localResourcesBuilder.setType(
+            DagTypeConverters.convertToDAGPlan(lr.getType()));
+        localResourcesBuilder.setVisibility(
+            DagTypeConverters.convertToDAGPlan(lr.getVisibility()));
+        if(lr.getType() == LocalResourceType.PATTERN){
+          if (lr.getPattern() == null || lr.getPattern().isEmpty()) {
+            throw new TezUncheckedException("LocalResource type set to pattern"
+                + " but pattern is null or empty");
+          }
+          localResourcesBuilder.setPattern(lr.getPattern());
+        }
+        taskConfigBuilder.addLocalResource(localResourcesBuilder);
+      }
+
+      if(vertex.getTaskEnvironment() != null){
+        for(String key : vertex.getTaskEnvironment().keySet()){
+          PlanKeyValuePair.Builder envSettingBuilder = PlanKeyValuePair.newBuilder();
+          envSettingBuilder.setKey(key);
+          envSettingBuilder.setValue(vertex.getTaskEnvironment().get(key));
+          taskConfigBuilder.addEnvironmentSetting(envSettingBuilder);
+        }
+      }
+
+      if(vertex.getTaskLocationsHint() != null ){
+        if(vertex.getTaskLocationsHint().getTaskLocationHints() != null){
+          for(TaskLocationHint hint : vertex.getTaskLocationsHint().getTaskLocationHints()){
+            PlanTaskLocationHint.Builder taskLocationHintBuilder = PlanTaskLocationHint.newBuilder();
+
+            if(hint.getDataLocalHosts() != null){
+              taskLocationHintBuilder.addAllHost(hint.getDataLocalHosts());
+            }
+            if(hint.getRacks() != null){
+              taskLocationHintBuilder.addAllRack(hint.getRacks());
+            }
+
+            vertexBuilder.addTaskLocationHint(taskLocationHintBuilder);
+          }
+        }
+      }
+
+      for(String inEdgeId : vertex.getInputEdgeIds()){
+        vertexBuilder.addInEdgeId(inEdgeId);
+      }
+
+      for(String outEdgeId : vertex.getOutputEdgeIds()){
+        vertexBuilder.addOutEdgeId(outEdgeId);
+      }
+
+      vertexBuilder.setTaskConfig(taskConfigBuilder);
+      dagBuilder.addVertex(vertexBuilder);
+    }
+
+    for(Edge edge : edges){
+      EdgePlan.Builder edgeBuilder = EdgePlan.newBuilder();
+      edgeBuilder.setId(edge.getId());
+      edgeBuilder.setInputVertexName(edge.getInputVertex().getVertexName());
+      edgeBuilder.setOutputVertexName(edge.getOutputVertex().getVertexName());
+      edgeBuilder.setDataMovementType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getDataMovementType()));
+      edgeBuilder.setDataSourceType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getDataSourceType()));
+      edgeBuilder.setSchedulingType(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getSchedulingType()));
+      edgeBuilder.setEdgeSource(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getEdgeSource()));
+      edgeBuilder.setEdgeDestination(DagTypeConverters.convertToDAGPlan(edge.getEdgeProperty().getEdgeDestination()));
+      dagBuilder.addEdge(edgeBuilder);
+    }
+
+    if(dagConf != null) {
+      Iterator<Entry<String, String>> iter = dagConf.iterator();
+      ConfigurationProto.Builder confProtoBuilder =
+          ConfigurationProto.newBuilder();
+      while (iter.hasNext()) {
+        Entry<String, String> entry = iter.next();
+        PlanKeyValuePair.Builder kvp = PlanKeyValuePair.newBuilder();
+        kvp.setKey(entry.getKey());
+        kvp.setValue(entry.getValue());
+        confProtoBuilder.addConfKeyValues(kvp);
+      }
+      dagBuilder.setDagKeyValues(confProtoBuilder);
+    }
+
+    return dagBuilder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
new file mode 100644
index 0000000..1fd78f1
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
@@ -0,0 +1,278 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.dag.api;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
+import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
+import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
+import org.apache.tez.dag.api.records.DAGProtos.ConfigurationProto;
+import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataMovementType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataSourceType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeSchedulingType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
+import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResource;
+import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResourceType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResourceVisibility;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+
+import com.google.protobuf.ByteString;
+
+
+public class DagTypeConverters {
+  
+  public static PlanLocalResourceVisibility convertToDAGPlan(LocalResourceVisibility visibility){
+    switch(visibility){
+      case PUBLIC : return PlanLocalResourceVisibility.PUBLIC;  
+      case PRIVATE : return PlanLocalResourceVisibility.PRIVATE;
+      case APPLICATION : return PlanLocalResourceVisibility.APPLICATION;
+      default : throw new RuntimeException("unknown 'visibility': " + visibility);
+    }
+  }
+  
+  public static LocalResourceVisibility convertFromDAGPlan(PlanLocalResourceVisibility visibility){
+    switch(visibility){
+      case PUBLIC : return LocalResourceVisibility.PUBLIC;  
+      case PRIVATE : return LocalResourceVisibility.PRIVATE;
+      case APPLICATION : return LocalResourceVisibility.APPLICATION;
+      default : throw new RuntimeException("unknown 'visibility': " + visibility);
+    }
+  }
+  
+  public static PlanEdgeDataSourceType convertToDAGPlan(DataSourceType sourceType){
+    switch(sourceType){
+      case PERSISTED : return PlanEdgeDataSourceType.PERSISTED;  
+      case PERSISTED_RELIABLE : return PlanEdgeDataSourceType.PERSISTED_RELIABLE;
+      case EPHEMERAL :  return PlanEdgeDataSourceType.EPHEMERAL;
+      default : throw new RuntimeException("unknown 'dataSourceType': " + sourceType);
+    }
+  }
+  
+  public static DataSourceType convertFromDAGPlan(PlanEdgeDataSourceType sourceType){
+    switch(sourceType){
+      case PERSISTED : return DataSourceType.PERSISTED;  
+      case PERSISTED_RELIABLE : return DataSourceType.PERSISTED_RELIABLE;
+      case EPHEMERAL :  return DataSourceType.EPHEMERAL;
+      default : throw new RuntimeException("unknown 'dataSourceType': " + sourceType);
+    }
+  }
+  
+  public static PlanEdgeDataMovementType convertToDAGPlan(DataMovementType type){
+    switch(type){
+      case ONE_TO_ONE : return PlanEdgeDataMovementType.ONE_TO_ONE;  
+      case BROADCAST : return PlanEdgeDataMovementType.BROADCAST;
+      case SCATTER_GATHER : return PlanEdgeDataMovementType.SCATTER_GATHER;
+      default : throw new RuntimeException("unknown 'dataMovementType': " + type);
+    }
+  }
+  
+  public static DataMovementType convertFromDAGPlan(PlanEdgeDataMovementType type){
+    switch(type){
+      case ONE_TO_ONE : return DataMovementType.ONE_TO_ONE;  
+      case BROADCAST : return DataMovementType.BROADCAST;
+      case SCATTER_GATHER : return DataMovementType.SCATTER_GATHER;
+      default : throw new IllegalArgumentException("unknown 'dataMovementType': " + type);
+    }
+  }
+  
+  public static PlanEdgeSchedulingType convertToDAGPlan(SchedulingType type){
+    switch(type){
+      case SEQUENTIAL : return PlanEdgeSchedulingType.SEQUENTIAL;  
+      case CONCURRENT : return PlanEdgeSchedulingType.CONCURRENT;
+      default : throw new RuntimeException("unknown 'SchedulingType': " + type);
+    }
+  }
+  
+  public static SchedulingType convertFromDAGPlan(PlanEdgeSchedulingType type){
+    switch(type){
+      case SEQUENTIAL : return SchedulingType.SEQUENTIAL;  
+      case CONCURRENT : return SchedulingType.CONCURRENT;
+      default : throw new IllegalArgumentException("unknown 'SchedulingType': " + type);
+    }
+  }
+  
+  public static PlanLocalResourceType convertToDAGPlan(LocalResourceType type) {
+    switch(type){
+    case ARCHIVE : return PlanLocalResourceType.ARCHIVE;
+    case FILE : return PlanLocalResourceType.FILE;
+    case PATTERN : return PlanLocalResourceType.PATTERN;
+    default : throw new IllegalArgumentException("unknown 'type': " + type);
+    }
+  }
+  
+  public static LocalResourceType convertFromDAGPlan(PlanLocalResourceType type) {
+    switch(type){
+    case ARCHIVE : return LocalResourceType.ARCHIVE;
+    case FILE : return LocalResourceType.FILE;
+    case PATTERN : return LocalResourceType.PATTERN;
+    default : throw new IllegalArgumentException("unknown 'type': " + type);
+    }
+  }
+
+  public static VertexLocationHint convertFromDAGPlan(
+      List<PlanTaskLocationHint> locationHints) {
+
+    List<TaskLocationHint> outputList = new ArrayList<TaskLocationHint>();  
+    
+    for(PlanTaskLocationHint inputHint : locationHints){
+      TaskLocationHint outputHint = new TaskLocationHint(
+          new HashSet<String>(inputHint.getHostList()),
+          new HashSet<String>(inputHint.getRackList()));
+      outputList.add(outputHint);
+    }
+    return new VertexLocationHint(outputList.size(), outputList);
+  }
+
+  // notes re HDFS URL handling:
+  //   Resource URLs in the protobuf message are strings of the form hdfs://host:port/path 
+  //   org.apache.hadoop.fs.Path.Path  is actually a URI type that allows any scheme
+  //   org.apache.hadoop.yarn.api.records.URL is a URL type used by YARN.
+  //   java.net.URL cannot be used out of the box as it rejects unknown schemes such as HDFS.
+  
+  public static String convertToDAGPlan(URL resource) {
+    // see above notes on HDFS URL handling
+    String out = resource.getScheme() + "://" + resource.getHost() + ":" + resource.getPort() 
+        + resource.getFile();
+    return out;
+  }
+
+  public static Map<String, LocalResource> createLocalResourceMapFromDAGPlan(
+      List<PlanLocalResource> localResourcesList) {
+    Map<String, LocalResource> map = new HashMap<String, LocalResource>();
+    for(PlanLocalResource res : localResourcesList){
+      LocalResource r = new LocalResourcePBImpl();
+      
+      //NOTE: have to check every optional field in protobuf generated classes for existence before accessing
+      //else we will receive a default value back, eg ""
+      if(res.hasPattern()){
+        r.setPattern(res.getPattern());
+      }
+      r.setResource(ConverterUtils.getYarnUrlFromPath(new Path(res.getUri())));  // see above notes on HDFS URL handling
+      r.setSize(res.getSize());
+      r.setTimestamp(res.getTimeStamp());
+      r.setType(DagTypeConverters.convertFromDAGPlan(res.getType()));
+      r.setVisibility(DagTypeConverters.convertFromDAGPlan(res.getVisibility()));
+      map.put(res.getName(), r);
+    }
+    return map;
+  }
+
+  public static Map<String, String> createEnvironmentMapFromDAGPlan(
+      List<PlanKeyValuePair> environmentSettingList) {  
+      
+    Map<String, String> map = new HashMap<String, String>();
+    for(PlanKeyValuePair setting : environmentSettingList){
+      map.put(setting.getKey(), setting.getValue());
+    }
+    
+    return map;
+  }
+  
+  public static Map<String, EdgePlan> createEdgePlanMapFromDAGPlan(List<EdgePlan> edgeList){
+    Map<String, EdgePlan> edgePlanMap =
+        new HashMap<String, EdgePlan>();
+    for(EdgePlan edgePlanItem : edgeList){
+      edgePlanMap.put(edgePlanItem.getId(), edgePlanItem);
+    }
+    return edgePlanMap;
+  }
+  
+  public static EdgeProperty createEdgePropertyMapFromDAGPlan(EdgePlan edge) {
+    return new EdgeProperty(
+        convertFromDAGPlan(edge.getDataMovementType()),
+        convertFromDAGPlan(edge.getDataSourceType()),
+        convertFromDAGPlan(edge.getSchedulingType()),
+        convertOutputDescriptorFromDAGPlan(edge.getEdgeSource()),
+        convertInputDescriptorFromDAGPlan(edge.getEdgeDestination())
+    );
+  }
+
+  public static Resource createResourceRequestFromTaskConfig(
+      PlanTaskConfiguration taskConfig) {
+    return Resource.newInstance(taskConfig.getMemoryMb(), taskConfig.getVirtualCores());
+  }
+
+  public static Map<String, String> convertConfFromProto(
+      ConfigurationProto confProto) {
+    List<PlanKeyValuePair> settingList = confProto.getConfKeyValuesList();
+    Map<String, String> map = new HashMap<String, String>();
+    for(PlanKeyValuePair setting: settingList){
+      map.put(setting.getKey(), setting.getValue());
+    }
+    return map;
+  }
+
+  public static TezEntityDescriptorProto convertToDAGPlan(
+      TezEntityDescriptor descriptor) {
+    TezEntityDescriptorProto.Builder builder = TezEntityDescriptorProto
+        .newBuilder();
+    builder.setClassName(descriptor.getClassName());
+    if (descriptor.getUserPayload() != null) {
+      builder
+          .setUserPayload(ByteString.copyFrom(descriptor.getUserPayload()));
+    }
+    return builder.build();
+  }
+
+  public static InputDescriptor convertInputDescriptorFromDAGPlan(
+      TezEntityDescriptorProto proto) {
+    String className = proto.getClassName();
+    byte[] bb = null;
+    if (proto.hasUserPayload()) {
+      bb = proto.getUserPayload().toByteArray();
+    }
+    return new InputDescriptor(className).setUserPayload(bb);
+  }
+
+  public static OutputDescriptor convertOutputDescriptorFromDAGPlan(
+      TezEntityDescriptorProto proto) {
+    String className = proto.getClassName();
+    byte[] bb = null;
+    if (proto.hasUserPayload()) {
+      bb =  proto.getUserPayload().toByteArray();
+    }
+    return new OutputDescriptor(className).setUserPayload(bb);
+  }
+
+  public static ProcessorDescriptor convertProcessorDescriptorFromDAGPlan(
+      TezEntityDescriptorProto proto) {
+    String className = proto.getClassName();
+    byte[] bb = null;
+    if (proto.hasUserPayload()) {
+      bb = proto.getUserPayload().toByteArray();
+    }
+    return new ProcessorDescriptor(className).setUserPayload(bb);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java b/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java
new file mode 100644
index 0000000..a893bc3
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/Edge.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.dag.api;
+
+public class Edge{
+  
+  private final Vertex inputVertex;
+  private final Vertex outputVertex;
+  private final EdgeProperty edgeProperty;
+    
+  public Edge(Vertex inputVertex, 
+               Vertex outputVertex, 
+               EdgeProperty edgeProperty) {
+    this.inputVertex = inputVertex;
+    this.outputVertex = outputVertex;
+    this.edgeProperty = edgeProperty;
+  }
+  
+  // RENAME to source and destination
+  public Vertex getInputVertex() {
+    return inputVertex;
+  }
+  
+  public Vertex getOutputVertex() {
+    return outputVertex;
+  }
+  
+  public EdgeProperty getEdgeProperty() {
+    return edgeProperty;
+  }
+  
+  /*
+   * Used to identify the edge in the configuration
+   */
+  public String getId() {
+    return String.valueOf(this.hashCode());
+  }
+ 
+  @Override
+  public String toString() {
+    return inputVertex + " -> " + outputVertex + " (" + edgeProperty + ")";
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java b/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
new file mode 100644
index 0000000..326d3d0
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/EdgeProperty.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.dag.api;
+
+public class EdgeProperty {
+  
+  /**
+   * Defines the manner of data movement between source and destination tasks.
+   * Determines which destination tasks have access to data produced on this
+   * edge by a source task. A destination task may choose to read any portion of
+   * the data available to it.
+   */
+  public enum DataMovementType {
+    /**
+     * Output on this edge produced by the i-th source task is available to the 
+     * i-th destination task.
+     */
+    ONE_TO_ONE,
+    /**
+     * Output on this edge produced by any source task is available to all
+     * destination tasks.
+     */
+    BROADCAST,
+    /**
+     * The i-th output on this edge produced by all source tasks is available to
+     * the same destination task. Source tasks scatter their outputs and they
+     * are gathered by designated destination tasks.
+     */
+    SCATTER_GATHER
+  }
+  
+  /**
+   * Determines the lifetime of the data produced on this edge by a source task.
+   */
+  public enum DataSourceType {
+    /**
+     * Data produced by the source is persisted and available even when the
+     * task is not running. The data may become unavailable and may cause the 
+     * source task to be re-executed.
+     */
+    PERSISTED,
+    /**
+     * Source data is stored reliably and will always be available
+     */
+    PERSISTED_RELIABLE,
+    /**
+     * Data produced by the source task is available only while the source task
+     * is running. This requires the destination task to run concurrently with 
+     * the source task.
+     */
+    EPHEMERAL
+  }
+  
+  /**
+   * Determines when the destination task is eligible to run, once the source  
+   * task is eligible to run.
+   */
+  public enum SchedulingType {
+    /**
+     * Destination task is eligible to run after one or more of its source tasks 
+     * have started or completed.
+     */
+    SEQUENTIAL,
+    /**
+     * Destination task must run concurrently with the source task
+     */
+    CONCURRENT
+  }
+  
+  DataMovementType dataMovementType;
+  DataSourceType dataSourceType;
+  SchedulingType schedulingType;
+  InputDescriptor inputDescriptor;
+  OutputDescriptor outputDescriptor;
+  
+  /**
+   * @param dataMovementType
+   * @param dataSourceType
+   * @param edgeSource
+   *          The {@link OutputDescriptor} that generates data on the edge.
+   * @param edgeDestination
+   *          The {@link InputDescriptor} which will consume data from the edge.
+   */
+  public EdgeProperty(DataMovementType dataMovementType, 
+                       DataSourceType dataSourceType,
+                       SchedulingType schedulingType,
+                       OutputDescriptor edgeSource,
+                       InputDescriptor edgeDestination) {
+    this.dataMovementType = dataMovementType;
+    this.dataSourceType = dataSourceType;
+    this.schedulingType = schedulingType;
+    this.inputDescriptor = edgeDestination;
+    this.outputDescriptor = edgeSource;
+  }
+  
+  public DataMovementType getDataMovementType() {
+    return dataMovementType;
+  }
+  
+  public DataSourceType getDataSourceType() {
+    return dataSourceType;
+  }
+  
+  public SchedulingType getSchedulingType() {
+    return schedulingType;
+  }
+  
+  /**
+   * Returns the {@link InputDescriptor} which will consume data from the edge.
+   * 
+   * @return
+   */
+  public InputDescriptor getEdgeDestination() {
+    return inputDescriptor;
+  }
+  
+  /**
+   * Returns the {@link OutputDescriptor} which produces data on the edge.
+   * 
+   * @return
+   */
+  public OutputDescriptor getEdgeSource() {
+    return outputDescriptor;
+  }
+  
+  @Override
+  public String toString() {
+    return "{ " + dataMovementType + " : " + inputDescriptor.getClassName()
+        + " >> " + dataSourceType + " >> " + outputDescriptor.getClassName() + " }";
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
new file mode 100644
index 0000000..dea9001
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/InputDescriptor.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+public class InputDescriptor extends TezEntityDescriptor {
+
+  public InputDescriptor(String inputClassName) {
+    super(inputClassName);
+  }
+
+  @Override
+  public InputDescriptor setUserPayload(byte[] userPayload) {
+    this.userPayload = userPayload;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
new file mode 100644
index 0000000..16fb9b1
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/OutputDescriptor.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+public class OutputDescriptor extends TezEntityDescriptor {
+
+  public OutputDescriptor(String outputClassName) {
+    super(outputClassName);
+  }
+
+  @Override
+  public OutputDescriptor setUserPayload(byte[] userPayload) {
+    this.userPayload = userPayload;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
new file mode 100644
index 0000000..092147d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/ProcessorDescriptor.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+public class ProcessorDescriptor extends TezEntityDescriptor {
+
+  public ProcessorDescriptor(String processorClassName) {
+    super(processorClassName);
+  }
+
+  public ProcessorDescriptor setUserPayload(byte[] userPayload) {
+    this.userPayload = userPayload;
+    return this;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
new file mode 100644
index 0000000..7447974
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -0,0 +1,223 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+import org.apache.hadoop.conf.Configuration;
+
+public class TezConfiguration extends Configuration {
+
+  public final static String TEZ_SITE_XML = "tez-site.xml";
+
+  static {
+    addDefaultResource(TEZ_SITE_XML);
+  }
+
+  public TezConfiguration() {
+    super();
+  }
+
+  public TezConfiguration(Configuration conf) {
+    super(conf);
+  }
+
+  public static final String TEZ_PREFIX = "tez.";
+  public static final String TEZ_AM_PREFIX = TEZ_PREFIX + "am.";
+  public static final String TEZ_TASK_PREFIX = TEZ_PREFIX + "task.";
+
+  public static final String TEZ_AM_STAGING_DIR = TEZ_PREFIX + "staging-dir";
+  public static final String TEZ_AM_STAGING_DIR_DEFAULT = "/tmp/tez/staging";
+
+  // TODO Should not be required once all tokens are handled via AppSubmissionContext
+  public static final String JOB_SUBMIT_DIR = TEZ_PREFIX + "jobSubmitDir";
+  public static final String APPLICATION_TOKENS_FILE = "appTokens";
+  public static final String TEZ_APPLICATION_MASTER_CLASS =
+      "org.apache.tez.dag.app.DAGAppMaster";
+
+  /** Root Logging level passed to the Tez app master.*/
+  public static final String TEZ_AM_LOG_LEVEL = TEZ_AM_PREFIX+"log.level";
+  public static final String TEZ_AM_LOG_LEVEL_DEFAULT = "INFO";
+
+  public static final String TEZ_AM_JAVA_OPTS = TEZ_AM_PREFIX
+      + "java.opts";
+  public static final String DEFAULT_TEZ_AM_JAVA_OPTS = " -Xmx1024m ";
+
+  public static final String TEZ_AM_CANCEL_DELEGATION_TOKEN = TEZ_AM_PREFIX +
+      "am.complete.cancel.delegation.tokens";
+  public static final boolean TEZ_AM_CANCEL_DELEGATION_TOKEN_DEFAULT = true;
+
+  public static final String TEZ_AM_TASK_LISTENER_THREAD_COUNT =
+      TEZ_AM_PREFIX + "task.listener.thread-count";
+  public static final int TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT = 30;
+
+  public static final String TEZ_AM_CONTAINER_LISTENER_THREAD_COUNT =
+      TEZ_AM_PREFIX + "container.listener.thread-count";
+  public static final int TEZ_AM_CONTAINER_LISTENER_THREAD_COUNT_DEFAULT = 30;
+
+  // TODO Some of the DAG properties are job specific and not AM specific. Rename accordingly.
+  // TODO Are any of these node blacklisting properties required. (other than for MR compat)
+  public static final String TEZ_AM_MAX_TASK_FAILURES_PER_NODE = TEZ_AM_PREFIX
+      + "maxtaskfailures.per.node";
+  public static final int TEZ_AM_MAX_TASK_FAILURES_PER_NODE_DEFAULT = 3;
+
+  public static final String TEZ_AM_MAX_TASK_ATTEMPTS =
+      TEZ_AM_PREFIX + "max.task.attempts";
+  public static final int TEZ_AM_MAX_TASK_ATTEMPTS_DEFAULT = 4;
+
+  public static final String TEZ_AM_NODE_BLACKLISTING_ENABLED = TEZ_AM_PREFIX
+      + "node-blacklisting.enabled";
+  public static final boolean TEZ_AM_NODE_BLACKLISTING_ENABLED_DEFAULT = true;
+  public static final String TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD = TEZ_AM_PREFIX
+      + "node-blacklisting.ignore-threshold-node-percent";
+  public static final int TEZ_AM_NODE_BLACKLISTING_IGNORE_THRESHOLD_DEFAULT = 33;
+
+  /** Number of threads to handle job client RPC requests.*/
+  public static final String TEZ_AM_CLIENT_THREAD_COUNT =
+      TEZ_AM_PREFIX + "client.am.thread-count";
+  public static final int TEZ_AM_CLIENT_THREAD_COUNT_DEFAULT = 1;
+  /**
+   * Range of ports that the AM can use when binding. Leave blank
+   * if you want all possible ports.
+   */
+  public static final String TEZ_AM_CLIENT_AM_PORT_RANGE =
+      TEZ_AM_PREFIX + "client.am.port-range";
+
+
+  public static final String TEZ_AM_RESOURCE_MEMORY_MB = TEZ_AM_PREFIX
+      + "resource.memory.mb";
+  public static final int TEZ_AM_RESOURCE_MEMORY_MB_DEFAULT = 1536;
+
+  public static final String TEZ_AM_RESOURCE_CPU_VCORES = TEZ_AM_PREFIX
+      + "resource.cpu.vcores";
+  public static final int TEZ_AM_RESOURCE_CPU_VCORES_DEFAULT = 1;
+
+  public static final String
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_SRC_FRACTION = TEZ_AM_PREFIX
+          + "shuffle-vertex-manager.min-src-fraction";
+  public static final float
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_SRC_FRACTION_DEFAULT = 0.25f;
+
+  public static final String
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MAX_SRC_FRACTION = TEZ_AM_PREFIX
+          + "shuffle-vertex-manager.max-src-fraction";
+  public static final float
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MAX_SRC_FRACTION_DEFAULT = 0.75f;
+
+  public static final String
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_ENABLE_AUTO_PARALLEL = TEZ_AM_PREFIX +
+          "shuffle-vertex-manager.enable.auto-parallel";
+  public static final boolean
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_ENABLE_AUTO_PARALLEL_DEFAULT = false;
+
+  public static final String
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_DESIRED_TASK_INPUT_SIZE = TEZ_AM_PREFIX +
+          "shuffle-vertex-manager.desired-task-input-size";
+  public static final long
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_DESIRED_TASK_INPUT_SIZE_DEFAULT =
+          1024*1024*100L;
+
+  public static final String
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_TASK_PARALLELISM = TEZ_AM_PREFIX +
+          "shuffle-vertex-manager.min-task-parallelism";
+  public static final int
+          TEZ_AM_SHUFFLE_VERTEX_MANAGER_MIN_TASK_PARALLELISM_DEFAULT = 1;
+
+  public static final String
+          TEZ_AM_SLOWSTART_DAG_SCHEDULER_MIN_SHUFFLE_RESOURCE_FRACTION = TEZ_AM_PREFIX
+          + "slowstart-dag-scheduler.min-resource-fraction";
+  public static final float
+          TEZ_AM_SLOWSTART_DAG_SCHEDULER_MIN_SHUFFLE_RESOURCE_FRACTION_DEFAULT = 0.5f;
+
+  public static final String TEZ_AM_AGGRESSIVE_SCHEDULING = TEZ_AM_PREFIX +
+      "aggressive.scheduling";
+  public static boolean TEZ_AM_AGGRESSIVE_SCHEDULING_DEFAULT = false;
+
+  /**
+   * The complete path to the serialized dag plan file
+   * <code>TEZ_AM_PLAN_PB_BINARY</code>. Used to make the plan available to
+   * individual tasks if needed. This will typically be a path in the job submit
+   * directory.
+   */
+  public static final String TEZ_AM_PLAN_REMOTE_PATH = TEZ_AM_PREFIX
+      + "dag-am-plan.remote.path";
+
+  public static final String TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX = TEZ_AM_PREFIX
+      + "am-rm.heartbeat.interval-ms.max";
+  public static final int TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX_DEFAULT = 1000;
+
+  public static final String TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX = TEZ_TASK_PREFIX
+      + "get-task.sleep.interval-ms.max";
+  public static final int TEZ_TASK_GET_TASK_SLEEP_INTERVAL_MS_MAX_DEFAULT = 500;
+
+  public static final String TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS = TEZ_TASK_PREFIX
+      + "am.heartbeat.interval-ms.max";
+  public static final int TEZ_TASK_AM_HEARTBEAT_INTERVAL_MS_DEFAULT = 100;
+
+  public static final String TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT = TEZ_TASK_PREFIX
+      + "max-events-per-heartbeat.max";
+  public static final int TEZ_TASK_MAX_EVENTS_PER_HEARTBEAT_DEFAULT = 100;
+
+  /**
+   * Configuration to specify whether container should be reused.
+   */
+  public static final String TEZ_AM_CONTAINER_REUSE_ENABLED = TEZ_AM_PREFIX
+      + "container.reuse.enabled";
+  public static final boolean TEZ_AM_CONTAINER_REUSE_ENABLED_DEFAULT = true;
+
+  /**
+   * Whether to reuse containers for rack local tasks. Active only if reuse is
+   * enabled.
+   */
+  public static final String TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED = TEZ_AM_PREFIX
+      + "container.reuse.rack-fallback.enabled";
+  public static final boolean TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED_DEFAULT = true;
+
+  /**
+   * Whether to reuse containers for non-local tasks. Active only if reuse is
+   * enabled.
+   */
+  public static final String TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED = TEZ_AM_PREFIX
+      + "container.reuse.non-local-fallback.enabled";
+  public static final boolean TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED_DEFAULT = false;
+
+  public static final String TEZ_AM_CONTAINER_REUSE_DELAY_ALLOCATION_MILLIS = TEZ_AM_PREFIX
+      + "container.reuse.delay-allocation-millis";
+  public static final long TEZ_AM_CONTAINER_REUSE_DELAY_ALLOCATION_MILLIS_DEFAULT = 3000l;
+
+  public static final String TEZ_PB_BINARY_CONF_NAME = "tez-conf.pb";
+  public static final String TEZ_PB_PLAN_BINARY_NAME = "tez-dag.pb";
+  public static final String TEZ_PB_PLAN_TEXT_NAME = "tez-dag.pb.txt";
+
+  /*
+   * Logger properties
+   */
+  public static final String TEZ_CONTAINER_LOG4J_PROPERTIES_FILE = "tez-container-log4j.properties";
+  public static final String TEZ_CONTAINER_LOGGER_NAME = "CLA";
+  public static final String TEZ_ROOT_LOGGER_NAME = "tez.root.logger";
+  public static final String TEZ_CONTAINER_LOG_FILE_NAME = "syslog";
+  public static final String TEZ_CONTAINER_ERR_FILE_NAME = "stderr";
+  public static final String TEZ_CONTAINER_OUT_FILE_NAME = "stdout";
+
+
+  public static final String TEZ_LIB_URIS =
+      TEZ_PREFIX + "lib.uris";
+
+  public static final String TEZ_APPLICATION_TYPE = "TEZ-MR*";
+
+  public static final String LOCAL_FRAMEWORK_NAME = "local-tez";
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
new file mode 100644
index 0000000..5463d65
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConstants.java
@@ -0,0 +1,29 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+/**
+ * Specifies all constant values in Tez
+ */
+public class TezConstants {
+
+  // Env variable names
+  public static final String TEZ_AM_IS_SESSION_ENV = "TEZ_AM_IS_SESSION";
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
new file mode 100644
index 0000000..9d4b2c4
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezEntityDescriptor.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+public abstract class TezEntityDescriptor {
+
+  protected byte[] userPayload;
+  private String className;
+
+  public TezEntityDescriptor(String className) {
+    this.className = className;
+  }
+
+  public byte[] getUserPayload() {
+    return this.userPayload;
+  }
+
+  public TezEntityDescriptor setUserPayload(byte[] userPayload) {
+    this.userPayload = userPayload;
+    return this;
+  }
+
+  public String getClassName() {
+    return this.className;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/TezException.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezException.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezException.java
new file mode 100644
index 0000000..e3b14e7
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezException.java
@@ -0,0 +1,31 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.api;
+
+/**
+ *  Base TezException
+ */
+public class TezException extends Exception {
+  private static final long serialVersionUID = 6337442733802964447L;
+  public TezException(Throwable cause) { super(cause); }
+  public TezException(String message) { super(message); }
+  public TezException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
new file mode 100644
index 0000000..f55f6dd
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezUncheckedException.java
@@ -0,0 +1,33 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.api;
+
+/**
+ * Base Tez Unchecked Exception
+ */
+public class TezUncheckedException extends RuntimeException {
+
+  private static final long serialVersionUID = -4956339297375386184L;
+  
+  public TezUncheckedException(Throwable cause) { super(cause); }
+  public TezUncheckedException(String message) { super(message); }
+  public TezUncheckedException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
new file mode 100644
index 0000000..900822b
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/Vertex.java
@@ -0,0 +1,153 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
+
+public class Vertex { // FIXME rename to Task
+
+  private final String vertexName;
+  private final ProcessorDescriptor processorDescriptor;
+
+  private final int parallelism;
+  private VertexLocationHint taskLocationsHint;
+  private final Resource taskResource;
+  private Map<String, LocalResource> taskLocalResources;
+  private Map<String, String> taskEnvironment;
+
+  private final List<Vertex> inputVertices = new ArrayList<Vertex>();
+  private final List<Vertex> outputVertices = new ArrayList<Vertex>();
+  private final List<String> inputEdgeIds = new ArrayList<String>();
+  private final List<String> outputEdgeIds = new ArrayList<String>();
+  private String javaOpts = "";
+
+
+  public Vertex(String vertexName,
+      ProcessorDescriptor processorDescriptor,
+      int parallelism,
+      Resource taskResource) {
+    this.vertexName = vertexName;
+    this.processorDescriptor = processorDescriptor;
+    this.parallelism = parallelism;
+    this.taskResource = taskResource;
+    if (parallelism == 0) {
+      throw new IllegalArgumentException("Parallelism cannot be 0");
+    }
+    if (taskResource == null) {
+      throw new IllegalArgumentException("Resource cannot be null");
+    }
+  }
+
+  public String getVertexName() { // FIXME rename to getName()
+    return vertexName;
+  }
+
+  public ProcessorDescriptor getProcessorDescriptor() {
+    return this.processorDescriptor;
+  }
+
+  public int getParallelism() {
+    return parallelism;
+  }
+
+  public Resource getTaskResource() {
+    return taskResource;
+  }
+
+  public Vertex setTaskLocationsHint(List<TaskLocationHint> locations) {
+    if (locations == null) {
+      return this;
+    }
+    assert locations.size() == parallelism;
+    taskLocationsHint = new VertexLocationHint(parallelism, locations);
+    return this;
+  }
+
+  // used internally to create parallelism location resource file
+  VertexLocationHint getTaskLocationsHint() {
+    return taskLocationsHint;
+  }
+
+  public Vertex setTaskLocalResources(Map<String, LocalResource> localResources) {
+    this.taskLocalResources = localResources;
+    return this;
+  }
+
+  public Map<String, LocalResource> getTaskLocalResources() {
+    return taskLocalResources;
+  }
+
+  public Vertex setTaskEnvironment(Map<String, String> environment) {
+    this.taskEnvironment = environment;
+    return this;
+  }
+
+  public Map<String, String> getTaskEnvironment() {
+    return taskEnvironment;
+  }
+
+  public Vertex setJavaOpts(String javaOpts){
+     this. javaOpts = javaOpts;
+     return this;
+  }
+
+  public String getJavaOpts(){
+	  return javaOpts;
+  }
+
+  @Override
+  public String toString() {
+    return "[" + vertexName + " : " + processorDescriptor.getClassName() + "]";
+  }
+
+  void addInputVertex(Vertex inputVertex, String edgeId) {
+    inputVertices.add(inputVertex);
+    inputEdgeIds.add(edgeId);
+  }
+
+  void addOutputVertex(Vertex outputVertex, String edgeId) {
+    outputVertices.add(outputVertex);
+    outputEdgeIds.add(edgeId);
+  }
+
+  List<Vertex> getInputVertices() {
+    return inputVertices;
+  }
+
+  List<Vertex> getOutputVertices() {
+    return outputVertices;
+  }
+
+  List<String> getInputEdgeIds() {
+    return inputEdgeIds;
+  }
+
+  List<String> getOutputEdgeIds() {
+    return outputEdgeIds;
+  }
+
+  // FIXME how do we support profiling? Can't profile all tasks.
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java b/tez-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
new file mode 100644
index 0000000..4f19314
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/VertexLocationHint.java
@@ -0,0 +1,154 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+public class VertexLocationHint  {
+
+  private final int numTasks;
+  private final List<TaskLocationHint> taskLocationHints;
+
+  public VertexLocationHint(int numTasks,
+      List<TaskLocationHint> taskLocationHints) {
+    this.numTasks = numTasks;
+    if (taskLocationHints != null) {
+      this.taskLocationHints = Collections.unmodifiableList(taskLocationHints);
+    } else {
+      this.taskLocationHints = null;
+    }
+  }
+
+  public int getNumTasks() {
+    return numTasks;
+  }
+
+  public List<TaskLocationHint> getTaskLocationHints() {
+    return taskLocationHints;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 7883;
+    int result = 1;
+    result = prime * result + numTasks;
+    if (taskLocationHints != null) {
+      result = prime * result + taskLocationHints.hashCode();
+    }
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    VertexLocationHint other = (VertexLocationHint) obj;
+    if (numTasks != other.numTasks) {
+      return false;
+    }
+    if (taskLocationHints != null) {
+      if (!taskLocationHints.equals(other.taskLocationHints)) {
+        return false;
+      }
+    } else if (other.taskLocationHints != null) {
+      return false;
+    }
+    return true;
+  }
+
+  public static class TaskLocationHint {
+
+    // Host names if any to be used
+    private final Set<String> hosts;
+    // Rack names if any to be used
+    private final Set<String> racks;
+
+    public TaskLocationHint(Set<String> hosts, Set<String> racks) {
+      if (hosts != null) {
+        this.hosts = Collections.unmodifiableSet(hosts);
+      } else {
+        this.hosts = null;
+      }
+      if (racks != null) {
+        this.racks = Collections.unmodifiableSet(racks);
+      } else {
+        this.racks = null;
+      }
+    }
+
+    public Set<String> getDataLocalHosts() {
+      return hosts;
+    }
+
+    public Set<String> getRacks() {
+      return racks;
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 9397;
+      int result = 1;
+      result = ( hosts != null) ?
+          prime * result + hosts.hashCode() :
+          result + prime;
+      result = ( racks != null) ?
+          prime * result + racks.hashCode() :
+          result + prime;
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) {
+        return true;
+      }
+      if (obj == null) {
+        return false;
+      }
+      if (getClass() != obj.getClass()) {
+        return false;
+      }
+      TaskLocationHint other = (TaskLocationHint) obj;
+      if (hosts != null) {
+        if (!hosts.equals(other.hosts)) {
+          return false;
+        }
+      } else if (other.hosts != null) {
+        return false;
+      }
+      if (racks != null) {
+        if (!racks.equals(other.racks)) {
+          return false;
+        }
+      } else if (other.racks != null) {
+        return false;
+      }
+      return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
new file mode 100644
index 0000000..9062e8e
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGClient.java
@@ -0,0 +1,67 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.api.client;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.tez.dag.api.TezException;
+
+/*
+ * Interface class for monitoring the <code>DAG</code> running in a Tez DAG
+ * Application Master.
+ */
+public interface DAGClient extends Closeable {
+
+  /**
+   * Get the YARN ApplicationId for the app running the DAG
+   * @return <code>ApplicationId</code>
+   */
+  public ApplicationId getApplicationId();
+
+  @Private
+  /**
+   * Get the YARN ApplicationReport for the app running the DAG. For performance
+   * reasons this may be stale copy and should be used to access static info. It
+   * may be null.
+   * @return <code>ApplicationReport</code> or null
+   */
+  public ApplicationReport getApplicationReport();
+
+  /**
+   * Get the status of the specified DAG
+   */
+  public DAGStatus getDAGStatus() throws IOException, TezException;
+
+  /**
+   * Get the status of a Vertex of a DAG
+   */
+  public VertexStatus getVertexStatus(String vertexName)
+      throws IOException, TezException;
+
+  /**
+   * Kill a running DAG
+   *
+   */
+  public void tryKillDAG() throws TezException, IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
new file mode 100644
index 0000000..d61173d
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
@@ -0,0 +1,130 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.api.client;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.tez.dag.api.records.DAGProtos.DAGStatusProtoOrBuilder;
+import org.apache.tez.dag.api.records.DAGProtos.StringProgressPairProto;
+import org.apache.tez.dag.api.TezUncheckedException;
+
+public class DAGStatus {
+
+  public enum State {
+    SUBMITTED,
+    INITING,
+    RUNNING,
+    SUCCEEDED,
+    KILLED,
+    FAILED,
+    ERROR,
+  };
+
+  DAGStatusProtoOrBuilder proxy = null;
+  Progress progress = null;
+  Map<String, Progress> vertexProgress = null;
+
+  public DAGStatus(DAGStatusProtoOrBuilder proxy) {
+    this.proxy = proxy;
+  }
+
+  public State getState() {
+    switch(proxy.getState()) {
+    case DAG_SUBMITTED:
+      return DAGStatus.State.SUBMITTED;
+    // For simplicity, initing/terminating states are presented as running
+    case DAG_INITING:
+    case DAG_TERMINATING:
+    case DAG_RUNNING:
+      return DAGStatus.State.RUNNING;
+    case DAG_SUCCEEDED:
+      return DAGStatus.State.SUCCEEDED;
+    case DAG_FAILED:
+      return DAGStatus.State.FAILED;
+    case DAG_KILLED:
+      return DAGStatus.State.KILLED;
+    case DAG_ERROR:
+      return DAGStatus.State.ERROR;
+    default:
+      throw new TezUncheckedException("Unsupported value for DAGStatus.State : " +
+                              proxy.getState());
+    }
+  }
+
+  public boolean isCompleted() {
+    State state = getState();
+    return (state == State.SUCCEEDED ||
+             state == State.FAILED ||
+             state == State.KILLED ||
+             state == State.ERROR);
+  }
+
+  public List<String> getDiagnostics() {
+    return proxy.getDiagnosticsList();
+  }
+
+  /**
+   * Gets overall progress value of the DAG.
+   *
+   * @return Progress of the DAG. Maybe null when the DAG is not running. Maybe
+   *         null when the DAG is running and the application master cannot be
+   *         reached - e.g. when the execution platform has restarted the
+   *         application master.
+   * @see Progress
+   */
+  public Progress getDAGProgress() {
+    if(progress == null && proxy.hasDAGProgress()) {
+      progress = new Progress(proxy.getDAGProgress());
+    }
+    return progress;
+  }
+
+  /**
+   * Get the progress of a vertex in the DAG
+   *
+   * @return Progress of the vertex. May be null when the DAG is not running.
+   *         Maybe null when the DAG is running and the application master
+   *         cannot be reached - e.g. when the execution platform has restarted
+   *         the application master.
+   * @see Progress
+   */
+  public Map<String, Progress> getVertexProgress() {
+    if(vertexProgress == null) {
+      if(proxy.getVertexProgressList() != null) {
+        List<StringProgressPairProto> kvList = proxy.getVertexProgressList();
+        vertexProgress = new HashMap<String, Progress>(kvList.size());
+        for(StringProgressPairProto kv : kvList){
+          vertexProgress.put(kv.getKey(), new Progress(kv.getProgress()));
+        }
+      }
+    }
+    return vertexProgress;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("status=" + getState()
+        + ", progress=" + getDAGProgress());
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
new file mode 100644
index 0000000..9577320
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/Progress.java
@@ -0,0 +1,67 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.api.client;
+
+import org.apache.tez.dag.api.records.DAGProtos.ProgressProtoOrBuilder;
+
+public class Progress {
+  
+  ProgressProtoOrBuilder proxy = null;
+  
+  Progress(ProgressProtoOrBuilder proxy) {
+    this.proxy = proxy;
+  }
+  
+  public int getTotalTaskCount() {
+    return proxy.getTotalTaskCount();
+  }
+
+  public int getSucceededTaskCount() {
+    return proxy.getSucceededTaskCount();
+  }
+
+  public int getRunningTaskCount() {
+    return proxy.getRunningTaskCount();
+  }
+
+  public int getFailedTaskCount() {
+    return proxy.getFailedTaskCount();
+  }
+
+  public int getKilledTaskCount() {
+    return proxy.getKilledTaskCount();
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("TotalTasks: ");
+    sb.append(getTotalTaskCount());
+    sb.append(" Succeeded: ");
+    sb.append(getSucceededTaskCount());
+    sb.append(" Running: ");
+    sb.append(getRunningTaskCount());
+    sb.append(" Failed: "); 
+    sb.append(getFailedTaskCount());
+    sb.append(" Killed: "); 
+    sb.append(getKilledTaskCount());
+    return sb.toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
new file mode 100644
index 0000000..ce5dbe0
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/VertexStatus.java
@@ -0,0 +1,78 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.dag.api.client;
+
+import java.util.List;
+
+import org.apache.tez.dag.api.records.DAGProtos.VertexStatusProtoOrBuilder;
+import org.apache.tez.dag.api.TezUncheckedException;
+
+public class VertexStatus {
+  
+  public enum State {
+    INITED,
+    RUNNING,
+    SUCCEEDED,
+    KILLED,
+    FAILED,
+    ERROR,
+    TERMINATING,
+  };
+  
+  VertexStatusProtoOrBuilder proxy = null;
+  Progress progress = null;
+  
+  public VertexStatus(VertexStatusProtoOrBuilder proxy) {
+    this.proxy = proxy;
+  }
+
+  public State getState() {
+    switch(proxy.getState()) {
+    case VERTEX_INITED:
+      return VertexStatus.State.INITED;
+    case VERTEX_RUNNING:
+      return VertexStatus.State.RUNNING;
+    case VERTEX_SUCCEEDED:
+      return VertexStatus.State.SUCCEEDED;
+    case VERTEX_FAILED:
+      return VertexStatus.State.FAILED;
+    case VERTEX_KILLED:
+      return VertexStatus.State.KILLED;
+    case VERTEX_ERROR:
+      return VertexStatus.State.ERROR;
+    case VERTEX_TERMINATING:
+      return VertexStatus.State.TERMINATING;
+    default:
+      throw new TezUncheckedException("Unsupported value for VertexStatus.State : " + 
+                              proxy.getState());
+    }    
+  }
+
+  public List<String> getDiagnostics() {
+    return proxy.getDiagnosticsList();
+  }
+
+  public Progress getProgress() {
+    if(progress == null && proxy.hasProgress()) {
+      progress = new Progress(proxy.getProgress());
+    }
+    return progress;    
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
new file mode 100644
index 0000000..a1ee18f
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPB.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.dag.api.client.rpc;
+
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.DAGClientAMProtocol;
+
+@ProtocolInfo(
+    protocolName = "org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolPB",
+    protocolVersion = 1)
+public interface DAGClientAMProtocolBlockingPB 
+                              extends DAGClientAMProtocol.BlockingInterface {
+
+}


[06/20] Rename *.new* packages back to what they should be, remove dead code from the old packages - mapreduce module - tez-engine module (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
index e5cc902..404dd8c 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
@@ -28,8 +28,8 @@ import org.apache.tez.common.ContainerTask;
 import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.common.records.ProceedToCompletionResponse;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.newapi.impl.TezHeartbeatResponse;
+import org.apache.tez.engine.api.impl.TezHeartbeatRequest;
+import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
 import org.apache.tez.engine.records.OutputContext;
 import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
----------------------------------------------------------------------
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 2bc327c..85e6653 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
@@ -44,14 +44,13 @@ import org.apache.hadoop.mapreduce.split.JobSplit;
 import org.apache.hadoop.mapreduce.split.JobSplit.SplitMetaInfo;
 import org.apache.hadoop.mapreduce.split.SplitMetaInfoReaderTez;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.OutputSpec;
-import org.apache.tez.common.TezEngineTaskContext;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.engine.api.Task;
-import org.apache.tez.engine.runtime.RuntimeUtils;
+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.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
@@ -174,9 +173,9 @@ public class MapUtils {
     outMeta.close();
   }
 
-  public static Task runMapProcessor(FileSystem fs, Path workDir,
+  public static LogicalIOProcessorRuntimeTask runMapProcessor(FileSystem fs, Path workDir,
       JobConf jobConf, int mapId, Path mapInput,
-      TezTaskUmbilicalProtocol umbilical,
+      TezUmbilical umbilical,
       String vertexName, List<InputSpec> inputSpecs,
       List<OutputSpec> outputSpecs) throws Exception {
     jobConf.setInputFormat(SequenceFileInputFormat.class);
@@ -185,14 +184,24 @@ public class MapUtils {
     ProcessorDescriptor mapProcessorDesc = new ProcessorDescriptor(
         MapProcessor.class.getName());
     writeSplitFiles(fs, jobConf, split);
-    TezEngineTaskContext taskContext = new TezEngineTaskContext(
-        TezTestUtils.getMockTaskAttemptId(0, 0, mapId, 0), "testuser",
-        "testJob", vertexName, mapProcessorDesc,
-        inputSpecs, outputSpecs);
-
-    Task t = RuntimeUtils.createRuntimeTask(taskContext);
-    t.initialize(jobConf, null, umbilical);
-    t.getProcessor().process(t.getInputs(), t.getOutputs());
-    return t;
+
+    TaskSpec taskSpec = new TaskSpec(
+        TezTestUtils.getMockTaskAttemptId(0, 0, mapId, 0),
+        "testuser",
+        vertexName,
+        mapProcessorDesc,
+        inputSpecs,
+        outputSpecs);
+    
+    // TODO NEWTEZ Fix umbilical access
+    LogicalIOProcessorRuntimeTask task = new LogicalIOProcessorRuntimeTask(
+        taskSpec,
+        1,
+        jobConf,
+        umbilical,
+        null);
+    task.initialize();
+    task.run();
+    return task;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
----------------------------------------------------------------------
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 84f1f81..2ecce8b 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,24 +31,24 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.tez.common.Constants;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.OutputSpec;
 import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.api.Task;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.engine.api.impl.InputSpec;
+import org.apache.tez.engine.api.impl.OutputSpec;
+import org.apache.tez.engine.api.impl.TezUmbilical;
+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.oldoutput.OldLocalOnFileSorterOutput;
-import org.apache.tez.mapreduce.TestUmbilicalProtocol;
+import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
+import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
-import org.apache.tez.mapreduce.hadoop.mapreduce.TezNullOutputCommitter;
 import org.apache.tez.mapreduce.input.SimpleInputLegacy;
-import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MapUtils;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -75,7 +75,7 @@ public class TestMapProcessor {
   
 
 
-  TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles();
+  
 
   public void setUpJobConf(JobConf job) {
     job.set(TezJobConfig.LOCAL_DIRS, workDir.toString());
@@ -97,8 +97,7 @@ public class TestMapProcessor {
     String vertexName = MultiStageMRConfigUtil.getInitialMapVertexName();
     JobConf jobConf = new JobConf(defaultConf);
     setUpJobConf(jobConf);
-    TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles();
-    mapOutputs.setConf(jobConf);
+    TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles(jobConf, "TODONEWTEZ_uniqueId");
 
     Configuration conf = MultiStageMRConfToTezTranslator.convertMRToLinearTez(jobConf);
     conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0);
@@ -111,20 +110,22 @@ public class TestMapProcessor {
     job.set(TezJobConfig.TASK_LOCAL_RESOURCE_DIR, new Path(workDir,
         "localized-resources").toUri().toString());
     
+    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(SimpleInputLegacy.class.getName()), 1);
+    OutputSpec mapOutputSpec = new OutputSpec("NullDestVertex", new OutputDescriptor(LocalOnFileSorterOutput.class.getName()), 1);
 
-    Task t = MapUtils.runMapProcessor(localFs, workDir, job, 0,
-        new Path(workDir, "map0"), new TestUmbilicalProtocol(), vertexName,
-        Collections.singletonList(new InputSpec("NullVertex", 0,
-            SimpleInputLegacy.class.getName())),
-        Collections.singletonList(new OutputSpec("FakeVertex", 1,
-            OldLocalOnFileSorterOutput.class.getName())));
+    // TODO NEWTEZ FIXME TezUmbilical handling
+    LogicalIOProcessorRuntimeTask t = MapUtils.runMapProcessor(localFs, workDir, job, 0,
+        new Path(workDir, "map0"), (TezUmbilical) null, vertexName,
+        Collections.singletonList(mapInputSpec),
+        Collections.singletonList(mapOutputSpec));
 
-    MRTask mrTask = (MRTask)t.getProcessor();
-    Assert.assertEquals(TezNullOutputCommitter.class.getName(), mrTask
-        .getCommitter().getClass().getName());
-    t.close();
+    // TODO NEWTEZ FIXME OutputCommitter verification
+//    MRTask mrTask = (MRTask)t.getProcessor();
+//    Assert.assertEquals(TezNullOutputCommitter.class.getName(), mrTask
+//        .getCommitter().getClass().getName());
+//    t.close();
 
-    Path mapOutputFile = mapOutputs.getInputFile(0);
+    Path mapOutputFile = mapOutputs.getInputFile(new InputAttemptIdentifier(0, 0));
     LOG.info("mapOutputFile = " + mapOutputFile);
     IFile.Reader reader =
         new IFile.Reader(job, localFs, mapOutputFile, null, null);

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b4950f98/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
----------------------------------------------------------------------
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 8bcd353..1d35f9b 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
@@ -32,18 +32,19 @@ import org.apache.hadoop.mapred.FileOutputFormat;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.tez.common.Constants;
-import org.apache.tez.common.InputSpec;
-import org.apache.tez.common.OutputSpec;
-import org.apache.tez.common.TezEngineTaskContext;
 import org.apache.tez.common.TezJobConfig;
+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.Task;
+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.task.local.output.TezLocalTaskOutputFiles;
 import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.lib.oldinput.LocalMergedInput;
-import org.apache.tez.engine.lib.oldoutput.OldLocalOnFileSorterOutput;
-import org.apache.tez.engine.runtime.RuntimeUtils;
-import org.apache.tez.mapreduce.TestUmbilicalProtocol;
+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.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
@@ -51,7 +52,6 @@ import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.input.SimpleInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;
-import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MapUtils;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
 import org.junit.After;
@@ -104,8 +104,7 @@ public class TestReduceProcessor {
     String reduceVertexName = MultiStageMRConfigUtil.getFinalReduceVertexName();
     JobConf jobConf = new JobConf(defaultConf);
     setUpJobConf(jobConf);
-    TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles();
-    mapOutputs.setConf(jobConf);
+    TezTaskOutput mapOutputs = new TezLocalTaskOutputFiles(jobConf, "TODONEWTEZ_uniqueId");
     
     Configuration conf = MultiStageMRConfToTezTranslator.convertMRToLinearTez(jobConf);
     conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 0);
@@ -117,14 +116,16 @@ public class TestReduceProcessor {
     mapConf.set(TezJobConfig.TASK_LOCAL_RESOURCE_DIR, new Path(workDir,
         "localized-resources").toUri().toString());
     
-    
+    InputSpec mapInputSpec = new InputSpec("NullSrcVertex", new InputDescriptor(
+        SimpleInputLegacy.class.getName()), 0);
+    OutputSpec mapOutputSpec = new OutputSpec("NullDestVertex", new OutputDescriptor(
+        LocalOnFileSorterOutput.class.getName()), 1);
     // Run a map
+    // TODO NEWTEZ FIX Umbilical creation
     MapUtils.runMapProcessor(localFs, workDir, mapConf, 0,
-        new Path(workDir, "map0"), new TestUmbilicalProtocol(), mapVertexName,
-        Collections.singletonList(new InputSpec("NullVertex", 0,
-            SimpleInputLegacy.class.getName())),
-        Collections.singletonList(new OutputSpec("FakeVertex", 1,
-            OldLocalOnFileSorterOutput.class.getName())));
+        new Path(workDir, "map0"), (TezUmbilical) null, mapVertexName,
+        Collections.singletonList(mapInputSpec),
+        Collections.singletonList(mapOutputSpec));
 
     LOG.info("Starting reduce...");
     
@@ -138,28 +139,43 @@ public class TestReduceProcessor {
     FileOutputFormat.setOutputPath(reduceConf, new Path(workDir, "output"));
     ProcessorDescriptor reduceProcessorDesc = new ProcessorDescriptor(
         ReduceProcessor.class.getName());
+    
+    InputSpec reduceInputSpec = new InputSpec(mapVertexName, new InputDescriptor(LocalMergedInput.class.getName()), 1);
+    OutputSpec reduceOutputSpec = new OutputSpec("NullDestinationVertex", new OutputDescriptor(SimpleOutput.class.getName()), 1);
+    
     // Now run a reduce
-    TezEngineTaskContext taskContext = new TezEngineTaskContext(
-        TezTestUtils.getMockTaskAttemptId(0, 1, 0, 0), "testUser",
-        "testJob", reduceVertexName, reduceProcessorDesc,
-        Collections.singletonList(new InputSpec(mapVertexName, 1,
-            LocalMergedInput.class.getName())),
-        Collections.singletonList(new OutputSpec("", 1,
-                SimpleOutput.class.getName())));
+    TaskSpec taskSpec = new TaskSpec(
+        TezTestUtils.getMockTaskAttemptId(0, 1, 0, 0),
+        "testUser",
+        reduceVertexName,
+        reduceProcessorDesc,
+        Collections.singletonList(reduceInputSpec),
+        Collections.singletonList(reduceOutputSpec));
+    
+    // TODO NEWTEZ FIXME Umbilical and jobToken
+    LogicalIOProcessorRuntimeTask task = new LogicalIOProcessorRuntimeTask(
+        taskSpec,
+        1,
+        reduceConf,
+        (TezUmbilical) null,
+        null);
     
-    Task t = RuntimeUtils.createRuntimeTask(taskContext);
-    t.initialize(reduceConf, null, new TestUmbilicalProtocol());
-    t.run();
-    MRTask mrTask = (MRTask)t.getProcessor();
+    task.initialize();
+    task.run();
+    
+//    MRTask mrTask = (MRTask)t.getProcessor();
 //    TODO NEWTEZ Verify the partitioner has been created
 //    Assert.assertNull(mrTask.getPartitioner());
-    t.close();
+    task.close();
     
     // Can this be done via some utility class ? MapOutputFile derivative, or
     // instantiating the OutputCommitter
+    
+
+    // TODO NEWTEZ FIXME uniqueId generation and event generation (mockTaskId will not work here)
     Path reduceOutputDir = new Path(new Path(workDir, "output"),
         "_temporary/0/" + IDConverter
-            .toMRTaskId(taskContext.getTaskAttemptId().getTaskID()));
+            .toMRTaskId(TezTestUtils.getMockTaskId(0, 1, 0)));
     Path reduceOutputFile = new Path(reduceOutputDir, "part-00000");
 
     SequenceFile.Reader reader = new SequenceFile.Reader(localFs,


[11/20] TEZ-443. Merge tez-dag-api and tez-engine-api into a single module - tez-api (part of TEZ-398). (sseth)

Posted by ss...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
index ab78d82..cc29e94 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.java
@@ -29,11 +29,11 @@ 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.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
 
 public class TezEngineUtils {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java b/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
index 6f86d61..b387b36 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.java
@@ -23,7 +23,6 @@ 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.common.TezTaskContext;
 import org.apache.tez.engine.common.sort.impl.IFile.Writer;
 import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java b/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
index 1cb89a7..546151f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
@@ -32,12 +32,12 @@ import org.apache.hadoop.util.ReflectionUtils;
 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.engine.newapi.TezInputContext;
 
 @SuppressWarnings({"rawtypes"})
 public class LocalShuffle {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
index 5c71644..b48bb0b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
@@ -46,12 +46,12 @@ import org.apache.hadoop.security.ssl.SSLFactory;
 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.engine.newapi.TezInputContext;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
index bf2be4e..b8792fb 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
@@ -46,6 +46,7 @@ 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;
@@ -56,7 +57,6 @@ 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.engine.newapi.TezInputContext;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
index f605b7c..15332a1 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.java
@@ -38,12 +38,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.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.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
index 98c7ab1..a8e5fe4 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
@@ -24,14 +24,14 @@ 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.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.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
-import org.apache.tez.engine.newapi.events.InputFailedEvent;
-import org.apache.tez.engine.newapi.events.InputInformationEvent;
 
 import com.google.common.base.Preconditions;
 import com.google.protobuf.InvalidProtocolBufferException;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
index acbd054..be75668 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
@@ -40,11 +40,11 @@ import org.apache.hadoop.conf.Configuration;
 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.engine.newapi.Event;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.events.InputReadErrorEvent;
 
 import com.google.common.collect.Lists;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
index f61670e..a6d1c5b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
@@ -64,12 +64,12 @@ import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext;
 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.newapi.TezOutputContext;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 import org.jboss.netty.bootstrap.ServerBootstrap;
 import org.jboss.netty.buffer.ChannelBuffers;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
index 8b4bd4e..5aa0ddf 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
@@ -43,6 +43,7 @@ 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;
@@ -50,7 +51,6 @@ 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.engine.newapi.TezOutputContext;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public abstract class ExternalSorter {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
index 952568e..1bf17a3 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
@@ -44,10 +44,10 @@ import org.apache.hadoop.util.IndexedSortable;
 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.engine.newapi.TezOutputContext;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class PipelinedSorter extends ExternalSorter {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
index 1ad31f7..6b48270 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.util.IndexedSortable;
 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;
@@ -49,7 +50,6 @@ 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.engine.newapi.TezOutputContext;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class DefaultSorter extends ExternalSorter implements IndexedSortable {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
index 53e6003..e2b3315 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
@@ -29,10 +29,10 @@ import org.apache.commons.logging.LogFactory;
 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.engine.newapi.TezOutputContext;
 
 public class InMemoryShuffleSorter extends DefaultSorter {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
index 8ae6bfe..ed57c61 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.java
@@ -22,10 +22,10 @@ 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.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.TezInputContext;
 
 /**
  * <code>LocalMergedInput</code> in an {@link LogicalInput} which shuffles intermediate

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
index c719fba..3db0632 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
@@ -29,14 +29,14 @@ import org.apache.tez.common.TezJobConfig;
 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.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.TezInputContext;
 
 /**
  * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
index c2126ca..44238fd 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
@@ -23,11 +23,11 @@ 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.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.Reader;
-import org.apache.tez.engine.newapi.TezInputContext;
 
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
index 218aa21..26a01c8 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
@@ -22,13 +22,13 @@ 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.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.Output;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.Writer;
 
 /**
  * {@link InMemorySortedOutput} is an {@link Output} which sorts key/value pairs 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
index 963276d..b24e10d 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
@@ -25,8 +25,8 @@ import org.apache.commons.logging.Log;
 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.engine.newapi.Event;
 
 public class LocalOnFileSorterOutput extends OnFileSortedOutput {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
index 7e0ca37..685722e 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
@@ -26,14 +26,14 @@ import org.apache.hadoop.conf.Configuration;
 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.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
index 37edde8..3ff603f 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
@@ -24,13 +24,13 @@ import java.util.List;
 
 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.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.events.DataMovementEvent;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
index 77299de..bfd898b 100644
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
+++ b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
@@ -38,6 +38,16 @@ import org.apache.hadoop.util.StringUtils;
 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;
@@ -49,16 +59,6 @@ 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.newapi.Event;
-import org.apache.tez.engine.newapi.Input;
-import org.apache.tez.engine.newapi.LogicalIOProcessor;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.Output;
-import org.apache.tez.engine.newapi.Processor;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezProcessorContext;
 import org.apache.tez.engine.shuffle.common.ShuffleUtils;
 
 import com.google.common.base.Preconditions;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml
index ea5016b..2c6b78e 100644
--- a/tez-mapreduce/pom.xml
+++ b/tez-mapreduce/pom.xml
@@ -27,7 +27,7 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-dag-api</artifactId>
+      <artifactId>tez-api</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/common/TezTaskStatus.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/common/TezTaskStatus.java b/tez-mapreduce/src/main/java/org/apache/tez/common/TezTaskStatus.java
new file mode 100644
index 0000000..45ea80e
--- /dev/null
+++ b/tez-mapreduce/src/main/java/org/apache/tez/common/TezTaskStatus.java
@@ -0,0 +1,105 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.common;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+
+// TODO NEWTEZ Get rid of this.
+public interface TezTaskStatus extends Writable {
+
+  //enumeration for reporting current phase of a task.
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public static enum Phase{STARTING, MAP, SHUFFLE, SORT, REDUCE, CLEANUP}
+
+  // what state is the task in?
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public static enum State {RUNNING, SUCCEEDED, FAILED, UNASSIGNED, KILLED, 
+                            COMMIT_PENDING, FAILED_UNCLEAN, KILLED_UNCLEAN}
+
+  public abstract TezTaskAttemptID getTaskAttemptId();
+
+  public abstract float getProgress();
+
+  public abstract void setProgress(float progress);
+
+  public abstract State getRunState();
+
+  public abstract void setRunState(State runState);
+
+  public abstract String getDiagnosticInfo();
+
+  public abstract void setDiagnosticInfo(String info);
+
+  // TODOTEZDAG Remove stateString / rename
+  public abstract String getStateString();
+
+  public abstract void setStateString(String stateString);
+
+  public abstract long getFinishTime();
+
+  public abstract void setFinishTime(long finishTime);
+  
+  // TODOTEZDAG Can shuffle / merge be made generic ? Otherwise just a single finish time.
+  public abstract long getShuffleFinishTime();
+
+  public abstract void setShuffleFinishTime(long shuffleFinishTime);
+  
+  public abstract long getMapFinishTime();
+
+  public abstract void setMapFinishTime(long mapFinishTime);
+  
+  public abstract long getSortFinishTime();
+  
+  public abstract void setSortFinishTime(long sortFinishTime);
+  
+  public abstract long getStartTime();
+  
+  public abstract void setStartTime(long startTime);
+
+  // TODOTEZDAG Remove phase
+  public abstract Phase getPhase();
+
+  public abstract void setPhase(Phase phase);
+
+  public abstract TezCounters getCounters();
+
+  public abstract void setCounters(TezCounters counters);
+
+  public abstract List<TezTaskAttemptID> getFailedDependencies();
+
+  public abstract void addFailedDependency(TezTaskAttemptID taskAttempttId);
+
+  public abstract void clearStatus();
+
+  public abstract void statusUpdate(float f, String string, TezCounters counters);
+
+  // TODOTEZDAG maybe remove ?
+  public abstract long getLocalOutputSize();
+
+  public abstract void setOutputSize(long l);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
----------------------------------------------------------------------
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 56e88c7..199bbfe 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,14 +42,14 @@ import org.apache.hadoop.util.Progress;
 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.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java
index 768d347..889c64c 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/TezTypeConverters.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.TypeConverter;
-import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEventStatus;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.tez.common.TezTaskStatus.Phase;
 import org.apache.tez.common.counters.CounterGroup;
@@ -30,7 +29,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.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 
 public class TezTypeConverters {
 
@@ -66,13 +64,6 @@ public class TezTypeConverters {
     return tezTaskAttemptId;
   }
 
-  public static TezDependentTaskCompletionEvent.Status toTez(
-      TaskAttemptCompletionEventStatus status) {
-    return TezDependentTaskCompletionEvent.Status.valueOf(status.toString());
-  }
-
-  
-  
   public static Counters fromTez(TezCounters tezCounters) {
     if (tezCounters == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
----------------------------------------------------------------------
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 f5e08dc..2a926d7 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,8 +3,8 @@ package org.apache.tez.mapreduce.hadoop.mapred;
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.Reporter;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.engine.api.TezProcessorContext;
+import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.common.Utils;
 
 public class MRReporter implements Reporter {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/MapContextImpl.java
----------------------------------------------------------------------
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 b0348c9..dcdb3ff 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.OutputCommitter;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.engine.api.TezTaskContext;
 
 /**
  * The context that is given to the {@link Mapper}.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
----------------------------------------------------------------------
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 d8548a4..4035c71 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,7 +24,7 @@ import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.common.Utils;
 
 // 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.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskInputOutputContextImpl.java
----------------------------------------------------------------------
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 c2920dc..05ea89c 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.RecordWriter;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskInputOutputContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
+import org.apache.tez.engine.api.TezTaskContext;
 
 /**
  * A context object that allows input and output from the task. It is only

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
index f03fdc7..598f801 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/SimpleInput.java
@@ -46,10 +46,10 @@ import org.apache.hadoop.util.ReflectionUtils;
 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.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.TezInputContext;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
index dd28ab8..d82c9e2 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/SimpleOutput.java
@@ -26,10 +26,10 @@ import org.apache.hadoop.util.ReflectionUtils;
 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.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezOutputContext;
+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;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
----------------------------------------------------------------------
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 af087c9..1a01466 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
@@ -71,12 +71,11 @@ import org.apache.tez.common.counters.TaskCounter;
 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.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.records.OutputContext;
 import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
@@ -422,14 +421,6 @@ public abstract class MRTask {
       InterruptedException {
   }
 
-  public void outputReady(MRTaskReporter reporter, OutputContext outputContext)
-      throws IOException,
-      InterruptedException {
-    LOG.info("Task: " + taskAttemptId + " reporting outputReady");
-    updateCounters();
-    statusUpdate();
-  }
-
   public void done(LogicalOutput output) throws IOException, InterruptedException {
     updateCounters();
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
----------------------------------------------------------------------
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 d32b4c0..22312f7 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,10 +24,10 @@ import org.apache.hadoop.mapred.Counters;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.newapi.TezInputContext;
-import org.apache.tez.engine.newapi.TezOutputContext;
-import org.apache.tez.engine.newapi.TezProcessorContext;
-import org.apache.tez.engine.newapi.TezTaskContext;
+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;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
----------------------------------------------------------------------
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 e1ed168..2084146 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,14 +35,14 @@ import org.apache.hadoop.util.ReflectionUtils;
 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.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalIOProcessor;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
 import org.apache.tez.mapreduce.hadoop.mapreduce.MapContextImpl;
 import org.apache.tez.mapreduce.input.SimpleInput;
 import org.apache.tez.mapreduce.input.SimpleInputLegacy;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
----------------------------------------------------------------------
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 b7288e8..9210187 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,16 +37,16 @@ import org.apache.hadoop.util.Progressable;
 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.output.OnFileSortedOutput;
-import org.apache.tez.engine.newapi.Event;
-import org.apache.tez.engine.newapi.LogicalIOProcessor;
-import org.apache.tez.engine.newapi.LogicalInput;
-import org.apache.tez.engine.newapi.LogicalOutput;
-import org.apache.tez.engine.newapi.TezProcessorContext;
 import org.apache.tez.mapreduce.input.ShuffledMergedInputLegacy;
 import org.apache.tez.mapreduce.output.SimpleOutput;
 import org.apache.tez.mapreduce.processor.MRTask;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
index 404dd8c..d5823f7 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilicalProtocol.java
@@ -30,8 +30,6 @@ import org.apache.tez.common.records.ProceedToCompletionResponse;
 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.engine.records.OutputContext;
-import org.apache.tez.engine.records.TezTaskDependencyCompletionEventsUpdate;
 
 public class TestUmbilicalProtocol implements TezTaskUmbilicalProtocol {
 
@@ -65,14 +63,6 @@ public class TestUmbilicalProtocol implements TezTaskUmbilicalProtocol {
   }
 
   @Override
-  public TezTaskDependencyCompletionEventsUpdate getDependentTasksCompletionEvents(
-      int fromEventIdx, int maxEventsToFetch,
-      TezTaskAttemptID reduce) {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
   public ContainerTask getTask(ContainerContext containerContext)
       throws IOException {
     // TODO Auto-generated method stub
@@ -86,13 +76,6 @@ public class TestUmbilicalProtocol implements TezTaskUmbilicalProtocol {
   }
 
   @Override
-  public void outputReady(TezTaskAttemptID taskAttemptId,
-      OutputContext outputContext) throws IOException {
-    // TODO Auto-generated method stub
-
-  }
-
-  @Override
   public ProceedToCompletionResponse proceedToCompletion(
       TezTaskAttemptID taskAttemptId) throws IOException {
     return proceedToCompletionResponse;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/d316f723/tez-yarn-client/pom.xml
----------------------------------------------------------------------
diff --git a/tez-yarn-client/pom.xml b/tez-yarn-client/pom.xml
index b142701..402126e 100644
--- a/tez-yarn-client/pom.xml
+++ b/tez-yarn-client/pom.xml
@@ -47,7 +47,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-dag-api</artifactId>
+      <artifactId>tez-api</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>