You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by hi...@apache.org on 2013/09/25 00:44:10 UTC

[01/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Updated Branches:
  refs/heads/TEZ-398 5d86b9350 -> b212ca1d2


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
new file mode 100644
index 0000000..ab7e5ba
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
@@ -0,0 +1,608 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.shuffle.common;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLConnection;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.crypto.SecretKey;
+import javax.net.ssl.HttpsURLConnection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Responsible for fetching inputs served by the ShuffleHandler for a single
+ * host. Construct using {@link FetcherBuilder}
+ */
+public class Fetcher implements Callable<FetchResult> {
+
+  private static final Log LOG = LogFactory.getLog(Fetcher.class);
+
+  private static final int UNIT_CONNECT_TIMEOUT = 60 * 1000;
+  private static final AtomicInteger fetcherIdGen = new AtomicInteger(0);
+
+  // Configurable fields.
+  private CompressionCodec codec;
+  private Decompressor decompressor;
+  private int connectionTimeout;
+  private int readTimeout;
+
+  private final SecretKey shuffleSecret;
+  private final Configuration conf;
+
+  private final FetcherCallback fetcherCallback;
+  private final FetchedInputAllocator inputManager;
+  private final ApplicationId appId;
+
+  private static boolean sslShuffle;
+  private static SSLFactory sslFactory;
+  private static boolean sslFactoryInited;
+
+  private final int fetcherIdentifier;
+  
+  // Parameters to track work.
+  private List<InputAttemptIdentifier> srcAttempts;
+  private String host;
+  private int port;
+  private int partition;
+
+  // Maps from the pathComponents (unique per srcTaskId) to the specific taskId
+  private Map<String, InputAttemptIdentifier> pathToAttemptMap;
+  private Set<InputAttemptIdentifier> remaining;
+
+  private URL url;
+  private String encHash;
+  private String msgToEncode;
+
+  private Fetcher(FetcherCallback fetcherCallback,
+      FetchedInputAllocator inputManager, ApplicationId appId, SecretKey shuffleSecret,
+      Configuration conf) {
+    this.fetcherCallback = fetcherCallback;
+    this.inputManager = inputManager;
+    this.shuffleSecret = shuffleSecret;
+    this.appId = appId;
+    this.conf = conf;
+
+    this.fetcherIdentifier = fetcherIdGen.getAndIncrement();
+    
+    // TODO NEWTEZ Ideally, move this out from here into a static initializer block.
+    synchronized (Fetcher.class) {
+      if (!sslFactoryInited) {
+        sslFactoryInited = true;
+        sslShuffle = conf.getBoolean(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL,
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL);
+        if (sslShuffle) {
+          sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
+          try {
+            sslFactory.init();
+          } catch (Exception ex) {
+            sslFactory.destroy();
+            throw new RuntimeException(ex);
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public FetchResult call() throws Exception {
+    if (srcAttempts.size() == 0) {
+      return new FetchResult(host, port, partition, srcAttempts);
+    }
+
+    for (InputAttemptIdentifier in : srcAttempts) {
+      pathToAttemptMap.put(in.getPathComponent(), in);
+    }
+
+    remaining = new HashSet<InputAttemptIdentifier>(srcAttempts);
+
+    HttpURLConnection connection;
+    try {
+      connection = connectToShuffleHandler(host, port, partition, srcAttempts);
+    } catch (IOException e) {
+      // ioErrs.increment(1);
+      // If connect did not succeed, just mark all the maps as failed,
+      // indirectly penalizing the host
+      for (Iterator<InputAttemptIdentifier> leftIter = remaining.iterator(); leftIter
+          .hasNext();) {
+        fetcherCallback.fetchFailed(host, leftIter.next(), true);
+        leftIter.remove();
+      }
+      return new FetchResult(host, port, partition, remaining);
+    }
+
+    DataInputStream input;
+
+    try {
+      input = new DataInputStream(connection.getInputStream());
+      validateConnectionResponse(connection, url, msgToEncode, encHash);
+    } catch (IOException e) {
+      // ioErrs.increment(1);
+      // If we got a read error at this stage, it implies there was a problem
+      // with the first map, typically lost map. So, penalize only that map
+      // and add the rest
+      InputAttemptIdentifier firstAttempt = srcAttempts.get(0);
+      fetcherCallback.fetchFailed(host, firstAttempt, false);
+      remaining.remove(firstAttempt);
+      return new FetchResult(host, port, partition, remaining);
+    }
+
+    // By this point, the connection is setup and the response has been
+    // validated.
+
+    // Loop through available map-outputs and fetch them
+    // On any error, faildTasks is not null and we exit
+    // after putting back the remaining maps to the
+    // yet_to_be_fetched list and marking the failed tasks.
+    InputAttemptIdentifier[] failedInputs = null;
+    while (!remaining.isEmpty() && failedInputs == null) {
+      failedInputs = fetchInputs(input);
+    }
+
+    if (failedInputs != null && failedInputs.length > 0) {
+      LOG.warn("copyInputs failed for tasks " + Arrays.toString(failedInputs));
+      for (InputAttemptIdentifier left : failedInputs) {
+        fetcherCallback.fetchFailed(host, left, false);
+        remaining.remove(left);
+      }
+    }
+
+    IOUtils.cleanup(LOG, input);
+
+    // Sanity check
+    if (failedInputs == null && !remaining.isEmpty()) {
+      throw new IOException("server didn't return all expected map outputs: "
+          + remaining.size() + " left.");
+    }
+
+    return new FetchResult(host, port, partition, remaining);
+
+  }
+
+  private InputAttemptIdentifier[] fetchInputs(DataInputStream input) {
+    FetchedInput fetchedInput = null;
+    InputAttemptIdentifier srcAttemptId = null;
+    long decompressedLength = -1;
+    long compressedLength = -1;
+
+    try {
+      long startTime = System.currentTimeMillis();
+      int responsePartition = -1;
+      // Read the shuffle header
+      try {
+        ShuffleHeader header = new ShuffleHeader();
+        header.readFields(input);
+        String pathComponent = header.getMapId();
+
+        srcAttemptId = pathToAttemptMap.get(pathComponent);
+        compressedLength = header.getCompressedLength();
+        decompressedLength = header.getUncompressedLength();
+        responsePartition = header.getPartition();
+      } catch (IllegalArgumentException e) {
+        // badIdErrs.increment(1);
+        LOG.warn("Invalid src id ", e);
+        // Don't know which one was bad, so consider all of them as bad
+        return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
+      }
+
+      // Do some basic sanity verification
+      if (!verifySanity(compressedLength, decompressedLength,
+          responsePartition, srcAttemptId)) {
+        return new InputAttemptIdentifier[] { srcAttemptId };
+      }
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("header: " + srcAttemptId + ", len: " + compressedLength
+            + ", decomp len: " + decompressedLength);
+      }
+
+      // Get the location for the map output - either in-memory or on-disk
+      fetchedInput = inputManager.allocate(decompressedLength, srcAttemptId);
+
+      // TODO NEWTEZ No concept of WAIT at the moment.
+      // // Check if we can shuffle *now* ...
+      // if (fetchedInput.getType() == FetchedInput.WAIT) {
+      // LOG.info("fetcher#" + id +
+      // " - MergerManager returned Status.WAIT ...");
+      // //Not an error but wait to process data.
+      // return EMPTY_ATTEMPT_ID_ARRAY;
+      // }
+
+      // Go!
+      LOG.info("fetcher" + " about to shuffle output of srcAttempt "
+          + fetchedInput.getInputAttemptIdentifier() + " decomp: "
+          + decompressedLength + " len: " + compressedLength + " to "
+          + fetchedInput.getType());
+
+      if (fetchedInput.getType() == Type.MEMORY) {
+        shuffleToMemory((MemoryFetchedInput) fetchedInput, input,
+            (int) decompressedLength, (int) compressedLength);
+      } else {
+        shuffleToDisk((DiskFetchedInput) fetchedInput, input, compressedLength);
+      }
+
+      // Inform the shuffle scheduler
+      long endTime = System.currentTimeMillis();
+      fetcherCallback.fetchSucceeded(host, srcAttemptId, fetchedInput,
+          compressedLength, (endTime - startTime));
+
+      // Note successful shuffle
+      remaining.remove(srcAttemptId);
+      // metrics.successFetch();
+      return null;
+    } catch (IOException ioe) {
+      // ioErrs.increment(1);
+      if (srcAttemptId == null || fetchedInput == null) {
+        LOG.info("fetcher" + " failed to read map header" + srcAttemptId
+            + " decomp: " + decompressedLength + ", " + compressedLength, ioe);
+        if (srcAttemptId == null) {
+          return remaining
+              .toArray(new InputAttemptIdentifier[remaining.size()]);
+        } else {
+          return new InputAttemptIdentifier[] { srcAttemptId };
+        }
+      }
+      LOG.warn("Failed to shuffle output of " + srcAttemptId + " from " + host,
+          ioe);
+
+      // Inform the shuffle-scheduler
+      try {
+        fetchedInput.abort();
+      } catch (IOException e) {
+        LOG.info("Failure to cleanup fetchedInput: " + fetchedInput);
+      }
+      // metrics.failedFetch();
+      return new InputAttemptIdentifier[] { srcAttemptId };
+    }
+  }
+
+  @SuppressWarnings("resource")
+  private void shuffleToMemory(MemoryFetchedInput fetchedInput,
+      InputStream input, int decompressedLength, int compressedLength)
+      throws IOException {
+    IFileInputStream checksumIn = new IFileInputStream(input, compressedLength,
+        conf);
+
+    input = checksumIn;
+
+    // Are map-outputs compressed?
+    if (codec != null) {
+      decompressor.reset();
+      input = codec.createInputStream(input, decompressor);
+    }
+    // Copy map-output into an in-memory buffer
+    byte[] shuffleData = fetchedInput.getBytes();
+
+    try {
+      IOUtils.readFully(input, shuffleData, 0, shuffleData.length);
+      // metrics.inputBytes(shuffleData.length);
+      LOG.info("Read " + shuffleData.length + " bytes from input for "
+          + fetchedInput.getInputAttemptIdentifier());
+    } catch (IOException ioe) {
+      // Close the streams
+      IOUtils.cleanup(LOG, input);
+      // Re-throw
+      throw ioe;
+    }
+  }
+
+  private void shuffleToDisk(DiskFetchedInput fetchedInput, InputStream input,
+      long compressedLength) throws IOException {
+    // Copy data to local-disk
+    OutputStream output = fetchedInput.getOutputStream();
+    long bytesLeft = compressedLength;
+    try {
+      final int BYTES_TO_READ = 64 * 1024;
+      byte[] buf = new byte[BYTES_TO_READ];
+      while (bytesLeft > 0) {
+        int n = input.read(buf, 0, (int) Math.min(bytesLeft, BYTES_TO_READ));
+        if (n < 0) {
+          throw new IOException("read past end of stream reading "
+              + fetchedInput.getInputAttemptIdentifier());
+        }
+        output.write(buf, 0, n);
+        bytesLeft -= n;
+        // metrics.inputBytes(n);
+      }
+
+      LOG.info("Read " + (compressedLength - bytesLeft)
+          + " bytes from input for " + fetchedInput.getInputAttemptIdentifier());
+
+      output.close();
+    } catch (IOException ioe) {
+      // Close the streams
+      IOUtils.cleanup(LOG, input, output);
+
+      // Re-throw
+      throw ioe;
+    }
+
+    // Sanity check
+    if (bytesLeft != 0) {
+      throw new IOException("Incomplete input received for "
+          + fetchedInput.getInputAttemptIdentifier() + " from " + host + " ("
+          + bytesLeft + " bytes missing of " + compressedLength + ")");
+    }
+  }
+
+  /**
+   * Do some basic verification on the input received -- Being defensive
+   * 
+   * @param compressedLength
+   * @param decompressedLength
+   * @param fetchPartition
+   * @param remaining
+   * @param mapId
+   * @return true/false, based on if the verification succeeded or not
+   */
+  private boolean verifySanity(long compressedLength, long decompressedLength,
+      int fetchPartition, InputAttemptIdentifier srcAttemptId) {
+    if (compressedLength < 0 || decompressedLength < 0) {
+      // wrongLengthErrs.increment(1);
+      LOG.warn(" invalid lengths in input header: id: " + srcAttemptId
+          + " len: " + compressedLength + ", decomp len: " + decompressedLength);
+      return false;
+    }
+
+    if (fetchPartition != this.partition) {
+      // wrongReduceErrs.increment(1);
+      LOG.warn(" data for the wrong reduce map: " + srcAttemptId + " len: "
+          + compressedLength + " decomp len: " + decompressedLength
+          + " for reduce " + fetchPartition);
+      return false;
+    }
+
+    // Sanity check
+    if (!remaining.contains(srcAttemptId)) {
+      // wrongMapErrs.increment(1);
+      LOG.warn("Invalid input. Received output for " + srcAttemptId);
+      return false;
+    }
+    return true;
+  }
+
+  private HttpURLConnection connectToShuffleHandler(String host, int port,
+      int partition, List<InputAttemptIdentifier> inputs) throws IOException {
+    try {
+      this.url = constructInputURL(host, port, partition, inputs);
+      HttpURLConnection connection = openConnection(url);
+
+      // generate hash of the url
+      this.msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
+      this.encHash = SecureShuffleUtils.hashFromString(msgToEncode,
+          shuffleSecret);
+
+      // put url hash into http header
+      connection.addRequestProperty(SecureShuffleUtils.HTTP_HEADER_URL_HASH,
+          encHash);
+      // set the read timeout
+      connection.setReadTimeout(readTimeout);
+      // put shuffle version into http header
+      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+
+      connect(connection, connectionTimeout);
+      return connection;
+    } catch (IOException e) {
+      LOG.warn("Failed to connect to " + host + " with " + srcAttempts.size()
+          + " inputs", e);
+      throw e;
+    }
+  }
+
+  private void validateConnectionResponse(HttpURLConnection connection,
+      URL url, String msgToEncode, String encHash) throws IOException {
+    int rc = connection.getResponseCode();
+    if (rc != HttpURLConnection.HTTP_OK) {
+      throw new IOException("Got invalid response code " + rc + " from " + url
+          + ": " + connection.getResponseMessage());
+    }
+
+    if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(connection
+        .getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
+        || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(connection
+            .getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))) {
+      throw new IOException("Incompatible shuffle response version");
+    }
+
+    // get the replyHash which is HMac of the encHash we sent to the server
+    String replyHash = connection
+        .getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH);
+    if (replyHash == null) {
+      throw new IOException("security validation of TT Map output failed");
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("url=" + msgToEncode + ";encHash=" + encHash + ";replyHash="
+          + replyHash);
+    }
+    // verify that replyHash is HMac of encHash
+    SecureShuffleUtils.verifyReply(replyHash, encHash, shuffleSecret);
+    LOG.info("for url=" + msgToEncode + " sent hash and receievd reply");
+  }
+
+  protected HttpURLConnection openConnection(URL url) throws IOException {
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    if (sslShuffle) {
+      HttpsURLConnection httpsConn = (HttpsURLConnection) conn;
+      try {
+        httpsConn.setSSLSocketFactory(sslFactory.createSSLSocketFactory());
+      } catch (GeneralSecurityException ex) {
+        throw new IOException(ex);
+      }
+      httpsConn.setHostnameVerifier(sslFactory.getHostnameVerifier());
+    }
+    return conn;
+  }
+
+  /**
+   * The connection establishment is attempted multiple times and is given up
+   * only on the last failure. Instead of connecting with a timeout of X, we try
+   * connecting with a timeout of x < X but multiple times.
+   */
+  private void connect(URLConnection connection, int connectionTimeout)
+      throws IOException {
+    int unit = 0;
+    if (connectionTimeout < 0) {
+      throw new IOException("Invalid timeout " + "[timeout = "
+          + connectionTimeout + " ms]");
+    } else if (connectionTimeout > 0) {
+      unit = Math.min(UNIT_CONNECT_TIMEOUT, connectionTimeout);
+    }
+    // set the connect timeout to the unit-connect-timeout
+    connection.setConnectTimeout(unit);
+    while (true) {
+      try {
+        connection.connect();
+        break;
+      } catch (IOException ioe) {
+        // update the total remaining connect-timeout
+        connectionTimeout -= unit;
+
+        // throw an exception if we have waited for timeout amount of time
+        // note that the updated value if timeout is used here
+        if (connectionTimeout == 0) {
+          throw ioe;
+        }
+
+        // reset the connect timeout for the last try
+        if (connectionTimeout < unit) {
+          unit = connectionTimeout;
+          // reset the connect time out for the final connect
+          connection.setConnectTimeout(unit);
+        }
+      }
+    }
+  }
+
+  private URL constructInputURL(String host, int port, int partition,
+      List<InputAttemptIdentifier> inputs) throws MalformedURLException {
+    StringBuilder url = ShuffleUtils.constructBaseURIForShuffleHandler(host,
+        port, partition, appId);
+    boolean first = true;
+    for (InputAttemptIdentifier input : inputs) {
+      if (first) {
+        first = false;
+        url.append(input.getPathComponent());
+      } else {
+        url.append(",").append(input.getPathComponent());
+      }
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("InputFetch URL for: " + host + " : " + url.toString());
+    }
+    return new URL(url.toString());
+  }
+
+  /**
+   * Builder for the construction of Fetchers
+   */
+  public static class FetcherBuilder {
+    private Fetcher fetcher;
+    private boolean workAssigned = false;
+
+    public FetcherBuilder(FetcherCallback fetcherCallback,
+        FetchedInputAllocator inputManager, ApplicationId appId,
+        SecretKey shuffleSecret, Configuration conf) {
+      this.fetcher = new Fetcher(fetcherCallback, inputManager, appId,
+          shuffleSecret, conf);
+    }
+
+    public FetcherBuilder setCompressionParameters(CompressionCodec codec,
+        Decompressor decompressor) {
+      fetcher.codec = codec;
+      fetcher.decompressor = decompressor;
+      return this;
+    }
+
+    public FetcherBuilder setConnectionParameters(int connectionTimeout,
+        int readTimeout) {
+      fetcher.connectionTimeout = connectionTimeout;
+      fetcher.readTimeout = readTimeout;
+      return this;
+    }
+
+    public FetcherBuilder assignWork(String host, int port, int partition,
+        List<InputAttemptIdentifier> inputs) {
+      fetcher.host = host;
+      fetcher.port = port;
+      fetcher.partition = partition;
+      fetcher.srcAttempts = inputs;
+      workAssigned = true;
+      return this;
+    }
+
+    public Fetcher build() {
+      Preconditions.checkState(workAssigned == true,
+          "Cannot build a fetcher withot assigning work to it");
+      return fetcher;
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return fetcherIdentifier;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    Fetcher other = (Fetcher) obj;
+    if (fetcherIdentifier != other.fetcherIdentifier)
+      return false;
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetcherCallback.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetcherCallback.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetcherCallback.java
new file mode 100644
index 0000000..18504b1
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetcherCallback.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.runtime.library.shuffle.common;
+
+import java.io.IOException;
+
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+public interface FetcherCallback {
+
+  public void fetchSucceeded(String host, InputAttemptIdentifier srcAttemptIdentifier, FetchedInput fetchedInput, long fetchedBytes, long copyDuration) throws IOException;
+  
+  public void fetchFailed(String host, InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/InputHost.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/InputHost.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/InputHost.java
new file mode 100644
index 0000000..66605dd
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/InputHost.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.shuffle.common;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+public class InputHost {
+
+  private final String host;
+  private final int port;
+
+  private final List<InputAttemptIdentifier> inputs = new LinkedList<InputAttemptIdentifier>();
+
+  public InputHost(String hostName, int port, ApplicationId appId) {
+    this.host = hostName;
+    this.port = port;
+  }
+
+  public String getHost() {
+    return this.host;
+  }
+
+  public int getPort() {
+    return this.port;
+  }
+
+  public synchronized int getNumPendingInputs() {
+    return inputs.size();
+  }
+  
+  public synchronized void addKnownInput(InputAttemptIdentifier srcAttempt) {
+    inputs.add(srcAttempt);
+  }
+
+  public synchronized List<InputAttemptIdentifier> clearAndGetPendingInputs() {
+    List<InputAttemptIdentifier> inputsCopy = new ArrayList<InputAttemptIdentifier>(
+        inputs);
+    inputs.clear();
+    return inputsCopy;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((host == null) ? 0 : host.hashCode());
+    result = prime * result + port;
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    InputHost other = (InputHost) obj;
+    if (host == null) {
+      if (other.host != null)
+        return false;
+    } else if (!host.equals(other.host))
+      return false;
+    if (port != other.port)
+      return false;
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/MemoryFetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/MemoryFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/MemoryFetchedInput.java
new file mode 100644
index 0000000..f56877f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/MemoryFetchedInput.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.runtime.library.shuffle.common;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.hadoop.io.BoundedByteArrayOutputStream;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+import com.google.common.base.Preconditions;
+
+public class MemoryFetchedInput extends FetchedInput {
+
+  private BoundedByteArrayOutputStream byteStream;
+
+  public MemoryFetchedInput(long size,
+      InputAttemptIdentifier inputAttemptIdentifier,
+      FetchedInputCallback callbackHandler) {
+    super(Type.MEMORY, size, inputAttemptIdentifier, callbackHandler);
+    this.byteStream = new BoundedByteArrayOutputStream((int) size);
+  }
+
+  @Override
+  public OutputStream getOutputStream() {
+    return byteStream;
+  }
+
+  @Override
+  public InputStream getInputStream() {
+    return new ByteArrayInputStream(byteStream.getBuffer());
+  }
+
+  public byte[] getBytes() {
+    return byteStream.getBuffer();
+  }
+  
+  @Override
+  public void commit() {
+    if (state == State.PENDING) {
+      state = State.COMMITTED;
+      notifyFetchComplete();
+    }
+  }
+
+  @Override
+  public void abort() {
+    if (state == State.PENDING) {
+      state = State.ABORTED;
+      notifyFetchFailure();
+    }
+  }
+  
+  @Override
+  public void free() {
+    Preconditions.checkState(
+        state == State.COMMITTED || state == State.ABORTED,
+        "FetchedInput can only be freed after it is committed or aborted");
+    if (state == State.COMMITTED) {
+      state = State.FREED;
+      this.byteStream = null;
+      notifyFreedResource();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "MemoryFetchedInput [inputAttemptIdentifier="
+        + inputAttemptIdentifier + ", size=" + size + ", type=" + type
+        + ", id=" + id + ", state=" + state + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/ShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/ShuffleUtils.java
new file mode 100644
index 0000000..7479e7e
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/ShuffleUtils.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.shuffle.common;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import javax.crypto.SecretKey;
+
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
+
+public class ShuffleUtils {
+
+  public static String SHUFFLE_HANDLER_SERVICE_ID = "mapreduce.shuffle";
+
+  public static SecretKey getJobTokenSecretFromTokenBytes(ByteBuffer meta)
+      throws IOException {
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    in.reset(meta);
+    Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>();
+    jt.readFields(in);
+    SecretKey sk = JobTokenSecretManager.createSecretKey(jt.getPassword());
+    return sk;
+  }
+
+  public static ByteBuffer convertJobTokenToBytes(
+      Token<JobTokenIdentifier> jobToken) throws IOException {
+    DataOutputBuffer dob = new DataOutputBuffer();
+    jobToken.write(dob);
+    ByteBuffer bb = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    return bb;
+  }
+
+  public static int deserializeShuffleProviderMetaData(ByteBuffer meta)
+      throws IOException {
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    try {
+      in.reset(meta);
+      int port = in.readInt();
+      return port;
+    } finally {
+      in.close();
+    }
+  }
+  
+  // TODO NEWTEZ handle ssl shuffle
+  public static StringBuilder constructBaseURIForShuffleHandler(String host, int port, int partition, ApplicationId appId) {
+    StringBuilder sb = new StringBuilder("http://");
+    sb.append(host);
+    sb.append(":");
+    sb.append(String.valueOf(port));
+    sb.append("/");
+    sb.append("mapOutput?job=");
+    sb.append(appId.toString().replace("application", "job"));
+    sb.append("&reduce=");
+    sb.append(String.valueOf(partition));
+    sb.append("&map=");
+    return sb;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/proto/ShufflePayloads.proto
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/proto/ShufflePayloads.proto b/tez-runtime-library/src/main/proto/ShufflePayloads.proto
new file mode 100644
index 0000000..1fb000f
--- /dev/null
+++ b/tez-runtime-library/src/main/proto/ShufflePayloads.proto
@@ -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.
+ */
+
+option java_package = "org.apache.tez.runtime.library.shuffle.impl";
+option java_outer_classname = "ShuffleUserPayloads";
+option java_generate_equals_and_hash = true;
+
+message DataMovementEventPayloadProto {
+  optional bool output_generated = 1;
+  optional string host = 2;
+  optional int32 port = 3;
+  optional string path_component = 4;
+  optional int32 run_duration = 5;
+} 
+
+message InputInformationEventPayloadProto {
+  optional int32 partition_range = 1;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 6496b55..a6d250f 100644
--- a/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
+++ b/tez-yarn-client/src/main/java/org/apache/tez/mapreduce/YARNRunner.java
@@ -95,8 +95,6 @@ import org.apache.tez.dag.api.Vertex;
 import org.apache.tez.dag.api.VertexLocationHint.TaskLocationHint;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.DeprecatedKeys;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
@@ -104,6 +102,8 @@ import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 import com.google.common.annotations.VisibleForTesting;
 


[07/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/output/FileBasedKVWriter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/output/FileBasedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/output/FileBasedKVWriter.java
new file mode 100644
index 0000000..8b19ce0
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/output/FileBasedKVWriter.java
@@ -0,0 +1,125 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.broadcast.output;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CompressionCodec;
+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.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
+import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+
+public class FileBasedKVWriter implements KVWriter {
+
+  public static final int INDEX_RECORD_LENGTH = 24;
+
+  private final Configuration conf;
+  private int numRecords = 0;
+
+  @SuppressWarnings("rawtypes")
+  private Class keyClass;
+  @SuppressWarnings("rawtypes")
+  private Class valClass;
+  private CompressionCodec codec;
+  private FileSystem rfs;
+  private IFile.Writer writer;
+
+  private TezTaskOutput ouputFileManager;
+
+  // TODO NEWTEZ Define Counters
+  // Number of records
+  // Time waiting for a write to complete, if that's possible.
+  // Size of key-value pairs written.
+
+  public FileBasedKVWriter(TezOutputContext outputContext) throws IOException {
+    this.conf = TezUtils.createConfFromUserPayload(outputContext
+        .getUserPayload());
+    this.conf.setStrings(TezJobConfig.LOCAL_DIRS,
+        outputContext.getWorkDirs());
+
+    this.rfs = ((LocalFileSystem) FileSystem.getLocal(this.conf)).getRaw();
+
+    // Setup serialization
+    keyClass = ConfigUtils.getIntermediateOutputKeyClass(this.conf);
+    valClass = ConfigUtils.getIntermediateOutputValueClass(this.conf);
+
+    // Setup compression
+    if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) {
+      Class<? extends CompressionCodec> codecClass = ConfigUtils
+          .getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, this.conf);
+    } else {
+      codec = null;
+    }
+
+    this.ouputFileManager = TezRuntimeUtils.instantiateTaskOutputManager(conf,
+        outputContext);
+
+    initWriter();
+  }
+
+  /**
+   * @return true if any output was generated. false otherwise
+   * @throws IOException
+   */
+  public boolean close() throws IOException {
+    this.writer.close();
+    TezIndexRecord rec = new TezIndexRecord(0, writer.getRawLength(),
+        writer.getCompressedLength());
+    TezSpillRecord sr = new TezSpillRecord(1);
+    sr.putIndex(rec, 0);
+
+    Path indexFile = ouputFileManager
+        .getOutputIndexFileForWrite(INDEX_RECORD_LENGTH);
+    sr.writeToFile(indexFile, conf);
+    return numRecords > 0;
+  }
+
+  @Override
+  public void write(Object key, Object value) throws IOException {
+    this.writer.append(key, value);
+    numRecords++;
+  }
+
+  public void initWriter() throws IOException {
+    Path outputFile = ouputFileManager.getOutputFileForWrite();
+
+    // TODO NEWTEZ Consider making the buffer size configurable. Also consider
+    // setting up an in-memory buffer which is occasionally flushed to disk so
+    // that the output does not block.
+
+    // TODO NEWTEZ maybe use appropriate counter
+    this.writer = new IFile.Writer(conf, rfs, outputFile, keyClass, valClass,
+        codec, null);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java
new file mode 100644
index 0000000..d1b7ced
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ConfigUtils.java
@@ -0,0 +1,148 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.common;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezJobConfig;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class ConfigUtils {
+
+  public static Class<? extends CompressionCodec> getIntermediateOutputCompressorClass(
+      Configuration conf, Class<DefaultCodec> defaultValue) {
+    Class<? extends CompressionCodec> codecClass = defaultValue;
+    String name = conf
+        .get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_COMPRESS_CODEC);
+    if (name != null) {
+      try {
+        codecClass = conf.getClassByName(name).asSubclass(
+            CompressionCodec.class);
+      } catch (ClassNotFoundException e) {
+        throw new IllegalArgumentException("Compression codec " + name
+            + " was not found.", e);
+      }
+    }
+    return codecClass;
+  }
+  
+  public static Class<? extends CompressionCodec> getIntermediateInputCompressorClass(
+      Configuration conf, Class<DefaultCodec> defaultValue) {
+    Class<? extends CompressionCodec> codecClass = defaultValue;
+    String name = conf
+        .get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_COMPRESS_CODEC);
+    if (name != null) {
+      try {
+        codecClass = conf.getClassByName(name).asSubclass(
+            CompressionCodec.class);
+      } catch (ClassNotFoundException e) {
+        throw new IllegalArgumentException("Compression codec " + name
+            + " was not found.", e);
+      }
+    }
+    return codecClass;
+  }
+
+
+  // TODO Move defaults over to a constants file.
+  
+  public static boolean shouldCompressIntermediateOutput(Configuration conf) {
+    return conf.getBoolean(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS, false);
+  }
+
+  public static boolean isIntermediateInputCompressed(Configuration conf) {
+    return conf.getBoolean(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_IS_COMPRESSED, false);
+  }
+
+  public static <V> Class<V> getIntermediateOutputValueClass(Configuration conf) {
+    Class<V> retv = (Class<V>) conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS, null,
+        Object.class);
+    return retv;
+  }
+  
+  public static <V> Class<V> getIntermediateInputValueClass(Configuration conf) {
+    Class<V> retv = (Class<V>) conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_VALUE_CLASS, null,
+        Object.class);
+    return retv;
+  }
+
+  public static <K> Class<K> getIntermediateOutputKeyClass(Configuration conf) {
+    Class<K> retv = (Class<K>) conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS, null,
+        Object.class);
+    return retv;
+  }
+
+  public static <K> Class<K> getIntermediateInputKeyClass(Configuration conf) {
+    Class<K> retv = (Class<K>) conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_CLASS, null,
+        Object.class);
+    return retv;
+  }
+
+  public static <K> RawComparator<K> getIntermediateOutputKeyComparator(Configuration conf) {
+    Class<? extends RawComparator> theClass = conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS, null,
+        RawComparator.class);
+    if (theClass != null)
+      return ReflectionUtils.newInstance(theClass, conf);
+    return WritableComparator.get(getIntermediateOutputKeyClass(conf).asSubclass(
+        WritableComparable.class));
+  }
+
+  public static <K> RawComparator<K> getIntermediateInputKeyComparator(Configuration conf) {
+    Class<? extends RawComparator> theClass = conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS, null,
+        RawComparator.class);
+    if (theClass != null)
+      return ReflectionUtils.newInstance(theClass, conf);
+    return WritableComparator.get(getIntermediateInputKeyClass(conf).asSubclass(
+        WritableComparable.class));
+  }
+
+  
+  
+  // TODO Fix name
+  public static <V> RawComparator<V> getInputKeySecondaryGroupingComparator(
+      Configuration conf) {
+    Class<? extends RawComparator> theClass = conf
+        .getClass(
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS,
+            null, RawComparator.class);
+    if (theClass == null) {
+      return getIntermediateInputKeyComparator(conf);
+    }
+
+    return ReflectionUtils.newInstance(theClass, conf);
+  }
+  
+  public static boolean useNewApi(Configuration conf) {
+    return conf.getBoolean("mapred.mapper.new-api", false);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java
new file mode 100644
index 0000000..33cd0f6
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/Constants.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.tez.runtime.library.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_RUNTIME_TASK_ATTEMPT_ID = 
+      "tez.runtime.task.attempt.id";
+
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING =
+      "file.out";
+
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING =
+      ".index";
+
+  public static final String TEZ_RUNTIME_TASK_INPUT_FILE_FORMAT_STRING =
+      "%s/task_%d.out"; 
+
+  public static final String TEZ_RUNTIME_JOB_CREDENTIALS =
+      "tez.runtime.job.credentials";
+  
+  @Private
+  public static final String TEZ_RUNTIME_TASK_MEMORY =
+      "tez.runtime.task.memory";
+  
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_DIR = "output";
+  
+  public static final String TEZ_RUNTIME_TASK_OUTPUT_MANAGER = 
+      "tez.runtime.task.local.output.manager";
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
new file mode 100644
index 0000000..a13f3f1
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+
+/**
+ * Container for a task number and an attempt number for the task.
+ */
+@Private
+public class InputAttemptIdentifier {
+
+  private final InputIdentifier inputIdentifier;
+  private final int attemptNumber;
+  private String pathComponent;
+  
+  public InputAttemptIdentifier(int taskIndex, int attemptNumber) {
+    this(new InputIdentifier(taskIndex), attemptNumber, null);
+  }
+  
+  public InputAttemptIdentifier(InputIdentifier inputIdentifier, int attemptNumber, String pathComponent) {
+    this.inputIdentifier = inputIdentifier;
+    this.attemptNumber = attemptNumber;
+    this.pathComponent = pathComponent;
+  }
+  
+  public InputAttemptIdentifier(int taskIndex, int attemptNumber, String pathComponent) {
+    this(new InputIdentifier(taskIndex), attemptNumber, pathComponent);
+  }
+
+  public InputIdentifier getInputIdentifier() {
+    return this.inputIdentifier;
+  }
+
+  public int getAttemptNumber() {
+    return attemptNumber;
+  }
+  
+  public String getPathComponent() {
+    return pathComponent;
+  }
+
+  // PathComponent does not need to be part of the hashCode and equals computation.
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + attemptNumber;
+    result = prime * result
+        + ((inputIdentifier == null) ? 0 : inputIdentifier.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;
+    InputAttemptIdentifier other = (InputAttemptIdentifier) obj;
+    if (attemptNumber != other.attemptNumber)
+      return false;
+    if (inputIdentifier == null) {
+      if (other.inputIdentifier != null)
+        return false;
+    } else if (!inputIdentifier.equals(other.inputIdentifier))
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "InputAttemptIdentifier [inputIdentifier=" + inputIdentifier
+        + ", attemptNumber=" + attemptNumber + ", pathComponent="
+        + pathComponent + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputIdentifier.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputIdentifier.java
new file mode 100644
index 0000000..f4ce190
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputIdentifier.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.runtime.library.common;
+
+public class InputIdentifier {
+
+  private final int srcTaskIndex;
+  
+  public InputIdentifier(int srcTaskIndex) {
+    this.srcTaskIndex = srcTaskIndex;
+  }
+
+  public int getSrcTaskIndex() {
+    return this.srcTaskIndex;
+  }
+
+  @Override
+  public int hashCode() {
+    return srcTaskIndex;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    InputIdentifier other = (InputIdentifier) obj;
+    if (srcTaskIndex != other.srcTaskIndex)
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    return "InputIdentifier [srcTaskIndex=" + srcTaskIndex + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.java
new file mode 100644
index 0000000..2381780
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/TezRuntimeUtils.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.runtime.library.common;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezTaskContext;
+import org.apache.tez.runtime.library.api.Partitioner;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
+
+public class TezRuntimeUtils {
+
+  private static final Log LOG = LogFactory
+      .getLog(TezRuntimeUtils.class);
+  
+  public static String getTaskIdentifier(String vertexName, int taskIndex) {
+    return String.format("%s_%06d", vertexName, taskIndex);
+  }
+
+  public static String getTaskAttemptIdentifier(int taskIndex,
+      int taskAttemptNumber) {
+    return String.format("%d_%d", taskIndex, taskAttemptNumber);
+  }
+
+  // TODO Maybe include a dag name in this.
+  public static String getTaskAttemptIdentifier(String vertexName,
+      int taskIndex, int taskAttemptNumber) {
+    return String.format("%s_%06d_%02d", vertexName, taskIndex,
+        taskAttemptNumber);
+  }
+
+  @SuppressWarnings("unchecked")
+  public static Combiner instantiateCombiner(Configuration conf, TezTaskContext taskContext) throws IOException {
+    Class<? extends Combiner> clazz;
+    String className = conf.get(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS);
+    if (className == null) {
+      LOG.info("No combiner specified via " + TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS + ". Combiner will not be used");
+      return null;
+    }
+    LOG.info("Using Combiner class: " + className);
+    try {
+      clazz = (Class<? extends Combiner>) conf.getClassByName(className);
+    } catch (ClassNotFoundException e) {
+      throw new IOException("Unable to load combiner class: " + className);
+    }
+    
+    Combiner combiner = null;
+    
+      Constructor<? extends Combiner> ctor;
+      try {
+        ctor = clazz.getConstructor(TezTaskContext.class);
+        combiner = ctor.newInstance(taskContext);
+      } catch (SecurityException e) {
+        throw new IOException(e);
+      } catch (NoSuchMethodException e) {
+        throw new IOException(e);
+      } catch (IllegalArgumentException e) {
+        throw new IOException(e);
+      } catch (InstantiationException e) {
+        throw new IOException(e);
+      } catch (IllegalAccessException e) {
+        throw new IOException(e);
+      } catch (InvocationTargetException e) {
+        throw new IOException(e);
+      }
+      return combiner;
+  }
+  
+  @SuppressWarnings("unchecked")
+  public static Partitioner instantiatePartitioner(Configuration conf)
+      throws IOException {
+    Class<? extends Partitioner> clazz;
+    try {
+      clazz = (Class<? extends Partitioner>) conf
+          .getClassByName(conf.get(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS));
+    } catch (ClassNotFoundException e) {
+      throw new IOException("Unable to find Partitioner class in config", e);
+    }
+
+    LOG.info("Using partitioner class: " + clazz.getName());
+
+    Partitioner partitioner = null;
+
+    try {
+      Constructor<? extends Partitioner> ctorWithConf = clazz
+          .getConstructor(Configuration.class);
+      partitioner = ctorWithConf.newInstance(conf);
+    } catch (SecurityException e) {
+      throw new IOException(e);
+    } catch (NoSuchMethodException e) {
+      try {
+        // Try a 0 argument constructor.
+        partitioner = clazz.newInstance();
+      } catch (InstantiationException e1) {
+        throw new IOException(e1);
+      } catch (IllegalAccessException e1) {
+        throw new IOException(e1);
+      }
+    } catch (IllegalArgumentException e) {
+      throw new IOException(e);
+    } catch (InstantiationException e) {
+      throw new IOException(e);
+    } catch (IllegalAccessException e) {
+      throw new IOException(e);
+    } catch (InvocationTargetException e) {
+      throw new IOException(e);
+    }
+    return partitioner;
+  }
+  
+  public static TezTaskOutput instantiateTaskOutputManager(Configuration conf, TezOutputContext outputContext) {
+    Class<?> clazz = conf.getClass(Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
+        TezTaskOutputFiles.class);
+    try {
+      Constructor<?> ctor = clazz.getConstructor(Configuration.class, String.class);
+      ctor.setAccessible(true);
+      TezTaskOutput instance = (TezTaskOutput) ctor.newInstance(conf, outputContext.getUniqueIdentifier());
+      return instance;
+    } catch (Exception e) {
+      throw new TezUncheckedException(
+          "Unable to instantiate configured TezOutputFileManager: "
+              + conf.get(Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
+                  TezTaskOutputFiles.class.getName()), e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java
new file mode 100644
index 0000000..fef3356
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/ValuesIterator.java
@@ -0,0 +1,194 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.common;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Iterates values while keys match in sorted input.
+ * 
+ * This class is not thread safe. Accessing methods from multiple threads will
+ * lead to corrupt data.
+ * 
+ */
+public class ValuesIterator<KEY,VALUE> {
+  protected TezRawKeyValueIterator in; //input iterator
+  private KEY key;               // current key
+  private KEY nextKey;
+  private VALUE value;             // current value
+  //private boolean hasNext;                      // more w/ this key
+  private boolean more;                         // more in file
+  private RawComparator<KEY> comparator;
+  private Deserializer<KEY> keyDeserializer;
+  private Deserializer<VALUE> valDeserializer;
+  private DataInputBuffer keyIn = new DataInputBuffer();
+  private DataInputBuffer valueIn = new DataInputBuffer();
+  private TezCounter inputKeyCounter;
+  private TezCounter inputValueCounter;
+  
+  private int keyCtr = 0;
+  private boolean hasMoreValues; // For the current key.
+  private boolean isFirstRecord = true;
+  
+  public ValuesIterator (TezRawKeyValueIterator in, 
+                         RawComparator<KEY> comparator, 
+                         Class<KEY> keyClass,
+                         Class<VALUE> valClass, Configuration conf,
+                         TezCounter inputKeyCounter,
+                         TezCounter inputValueCounter)
+    throws IOException {
+    this.in = in;
+    this.comparator = comparator;
+    this.inputKeyCounter = inputKeyCounter;
+    this.inputValueCounter = inputValueCounter;
+    SerializationFactory serializationFactory = new SerializationFactory(conf);
+    this.keyDeserializer = serializationFactory.getDeserializer(keyClass);
+    this.keyDeserializer.open(keyIn);
+    this.valDeserializer = serializationFactory.getDeserializer(valClass);
+    this.valDeserializer.open(this.valueIn);
+  }
+
+  TezRawKeyValueIterator getRawIterator() { return in; }
+
+  /**
+   * Move to the next K-Vs pair
+   * @return true if another pair exists, otherwise false.
+   * @throws IOException 
+   */
+  public boolean moveToNext() throws IOException {
+    if (isFirstRecord) {
+      readNextKey();
+      key = nextKey;
+      nextKey = null;
+      hasMoreValues = more;
+      isFirstRecord = false;
+    } else {
+      nextKey();
+    }
+    return more;
+  }
+  
+  /** The current key. */
+  public KEY getKey() { 
+    return key; 
+  }
+  
+  // TODO NEWTEZ Maybe add another method which returns an iterator instead of iterable
+  
+  public Iterable<VALUE> getValues() {
+    return new Iterable<VALUE>() {
+
+      @Override
+      public Iterator<VALUE> iterator() {
+        
+        return new Iterator<VALUE>() {
+
+          private final int keyNumber = keyCtr;
+          
+          @Override
+          public boolean hasNext() {
+            return hasMoreValues;
+          }
+
+          @Override
+          public VALUE next() {
+            if (!hasMoreValues) {
+              throw new NoSuchElementException("iterate past last value");
+            }
+            Preconditions
+                .checkState(
+                    keyNumber == keyCtr,
+                    "Cannot use values iterator on the previous K-V pair after moveToNext has been invoked to move to the next K-V pair");
+            
+            try {
+              readNextValue();
+              readNextKey();
+            } catch (IOException ie) {
+              throw new RuntimeException("problem advancing post rec#"+keyCtr, ie);
+            }
+            inputValueCounter.increment(1);
+            return value;
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException("Cannot remove elements");
+          }
+        };
+      }
+    };
+  }
+  
+  
+
+  /** Start processing next unique key. */
+  private void nextKey() throws IOException {
+    // read until we find a new key
+    while (hasMoreValues) { 
+      readNextKey();
+    }
+    if (more) {
+      inputKeyCounter.increment(1);
+      ++keyCtr;
+    }
+    
+    // move the next key to the current one
+    KEY tmpKey = key;
+    key = nextKey;
+    nextKey = tmpKey;
+    hasMoreValues = more;
+  }
+
+  /** 
+   * read the next key - which may be the same as the current key.
+   */
+  private void readNextKey() throws IOException {
+    more = in.next();
+    if (more) {      
+      DataInputBuffer nextKeyBytes = in.getKey();
+      keyIn.reset(nextKeyBytes.getData(), nextKeyBytes.getPosition(), nextKeyBytes.getLength());
+      nextKey = keyDeserializer.deserialize(nextKey);
+      hasMoreValues = key != null && (comparator.compare(key, nextKey) == 0);
+    } else {
+      hasMoreValues = false;
+    }
+  }
+
+  /**
+   * Read the next value
+   * @throws IOException
+   */
+  private void readNextValue() throws IOException {
+    DataInputBuffer nextValueBytes = in.getValue();
+    valueIn.reset(nextValueBytes.getData(), nextValueBytes.getPosition(), nextValueBytes.getLength());
+    value = valDeserializer.deserialize(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/YARNMaster.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/YARNMaster.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/YARNMaster.java
new file mode 100644
index 0000000..8709e05
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/YARNMaster.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.runtime.library.common;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+@Private
+@Unstable
+public class YARNMaster {
+  
+  public enum State {
+    INITIALIZING, RUNNING;
+  }
+
+  public static String getMasterUserName(Configuration conf) {
+    return conf.get(YarnConfiguration.RM_PRINCIPAL);
+  }
+  
+  public static InetSocketAddress getMasterAddress(Configuration conf) {
+    return conf.getSocketAddr(
+        YarnConfiguration.RM_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_ADDRESS,
+        YarnConfiguration.DEFAULT_RM_PORT);
+  }
+
+  public static String getMasterPrincipal(Configuration conf) 
+  throws IOException {
+    String masterHostname = getMasterAddress(conf).getHostName();
+    // get kerberos principal for use as delegation token renewer
+    return SecurityUtil.getServerPrincipal(
+        getMasterUserName(conf), masterHostname);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/combine/Combiner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/combine/Combiner.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/combine/Combiner.java
new file mode 100644
index 0000000..5b10590
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/combine/Combiner.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.runtime.library.common.combine;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+
+/**
+ *<b>Combiner Initialization</b></p> The Combiner class is picked up
+ * using the TEZ_RUNTIME_COMBINER_CLASS attribute in {@link TezJobConfig}
+ * 
+ * 
+ * Partitioners need to provide a single argument ({@link TezTaskContext})
+ * constructor.
+ */
+@Unstable
+@LimitedPrivate("mapreduce")
+public interface Combiner {
+  public void combine(TezRawKeyValueIterator rawIter, Writer writer)
+      throws InterruptedException, IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/localshuffle/LocalShuffle.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/localshuffle/LocalShuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/localshuffle/LocalShuffle.java
new file mode 100644
index 0000000..b40df6f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/localshuffle/LocalShuffle.java
@@ -0,0 +1,120 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.common.localshuffle;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+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.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.task.local.output.TezLocalTaskOutputFiles;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+
+@SuppressWarnings({"rawtypes"})
+public class LocalShuffle {
+
+  // TODO NEWTEZ This is broken.
+
+  private final TezInputContext inputContext;
+  private final Configuration conf;
+  private final int numInputs;
+
+  private final Class keyClass;
+  private final Class valClass;
+  private final RawComparator comparator;
+
+  private final FileSystem rfs;
+  private final int sortFactor;
+  
+  private final TezCounter spilledRecordsCounter;
+  private final CompressionCodec codec;
+  private final TezTaskOutput mapOutputFile;
+
+  public LocalShuffle(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
+    this.inputContext = inputContext;
+    this.conf = conf;
+    this.numInputs = numInputs;
+    
+    this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
+    this.valClass = ConfigUtils.getIntermediateInputValueClass(conf);
+    this.comparator = ConfigUtils.getIntermediateInputKeyComparator(conf);
+    
+    this.sortFactor =
+        conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
+    
+    this.rfs = FileSystem.getLocal(conf).getRaw();
+
+    this.spilledRecordsCounter = inputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
+    
+ // compression
+    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
+      Class<? extends CompressionCodec> codecClass =
+          ConfigUtils.getIntermediateInputCompressorClass(conf, DefaultCodec.class);
+      this.codec = ReflectionUtils.newInstance(codecClass, conf);
+    } else {
+      this.codec = null;
+    }
+    
+    // Always local
+    this.mapOutputFile = new TezLocalTaskOutputFiles(conf, inputContext.getUniqueIdentifier());
+  }
+ 
+  
+  public TezRawKeyValueIterator run() throws IOException {
+    // Copy is complete, obviously! 
+
+    
+    // Merge
+    return TezMerger.merge(conf, rfs, 
+        keyClass, valClass,
+        codec, 
+        getMapFiles(),
+        false, 
+        sortFactor,
+        new Path(inputContext.getUniqueIdentifier()), // TODO NEWTEZ This is likely broken 
+        comparator,
+        null, spilledRecordsCounter, null, null);
+  }
+  
+  private Path[] getMapFiles() 
+  throws IOException {
+    List<Path> fileList = new ArrayList<Path>();
+      // for local jobs
+      for(int i = 0; i < numInputs; ++i) {
+        //fileList.add(mapOutputFile.getInputFile(i));
+      }
+      
+    return fileList.toArray(new Path[0]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenIdentifier.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenIdentifier.java
new file mode 100644
index 0000000..4b916fa
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenIdentifier.java
@@ -0,0 +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.tez.runtime.library.common.security;
+
+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.Text;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+/**
+ * The token identifier for job token
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class JobTokenIdentifier extends TokenIdentifier {
+  private Text jobid;
+  public final static Text KIND_NAME = new Text("mapreduce.job");
+  
+  /**
+   * Default constructor
+   */
+  public JobTokenIdentifier() {
+    this.jobid = new Text();
+  }
+
+  /**
+   * Create a job token identifier from a jobid
+   * @param jobid the jobid to use
+   */
+  public JobTokenIdentifier(Text jobid) {
+    this.jobid = jobid;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public Text getKind() {
+    return KIND_NAME;
+  }
+  
+  /** {@inheritDoc} */
+  @Override
+  public UserGroupInformation getUser() {
+    if (jobid == null || "".equals(jobid.toString())) {
+      return null;
+    }
+    return UserGroupInformation.createRemoteUser(jobid.toString());
+  }
+  
+  /**
+   * Get the jobid
+   * @return the jobid
+   */
+  public Text getJobId() {
+    return jobid;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    jobid.readFields(in);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public void write(DataOutput out) throws IOException {
+    jobid.write(out);
+  }
+
+  @InterfaceAudience.Private
+  public static class Renewer extends Token.TrivialRenewer {
+    @Override
+    protected Text getKind() {
+      return KIND_NAME;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSecretManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSecretManager.java
new file mode 100644
index 0000000..a03ee94
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSecretManager.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.runtime.library.common.security;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+import javax.crypto.SecretKey;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * SecretManager for job token. It can be used to cache generated job tokens.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class JobTokenSecretManager extends SecretManager<JobTokenIdentifier> {
+  private final SecretKey masterKey;
+  private final Map<String, SecretKey> currentJobTokens;
+
+  /**
+   * Convert the byte[] to a secret key
+   * @param key the byte[] to create the secret key from
+   * @return the secret key
+   */
+  public static SecretKey createSecretKey(byte[] key) {
+    return SecretManager.createSecretKey(key);
+  }
+  
+  /**
+   * Compute the HMAC hash of the message using the key
+   * @param msg the message to hash
+   * @param key the key to use
+   * @return the computed hash
+   */
+  public static byte[] computeHash(byte[] msg, SecretKey key) {
+    return createPassword(msg, key);
+  }
+  
+  /**
+   * Default constructor
+   */
+  public JobTokenSecretManager() {
+    this.masterKey = generateSecret();
+    this.currentJobTokens = new TreeMap<String, SecretKey>();
+  }
+  
+  /**
+   * Create a new password/secret for the given job token identifier.
+   * @param identifier the job token identifier
+   * @return token password/secret
+   */
+  @Override
+  public byte[] createPassword(JobTokenIdentifier identifier) {
+    byte[] result = createPassword(identifier.getBytes(), masterKey);
+    return result;
+  }
+
+  /**
+   * Add the job token of a job to cache
+   * @param jobId the job that owns the token
+   * @param token the job token
+   */
+  public void addTokenForJob(String jobId, Token<JobTokenIdentifier> token) {
+    SecretKey tokenSecret = createSecretKey(token.getPassword());
+    synchronized (currentJobTokens) {
+      currentJobTokens.put(jobId, tokenSecret);
+    }
+  }
+
+  /**
+   * Remove the cached job token of a job from cache
+   * @param jobId the job whose token is to be removed
+   */
+  public void removeTokenForJob(String jobId) {
+    synchronized (currentJobTokens) {
+      currentJobTokens.remove(jobId);
+    }
+  }
+  
+  /**
+   * Look up the token password/secret for the given jobId.
+   * @param jobId the jobId to look up
+   * @return token password/secret as SecretKey
+   * @throws InvalidToken
+   */
+  public SecretKey retrieveTokenSecret(String jobId) throws InvalidToken {
+    SecretKey tokenSecret = null;
+    synchronized (currentJobTokens) {
+      tokenSecret = currentJobTokens.get(jobId);
+    }
+    if (tokenSecret == null) {
+      throw new InvalidToken("Can't find job token for job " + jobId + " !!");
+    }
+    return tokenSecret;
+  }
+  
+  /**
+   * Look up the token password/secret for the given job token identifier.
+   * @param identifier the job token identifier to look up
+   * @return token password/secret as byte[]
+   * @throws InvalidToken
+   */
+  @Override
+  public byte[] retrievePassword(JobTokenIdentifier identifier)
+      throws InvalidToken {
+    return retrieveTokenSecret(identifier.getJobId().toString()).getEncoded();
+  }
+
+  /**
+   * Create an empty job token identifier
+   * @return a newly created empty job token identifier
+   */
+  @Override
+  public JobTokenIdentifier createIdentifier() {
+    return new JobTokenIdentifier();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSelector.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSelector.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSelector.java
new file mode 100644
index 0000000..b8227ab
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/JobTokenSelector.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common.security;
+
+import java.util.Collection;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.TokenSelector;
+
+/**
+ * Look through tokens to find the first job token that matches the service
+ * and return it.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class JobTokenSelector implements TokenSelector<JobTokenIdentifier> {
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public Token<JobTokenIdentifier> selectToken(Text service,
+      Collection<Token<? extends TokenIdentifier>> tokens) {
+    if (service == null) {
+      return null;
+    }
+    for (Token<? extends TokenIdentifier> token : tokens) {
+      if (JobTokenIdentifier.KIND_NAME.equals(token.getKind())
+          && service.equals(token.getService())) {
+        return (Token<JobTokenIdentifier>) token;
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/Master.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/Master.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/Master.java
new file mode 100644
index 0000000..5b91e0f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/Master.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.runtime.library.common.security;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+@Private
+@Unstable
+public class Master {
+
+  public enum State {
+    INITIALIZING, RUNNING;
+  }
+
+  public static String getMasterUserName(Configuration conf) {
+    return conf.get(YarnConfiguration.RM_PRINCIPAL);
+  }
+
+  public static InetSocketAddress getMasterAddress(Configuration conf) {
+    return conf
+        .getSocketAddr(YarnConfiguration.RM_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_ADDRESS,
+            YarnConfiguration.DEFAULT_RM_PORT);
+  }
+
+  public static String getMasterPrincipal(Configuration conf)
+      throws IOException {
+    String masterHostname = getMasterAddress(conf).getHostName();
+    // get kerberos principal for use as delegation token renewer
+    return SecurityUtil.getServerPrincipal(getMasterUserName(conf),
+        masterHostname);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java
new file mode 100644
index 0000000..b2d382c
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/SecureShuffleUtils.java
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.common.security;
+
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.net.URL;
+
+import javax.crypto.SecretKey;
+import javax.servlet.http.HttpServletRequest;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.WritableComparator;
+
+/**
+ * 
+ * utilities for generating kyes, hashes and verifying them for shuffle
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SecureShuffleUtils {
+  public static final String HTTP_HEADER_URL_HASH = "UrlHash";
+  public static final String HTTP_HEADER_REPLY_URL_HASH = "ReplyHash";
+  
+  /**
+   * Base64 encoded hash of msg
+   * @param msg
+   */
+  public static String generateHash(byte[] msg, SecretKey key) {
+    return new String(Base64.encodeBase64(generateByteHash(msg, key)));
+  }
+  
+  /**
+   * calculate hash of msg
+   * @param msg
+   * @return
+   */
+  private static byte[] generateByteHash(byte[] msg, SecretKey key) {
+    return JobTokenSecretManager.computeHash(msg, key);
+  }
+  
+  /**
+   * verify that hash equals to HMacHash(msg)
+   * @param newHash
+   * @return true if is the same
+   */
+  private static boolean verifyHash(byte[] hash, byte[] msg, SecretKey key) {
+    byte[] msg_hash = generateByteHash(msg, key);
+    return WritableComparator.compareBytes(msg_hash, 0, msg_hash.length, hash, 0, hash.length) == 0;
+  }
+  
+  /**
+   * Aux util to calculate hash of a String
+   * @param enc_str
+   * @param key
+   * @return Base64 encodedHash
+   * @throws IOException
+   */
+  public static String hashFromString(String enc_str, SecretKey key) 
+  throws IOException {
+    return generateHash(enc_str.getBytes(), key); 
+  }
+  
+  /**
+   * verify that base64Hash is same as HMacHash(msg)  
+   * @param base64Hash (Base64 encoded hash)
+   * @param msg
+   * @throws IOException if not the same
+   */
+  public static void verifyReply(String base64Hash, String msg, SecretKey key)
+  throws IOException {
+    byte[] hash = Base64.decodeBase64(base64Hash.getBytes());
+    
+    boolean res = verifyHash(hash, msg.getBytes(), key);
+    
+    if(res != true) {
+      throw new IOException("Verification of the hashReply failed");
+    }
+  }
+  
+  /**
+   * Shuffle specific utils - build string for encoding from URL
+   * @param url
+   * @return string for encoding
+   */
+  public static String buildMsgFrom(URL url) {
+    return buildMsgFrom(url.getPath(), url.getQuery(), url.getPort());
+  }
+  /**
+   * Shuffle specific utils - build string for encoding from URL
+   * @param request
+   * @return string for encoding
+   */
+  public static String buildMsgFrom(HttpServletRequest request ) {
+    return buildMsgFrom(request.getRequestURI(), request.getQueryString(),
+        request.getLocalPort());
+  }
+  /**
+   * Shuffle specific utils - build string for encoding from URL
+   * @param uri_path
+   * @param uri_query
+   * @return string for encoding
+   */
+  private static String buildMsgFrom(String uri_path, String uri_query, int port) {
+    return String.valueOf(port) + uri_path + "?" + uri_query;
+  }
+  
+  
+  /**
+   * byte array to Hex String
+   * @param ba
+   * @return string with HEX value of the key
+   */
+  public static String toHex(byte[] ba) {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintStream ps = new PrintStream(baos);
+    for(byte b: ba) {
+      ps.printf("%x", b);
+    }
+    return baos.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/TokenCache.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/TokenCache.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/TokenCache.java
new file mode 100644
index 0000000..33373b7
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/security/TokenCache.java
@@ -0,0 +1,205 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common.security;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+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.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.tez.common.TezJobConfig;
+
+
+/**
+ * This class provides user facing APIs for transferring secrets from
+ * the job client to the tasks.
+ * The secrets can be stored just before submission of jobs and read during
+ * the task execution.  
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class TokenCache {
+  
+  private static final Log LOG = LogFactory.getLog(TokenCache.class);
+
+  
+  /**
+   * auxiliary method to get user's secret keys..
+   * @param alias
+   * @return secret key from the storage
+   */
+  public static byte[] getSecretKey(Credentials credentials, Text alias) {
+    if(credentials == null)
+      return null;
+    return credentials.getSecretKey(alias);
+  }
+  
+  /**
+   * Convenience method to obtain delegation tokens from namenodes 
+   * corresponding to the paths passed.
+   * @param credentials
+   * @param ps array of paths
+   * @param conf configuration
+   * @throws IOException
+   */
+  public static void obtainTokensForNamenodes(Credentials credentials,
+      Path[] ps, Configuration conf) throws IOException {
+    if (!UserGroupInformation.isSecurityEnabled()) {
+      return;
+    }
+    obtainTokensForNamenodesInternal(credentials, ps, conf);
+  }
+
+  /**
+   * Remove jobtoken referrals which don't make sense in the context
+   * of the task execution.
+   *
+   * @param conf
+   */
+  public static void cleanUpTokenReferral(Configuration conf) {
+    conf.unset(TezJobConfig.DAG_CREDENTIALS_BINARY);
+  }
+
+  static void obtainTokensForNamenodesInternal(Credentials credentials,
+      Path[] ps, Configuration conf) throws IOException {
+    Set<FileSystem> fsSet = new HashSet<FileSystem>();
+    for(Path p: ps) {
+      fsSet.add(p.getFileSystem(conf));
+    }
+    for (FileSystem fs : fsSet) {
+      obtainTokensForNamenodesInternal(fs, credentials, conf);
+    }
+  }
+
+  /**
+   * get delegation token for a specific FS
+   * @param fs
+   * @param credentials
+   * @param p
+   * @param conf
+   * @throws IOException
+   */
+  static void obtainTokensForNamenodesInternal(FileSystem fs, 
+      Credentials credentials, Configuration conf) throws IOException {
+    String delegTokenRenewer = Master.getMasterPrincipal(conf);
+    if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
+      throw new IOException(
+          "Can't get Master Kerberos principal for use as renewer");
+    }
+    mergeBinaryTokens(credentials, conf);
+
+    final Token<?> tokens[] = fs.addDelegationTokens(delegTokenRenewer,
+                                                     credentials);
+    if (tokens != null) {
+      for (Token<?> token : tokens) {
+        LOG.info("Got dt for " + fs.getUri() + "; "+token);
+      }
+    }
+  }
+
+  private static void mergeBinaryTokens(Credentials creds, Configuration conf) {
+    String binaryTokenFilename =
+        conf.get(TezJobConfig.DAG_CREDENTIALS_BINARY);
+    if (binaryTokenFilename != null) {
+      Credentials binary;
+      try {
+        binary = Credentials.readTokenStorageFile(
+            new Path("file:///" +  binaryTokenFilename), conf);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      // supplement existing tokens with the tokens in the binary file
+      creds.mergeAll(binary);
+    }
+  }
+  
+  /**
+   * file name used on HDFS for generated job token
+   */
+  @InterfaceAudience.Private
+  public static final String JOB_TOKEN_HDFS_FILE = "jobToken";
+
+  /**
+   * conf setting for job tokens cache file name
+   */
+  @InterfaceAudience.Private
+  public static final String JOB_TOKENS_FILENAME = "mapreduce.job.jobTokenFile";
+  private static final Text JOB_TOKEN = new Text("JobToken");
+  private static final Text SHUFFLE_TOKEN = new Text("MapReduceShuffleToken");
+  
+  /**
+   * load job token from a file
+   * @param conf
+   * @throws IOException
+   */
+  @InterfaceAudience.Private
+  public static Credentials loadTokens(String jobTokenFile, Configuration conf) 
+  throws IOException {
+    Path localJobTokenFile = new Path ("file:///" + jobTokenFile);
+
+    Credentials ts = Credentials.readTokenStorageFile(localJobTokenFile, conf);
+
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Task: Loaded jobTokenFile from: "+
+          localJobTokenFile.toUri().getPath() 
+          +"; num of sec keys  = " + ts.numberOfSecretKeys() +
+          " Number of tokens " +  ts.numberOfTokens());
+    }
+    return ts;
+  }
+  /**
+   * store job token
+   * @param t
+   */
+  @InterfaceAudience.Private
+  public static void setJobToken(Token<? extends TokenIdentifier> t, 
+      Credentials credentials) {
+    credentials.addToken(JOB_TOKEN, t);
+  }
+  /**
+   * 
+   * @return job token
+   */
+  @SuppressWarnings("unchecked")
+  @InterfaceAudience.Private
+  public static Token<JobTokenIdentifier> getJobToken(Credentials credentials) {
+    return (Token<JobTokenIdentifier>) credentials.getToken(JOB_TOKEN);
+  }
+
+  @InterfaceAudience.Private
+  public static void setShuffleSecretKey(byte[] key, Credentials credentials) {
+    credentials.addSecretKey(SHUFFLE_TOKEN, key);
+  }
+
+  @InterfaceAudience.Private
+  public static byte[] getShuffleSecretKey(Credentials credentials) {
+    return getSecretKey(credentials, SHUFFLE_TOKEN);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ExceptionReporter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ExceptionReporter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ExceptionReporter.java
new file mode 100644
index 0000000..a872ba1
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ExceptionReporter.java
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.common.shuffle.impl;
+
+/**
+ * An interface for reporting exceptions to other threads
+ */
+interface ExceptionReporter {
+  void reportException(Throwable t);
+}


[02/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/SortBufferInputStream.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/SortBufferInputStream.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/SortBufferInputStream.java
new file mode 100644
index 0000000..b7b1e82
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/SortBufferInputStream.java
@@ -0,0 +1,271 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.common.sort.impl.dflt;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.IntBuffer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.BoundedByteArrayOutputStream;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.InMemoryWriter;
+import org.apache.tez.runtime.library.common.sort.impl.dflt.DefaultSorter.InMemValBytes;
+
+  public class SortBufferInputStream extends InputStream {
+
+  private static final Log LOG = LogFactory.getLog(SortBufferInputStream.class);
+  
+  private final InMemoryShuffleSorter sorter;
+  private InMemoryWriter sortOutput;
+  
+  private int mend;
+  private int recIndex;
+  private final byte[] kvbuffer;       
+  private final IntBuffer kvmeta;
+  private final int partitionBytes;
+  private final int partition;
+  
+  byte[] dualBuf = new byte[8192];
+  DualBufferOutputStream out;
+  private int readBytes = 0;
+  
+  public SortBufferInputStream(
+      InMemoryShuffleSorter sorter, int partition) {
+    this.sorter = sorter;
+    this.partitionBytes = 
+        (int)sorter.getShuffleHeader(partition).getCompressedLength();
+    this.partition = partition;
+    this.mend = sorter.getMetaEnd();
+    this.recIndex = sorter.getSpillIndex(partition);
+    this.kvbuffer = sorter.kvbuffer;
+    this.kvmeta = sorter.kvmeta;
+    out = new DualBufferOutputStream(null, 0, 0, dualBuf);
+    sortOutput = new InMemoryWriter(out);
+  }
+  
+  byte[] one = new byte[1];
+  
+  @Override
+  public int read() throws IOException {
+    int b = read(one, 0, 1);
+    return (b == -1) ? b : one[0]; 
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (available() == 0) {
+      return -1;
+    }
+    
+    int currentOffset = off;
+    int currentLength = len;
+    int currentReadBytes = 0;
+    
+    // Check if there is residual data in the dualBuf
+    int residualLen = out.getCurrent();
+    if (residualLen > 0) {
+      int readable = Math.min(currentLength, residualLen);
+      System.arraycopy(dualBuf, 0, b, currentOffset, readable);
+      currentOffset += readable;
+      currentReadBytes += readable;
+      out.setCurrentPointer(-readable);
+      
+      // buffer has less capacity
+      currentLength -= readable;
+      
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("XXX read_residual:" +
+            " readable=" + readable +
+            " readBytes=" + readBytes);
+      }
+    }
+    
+    // Now, use the provided buffer
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("XXX read: out.reset" +
+          " b=" + b + 
+          " currentOffset=" + currentOffset + 
+          " currentLength=" + currentLength +
+          " recIndex=" + recIndex);
+    }
+    out.reset(b, currentOffset, currentLength);
+    
+    // Read from sort-buffer into the provided buffer, space permitting
+    DataInputBuffer key = new DataInputBuffer();
+    final InMemValBytes value = sorter.createInMemValBytes();
+    
+    int kvPartition = 0;
+    int numRec = 0;
+    for (;
+         currentLength > 0 && recIndex < mend && 
+             (kvPartition = getKVPartition(recIndex)) == partition;
+        ++recIndex) {
+      
+      final int kvoff = sorter.offsetFor(recIndex);
+      
+      int keyLen = 
+          (kvmeta.get(kvoff + InMemoryShuffleSorter.VALSTART) - 
+              kvmeta.get(kvoff + InMemoryShuffleSorter.KEYSTART));
+      key.reset(
+          kvbuffer, 
+          kvmeta.get(kvoff + InMemoryShuffleSorter.KEYSTART),
+          keyLen
+          );
+      
+      int valLen = sorter.getVBytesForOffset(kvoff, value);
+
+      int recLen = 
+          (keyLen + WritableUtils.getVIntSize(keyLen)) + 
+          (valLen + WritableUtils.getVIntSize(valLen));
+      
+      currentReadBytes += recLen;
+      currentOffset += recLen;
+      currentLength -= recLen;
+
+      // Write out key/value into the in-mem ifile
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("XXX read: sortOutput.append" +
+            " #rec=" + ++numRec +
+            " recIndex=" + recIndex + " kvoff=" + kvoff + 
+            " keyLen=" + keyLen + " valLen=" + valLen + " recLen=" + recLen +
+            " readBytes=" + readBytes +
+            " currentReadBytes="  + currentReadBytes +
+            " currentLength=" + currentLength);
+      }
+      sortOutput.append(key, value);
+    }
+
+    // If we are at the end of the segment, close the ifile
+    if (currentLength > 0 && 
+        (recIndex == mend || kvPartition != partition)) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("XXX About to call close:" +
+            " currentLength=" + currentLength + 
+            " recIndex=" + recIndex + " mend=" + mend + 
+            " kvPartition=" + kvPartition + " partitino=" + partition);
+      }
+      sortOutput.close();
+      currentReadBytes += 
+          (InMemoryShuffleSorter.IFILE_EOF_LENGTH + 
+              InMemoryShuffleSorter.IFILE_CHECKSUM_LENGTH);
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("XXX Hmm..." +
+            " currentLength=" + currentLength + 
+            " recIndex=" + recIndex + " mend=" + mend + 
+            " kvPartition=" + kvPartition + " partitino=" + partition);
+      }
+    }
+    
+    int retVal = Math.min(currentReadBytes, len);
+    readBytes += retVal;
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("XXX read: done" +
+          " retVal=" + retVal + 
+          " currentReadBytes=" + currentReadBytes +
+          " len=" + len + 
+          " readBytes=" + readBytes +
+          " partitionBytes=" + partitionBytes +
+          " residualBytes=" + out.getCurrent());
+    }
+    return retVal;
+  }
+
+  private int getKVPartition(int recIndex) {
+    return kvmeta.get(
+        sorter.offsetFor(recIndex) + InMemoryShuffleSorter.PARTITION);
+  }
+  
+  @Override
+  public int available() throws IOException {
+    return (partitionBytes - readBytes);
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close();
+  }
+
+  @Override
+  public boolean markSupported() {
+    return false;
+  }
+  
+  static class DualBufferOutputStream extends BoundedByteArrayOutputStream {
+
+    byte[] dualBuf;
+    int currentPointer = 0;
+    byte[] one = new byte[1];
+    
+    public DualBufferOutputStream(
+        byte[] buf, int offset, int length, 
+        byte[] altBuf) {
+      super(buf, offset, length);
+      this.dualBuf = altBuf;
+    }
+    
+    public void reset(byte[] b, int off, int len) {
+      super.resetBuffer(b, off, len);
+    }
+
+    @Override
+    public void write(int b) throws IOException {
+      one[0] = (byte)b;
+      write(one, 0, 1);
+    }
+
+    @Override
+    public void write(byte[] b) throws IOException {
+      write(b, 0, b.length);
+    }
+
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException {
+      int available = super.available();
+      if (available >= len) {
+        super.write(b, off, len);
+      } else {
+        super.write(b, off, available);
+        System.arraycopy(b, off+available, dualBuf, currentPointer, len-available);
+        currentPointer += (len - available);
+      }
+    }
+    
+    int getCurrent() {
+      return currentPointer;
+    }
+    
+    void setCurrentPointer(int delta) {
+      if ((currentPointer + delta) > dualBuf.length) {
+        throw new IndexOutOfBoundsException("Trying to set dualBuf 'current'" +
+        		" marker to " + (currentPointer+delta) + " when " +
+        		" dualBuf.length is " + dualBuf.length);
+      }
+      currentPointer = (currentPointer + delta) % dualBuf.length;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/impl/ValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/impl/ValuesIterator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/impl/ValuesIterator.java
new file mode 100644
index 0000000..88cb750
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/impl/ValuesIterator.java
@@ -0,0 +1,149 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.common.task.impl;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.serializer.Deserializer;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.util.Progressable;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+
+
+/**
+ * Iterates values while keys match in sorted input.
+ *
+ * Usage: Call moveToNext to move to the next k, v pair. This returns true if another exists,
+ * followed by getKey() and getValues() to get the current key and list of values.
+ * 
+ */
+public class ValuesIterator<KEY,VALUE> implements Iterator<VALUE> {
+  protected TezRawKeyValueIterator in; //input iterator
+  private KEY key;               // current key
+  private KEY nextKey;
+  private VALUE value;             // current value
+  private boolean hasNext;                      // more w/ this key
+  private boolean more;                         // more in file
+  private RawComparator<KEY> comparator;
+  protected Progressable reporter;
+  private Deserializer<KEY> keyDeserializer;
+  private Deserializer<VALUE> valDeserializer;
+  private DataInputBuffer keyIn = new DataInputBuffer();
+  private DataInputBuffer valueIn = new DataInputBuffer();
+  
+  public ValuesIterator (TezRawKeyValueIterator in, 
+                         RawComparator<KEY> comparator, 
+                         Class<KEY> keyClass,
+                         Class<VALUE> valClass, Configuration conf, 
+                         Progressable reporter)
+    throws IOException {
+    this.in = in;
+    this.comparator = comparator;
+    this.reporter = reporter;
+    SerializationFactory serializationFactory = new SerializationFactory(conf);
+    this.keyDeserializer = serializationFactory.getDeserializer(keyClass);
+    this.keyDeserializer.open(keyIn);
+    this.valDeserializer = serializationFactory.getDeserializer(valClass);
+    this.valDeserializer.open(this.valueIn);
+    readNextKey();
+    key = nextKey;
+    nextKey = null; // force new instance creation
+    hasNext = more;
+  }
+
+  TezRawKeyValueIterator getRawIterator() { return in; }
+  
+  /// Iterator methods
+
+  public boolean hasNext() { return hasNext; }
+
+  private int ctr = 0;
+  public VALUE next() {
+    if (!hasNext) {
+      throw new NoSuchElementException("iterate past last value");
+    }
+    try {
+      readNextValue();
+      readNextKey();
+    } catch (IOException ie) {
+      throw new RuntimeException("problem advancing post rec#"+ctr, ie);
+    }
+    reporter.progress();
+    return value;
+  }
+
+  public void remove() { throw new RuntimeException("not implemented"); }
+
+  /// Auxiliary methods
+
+  /** Start processing next unique key. */
+  public void nextKey() throws IOException {
+    // read until we find a new key
+    while (hasNext) { 
+      readNextKey();
+    }
+    ++ctr;
+    
+    // move the next key to the current one
+    KEY tmpKey = key;
+    key = nextKey;
+    nextKey = tmpKey;
+    hasNext = more;
+  }
+
+  /** True iff more keys remain. */
+  public boolean more() { 
+    return more; 
+  }
+
+  /** The current key. */
+  public KEY getKey() { 
+    return key; 
+  }
+
+  /** 
+   * read the next key 
+   */
+  private void readNextKey() throws IOException {
+    more = in.next();
+    if (more) {
+      DataInputBuffer nextKeyBytes = in.getKey();
+      keyIn.reset(nextKeyBytes.getData(), nextKeyBytes.getPosition(), nextKeyBytes.getLength());
+      nextKey = keyDeserializer.deserialize(nextKey);
+      hasNext = key != null && (comparator.compare(key, nextKey) == 0);
+    } else {
+      hasNext = false;
+    }
+  }
+
+  /**
+   * Read the next value
+   * @throws IOException
+   */
+  private void readNextValue() throws IOException {
+    DataInputBuffer nextValueBytes = in.getValue();
+    valueIn.reset(nextValueBytes.getData(), nextValueBytes.getPosition(), nextValueBytes.getLength());
+    value = valDeserializer.deserialize(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezLocalTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezLocalTaskOutputFiles.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezLocalTaskOutputFiles.java
new file mode 100644
index 0000000..30d28f0
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezLocalTaskOutputFiles.java
@@ -0,0 +1,249 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common.task.local.output;
+
+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.TezJobConfig;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+/**
+ * Manipulate the working area for the transient store for maps and reduces.
+ *
+ * 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.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_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.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_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.TEZ_RUNTIME_TASK_OUTPUT_DIR
+        + Path.SEPARATOR + Constants.TEZ_RUNTIME_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_RUNTIME_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.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_RUNTIME_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.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_RUNTIME_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_RUNTIME_TASK_OUTPUT_FILENAME_STRING + Constants.TEZ_RUNTIME_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.TEZ_RUNTIME_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.TEZ_RUNTIME_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.TEZ_RUNTIME_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.TEZ_RUNTIME_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_RUNTIME_TASK_INPUT_FILE_FORMAT_STRING, 
+        Constants.TEZ_RUNTIME_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_RUNTIME_TASK_INPUT_FILE_FORMAT_STRING, Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR, taskId),
+        size, conf);
+  }
+
+  /** Removes all of the files related to a task. */
+  @Override
+  public void removeAll()
+      throws IOException {
+    deleteLocalFiles(Constants.TEZ_RUNTIME_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/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutput.java
new file mode 100644
index 0000000..d3e7d27
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutput.java
@@ -0,0 +1,165 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common.task.local.output;
+
+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.runtime.library.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/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java
new file mode 100644
index 0000000..2c18b4e
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/task/local/output/TezTaskOutputFiles.java
@@ -0,0 +1,246 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common.task.local.output;
+
+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.TezJobConfig;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+/**
+ * Manipulate the working area for the transient store for maps and reduces.
+ *
+ * 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.TEZ_RUNTIME_TASK_OUTPUT_DIR + "/"
+          + uniqueId);
+    }
+    return new Path(Constants.TEZ_RUNTIME_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_RUNTIME_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_RUNTIME_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_RUNTIME_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.TEZ_RUNTIME_TASK_OUTPUT_DIR);
+    Path attemptOutputDir = new Path(outputDir, uniqueId);
+    return new Path(attemptOutputDir, Constants.TEZ_RUNTIME_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_RUNTIME_TASK_OUTPUT_FILENAME_STRING +
+                                      Constants.TEZ_RUNTIME_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_RUNTIME_TASK_OUTPUT_FILENAME_STRING +
+                                      Constants.TEZ_RUNTIME_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.TEZ_RUNTIME_TASK_OUTPUT_DIR);
+    Path attemptOutputDir = new Path(outputDir, uniqueId);
+    return new Path(attemptOutputDir, Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING +
+                                      Constants.TEZ_RUNTIME_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/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/hadoop/compat/NullProgressable.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/hadoop/compat/NullProgressable.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/hadoop/compat/NullProgressable.java
new file mode 100644
index 0000000..eb8d176
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/hadoop/compat/NullProgressable.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.runtime.library.hadoop.compat;
+
+import org.apache.hadoop.util.Progressable;
+
+public class NullProgressable implements Progressable {
+
+  public NullProgressable() {
+    // TODO Auto-generated constructor stub
+  }
+
+  @Override
+  public void progress() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/LocalMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/LocalMergedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/LocalMergedInput.java
new file mode 100644
index 0000000..3aec247
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/LocalMergedInput.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.runtime.library.input;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.localshuffle.LocalShuffle;
+
+/**
+ * <code>LocalMergedInput</code> in an {@link LogicalInput} which shuffles intermediate
+ * sorted data, merges them and provides key/<values> to the consumer. 
+ */
+public class LocalMergedInput extends ShuffledMergedInputLegacy {
+
+  @Override
+  public List<Event> initialize(TezInputContext inputContext) throws IOException {
+    this.inputContext = inputContext;
+    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
+
+    LocalShuffle localShuffle = new LocalShuffle(inputContext, conf, numInputs);
+    rawIter = localShuffle.run();
+    createValuesIterator();
+    return Collections.emptyList();
+  }
+
+  @Override
+  public List<Event> close() throws IOException {
+    rawIter.close();
+    return Collections.emptyList();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInput.java
new file mode 100644
index 0000000..771ac1b
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInput.java
@@ -0,0 +1,179 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.input;
+
+import java.io.IOException;
+import java.util.Collections;
+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.io.RawComparator;
+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.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.ValuesIterator;
+import org.apache.tez.runtime.library.common.shuffle.impl.Shuffle;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+
+/**
+ * <code>ShuffleMergedInput</code> in a {@link LogicalInput} which shuffles
+ * 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.
+ *
+ */
+public class ShuffledMergedInput implements LogicalInput {
+
+  static final Log LOG = LogFactory.getLog(ShuffledMergedInput.class);
+
+  protected TezInputContext inputContext;
+  protected TezRawKeyValueIterator rawIter = null;
+  protected Configuration conf;
+  protected int numInputs = 0;
+  protected Shuffle shuffle;
+  @SuppressWarnings("rawtypes")
+  protected ValuesIterator vIter;
+
+  private TezCounter inputKeyCounter;
+  private TezCounter inputValueCounter;
+
+  @Override
+  public List<Event> initialize(TezInputContext inputContext) throws IOException {
+    this.inputContext = inputContext;
+    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
+
+    this.inputKeyCounter = inputContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_GROUPS);
+    this.inputValueCounter = inputContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_RECORDS);
+    this.conf.setStrings(TezJobConfig.LOCAL_DIRS,
+        inputContext.getWorkDirs());
+
+    // Start the shuffle - copy and merge.
+    shuffle = new Shuffle(inputContext, this.conf, numInputs);
+    shuffle.run();
+
+    return Collections.emptyList();
+  }
+
+  /**
+   * Check if the input is ready for consumption
+   *
+   * @return true if the input is ready for consumption, or if an error occurred
+   *         processing fetching the input. false if the shuffle and merge are
+   *         still in progress
+   */
+  public boolean isInputReady() {
+    return shuffle.isInputReady();
+  }
+
+  /**
+   * Waits for the input to become ready for consumption
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public void waitForInputReady() throws IOException, InterruptedException {
+    rawIter = shuffle.waitForInput();
+    createValuesIterator();
+  }
+
+  @Override
+  public List<Event> close() throws IOException {
+    rawIter.close();
+    return Collections.emptyList();
+  }
+
+  /**
+   * Get a KVReader for the Input.</p> This method will block until the input is
+   * ready - i.e. the copy and merge stages are complete. Users can use the
+   * isInputReady method to check if the input is ready, which gives an
+   * indication of whether this method will block or not.
+   *
+   * NOTE: All values for the current K-V pair must be read prior to invoking
+   * moveToNext. Once moveToNext() is called, the valueIterator from the
+   * previous K-V pair will throw an Exception
+   *
+   * @return a KVReader over the sorted input.
+   */
+  @Override
+  public KVReader getReader() throws IOException {
+    if (rawIter == null) {
+      try {
+        waitForInputReady();
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        throw new IOException("Interrupted while waiting for input ready", e);
+      }
+    }
+    return new KVReader() {
+
+      @Override
+      public boolean next() throws IOException {
+        return vIter.moveToNext();
+      }
+
+      @SuppressWarnings("unchecked")
+      @Override
+      public KVRecord getCurrentKV() {
+        return new KVRecord(vIter.getKey(), vIter.getValues());
+      }
+    };
+  }
+
+  @Override
+  public void handleEvents(List<Event> inputEvents) {
+    shuffle.handleEvents(inputEvents);
+  }
+
+  @Override
+  public void setNumPhysicalInputs(int numInputs) {
+    this.numInputs = numInputs;
+  }
+
+  @SuppressWarnings({ "rawtypes", "unchecked" })
+  protected void createValuesIterator()
+      throws IOException {
+    vIter = new ValuesIterator(rawIter,
+        (RawComparator) ConfigUtils.getIntermediateInputKeyComparator(conf),
+        ConfigUtils.getIntermediateInputKeyClass(conf),
+        ConfigUtils.getIntermediateInputValueClass(conf), conf, inputKeyCounter, inputValueCounter);
+
+  }
+
+  // This functionality is currently broken. If there's inputs which need to be
+  // written to disk, there's a possibility that inputs from the different
+  // sources could clobber each others' output. Also the current structures do
+  // not have adequate information to de-dupe these (vertex name)
+//  public void mergeWith(ShuffledMergedInput other) {
+//    this.numInputs += other.getNumPhysicalInputs();
+//  }
+//
+//  public int getNumPhysicalInputs() {
+//    return this.numInputs;
+//  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInputLegacy.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInputLegacy.java
new file mode 100644
index 0000000..97e19d8
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledMergedInputLegacy.java
@@ -0,0 +1,30 @@
+/**
+ * <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.runtime.library.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+
+@LimitedPrivate("mapreduce")
+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/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledUnorderedKVInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledUnorderedKVInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledUnorderedKVInput.java
new file mode 100644
index 0000000..42b2e00
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/input/ShuffledUnorderedKVInput.java
@@ -0,0 +1,76 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.input;
+
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.Reader;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.broadcast.input.BroadcastShuffleManager;
+
+import com.google.common.base.Preconditions;
+
+public class ShuffledUnorderedKVInput implements LogicalInput {
+
+  private Configuration conf;
+  private int numInputs = -1;
+  private BroadcastShuffleManager shuffleManager;
+  
+  
+  
+  public ShuffledUnorderedKVInput() {
+  }
+
+  @Override
+  public List<Event> initialize(TezInputContext inputContext) throws Exception {
+    Preconditions.checkArgument(numInputs != -1, "Number of Inputs has not been set");
+    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
+    this.conf.setStrings(TezJobConfig.LOCAL_DIRS, inputContext.getWorkDirs());
+    
+    this.shuffleManager = new BroadcastShuffleManager(inputContext, conf, numInputs);
+    return null;
+  }
+
+  @Override
+  public Reader getReader() throws Exception {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public void handleEvents(List<Event> inputEvents) {
+    shuffleManager.handleEvents(inputEvents);
+  }
+
+  @Override
+  public List<Event> close() throws Exception {
+    this.shuffleManager.shutdown();
+    return null;
+  }
+
+  @Override
+  public void setNumPhysicalInputs(int numInputs) {
+    this.numInputs = numInputs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/InMemorySortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/InMemorySortedOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/InMemorySortedOutput.java
new file mode 100644
index 0000000..2ec6b2a
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/InMemorySortedOutput.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.output;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.Output;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.Writer;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.sort.impl.dflt.InMemoryShuffleSorter;
+
+/**
+ * {@link InMemorySortedOutput} is an {@link Output} which sorts key/value pairs 
+ * written to it and persists it to a file.
+ */
+public class InMemorySortedOutput implements LogicalOutput {
+  
+  protected InMemoryShuffleSorter sorter;
+  protected int numTasks;
+  protected TezOutputContext outputContext;
+  
+
+  @Override
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws IOException {
+    this.outputContext = outputContext;
+    this.sorter = new InMemoryShuffleSorter();
+    sorter.initialize(outputContext, TezUtils.createConfFromUserPayload(outputContext.getUserPayload()), numTasks);
+    return Collections.emptyList();
+  }
+
+  @Override
+  public Writer getWriter() throws IOException {
+    return new KVWriter() {
+      
+      @Override
+      public void write(Object key, Object value) throws IOException {
+        sorter.write(key, value);
+      }
+    };
+  }
+
+  @Override
+  public void handleEvents(List<Event> outputEvents) {
+    // No events expected.
+  }
+
+  @Override
+  public void setNumPhysicalOutputs(int numOutputs) {
+    this.numTasks = numOutputs;
+  }
+  
+  @Override
+  public List<Event> close() throws IOException {
+    sorter.flush();
+    sorter.close();
+    // TODO NEWTEZ Event generation
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/LocalOnFileSorterOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/LocalOnFileSorterOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/LocalOnFileSorterOutput.java
new file mode 100644
index 0000000..a19d5e1
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/LocalOnFileSorterOutput.java
@@ -0,0 +1,63 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.output;
+
+import java.io.IOException;
+import java.util.List;
+
+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.runtime.api.Event;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+
+public class LocalOnFileSorterOutput extends OnFileSortedOutput {
+
+  private static final Log LOG = LogFactory.getLog(LocalOnFileSorterOutput.class);
+
+  
+
+  @Override
+  public List<Event> close() throws IOException {
+    LOG.debug("Closing LocalOnFileSorterOutput");
+    super.close();
+
+    TezTaskOutput mapOutputFile = sorter.getMapOutput();
+    FileSystem localFs = FileSystem.getLocal(conf);
+
+    Path src = mapOutputFile.getOutputFile();
+    Path dst =
+        mapOutputFile.getInputFileForWrite(
+            outputContext.getTaskIndex(),
+            localFs.getFileStatus(src).getLen());
+
+    LOG.info("Renaming src = " + src + ", dst = " + dst);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Renaming src = " + src + ", dst = " + dst);
+    }
+    localFs.rename(src, dst);
+    return null;
+  }
+  
+  @Override
+  protected List<Event> generateDataMovementEventsOnClose() throws IOException {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileSortedOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileSortedOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileSortedOutput.java
new file mode 100644
index 0000000..42e1eeb
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileSortedOutput.java
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.output;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+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.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.sort.impl.ExternalSorter;
+import org.apache.tez.runtime.library.common.sort.impl.dflt.DefaultSorter;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
+
+import com.google.common.collect.Lists;
+
+/**
+ * <code>OnFileSortedOutput</code> is an {@link LogicalOutput} which sorts key/value pairs 
+ * written to it and persists it to a file.
+ */
+public class OnFileSortedOutput implements LogicalOutput {
+  
+  protected ExternalSorter sorter;
+  protected Configuration conf;
+  protected int numOutputs;
+  protected TezOutputContext outputContext;
+  private long startTime;
+  private long endTime;
+  
+  
+  @Override
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws IOException {
+    this.startTime = System.nanoTime();
+    this.outputContext = outputContext;
+    sorter = new DefaultSorter();
+    this.conf = TezUtils.createConfFromUserPayload(outputContext.getUserPayload());
+    // Initializing this parametr in this conf since it is used in multiple
+    // places (wherever LocalDirAllocator is used) - TezTaskOutputFiles,
+    // TezMerger, etc.
+    this.conf.setStrings(TezJobConfig.LOCAL_DIRS, outputContext.getWorkDirs());
+    sorter.initialize(outputContext, conf, numOutputs);
+    return Collections.emptyList();
+  }
+
+  @Override
+  public KVWriter getWriter() throws IOException {
+    return new KVWriter() {
+      @Override
+      public void write(Object key, Object value) throws IOException {
+        sorter.write(key, value);
+      }
+    };
+  }
+
+  @Override
+  public void handleEvents(List<Event> outputEvents) {
+    // Not expecting any events.
+  }
+
+  @Override
+  public void setNumPhysicalOutputs(int numOutputs) {
+    this.numOutputs = numOutputs;
+  }
+
+  @Override
+  public List<Event> close() throws IOException {
+    sorter.flush();
+    sorter.close();
+    this.endTime = System.nanoTime();
+
+   return generateDataMovementEventsOnClose();
+  }
+  
+  protected List<Event> generateDataMovementEventsOnClose() throws IOException {
+    String host = System.getenv(ApplicationConstants.Environment.NM_HOST
+        .toString());
+    ByteBuffer shuffleMetadata = outputContext
+        .getServiceProviderMetaData(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID);
+    int shufflePort = ShuffleUtils.deserializeShuffleProviderMetaData(shuffleMetadata);
+
+    DataMovementEventPayloadProto.Builder payloadBuilder = DataMovementEventPayloadProto
+        .newBuilder();
+    payloadBuilder.setHost(host);
+    payloadBuilder.setPort(shufflePort);
+    payloadBuilder.setPathComponent(outputContext.getUniqueIdentifier());
+    payloadBuilder.setRunDuration((int) ((endTime - startTime) / 1000));
+    DataMovementEventPayloadProto payloadProto = payloadBuilder.build();
+    byte[] payloadBytes = payloadProto.toByteArray();
+
+    List<Event> events = Lists.newArrayListWithCapacity(numOutputs);
+
+    for (int i = 0; i < numOutputs; i++) {
+      DataMovementEvent event = new DataMovementEvent(i, payloadBytes);
+      events.add(event);
+    }
+    return events;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileUnorderedKVOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileUnorderedKVOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileUnorderedKVOutput.java
new file mode 100644
index 0000000..dd18149
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/output/OnFileUnorderedKVOutput.java
@@ -0,0 +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.tez.runtime.library.output;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.broadcast.output.FileBasedKVWriter;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class OnFileUnorderedKVOutput implements LogicalOutput {
+
+  private TezOutputContext outputContext;
+  private FileBasedKVWriter kvWriter;
+
+  public OnFileUnorderedKVOutput() {
+  }
+
+  @Override
+  public List<Event> initialize(TezOutputContext outputContext)
+      throws Exception {
+    this.outputContext = outputContext;
+    this.kvWriter = new FileBasedKVWriter(outputContext);
+    return Collections.emptyList();
+  }
+
+  @Override
+  public KVWriter getWriter() throws Exception {
+    return kvWriter;
+  }
+
+  @Override
+  public void handleEvents(List<Event> outputEvents) {
+    throw new TezUncheckedException("Not expecting any events");
+  }
+
+  @Override
+  public List<Event> close() throws Exception {
+    boolean outputGenerated = this.kvWriter.close();
+    DataMovementEventPayloadProto.Builder payloadBuilder = DataMovementEventPayloadProto
+        .newBuilder();
+
+    String host = System.getenv(ApplicationConstants.Environment.NM_HOST
+        .toString());
+    ByteBuffer shuffleMetadata = outputContext
+        .getServiceProviderMetaData(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID);
+    int shufflePort = ShuffleUtils
+        .deserializeShuffleProviderMetaData(shuffleMetadata);
+    payloadBuilder.setOutputGenerated(outputGenerated);
+    if (outputGenerated) {
+      payloadBuilder.setHost(host);
+      payloadBuilder.setPort(shufflePort);
+      payloadBuilder.setPathComponent(outputContext.getUniqueIdentifier());
+    }
+    DataMovementEventPayloadProto payloadProto = payloadBuilder.build();
+
+    DataMovementEvent dmEvent = new DataMovementEvent(0,
+        payloadProto.toByteArray());
+    List<Event> events = Lists.newArrayListWithCapacity(1);
+    events.add(dmEvent);
+    return events;
+  }
+
+  @Override
+  public void setNumPhysicalOutputs(int numOutputs) {
+    Preconditions.checkArgument(numOutputs == 1,
+        "Number of outputs can only be 1 for " + this.getClass().getName());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
new file mode 100644
index 0000000..a98ce63
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.shuffle.common;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
+
+import com.google.common.base.Preconditions;
+
+public class DiskFetchedInput extends FetchedInput {
+
+  private static final Log LOG = LogFactory.getLog(DiskFetchedInput.class);
+  
+  private final FileSystem localFS;
+  private final Path tmpOutputPath;
+  private final Path outputPath;
+
+  public DiskFetchedInput(long size,
+      InputAttemptIdentifier inputAttemptIdentifier,
+      FetchedInputCallback callbackHandler, Configuration conf,
+      LocalDirAllocator localDirAllocator, TezTaskOutputFiles filenameAllocator)
+      throws IOException {
+    super(Type.DISK, size, inputAttemptIdentifier, callbackHandler);
+
+    this.localFS = FileSystem.getLocal(conf);
+    this.outputPath = filenameAllocator.getInputFileForWrite(
+        this.inputAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(), size);
+    this.tmpOutputPath = outputPath.suffix(String.valueOf(id));
+  }
+
+  @Override
+  public OutputStream getOutputStream() throws IOException {
+    return localFS.create(tmpOutputPath);
+  }
+
+  @Override
+  public InputStream getInputStream() throws IOException {
+    return localFS.open(outputPath);
+  }
+
+  @Override
+  public void commit() throws IOException {
+    if (state == State.PENDING) {
+      state = State.COMMITTED;
+      localFS.rename(tmpOutputPath, outputPath);
+      notifyFetchComplete();
+    }
+  }
+
+  @Override
+  public void abort() throws IOException {
+    if (state == State.PENDING) {
+      state = State.ABORTED;
+      // TODO NEWTEZ Maybe defer this to container cleanup
+      localFS.delete(tmpOutputPath, false);
+      notifyFetchFailure();
+    }
+  }
+  
+  @Override
+  public void free() {
+    Preconditions.checkState(
+        state == State.COMMITTED || state == State.ABORTED,
+        "FetchedInput can only be freed after it is committed or aborted");
+    if (state == State.COMMITTED) {
+      state = State.FREED;
+      try {
+        // TODO NEWTEZ Maybe defer this to container cleanup
+        localFS.delete(outputPath, false);
+      } catch (IOException e) {
+        // Ignoring the exception, will eventually be cleaned by container
+        // cleanup.
+        LOG.warn("Failed to remvoe file : " + outputPath.toString());
+      }
+      notifyFreedResource();
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "DiskFetchedInput [outputPath=" + outputPath
+        + ", inputAttemptIdentifier=" + inputAttemptIdentifier + ", size="
+        + size + ", type=" + type + ", id=" + id + ", state=" + state + "]";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchResult.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchResult.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchResult.java
new file mode 100644
index 0000000..df38b07
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchResult.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.runtime.library.shuffle.common;
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+public class FetchResult {
+
+  private final String host;
+  private final int port;
+  private final int partition;
+  private final Iterable<InputAttemptIdentifier> pendingInputs;
+
+  public FetchResult(String host, int port, int partition,
+      Iterable<InputAttemptIdentifier> pendingInputs) {
+    this.host = host;
+    this.port = port;
+    this.partition = partition;
+    this.pendingInputs = pendingInputs;
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public int getPartition() {
+    return partition;
+  }
+
+  public Iterable<InputAttemptIdentifier> getPendingInputs() {
+    return pendingInputs;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
new file mode 100644
index 0000000..8f3c407
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.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.runtime.library.shuffle.common;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+@Private
+public abstract class FetchedInput {
+  
+  public static enum Type {
+    WAIT, // TODO NEWTEZ Implement this, only if required.
+    MEMORY,
+    DISK,
+  }
+  
+  protected static enum State {
+    PENDING, COMMITTED, ABORTED, FREED
+  }
+
+  private static AtomicInteger ID_GEN = new AtomicInteger(0);
+
+  protected InputAttemptIdentifier inputAttemptIdentifier;
+  protected final long size;
+  protected final Type type;
+  protected final FetchedInputCallback callback;
+  protected final int id;
+  protected State state;
+
+  public FetchedInput(Type type, long size,
+      InputAttemptIdentifier inputAttemptIdentifier,
+      FetchedInputCallback callbackHandler) {
+    this.type = type;
+    this.size = size;
+    this.inputAttemptIdentifier = inputAttemptIdentifier;
+    this.callback = callbackHandler;
+    this.id = ID_GEN.getAndIncrement();
+    this.state = State.PENDING;
+  }
+
+  public Type getType() {
+    return this.type;
+  }
+
+  public long getSize() {
+    return this.size;
+  }
+
+  public InputAttemptIdentifier getInputAttemptIdentifier() {
+    return this.inputAttemptIdentifier;
+  }
+
+  /**
+   * Inform the Allocator about a committed resource.
+   * This should be called by commit
+   */
+  public void notifyFetchComplete() {
+    this.callback.fetchComplete(this);
+  }
+  
+  /**
+   * Inform the Allocator about a failed resource.
+   * This should be called by abort
+   */
+  public void notifyFetchFailure() {
+    this.callback.fetchFailed(this);
+  }
+  
+  /**
+   * Inform the Allocator about a completed resource being released.
+   * This should be called by free
+   */
+  public void notifyFreedResource() {
+    this.callback.freeResources(this);
+  }
+  
+  /**
+   * Returns the output stream to be used to write fetched data. Users are
+   * expected to close the OutputStream when they're done
+   */
+  public abstract OutputStream getOutputStream() throws IOException;
+
+  /**
+   * Return an input stream to be used to read the previously fetched data.
+   * Users are expected to close the InputStream when they're done
+   */
+  public abstract InputStream getInputStream() throws IOException;
+
+  /**
+   * Commit the output. Should be idempotent
+   */
+  public abstract void commit() throws IOException;
+
+  /**
+   * Abort the output. Should be idempotent
+   */
+  public abstract void abort() throws IOException;
+
+  /**
+   * Called when this input has been consumed, so that resources can be
+   * reclaimed.
+   */
+  public abstract void free();
+  
+  @Override
+  public int hashCode() {
+    return id;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    FetchedInput other = (FetchedInput) obj;
+    if (id != other.id)
+      return false;
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
new file mode 100644
index 0000000..1d60b68
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.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.runtime.library.shuffle.common;
+
+import java.io.IOException;
+
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+public interface FetchedInputAllocator {
+
+  public FetchedInput allocate(long size,
+      InputAttemptIdentifier inputAttemptIdentifier) throws IOException;
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputCallback.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputCallback.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputCallback.java
new file mode 100644
index 0000000..a22ce45
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputCallback.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.runtime.library.shuffle.common;
+
+public interface FetchedInputCallback {
+  
+  public void fetchComplete(FetchedInput fetchedInput);
+  
+  public void fetchFailed(FetchedInput fetchedInput);
+  
+  public void freeResources(FetchedInput fetchedInput);
+  
+}


[04/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
new file mode 100644
index 0000000..4ce82d5
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFile.java
@@ -0,0 +1,559 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.common.sort.impl;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+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.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.BufferUtils;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.Compressor;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.tez.common.counters.TezCounter;
+
+/**
+ * <code>IFile</code> is the simple <key-len, value-len, key, value> format
+ * for the intermediate map-outputs in Map-Reduce.
+ *
+ * There is a <code>Writer</code> to write out map-outputs in this format and 
+ * a <code>Reader</code> to read files of this format.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class IFile {
+  private static final Log LOG = LogFactory.getLog(IFile.class);
+  public static final int EOF_MARKER = -1; // End of File Marker
+  public static final int RLE_MARKER = -2; // Repeat same key marker
+  public static final DataInputBuffer REPEAT_KEY = new DataInputBuffer();
+    
+  /**
+   * <code>IFile.Writer</code> to write out intermediate map-outputs. 
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  public static class Writer {
+    FSDataOutputStream out;
+    boolean ownOutputStream = false;
+    long start = 0;
+    FSDataOutputStream rawOut;
+    AtomicBoolean closed = new AtomicBoolean(false);
+    
+    CompressionOutputStream compressedOut;
+    Compressor compressor;
+    boolean compressOutput = false;
+    
+    long decompressedBytesWritten = 0;
+    long compressedBytesWritten = 0;
+
+    // Count records written to disk
+    private long numRecordsWritten = 0;
+    private final TezCounter writtenRecordsCounter;
+
+    IFileOutputStream checksumOut;
+
+    Class keyClass;
+    Class valueClass;
+    Serializer keySerializer;
+    Serializer valueSerializer;
+    
+    DataOutputBuffer buffer = new DataOutputBuffer();
+    DataOutputBuffer previous = new DataOutputBuffer();
+    
+    // de-dup keys or not
+    private boolean rle = false;
+
+    public Writer(Configuration conf, FileSystem fs, Path file, 
+                  Class keyClass, Class valueClass,
+                  CompressionCodec codec,
+                  TezCounter writesCounter) throws IOException {
+      this(conf, fs.create(file), keyClass, valueClass, codec,
+           writesCounter);
+      ownOutputStream = true;
+    }
+    
+    protected Writer(TezCounter writesCounter) {
+      writtenRecordsCounter = writesCounter;
+    }
+
+    public Writer(Configuration conf, FSDataOutputStream out, 
+        Class keyClass, Class valueClass,
+        CompressionCodec codec, TezCounter writesCounter)
+        throws IOException {
+      this.writtenRecordsCounter = writesCounter;
+      this.checksumOut = new IFileOutputStream(out);
+      this.rawOut = out;
+      this.start = this.rawOut.getPos();
+      if (codec != null) {
+        this.compressor = CodecPool.getCompressor(codec);
+        if (this.compressor != null) {
+          this.compressor.reset();
+          this.compressedOut = codec.createOutputStream(checksumOut, compressor);
+          this.out = new FSDataOutputStream(this.compressedOut,  null);
+          this.compressOutput = true;
+        } else {
+          LOG.warn("Could not obtain compressor from CodecPool");
+          this.out = new FSDataOutputStream(checksumOut,null);
+        }
+      } else {
+        this.out = new FSDataOutputStream(checksumOut,null);
+      }
+      
+      this.keyClass = keyClass;
+      this.valueClass = valueClass;
+
+      if (keyClass != null) {
+        SerializationFactory serializationFactory = 
+          new SerializationFactory(conf);
+        this.keySerializer = serializationFactory.getSerializer(keyClass);
+        this.keySerializer.open(buffer);
+        this.valueSerializer = serializationFactory.getSerializer(valueClass);
+        this.valueSerializer.open(buffer);
+      }
+    }
+
+    public Writer(Configuration conf, FileSystem fs, Path file) 
+    throws IOException {
+      this(conf, fs, file, null, null, null, null);
+    }
+
+    public void close() throws IOException {
+      if (closed.getAndSet(true)) {
+        throw new IOException("Writer was already closed earlier");
+      }
+
+      // When IFile writer is created by BackupStore, we do not have
+      // Key and Value classes set. So, check before closing the
+      // serializers
+      if (keyClass != null) {
+        keySerializer.close();
+        valueSerializer.close();
+      }
+
+      // Write EOF_MARKER for key/value length
+      WritableUtils.writeVInt(out, EOF_MARKER);
+      WritableUtils.writeVInt(out, EOF_MARKER);
+      decompressedBytesWritten += 2 * WritableUtils.getVIntSize(EOF_MARKER);
+      
+      //Flush the stream
+      out.flush();
+  
+      if (compressOutput) {
+        // Flush
+        compressedOut.finish();
+        compressedOut.resetState();
+      }
+      
+      // Close the underlying stream iff we own it...
+      if (ownOutputStream) {
+        out.close();
+      }
+      else {
+        // Write the checksum
+        checksumOut.finish();
+      }
+
+      compressedBytesWritten = rawOut.getPos() - start;
+
+      if (compressOutput) {
+        // Return back the compressor
+        CodecPool.returnCompressor(compressor);
+        compressor = null;
+      }
+
+      out = null;
+      if(writtenRecordsCounter != null) {
+        writtenRecordsCounter.increment(numRecordsWritten);
+      }
+    }
+
+    public void append(Object key, Object value) throws IOException {
+      if (key.getClass() != keyClass)
+        throw new IOException("wrong key class: "+ key.getClass()
+                              +" is not "+ keyClass);
+      if (value.getClass() != valueClass)
+        throw new IOException("wrong value class: "+ value.getClass()
+                              +" is not "+ valueClass);
+      
+      boolean sameKey = false;
+
+      // Append the 'key'
+      keySerializer.serialize(key);
+      int keyLength = buffer.getLength();
+      if (keyLength < 0) {
+        throw new IOException("Negative key-length not allowed: " + keyLength + 
+                              " for " + key);
+      }     
+      
+      if(keyLength == previous.getLength()) {
+        sameKey = (BufferUtils.compare(previous, buffer) == 0);       
+      }
+      
+      if(!sameKey) {
+        BufferUtils.copy(buffer, previous);
+      }
+
+      // Append the 'value'
+      valueSerializer.serialize(value);
+      int valueLength = buffer.getLength() - keyLength;
+      if (valueLength < 0) {
+        throw new IOException("Negative value-length not allowed: " + 
+                              valueLength + " for " + value);
+      }
+      
+      if(sameKey) {        
+        WritableUtils.writeVInt(out, RLE_MARKER);                   // Same key as previous
+        WritableUtils.writeVInt(out, valueLength);                  // value length
+        out.write(buffer.getData(), keyLength, buffer.getLength()); // only the value
+        // Update bytes written
+        decompressedBytesWritten += 0 + valueLength + 
+                                    WritableUtils.getVIntSize(RLE_MARKER) + 
+                                    WritableUtils.getVIntSize(valueLength);
+      } else {        
+        // Write the record out        
+        WritableUtils.writeVInt(out, keyLength);                  // key length
+        WritableUtils.writeVInt(out, valueLength);                // value length
+        out.write(buffer.getData(), 0, buffer.getLength());       // data
+        // Update bytes written
+        decompressedBytesWritten += keyLength + valueLength + 
+                                    WritableUtils.getVIntSize(keyLength) + 
+                                    WritableUtils.getVIntSize(valueLength);
+      }
+
+      // Reset
+      buffer.reset();
+      
+      
+      ++numRecordsWritten;
+    }
+    
+    public void append(DataInputBuffer key, DataInputBuffer value)
+    throws IOException {
+      int keyLength = key.getLength() - key.getPosition();
+      if (keyLength < 0) {
+        throw new IOException("Negative key-length not allowed: " + keyLength + 
+                              " for " + key);
+      }
+      
+      int valueLength = value.getLength() - value.getPosition();
+      if (valueLength < 0) {
+        throw new IOException("Negative value-length not allowed: " + 
+                              valueLength + " for " + value);
+      }
+      
+      boolean sameKey = false;
+      
+      if(rle && keyLength == previous.getLength()) {
+        sameKey = (keyLength != 0) && (BufferUtils.compare(previous, key) == 0);        
+      }
+      
+      if(rle && sameKey) {
+        WritableUtils.writeVInt(out, RLE_MARKER);
+        WritableUtils.writeVInt(out, valueLength);        
+        out.write(value.getData(), value.getPosition(), valueLength);
+
+        // Update bytes written
+        decompressedBytesWritten += 0 + valueLength
+            + WritableUtils.getVIntSize(RLE_MARKER)
+            + WritableUtils.getVIntSize(valueLength);
+      } else {
+        WritableUtils.writeVInt(out, keyLength);
+        WritableUtils.writeVInt(out, valueLength);
+        out.write(key.getData(), key.getPosition(), keyLength);
+        out.write(value.getData(), value.getPosition(), valueLength);
+
+        // Update bytes written
+        decompressedBytesWritten += keyLength + valueLength
+            + WritableUtils.getVIntSize(keyLength)
+            + WritableUtils.getVIntSize(valueLength);
+                
+        BufferUtils.copy(key, previous);        
+      }
+      ++numRecordsWritten;
+    }
+    
+    // Required for mark/reset
+    public DataOutputStream getOutputStream () {
+      return out;
+    }
+    
+    // Required for mark/reset
+    public void updateCountersForExternalAppend(long length) {
+      ++numRecordsWritten;
+      decompressedBytesWritten += length;
+    }
+    
+    public long getRawLength() {
+      return decompressedBytesWritten;
+    }
+    
+    public long getCompressedLength() {
+      return compressedBytesWritten;
+    }
+    
+    public void setRLE(boolean rle) {
+      this.rle = rle;
+      previous.reset();
+    }
+
+  }
+
+  /**
+   * <code>IFile.Reader</code> to read intermediate map-outputs. 
+   */
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public static class Reader {
+    
+    public enum KeyState {NO_KEY, NEW_KEY, SAME_KEY};
+    
+    private static final int DEFAULT_BUFFER_SIZE = 128*1024;
+
+    // Count records read from disk
+    private long numRecordsRead = 0;
+    private final TezCounter readRecordsCounter;
+
+    final InputStream in;        // Possibly decompressed stream that we read
+    Decompressor decompressor;
+    public long bytesRead = 0;
+    protected final long fileLength;
+    protected boolean eof = false;
+    final IFileInputStream checksumIn;
+    
+    protected byte[] buffer = null;
+    protected int bufferSize = DEFAULT_BUFFER_SIZE;
+    protected DataInputStream dataIn;
+
+    protected int recNo = 1;
+    protected int prevKeyLength;
+    protected int currentKeyLength;
+    protected int currentValueLength;
+    byte keyBytes[] = new byte[0];
+    
+    
+    /**
+     * Construct an IFile Reader.
+     * 
+     * @param conf Configuration File 
+     * @param fs  FileSystem
+     * @param file Path of the file to be opened. This file should have
+     *             checksum bytes for the data at the end of the file.
+     * @param codec codec
+     * @param readsCounter Counter for records read from disk
+     * @throws IOException
+     */
+    public Reader(Configuration conf, FileSystem fs, Path file,
+                  CompressionCodec codec,
+                  TezCounter readsCounter) throws IOException {
+      this(conf, fs.open(file), 
+           fs.getFileStatus(file).getLen(),
+           codec, readsCounter);
+    }
+
+    /**
+     * Construct an IFile Reader.
+     * 
+     * @param conf Configuration File 
+     * @param in   The input stream
+     * @param length Length of the data in the stream, including the checksum
+     *               bytes.
+     * @param codec codec
+     * @param readsCounter Counter for records read from disk
+     * @throws IOException
+     */
+    public Reader(Configuration conf, InputStream in, long length, 
+                  CompressionCodec codec,
+                  TezCounter readsCounter) throws IOException {
+      readRecordsCounter = readsCounter;
+      checksumIn = new IFileInputStream(in,length, conf);
+      if (codec != null) {
+        decompressor = CodecPool.getDecompressor(codec);
+        if (decompressor != null) {
+          this.in = codec.createInputStream(checksumIn, decompressor);
+        } else {
+          LOG.warn("Could not obtain decompressor from CodecPool");
+          this.in = checksumIn;
+        }
+      } else {
+        this.in = checksumIn;
+      }
+      this.dataIn = new DataInputStream(this.in);
+      this.fileLength = length;
+      
+      if (conf != null) {
+        bufferSize = conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE);
+      }
+    }
+    
+    public long getLength() { 
+      return fileLength - checksumIn.getSize();
+    }
+    
+    public long getPosition() throws IOException {    
+      return checksumIn.getPosition(); 
+    }
+    
+    /**
+     * Read upto len bytes into buf starting at offset off.
+     * 
+     * @param buf buffer 
+     * @param off offset
+     * @param len length of buffer
+     * @return the no. of bytes read
+     * @throws IOException
+     */
+    private int readData(byte[] buf, int off, int len) throws IOException {
+      int bytesRead = 0;
+      while (bytesRead < len) {
+        int n = IOUtils.wrappedReadForCompressedData(in, buf, off + bytesRead,
+            len - bytesRead);
+        if (n < 0) {
+          return bytesRead;
+        }
+        bytesRead += n;
+      }
+      return len;
+    }
+    
+    protected boolean positionToNextRecord(DataInput dIn) throws IOException {
+      // Sanity check
+      if (eof) {
+        throw new EOFException("Completed reading " + bytesRead);
+      }
+      
+      // Read key and value lengths
+      prevKeyLength = currentKeyLength;
+      currentKeyLength = WritableUtils.readVInt(dIn);
+      currentValueLength = WritableUtils.readVInt(dIn);
+      bytesRead += WritableUtils.getVIntSize(currentKeyLength) +
+                   WritableUtils.getVIntSize(currentValueLength);
+      
+      // Check for EOF
+      if (currentKeyLength == EOF_MARKER && currentValueLength == EOF_MARKER) {
+        eof = true;
+        return false;
+      }      
+      
+      // Sanity check
+      if (currentKeyLength != RLE_MARKER && currentKeyLength < 0) {
+        throw new IOException("Rec# " + recNo + ": Negative key-length: " + 
+                              currentKeyLength);
+      }
+      if (currentValueLength < 0) {
+        throw new IOException("Rec# " + recNo + ": Negative value-length: " + 
+                              currentValueLength);
+      }
+            
+      return true;
+    }
+    
+    public boolean nextRawKey(DataInputBuffer key) throws IOException {
+      return readRawKey(key) != KeyState.NO_KEY;
+    }
+    
+    public KeyState readRawKey(DataInputBuffer key) throws IOException {
+      if (!positionToNextRecord(dataIn)) {
+        return KeyState.NO_KEY;
+      }
+      if(currentKeyLength == RLE_MARKER) {
+        currentKeyLength = prevKeyLength;
+        // no data to read
+        key.reset(keyBytes, currentKeyLength);
+        return KeyState.SAME_KEY;
+      }
+      if (keyBytes.length < currentKeyLength) {
+        keyBytes = new byte[currentKeyLength << 1];
+      }
+      int i = readData(keyBytes, 0, currentKeyLength);
+      if (i != currentKeyLength) {
+        throw new IOException ("Asked for " + currentKeyLength + " Got: " + i);
+      }
+      key.reset(keyBytes, currentKeyLength);
+      bytesRead += currentKeyLength;
+      return KeyState.NEW_KEY;
+    }
+    
+    public void nextRawValue(DataInputBuffer value) throws IOException {
+      final byte[] valBytes = 
+        ((value.getData().length < currentValueLength) || (value.getData() == keyBytes))
+        ? new byte[currentValueLength << 1]
+        : value.getData();
+      int i = readData(valBytes, 0, currentValueLength);
+      if (i != currentValueLength) {
+        throw new IOException ("Asked for " + currentValueLength + " Got: " + i);
+      }
+      value.reset(valBytes, currentValueLength);
+      
+      // Record the bytes read
+      bytesRead += currentValueLength;
+
+      ++recNo;
+      ++numRecordsRead;
+    }
+    
+    public void close() throws IOException {
+      // Close the underlying stream
+      in.close();
+      
+      // Release the buffer
+      dataIn = null;
+      buffer = null;
+      if(readRecordsCounter != null) {
+        readRecordsCounter.increment(numRecordsRead);
+      }
+
+      // Return the decompressor
+      if (decompressor != null) {
+        decompressor.reset();
+        CodecPool.returnDecompressor(decompressor);
+        decompressor = null;
+      }
+    }
+    
+    public void reset(int offset) {
+      return;
+    }
+
+    public void disableChecksumValidation() {
+      checksumIn.disableChecksumValidation();
+    }
+
+  }    
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileInputStream.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileInputStream.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileInputStream.java
new file mode 100644
index 0000000..e828c0b
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileInputStream.java
@@ -0,0 +1,276 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common.sort.impl;
+
+import java.io.EOFException;
+import java.io.FileDescriptor;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+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.ChecksumException;
+import org.apache.hadoop.fs.HasFileDescriptor;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.ReadaheadPool;
+import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.tez.common.TezJobConfig;
+/**
+ * A checksum input stream, used for IFiles.
+ * Used to validate the checksum of files created by {@link IFileOutputStream}. 
+*/
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class IFileInputStream extends InputStream {
+  
+  private final InputStream in; //The input stream to be verified for checksum.
+  private final FileDescriptor inFd; // the file descriptor, if it is known
+  private final long length; //The total length of the input file
+  private final long dataLength;
+  private DataChecksum sum;
+  private long currentOffset = 0;
+  private final byte b[] = new byte[1];
+  private byte csum[] = null;
+  private int checksumSize;
+  private byte[] buffer;
+  private int offset;
+
+  private ReadaheadRequest curReadahead = null;
+  private ReadaheadPool raPool = ReadaheadPool.getInstance();
+  private boolean readahead;
+  private int readaheadLength;
+
+  public static final Log LOG = LogFactory.getLog(IFileInputStream.class);
+
+  private boolean disableChecksumValidation = false;
+  
+  /**
+   * Create a checksum input stream that reads
+   * @param in The input stream to be verified for checksum.
+   * @param len The length of the input stream including checksum bytes.
+   */
+  public IFileInputStream(InputStream in, long len, Configuration conf) {
+    this.in = in;
+    this.inFd = getFileDescriptorIfAvail(in);
+    sum = DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 
+        Integer.MAX_VALUE);
+    checksumSize = sum.getChecksumSize();
+    buffer = new byte[4096];
+    offset = 0;
+    length = len;
+    dataLength = length - checksumSize;
+
+    conf = (conf != null) ? conf : new Configuration();
+    readahead = conf.getBoolean(TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD);
+    readaheadLength = conf.getInt(TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD_BYTES,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD_BYTES);
+
+    doReadahead();
+  }
+
+  private static FileDescriptor getFileDescriptorIfAvail(InputStream in) {
+    FileDescriptor fd = null;
+    try {
+      if (in instanceof HasFileDescriptor) {
+        fd = ((HasFileDescriptor)in).getFileDescriptor();
+      } else if (in instanceof FileInputStream) {
+        fd = ((FileInputStream)in).getFD();
+      }
+    } catch (IOException e) {
+      LOG.info("Unable to determine FileDescriptor", e);
+    }
+    return fd;
+  }
+
+  /**
+   * Close the input stream. Note that we need to read to the end of the
+   * stream to validate the checksum.
+   */
+  @Override
+  public void close() throws IOException {
+
+    if (curReadahead != null) {
+      curReadahead.cancel();
+    }
+    if (currentOffset < dataLength) {
+      byte[] t = new byte[Math.min((int)
+            (Integer.MAX_VALUE & (dataLength - currentOffset)), 32 * 1024)];
+      while (currentOffset < dataLength) {
+        int n = read(t, 0, t.length);
+        if (0 == n) {
+          throw new EOFException("Could not validate checksum");
+        }
+      }
+    }
+    in.close();
+  }
+  
+  @Override
+  public long skip(long n) throws IOException {
+   throw new IOException("Skip not supported for IFileInputStream");
+  }
+  
+  public long getPosition() {
+    return (currentOffset >= dataLength) ? dataLength : currentOffset;
+  }
+  
+  public long getSize() {
+    return checksumSize;
+  }
+
+  private void checksum(byte[] b, int off, int len) {
+    if(len >= buffer.length) {
+      sum.update(buffer, 0, offset);
+      offset = 0;
+      sum.update(b, off, len);
+      return;
+    }
+    final int remaining = buffer.length - offset;
+    if(len > remaining) {
+      sum.update(buffer, 0, offset);
+      offset = 0;
+    }
+    /* now we should have len < buffer.length */
+    System.arraycopy(b, off, buffer, offset, len);
+    offset += len;
+  }
+  
+  /**
+   * Read bytes from the stream.
+   * At EOF, checksum is validated, but the checksum
+   * bytes are not passed back in the buffer. 
+   */
+  public int read(byte[] b, int off, int len) throws IOException {
+
+    if (currentOffset >= dataLength) {
+      return -1;
+    }
+
+    doReadahead();
+
+    return doRead(b,off,len);
+  }
+
+  private void doReadahead() {
+    if (raPool != null && inFd != null && readahead) {
+      curReadahead = raPool.readaheadStream(
+          "ifile", inFd,
+          currentOffset, readaheadLength, dataLength,
+          curReadahead);
+    }
+  }
+
+  /**
+   * Read bytes from the stream.
+   * At EOF, checksum is validated and sent back
+   * as the last four bytes of the buffer. The caller should handle
+   * these bytes appropriately
+   */
+  public int readWithChecksum(byte[] b, int off, int len) throws IOException {
+
+    if (currentOffset == length) {
+      return -1;
+    }
+    else if (currentOffset >= dataLength) {
+      // If the previous read drained off all the data, then just return
+      // the checksum now. Note that checksum validation would have 
+      // happened in the earlier read
+      int lenToCopy = (int) (checksumSize - (currentOffset - dataLength));
+      if (len < lenToCopy) {
+        lenToCopy = len;
+      }
+      System.arraycopy(csum, (int) (currentOffset - dataLength), b, off, 
+          lenToCopy);
+      currentOffset += lenToCopy;
+      return lenToCopy;
+    }
+
+    int bytesRead = doRead(b,off,len);
+
+    if (currentOffset == dataLength) {
+      if (len >= bytesRead + checksumSize) {
+        System.arraycopy(csum, 0, b, off + bytesRead, checksumSize);
+        bytesRead += checksumSize;
+        currentOffset += checksumSize;
+      }
+    }
+    return bytesRead;
+  }
+
+  private int doRead(byte[]b, int off, int len) throws IOException {
+    
+    // If we are trying to read past the end of data, just read
+    // the left over data
+    if (currentOffset + len > dataLength) {
+      len = (int) dataLength - (int)currentOffset;
+    }
+    
+    int bytesRead = in.read(b, off, len);
+
+    if (bytesRead < 0) {
+      throw new ChecksumException("Checksum Error", 0);
+    }
+
+    checksum(b, off, bytesRead);
+
+    currentOffset += bytesRead;
+
+    if (disableChecksumValidation) {
+      return bytesRead;
+    }
+    
+    if (currentOffset == dataLength) {
+      // The last four bytes are checksum. Strip them and verify
+      sum.update(buffer, 0, offset);
+      csum = new byte[checksumSize];
+      IOUtils.readFully(in, csum, 0, checksumSize);
+      if (!sum.compare(csum, 0)) {
+        throw new ChecksumException("Checksum Error", 0);
+      }
+    }
+    return bytesRead;
+  }
+
+
+  @Override
+  public int read() throws IOException {    
+    b[0] = 0;
+    int l = read(b,0,1);
+    if (l < 0)  return l;
+    
+    // Upgrade the b[0] to an int so as not to misinterpret the
+    // first bit of the byte as a sign bit
+    int result = 0xFF & b[0];
+    return result;
+  }
+
+  public byte[] getChecksum() {
+    return csum;
+  }
+
+  void disableChecksumValidation() {
+    disableChecksumValidation = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java
new file mode 100644
index 0000000..3198446
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/IFileOutputStream.java
@@ -0,0 +1,129 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common.sort.impl;
+
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.DataChecksum;
+/**
+ * A Checksum output stream.
+ * Checksum for the contents of the file is calculated and
+ * appended to the end of the file on close of the stream.
+ * Used for IFiles
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class IFileOutputStream extends FilterOutputStream {
+
+  /**
+   * The output stream to be checksummed.
+   */
+  private final DataChecksum sum;
+  private byte[] barray;
+  private byte[] buffer;
+  private int offset;
+  private boolean closed = false;
+  private boolean finished = false;
+
+  /**
+   * Create a checksum output stream that writes
+   * the bytes to the given stream.
+   * @param out
+   */
+  public IFileOutputStream(OutputStream out) {
+    super(out);
+    sum = DataChecksum.newDataChecksum(DataChecksum.Type.CRC32,
+        Integer.MAX_VALUE);
+    barray = new byte[sum.getChecksumSize()];
+    buffer = new byte[4096];
+    offset = 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (closed) {
+      return;
+    }
+    closed = true;
+    finish();
+    out.close();
+  }
+
+  /**
+   * Finishes writing data to the output stream, by writing
+   * the checksum bytes to the end. The underlying stream is not closed.
+   * @throws IOException
+   */
+  public void finish() throws IOException {
+    if (finished) {
+      return;
+    }
+    finished = true;
+    sum.update(buffer, 0, offset);
+    sum.writeValue(barray, 0, false);
+    out.write (barray, 0, sum.getChecksumSize());
+    out.flush();
+  }
+
+  private void checksum(byte[] b, int off, int len) {
+    if(len >= buffer.length) {
+      sum.update(buffer, 0, offset);
+      offset = 0;
+      sum.update(b, off, len);
+      return;
+    }
+    final int remaining = buffer.length - offset;
+    if(len > remaining) {
+      sum.update(buffer, 0, offset);
+      offset = 0;
+    }
+    /*
+    // FIXME if needed re-enable this in debug mode
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("XXX checksum" +
+          " b=" + b + " off=" + off +
+          " buffer=" + " offset=" + offset +
+          " len=" + len);
+    }
+    */
+    /* now we should have len < buffer.length */
+    System.arraycopy(b, off, buffer, offset, len);
+    offset += len;
+  }
+
+  /**
+   * Write bytes to the stream.
+   */
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    checksum(b, off, len);
+    out.write(b,off,len);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    barray[0] = (byte) (b & 0xFF);
+    write(barray,0,1);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
new file mode 100644
index 0000000..1b153ca
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/PipelinedSorter.java
@@ -0,0 +1,932 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.tez.runtime.library.common.sort.impl;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.IntBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+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.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.HashComparator;
+import org.apache.hadoop.io.RawComparator;
+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.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger.Segment;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class PipelinedSorter extends ExternalSorter {
+  
+  private static final Log LOG = LogFactory.getLog(PipelinedSorter.class);
+  
+  /**
+   * The size of each record in the index file for the map-outputs.
+   */
+  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
+
+  private final static int APPROX_HEADER_LENGTH = 150;
+    
+  int partitionBits;
+  
+  private static final int PARTITION = 0;        // partition offset in acct
+  private static final int KEYSTART = 1;         // key offset in acct
+  private static final int VALSTART = 2;         // val offset in acct
+  private static final int VALLEN = 3;           // val len in acct
+  private static final int NMETA = 4;            // num meta ints
+  private static final int METASIZE = NMETA * 4; // size in bytes
+
+  // spill accounting
+  volatile Throwable sortSpillException = null;
+
+  int numSpills = 0;
+  int minSpillsForCombine;
+  private HashComparator hasher;
+  // SortSpans  
+  private SortSpan span;
+  private ByteBuffer largeBuffer;
+  // Merger
+  private SpanMerger merger; 
+  private ExecutorService sortmaster;
+
+  final ArrayList<TezSpillRecord> indexCacheList =
+    new ArrayList<TezSpillRecord>();
+  private int totalIndexCacheMemory;
+  private int indexCacheMemoryLimit;
+
+  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
+    super.initialize(outputContext, conf, numOutputs);
+    
+    partitionBits = bitcount(partitions)+1;
+   
+    //sanity checks
+    final float spillper =
+      this.conf.getFloat(
+          TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT, 
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SORT_SPILL_PERCENT);
+    final int sortmb = 
+        this.conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_IO_SORT_MB, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_MB);
+    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES,
+                                       TezJobConfig.DEFAULT_TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES);
+    if (spillper > (float)1.0 || spillper <= (float)0.0) {
+      throw new IOException("Invalid \"" + TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT +
+          "\": " + spillper);
+    }
+    if ((sortmb & 0x7FF) != sortmb) {
+      throw new IOException(
+          "Invalid \"" + TezJobConfig.TEZ_RUNTIME_IO_SORT_MB + "\": " + sortmb);
+    }
+    
+    // buffers and accounting
+    int maxMemUsage = sortmb << 20;
+    maxMemUsage -= maxMemUsage % METASIZE;
+    largeBuffer = ByteBuffer.allocate(maxMemUsage);
+    LOG.info(TezJobConfig.TEZ_RUNTIME_IO_SORT_MB + " = " + sortmb);
+    // TODO: configurable setting?
+    span = new SortSpan(largeBuffer, 1024*1024, 16);
+    merger = new SpanMerger(comparator);
+    final int sortThreads = 
+            this.conf.getInt(
+                TezJobConfig.TEZ_RUNTIME_SORT_THREADS, 
+                TezJobConfig.DEFAULT_TEZ_RUNTIME_SORT_THREADS);
+    sortmaster = Executors.newFixedThreadPool(sortThreads);
+
+    // k/v serialization    
+    if(comparator instanceof HashComparator) {
+      hasher = (HashComparator)comparator;
+      LOG.info("Using the HashComparator");
+    } else {
+      hasher = null;
+    }    
+    valSerializer.open(span.out);
+    keySerializer.open(span.out);
+    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_COMBINE_MIN_SPILLS, 3);
+  }
+
+  private int bitcount(int n) {
+    int bit = 0;
+    while(n!=0) {
+      bit++;
+      n >>= 1;
+    }
+    return bit;
+  }
+  
+  public void sort() throws IOException {
+    SortSpan newSpan = span.next();
+
+    if(newSpan == null) {
+      // sort in the same thread, do not wait for the thread pool
+      merger.add(span.sort(sorter, comparator));
+      spill();
+      int items = 1024*1024;
+      int perItem = 16;
+      if(span.length() != 0) {
+        items = span.length();
+        perItem = span.kvbuffer.limit()/items;
+        items = (largeBuffer.capacity())/(METASIZE+perItem);
+        if(items > 1024*1024) {
+            // our goal is to have 1M splits and sort early
+            items = 1024*1024;
+        }
+      }      
+      span = new SortSpan(largeBuffer, items, perItem);
+    } else {
+      // queue up the sort
+      SortTask task = new SortTask(span, sorter, comparator);
+      Future<SpanIterator> future = sortmaster.submit(task);
+      merger.add(future);
+      span = newSpan;
+    }
+    valSerializer.open(span.out);
+    keySerializer.open(span.out);
+  }
+
+  @Override
+  public void write(Object key, Object value) 
+      throws IOException {
+    collect(
+        key, value, partitioner.getPartition(key, value, partitions));
+  }
+
+  /**
+   * Serialize the key, value to intermediate storage.
+   * When this method returns, kvindex must refer to sufficient unused
+   * storage to store one METADATA.
+   */
+  synchronized void collect(Object key, Object value, final int partition
+                                   ) throws IOException {
+    if (key.getClass() != keyClass) {
+      throw new IOException("Type mismatch in key from map: expected "
+                            + keyClass.getName() + ", received "
+                            + key.getClass().getName());
+    }
+    if (value.getClass() != valClass) {
+      throw new IOException("Type mismatch in value from map: expected "
+                            + valClass.getName() + ", received "
+                            + value.getClass().getName());
+    }
+    if (partition < 0 || partition >= partitions) {
+      throw new IOException("Illegal partition for " + key + " (" +
+          partition + ")");
+    }
+    if(span.kvmeta.remaining() < METASIZE) {
+      this.sort();
+    }
+    int keystart = span.kvbuffer.position();
+    int valstart = -1;
+    int valend = -1;
+    try { 
+      keySerializer.serialize(key);
+      valstart = span.kvbuffer.position();      
+      valSerializer.serialize(value);
+      valend = span.kvbuffer.position();
+    } catch(BufferOverflowException overflow) {
+      // restore limit
+      span.kvbuffer.position(keystart);
+      this.sort();
+      // try again
+      this.collect(key, value, partition);
+      return;
+    }
+
+    int prefix = 0;
+
+    if(hasher != null) {
+      prefix = hasher.getHashCode(key);
+    }
+
+    prefix = (partition << (32 - partitionBits)) | (prefix >>> partitionBits);
+
+    /* maintain order as in PARTITION, KEYSTART, VALSTART, VALLEN */
+    span.kvmeta.put(prefix);
+    span.kvmeta.put(keystart);
+    span.kvmeta.put(valstart);
+    span.kvmeta.put(valend - valstart);
+    if((valstart - keystart) > span.keymax) {
+      span.keymax = (valstart - keystart);
+    }
+    if((valend - valstart) > span.valmax) {
+      span.valmax = (valend - valstart);
+    }
+    mapOutputRecordCounter.increment(1);
+    mapOutputByteCounter.increment(valend - keystart);
+  }
+
+  public void spill() throws IOException { 
+    // create spill file
+    final long size = largeBuffer.capacity() + 
+      (partitions * APPROX_HEADER_LENGTH);
+    final TezSpillRecord spillRec = new TezSpillRecord(partitions);
+    final Path filename =
+      mapOutputFile.getSpillFileForWrite(numSpills, size);    
+    FSDataOutputStream out = rfs.create(filename, true, 4096);
+
+    try {
+      merger.ready(); // wait for all the future results from sort threads
+      LOG.info("Spilling to " + filename.toString());
+      for (int i = 0; i < partitions; ++i) {
+        TezRawKeyValueIterator kvIter = merger.filter(i);
+        //write merged output to disk
+        long segmentStart = out.getPos();
+        Writer writer =
+          new Writer(conf, out, keyClass, valClass, codec,
+              spilledRecordsCounter);
+        writer.setRLE(merger.needsRLE());
+        if (combiner == null) {
+          while(kvIter.next()) {
+            writer.append(kvIter.getKey(), kvIter.getValue());
+          }
+        } else {          
+          runCombineProcessor(kvIter, writer);
+        }
+        //close
+        writer.close();
+
+        // record offsets
+        final TezIndexRecord rec = 
+            new TezIndexRecord(
+                segmentStart, 
+                writer.getRawLength(), 
+                writer.getCompressedLength());
+        spillRec.putIndex(rec, i);
+      }
+
+      Path indexFilename =
+        mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
+            * MAP_OUTPUT_INDEX_RECORD_LENGTH);
+      // TODO: cache
+      spillRec.writeToFile(indexFilename, conf);
+      ++numSpills;
+    } catch(InterruptedException ie) {
+      // TODO:the combiner has been interrupted
+    } finally {
+      out.close();
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    final String uniqueIdentifier = outputContext.getUniqueIdentifier();
+    Path finalOutputFile =
+        mapOutputFile.getOutputFileForWrite(0); //TODO
+    Path finalIndexFile =
+        mapOutputFile.getOutputIndexFileForWrite(0); //TODO
+
+    LOG.info("Starting flush of map output");
+    span.end();
+    merger.add(span.sort(sorter, comparator));
+    spill();
+    sortmaster.shutdown();
+
+    largeBuffer = null;
+
+    if(numSpills == 1) {
+      // someday be able to pass this directly to shuffle
+      // without writing to disk
+      final Path filename =
+          mapOutputFile.getSpillFile(0);
+      Path indexFilename =
+              mapOutputFile.getSpillIndexFile(0);
+      sameVolRename(filename, finalOutputFile);
+      sameVolRename(indexFilename, finalIndexFile);
+      return;
+    }
+    
+    //The output stream for the final single output file
+    FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
+
+    TezMerger.considerFinalMergeForProgress();
+
+    final TezSpillRecord spillRec = new TezSpillRecord(partitions);
+    final ArrayList<TezSpillRecord> indexCacheList = new ArrayList<TezSpillRecord>();
+
+    for(int i = 0; i < numSpills; i++) {
+      // TODO: build this cache before
+      Path indexFilename = mapOutputFile.getSpillIndexFile(i);
+      TezSpillRecord spillIndex = new TezSpillRecord(indexFilename, conf);
+      indexCacheList.add(spillIndex);
+    }
+    
+    for (int parts = 0; parts < partitions; parts++) {
+      //create the segments to be merged
+      List<Segment> segmentList =
+          new ArrayList<Segment>(numSpills);
+      for(int i = 0; i < numSpills; i++) {
+        Path spillFilename = mapOutputFile.getSpillFile(i);
+        TezIndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
+
+        Segment s =
+            new Segment(conf, rfs, spillFilename, indexRecord.getStartOffset(),
+                             indexRecord.getPartLength(), codec, true);
+        segmentList.add(i, s);
+      }
+
+      int mergeFactor = 
+              this.conf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 
+                  TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
+      // sort the segments only if there are intermediate merges
+      boolean sortSegments = segmentList.size() > mergeFactor;
+      //merge
+      TezRawKeyValueIterator kvIter = TezMerger.merge(conf, rfs,
+                     keyClass, valClass, codec,
+                     segmentList, mergeFactor,
+                     new Path(uniqueIdentifier),
+                     (RawComparator)ConfigUtils.getIntermediateOutputKeyComparator(conf), 
+                     nullProgressable, sortSegments,
+                     null, spilledRecordsCounter,
+                     null); // Not using any Progress in TezMerger. Should just work.
+
+      //write merged output to disk
+      long segmentStart = finalOut.getPos();
+      Writer writer =
+          new Writer(conf, finalOut, keyClass, valClass, codec,
+                           spilledRecordsCounter);
+      writer.setRLE(merger.needsRLE());
+      if (combiner == null || numSpills < minSpillsForCombine) {
+        TezMerger.writeFile(kvIter, writer, nullProgressable, conf);
+      } else {
+        runCombineProcessor(kvIter, writer);
+      }
+
+      //close
+      writer.close();
+
+      // record offsets
+      final TezIndexRecord rec = 
+          new TezIndexRecord(
+              segmentStart, 
+              writer.getRawLength(), 
+              writer.getCompressedLength());
+      spillRec.putIndex(rec, parts);
+    }
+
+    spillRec.writeToFile(finalIndexFile, conf);
+    finalOut.close();
+    for(int i = 0; i < numSpills; i++) {
+      Path indexFilename = mapOutputFile.getSpillIndexFile(i);
+      Path spillFilename = mapOutputFile.getSpillFile(i);
+      rfs.delete(indexFilename,true);
+      rfs.delete(spillFilename,true);
+    }
+  }
+
+  public void close() { }
+
+  private interface PartitionedRawKeyValueIterator extends TezRawKeyValueIterator {
+    int getPartition();
+  }
+
+  private class BufferStreamWrapper extends OutputStream 
+  {
+    private final ByteBuffer out;
+    public BufferStreamWrapper(ByteBuffer out) {
+      this.out = out;
+    }
+    
+    @Override
+    public void write(int b) throws IOException { out.put((byte)b); }
+    @Override
+    public void write(byte[] b) throws IOException { out.put(b); }
+    @Override
+    public void write(byte[] b, int off, int len) throws IOException { out.put(b, off, len); }
+  }
+
+  protected class InputByteBuffer extends DataInputBuffer {
+    private byte[] buffer = new byte[256]; 
+    private ByteBuffer wrapped = ByteBuffer.wrap(buffer);
+    private void resize(int length) {
+      if(length > buffer.length) {
+        buffer = new byte[length];
+        wrapped = ByteBuffer.wrap(buffer);
+      }
+      wrapped.limit(length);
+    }
+    public void reset(ByteBuffer b, int start, int length) {
+      resize(length);
+      b.position(start);
+      b.get(buffer, 0, length);
+      super.reset(buffer, 0, length);
+    }
+    // clone-ish function
+    public void reset(DataInputBuffer clone) {
+      byte[] data = clone.getData();
+      int start = clone.getPosition();
+      int length = clone.getLength();
+      resize(length);
+      System.arraycopy(data, start, buffer, 0, length);
+      super.reset(buffer, 0, length);
+    }
+  }
+
+  private class SortSpan  implements IndexedSortable {
+    final IntBuffer kvmeta;
+    final ByteBuffer kvbuffer;
+    final DataOutputStream out;    
+    private RawComparator comparator; 
+    final int imeta[] = new int[NMETA];
+    final int jmeta[] = new int[NMETA];
+    int keymax = 1;
+    int valmax = 1;
+    private int i,j;
+    private byte[] ki;
+    private byte[] kj;
+    private int index = 0;
+    private InputByteBuffer hay = new InputByteBuffer();
+    private long eq = 0;
+
+    public SortSpan(ByteBuffer source, int maxItems, int perItem) {
+      int capacity = source.remaining(); 
+      int metasize = METASIZE*maxItems;
+      int dataSize = maxItems * perItem;
+      if(capacity < (metasize+dataSize)) {
+        // try to allocate less meta space, because we have sample data
+        metasize = METASIZE*(capacity/(perItem+METASIZE));
+      }
+      ByteBuffer reserved = source.duplicate();
+      reserved.mark();
+      LOG.info("reserved.remaining() = "+reserved.remaining());
+      LOG.info("reserved.size = "+metasize);
+      reserved.position(metasize);
+      kvbuffer = reserved.slice();
+      reserved.flip();
+      reserved.limit(metasize);
+      kvmeta = reserved
+                .slice()
+                .order(ByteOrder.nativeOrder())
+               .asIntBuffer();
+      out = new DataOutputStream(
+              new BufferStreamWrapper(kvbuffer));
+    }
+
+    public SpanIterator sort(IndexedSorter sorter, RawComparator comparator) {
+    	this.comparator = comparator;
+      ki = new byte[keymax];
+      kj = new byte[keymax];
+      LOG.info("begin sorting Span"+index + " ("+length()+")");
+      if(length() > 1) {
+        sorter.sort(this, 0, length(), nullProgressable);
+      }
+      LOG.info("done sorting Span"+index);
+      return new SpanIterator(this);
+    }
+
+    int offsetFor(int i) {
+      return (i * NMETA);
+    }
+
+    public void swap(final int mi, final int mj) {
+      final int kvi = offsetFor(mi);
+      final int kvj = offsetFor(mj);
+
+      kvmeta.position(kvi); kvmeta.get(imeta);
+      kvmeta.position(kvj); kvmeta.get(jmeta);
+      kvmeta.position(kvj); kvmeta.put(imeta);
+      kvmeta.position(kvi); kvmeta.put(jmeta);
+
+      if(i == mi || j == mj) i = -1;
+      if(i == mi || j == mj) j = -1;
+    }
+
+    public int compare(final int mi, final int mj) {
+      final int kvi = offsetFor(mi);
+      final int kvj = offsetFor(mj);
+      final int kvip = kvmeta.get(kvi + PARTITION);
+      final int kvjp = kvmeta.get(kvj + PARTITION);
+      // sort by partition      
+      if (kvip != kvjp) {
+        return kvip - kvjp;
+      }
+      
+      final int istart = kvmeta.get(kvi + KEYSTART);
+      final int jstart = kvmeta.get(kvj + KEYSTART);
+      final int ilen   = kvmeta.get(kvi + VALSTART) - istart;
+      final int jlen   = kvmeta.get(kvj + VALSTART) - jstart;
+
+      kvbuffer.position(istart);
+      kvbuffer.get(ki, 0, ilen);
+      kvbuffer.position(jstart);
+      kvbuffer.get(kj, 0, jlen);
+      // sort by key
+      final int cmp = comparator.compare(ki, 0, ilen, kj, 0, jlen);
+      if(cmp == 0) eq++;
+      return cmp;
+    }
+
+    public SortSpan next() {
+      ByteBuffer remaining = end();
+      if(remaining != null) {
+        int items = length();
+        int perItem = kvbuffer.position()/items;
+        SortSpan newSpan = new SortSpan(remaining, items, perItem);
+        newSpan.index = index+1;
+        return newSpan;
+      }
+      return null;
+    }
+
+    public int length() {
+      return kvmeta.limit()/NMETA;
+    }
+
+    public ByteBuffer end() {
+      ByteBuffer remaining = kvbuffer.duplicate();
+      remaining.position(kvbuffer.position());
+      remaining = remaining.slice();
+      kvbuffer.limit(kvbuffer.position());
+      kvmeta.limit(kvmeta.position());
+      int items = length();
+      if(items == 0) {
+        return null;
+      }
+      int perItem = kvbuffer.position()/items;
+      LOG.info(String.format("Span%d.length = %d, perItem = %d", index, length(), perItem));
+      if(remaining.remaining() < NMETA+perItem) {
+        return null;
+      }
+      return remaining;
+    }
+
+    private int compareInternal(DataInputBuffer needle, int needlePart, int index) {
+      int cmp = 0;
+      int keystart;
+      int valstart;
+      int partition;
+      partition = kvmeta.get(span.offsetFor(index) + PARTITION);
+      if(partition != needlePart) {
+          cmp = (partition-needlePart);
+      } else {
+        keystart = kvmeta.get(span.offsetFor(index) + KEYSTART);
+        valstart = kvmeta.get(span.offsetFor(index) + VALSTART);
+        // hay is allocated ahead of time
+        hay.reset(kvbuffer, keystart, valstart - keystart);
+        cmp = comparator.compare(hay.getData(), 
+            hay.getPosition(), hay.getLength(),
+            needle.getData(), 
+            needle.getPosition(), needle.getLength());
+      }
+      return cmp;
+    }
+    
+    public long getEq() {
+      return eq;
+    }
+    
+    @Override
+    public String toString() {
+        return String.format("Span[%d,%d]", NMETA*kvmeta.capacity(), kvbuffer.limit());
+    }
+  }
+
+  private class SpanIterator implements PartitionedRawKeyValueIterator, Comparable<SpanIterator> {
+    private int kvindex = -1;
+    private int maxindex;
+    private IntBuffer kvmeta;
+    private ByteBuffer kvbuffer;
+    private SortSpan span;
+    private InputByteBuffer key = new InputByteBuffer();
+    private InputByteBuffer value = new InputByteBuffer();
+    private Progress progress = new Progress();
+
+    private final int minrun = (1 << 4);
+
+    public SpanIterator(SortSpan span) {
+      this.kvmeta = span.kvmeta;
+      this.kvbuffer = span.kvbuffer;
+      this.span = span;
+      this.maxindex = (kvmeta.limit()/NMETA) - 1;
+    }
+
+    public DataInputBuffer getKey() throws IOException {
+      final int keystart = kvmeta.get(span.offsetFor(kvindex) + KEYSTART);
+      final int valstart = kvmeta.get(span.offsetFor(kvindex) + VALSTART);
+      key.reset(kvbuffer, keystart, valstart - keystart);
+      return key;
+    }
+
+    public DataInputBuffer getValue() throws IOException {
+      final int valstart = kvmeta.get(span.offsetFor(kvindex) + VALSTART);
+      final int vallen = kvmeta.get(span.offsetFor(kvindex) + VALLEN);
+      value.reset(kvbuffer, valstart, vallen);
+      return value;
+    }
+
+    public boolean next() throws IOException {
+      // caveat: since we use this as a comparable in the merger 
+      if(kvindex == maxindex) return false;
+      if(kvindex % 100 == 0) {
+          progress.set((kvindex-maxindex) / maxindex);
+      }
+      kvindex += 1;
+      return true;
+    }
+
+    public void close() throws IOException {
+    }
+
+    public Progress getProgress() { 
+      return progress;
+    }
+
+    public int getPartition() {
+      final int partition = kvmeta.get(span.offsetFor(kvindex) + PARTITION);
+      return partition;
+    }
+
+    public int size() {
+      return (maxindex - kvindex);
+    }
+
+    public int compareTo(SpanIterator other) {
+      try {
+        return span.compareInternal(other.getKey(), other.getPartition(), kvindex);
+      } catch(IOException ie) {
+        // since we're not reading off disk, how could getKey() throw exceptions?
+      }
+      return -1;
+    }
+    
+    @Override
+    public String toString() {
+        return String.format("SpanIterator<%d:%d> (span=%s)", kvindex, maxindex, span.toString());
+    }
+
+    /**
+     * bisect returns the next insertion point for a given raw key, skipping keys
+     * which are <= needle using a binary search instead of a linear comparison.
+     * This is massively efficient when long strings of identical keys occur.
+     * @param needle 
+     * @param needlePart
+     * @return
+     */
+    int bisect(DataInputBuffer needle, int needlePart) {
+      int start = kvindex;
+      int end = maxindex-1;
+      int mid = start;
+      int cmp = 0;
+
+      if(end - start < minrun) {
+        return 0;
+      }
+
+      if(span.compareInternal(needle, needlePart, start) > 0) {
+        return kvindex;
+      }
+      
+      // bail out early if we haven't got a min run 
+      if(span.compareInternal(needle, needlePart, start+minrun) > 0) {
+        return 0;
+      }
+
+      if(span.compareInternal(needle, needlePart, end) < 0) {
+        return end - kvindex;
+      }
+      
+      boolean found = false;
+      
+      // we sort 100k items, the max it can do is 20 loops, but break early
+      for(int i = 0; start < end && i < 16; i++) {
+        mid = start + (end - start)/2;
+        cmp = span.compareInternal(needle, needlePart, mid);
+        if(cmp == 0) {
+          start = mid;
+          found = true;
+        } else if(cmp < 0) {
+          start = mid; 
+          found = true;
+        }
+        if(cmp > 0) {
+          end = mid;
+        }
+      }
+
+      if(found) {
+        return start - kvindex;
+      }
+      return 0;
+    }
+  }
+
+  private class SortTask implements Callable<SpanIterator> {
+    private final SortSpan sortable;
+    private final IndexedSorter sorter;
+    private final RawComparator comparator;
+    
+    public SortTask(SortSpan sortable, 
+              IndexedSorter sorter, RawComparator comparator) {
+        this.sortable = sortable;
+        this.sorter = sorter;
+        this.comparator = comparator;
+    }
+
+    public SpanIterator call() {
+      return sortable.sort(sorter, comparator);
+    }
+  }
+
+  private class PartitionFilter implements TezRawKeyValueIterator {
+    private final PartitionedRawKeyValueIterator iter;
+    private int partition;
+    private boolean dirty = false;
+    public PartitionFilter(PartitionedRawKeyValueIterator iter) {
+      this.iter = iter;
+    }
+    public DataInputBuffer getKey() throws IOException { return iter.getKey(); }
+    public DataInputBuffer getValue() throws IOException { return iter.getValue(); }
+    public void close() throws IOException { }
+    public Progress getProgress() {
+      return new Progress();
+    }
+    public boolean next() throws IOException {
+      if(dirty || iter.next()) { 
+        int prefix = iter.getPartition();
+
+        if((prefix >>> (32 - partitionBits)) == partition) {
+          dirty = false; // we found what we were looking for, good
+          return true;
+        } else if(!dirty) {
+          dirty = true; // we did a lookahead and failed to find partition
+        }
+      }
+      return false;
+    }
+
+    public void reset(int partition) {
+      this.partition = partition;
+    }
+
+    public int getPartition() {
+      return this.partition;
+    }
+  }
+
+  private class SpanHeap extends java.util.PriorityQueue<SpanIterator> {
+    public SpanHeap() {
+      super(256);
+    }
+    /**
+     * {@link PriorityQueue}.poll() by a different name 
+     * @return
+     */
+    public SpanIterator pop() {
+      return this.poll();
+    }
+  }
+
+  private class SpanMerger implements PartitionedRawKeyValueIterator {
+    private final RawComparator comparator;
+    InputByteBuffer key = new InputByteBuffer();
+    InputByteBuffer value = new InputByteBuffer();
+    int partition;
+
+    private ArrayList< Future<SpanIterator>> futures = new ArrayList< Future<SpanIterator>>();
+
+    private SpanHeap heap = new SpanHeap();
+    private PartitionFilter partIter;
+
+    private int gallop = 0;
+    private SpanIterator horse;
+    private long total = 0;
+    private long count = 0;
+    private long eq = 0;
+    
+    public SpanMerger(RawComparator comparator) {
+      this.comparator = comparator;
+      partIter = new PartitionFilter(this);
+    }
+
+    public void add(SpanIterator iter) throws IOException{
+      if(iter.next()) {
+        heap.add(iter);
+      }
+    }
+
+    public void add(Future<SpanIterator> iter) throws IOException{
+      this.futures.add(iter);
+    }
+
+    public boolean ready() throws IOException, InterruptedException {
+      try {
+        SpanIterator iter = null;
+        while(this.futures.size() > 0) {
+          Future<SpanIterator> futureIter = this.futures.remove(0);
+          iter = futureIter.get();
+          this.add(iter);
+        }
+        
+        StringBuilder sb = new StringBuilder();
+        for(SpanIterator sp: heap) {
+            sb.append(sp.toString());
+            sb.append(",");
+            total += sp.span.length();
+            eq += sp.span.getEq();
+        }
+        LOG.info("Heap = " + sb.toString());
+        return true;
+      } catch(Exception e) {
+        LOG.info(e.toString());
+        return false;
+      }
+    }
+
+    private SpanIterator pop() throws IOException {
+      if(gallop > 0) {
+        gallop--;
+        return horse;
+      }
+      SpanIterator current = heap.pop();
+      SpanIterator next = heap.peek();
+      if(next != null && current != null &&
+        ((Object)horse) == ((Object)current)) {
+        // TODO: a better threshold check
+        gallop = current.bisect(next.getKey(), next.getPartition())-1;
+      }
+      horse = current;
+      return current;
+    }
+    
+    public boolean needsRLE() {
+      return (eq > 0.1 * total);
+    }
+    
+    private SpanIterator peek() throws IOException {
+    	if(gallop > 0) {
+            return horse;
+        }
+    	return heap.peek();
+    }
+
+    public boolean next() throws IOException {
+      SpanIterator current = pop();
+
+      if(current != null) {
+        // keep local copies, since add() will move it all out
+        key.reset(current.getKey());
+        value.reset(current.getValue());
+        partition = current.getPartition();
+        if(gallop <= 0) {
+          this.add(current);
+        } else {
+          // galloping
+          current.next();
+        }
+        return true;
+      }
+      return false;
+    }
+
+    public DataInputBuffer getKey() throws IOException { return key; }
+    public DataInputBuffer getValue() throws IOException { return value; }
+    public int getPartition() { return partition; }
+
+    public void close() throws IOException {
+    }
+
+    public Progress getProgress() {
+      // TODO
+      return new Progress();
+    }
+
+    public TezRawKeyValueIterator filter(int partition) {
+      partIter.reset(partition);
+      return partIter;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezIndexRecord.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezIndexRecord.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezIndexRecord.java
new file mode 100644
index 0000000..95ae8eb
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezIndexRecord.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.runtime.library.common.sort.impl;
+
+public class TezIndexRecord {
+  private long startOffset;
+  private long rawLength;
+  private long partLength;
+
+  public TezIndexRecord() { }
+
+  public TezIndexRecord(long startOffset, long rawLength, long partLength) {
+    this.startOffset = startOffset;
+    this.rawLength = rawLength;
+    this.partLength = partLength;
+  }
+
+  public long getStartOffset() {
+    return startOffset;
+  }
+
+  public long getRawLength() {
+    return rawLength;
+  }
+
+  public long getPartLength() {
+    return partLength;
+  }
+}


[11/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index a984b0f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInput.java
+++ /dev/null
@@ -1,179 +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.input;
-
-import java.io.IOException;
-import java.util.Collections;
-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.io.RawComparator;
-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;
-
-/**
- * <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.
- *
- */
-public class ShuffledMergedInput implements LogicalInput {
-
-  static final Log LOG = LogFactory.getLog(ShuffledMergedInput.class);
-
-  protected TezInputContext inputContext;
-  protected TezRawKeyValueIterator rawIter = null;
-  protected Configuration conf;
-  protected int numInputs = 0;
-  protected Shuffle shuffle;
-  @SuppressWarnings("rawtypes")
-  protected ValuesIterator vIter;
-
-  private TezCounter inputKeyCounter;
-  private TezCounter inputValueCounter;
-
-  @Override
-  public List<Event> initialize(TezInputContext inputContext) throws IOException {
-    this.inputContext = inputContext;
-    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
-
-    this.inputKeyCounter = inputContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_GROUPS);
-    this.inputValueCounter = inputContext.getCounters().findCounter(TaskCounter.REDUCE_INPUT_RECORDS);
-    this.conf.setStrings(TezJobConfig.LOCAL_DIRS,
-        inputContext.getWorkDirs());
-
-    // Start the shuffle - copy and merge.
-    shuffle = new Shuffle(inputContext, this.conf, numInputs);
-    shuffle.run();
-
-    return Collections.emptyList();
-  }
-
-  /**
-   * Check if the input is ready for consumption
-   *
-   * @return true if the input is ready for consumption, or if an error occurred
-   *         processing fetching the input. false if the shuffle and merge are
-   *         still in progress
-   */
-  public boolean isInputReady() {
-    return shuffle.isInputReady();
-  }
-
-  /**
-   * Waits for the input to become ready for consumption
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public void waitForInputReady() throws IOException, InterruptedException {
-    rawIter = shuffle.waitForInput();
-    createValuesIterator();
-  }
-
-  @Override
-  public List<Event> close() throws IOException {
-    rawIter.close();
-    return Collections.emptyList();
-  }
-
-  /**
-   * Get a KVReader for the Input.</p> This method will block until the input is
-   * ready - i.e. the copy and merge stages are complete. Users can use the
-   * isInputReady method to check if the input is ready, which gives an
-   * indication of whether this method will block or not.
-   *
-   * NOTE: All values for the current K-V pair must be read prior to invoking
-   * moveToNext. Once moveToNext() is called, the valueIterator from the
-   * previous K-V pair will throw an Exception
-   *
-   * @return a KVReader over the sorted input.
-   */
-  @Override
-  public KVReader getReader() throws IOException {
-    if (rawIter == null) {
-      try {
-        waitForInputReady();
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        throw new IOException("Interrupted while waiting for input ready", e);
-      }
-    }
-    return new KVReader() {
-
-      @Override
-      public boolean next() throws IOException {
-        return vIter.moveToNext();
-      }
-
-      @SuppressWarnings("unchecked")
-      @Override
-      public KVRecord getCurrentKV() {
-        return new KVRecord(vIter.getKey(), vIter.getValues());
-      }
-    };
-  }
-
-  @Override
-  public void handleEvents(List<Event> inputEvents) {
-    shuffle.handleEvents(inputEvents);
-  }
-
-  @Override
-  public void setNumPhysicalInputs(int numInputs) {
-    this.numInputs = numInputs;
-  }
-
-  @SuppressWarnings({ "rawtypes", "unchecked" })
-  protected void createValuesIterator()
-      throws IOException {
-    vIter = new ValuesIterator(rawIter,
-        (RawComparator) ConfigUtils.getIntermediateInputKeyComparator(conf),
-        ConfigUtils.getIntermediateInputKeyClass(conf),
-        ConfigUtils.getIntermediateInputValueClass(conf), conf, inputKeyCounter, inputValueCounter);
-
-  }
-
-  // This functionality is currently broken. If there's inputs which need to be
-  // written to disk, there's a possibility that inputs from the different
-  // sources could clobber each others' output. Also the current structures do
-  // not have adequate information to de-dupe these (vertex name)
-//  public void mergeWith(ShuffledMergedInput other) {
-//    this.numInputs += other.getNumPhysicalInputs();
-//  }
-//
-//  public int getNumPhysicalInputs() {
-//    return this.numInputs;
-//  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java b/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java
deleted file mode 100644
index f2da031..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledMergedInputLegacy.java
+++ /dev/null
@@ -1,30 +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.engine.lib.input;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-@LimitedPrivate("mapreduce")
-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/b212ca1d/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
deleted file mode 100644
index 44238fd..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/ShuffledUnorderedKVInput.java
+++ /dev/null
@@ -1,76 +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.input;
-
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.Reader;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.broadcast.input.BroadcastShuffleManager;
-
-import com.google.common.base.Preconditions;
-
-public class ShuffledUnorderedKVInput implements LogicalInput {
-
-  private Configuration conf;
-  private int numInputs = -1;
-  private BroadcastShuffleManager shuffleManager;
-  
-  
-  
-  public ShuffledUnorderedKVInput() {
-  }
-
-  @Override
-  public List<Event> initialize(TezInputContext inputContext) throws Exception {
-    Preconditions.checkArgument(numInputs != -1, "Number of Inputs has not been set");
-    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
-    this.conf.setStrings(TezJobConfig.LOCAL_DIRS, inputContext.getWorkDirs());
-    
-    this.shuffleManager = new BroadcastShuffleManager(inputContext, conf, numInputs);
-    return null;
-  }
-
-  @Override
-  public Reader getReader() throws Exception {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override
-  public void handleEvents(List<Event> inputEvents) {
-    shuffleManager.handleEvents(inputEvents);
-  }
-
-  @Override
-  public List<Event> close() throws Exception {
-    this.shuffleManager.shutdown();
-    return null;
-  }
-
-  @Override
-  public void setNumPhysicalInputs(int numInputs) {
-    this.numInputs = numInputs;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 26a01c8..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/InMemorySortedOutput.java
+++ /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.
- */
-package org.apache.tez.engine.lib.output;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.KVWriter;
-import org.apache.tez.engine.api.LogicalOutput;
-import org.apache.tez.engine.api.Output;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.Writer;
-import org.apache.tez.engine.common.sort.impl.dflt.InMemoryShuffleSorter;
-
-/**
- * {@link InMemorySortedOutput} is an {@link Output} which sorts key/value pairs 
- * written to it and persists it to a file.
- */
-public class InMemorySortedOutput implements LogicalOutput {
-  
-  protected InMemoryShuffleSorter sorter;
-  protected int numTasks;
-  protected TezOutputContext outputContext;
-  
-
-  @Override
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws IOException {
-    this.outputContext = outputContext;
-    this.sorter = new InMemoryShuffleSorter();
-    sorter.initialize(outputContext, TezUtils.createConfFromUserPayload(outputContext.getUserPayload()), numTasks);
-    return Collections.emptyList();
-  }
-
-  @Override
-  public Writer getWriter() throws IOException {
-    return new KVWriter() {
-      
-      @Override
-      public void write(Object key, Object value) throws IOException {
-        sorter.write(key, value);
-      }
-    };
-  }
-
-  @Override
-  public void handleEvents(List<Event> outputEvents) {
-    // No events expected.
-  }
-
-  @Override
-  public void setNumPhysicalOutputs(int numOutputs) {
-    this.numTasks = numOutputs;
-  }
-  
-  @Override
-  public List<Event> close() throws IOException {
-    sorter.flush();
-    sorter.close();
-    // TODO NEWTEZ Event generation
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 7fd26d7..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/LocalOnFileSorterOutput.java
+++ /dev/null
@@ -1,63 +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.output;
-
-import java.io.IOException;
-import java.util.List;
-
-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;
-
-public class LocalOnFileSorterOutput extends OnFileSortedOutput {
-
-  private static final Log LOG = LogFactory.getLog(LocalOnFileSorterOutput.class);
-
-  
-
-  @Override
-  public List<Event> close() throws IOException {
-    LOG.debug("Closing LocalOnFileSorterOutput");
-    super.close();
-
-    TezTaskOutput mapOutputFile = sorter.getMapOutput();
-    FileSystem localFs = FileSystem.getLocal(conf);
-
-    Path src = mapOutputFile.getOutputFile();
-    Path dst =
-        mapOutputFile.getInputFileForWrite(
-            outputContext.getTaskIndex(),
-            localFs.getFileStatus(src).getLen());
-
-    LOG.info("Renaming src = " + src + ", dst = " + dst);
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Renaming src = " + src + ", dst = " + dst);
-    }
-    localFs.rename(src, dst);
-    return null;
-  }
-  
-  @Override
-  protected List<Event> generateDataMovementEventsOnClose() throws IOException {
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 9c9eba0..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileSortedOutput.java
+++ /dev/null
@@ -1,123 +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.output;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.List;
-
-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.shuffle.common.ShuffleUtils;
-
-import com.google.common.collect.Lists;
-
-/**
- * <code>OnFileSortedOutput</code> is an {@link LogicalOutput} which sorts key/value pairs 
- * written to it and persists it to a file.
- */
-public class OnFileSortedOutput implements LogicalOutput {
-  
-  protected ExternalSorter sorter;
-  protected Configuration conf;
-  protected int numOutputs;
-  protected TezOutputContext outputContext;
-  private long startTime;
-  private long endTime;
-  
-  
-  @Override
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws IOException {
-    this.startTime = System.nanoTime();
-    this.outputContext = outputContext;
-    sorter = new DefaultSorter();
-    this.conf = TezUtils.createConfFromUserPayload(outputContext.getUserPayload());
-    // Initializing this parametr in this conf since it is used in multiple
-    // places (wherever LocalDirAllocator is used) - TezTaskOutputFiles,
-    // TezMerger, etc.
-    this.conf.setStrings(TezJobConfig.LOCAL_DIRS, outputContext.getWorkDirs());
-    sorter.initialize(outputContext, conf, numOutputs);
-    return Collections.emptyList();
-  }
-
-  @Override
-  public KVWriter getWriter() throws IOException {
-    return new KVWriter() {
-      @Override
-      public void write(Object key, Object value) throws IOException {
-        sorter.write(key, value);
-      }
-    };
-  }
-
-  @Override
-  public void handleEvents(List<Event> outputEvents) {
-    // Not expecting any events.
-  }
-
-  @Override
-  public void setNumPhysicalOutputs(int numOutputs) {
-    this.numOutputs = numOutputs;
-  }
-
-  @Override
-  public List<Event> close() throws IOException {
-    sorter.flush();
-    sorter.close();
-    this.endTime = System.nanoTime();
-
-   return generateDataMovementEventsOnClose();
-  }
-  
-  protected List<Event> generateDataMovementEventsOnClose() throws IOException {
-    String host = System.getenv(ApplicationConstants.Environment.NM_HOST
-        .toString());
-    ByteBuffer shuffleMetadata = outputContext
-        .getServiceProviderMetaData(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID);
-    int shufflePort = ShuffleUtils.deserializeShuffleProviderMetaData(shuffleMetadata);
-
-    DataMovementEventPayloadProto.Builder payloadBuilder = DataMovementEventPayloadProto
-        .newBuilder();
-    payloadBuilder.setHost(host);
-    payloadBuilder.setPort(shufflePort);
-    payloadBuilder.setPathComponent(outputContext.getUniqueIdentifier());
-    payloadBuilder.setRunDuration((int) ((endTime - startTime) / 1000));
-    DataMovementEventPayloadProto payloadProto = payloadBuilder.build();
-    byte[] payloadBytes = payloadProto.toByteArray();
-
-    List<Event> events = Lists.newArrayListWithCapacity(numOutputs);
-
-    for (int i = 0; i < numOutputs; i++) {
-      DataMovementEvent event = new DataMovementEvent(i, payloadBytes);
-      events.add(event);
-    }
-    return events;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 3ff603f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/output/OnFileUnorderedKVOutput.java
+++ /dev/null
@@ -1,98 +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.output;
-
-import java.nio.ByteBuffer;
-import java.util.Collections;
-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.shuffle.common.ShuffleUtils;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
-public class OnFileUnorderedKVOutput implements LogicalOutput {
-
-  private TezOutputContext outputContext;
-  private FileBasedKVWriter kvWriter;
-
-  public OnFileUnorderedKVOutput() {
-  }
-
-  @Override
-  public List<Event> initialize(TezOutputContext outputContext)
-      throws Exception {
-    this.outputContext = outputContext;
-    this.kvWriter = new FileBasedKVWriter(outputContext);
-    return Collections.emptyList();
-  }
-
-  @Override
-  public KVWriter getWriter() throws Exception {
-    return kvWriter;
-  }
-
-  @Override
-  public void handleEvents(List<Event> outputEvents) {
-    throw new TezUncheckedException("Not expecting any events");
-  }
-
-  @Override
-  public List<Event> close() throws Exception {
-    boolean outputGenerated = this.kvWriter.close();
-    DataMovementEventPayloadProto.Builder payloadBuilder = DataMovementEventPayloadProto
-        .newBuilder();
-
-    String host = System.getenv(ApplicationConstants.Environment.NM_HOST
-        .toString());
-    ByteBuffer shuffleMetadata = outputContext
-        .getServiceProviderMetaData(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID);
-    int shufflePort = ShuffleUtils
-        .deserializeShuffleProviderMetaData(shuffleMetadata);
-    payloadBuilder.setOutputGenerated(outputGenerated);
-    if (outputGenerated) {
-      payloadBuilder.setHost(host);
-      payloadBuilder.setPort(shufflePort);
-      payloadBuilder.setPathComponent(outputContext.getUniqueIdentifier());
-    }
-    DataMovementEventPayloadProto payloadProto = payloadBuilder.build();
-
-    DataMovementEvent dmEvent = new DataMovementEvent(0,
-        payloadProto.toByteArray());
-    List<Event> events = Lists.newArrayListWithCapacity(1);
-    events.add(dmEvent);
-    return events;
-  }
-
-  @Override
-  public void setNumPhysicalOutputs(int numOutputs) {
-    Preconditions.checkArgument(numOutputs == 1,
-        "Number of outputs can only be 1 for " + this.getClass().getName());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 29063f9..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/LogicalIOProcessorRuntimeTask.java
+++ /dev/null
@@ -1,475 +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.newruntime;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.token.Token;
-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;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezInputContextImpl;
-import org.apache.tez.engine.api.impl.TezOutputContextImpl;
-import org.apache.tez.engine.api.impl.TezProcessorContextImpl;
-import org.apache.tez.engine.api.impl.TezUmbilical;
-import org.apache.tez.engine.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-@Private
-public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
-
-  private static final Log LOG = LogFactory
-      .getLog(LogicalIOProcessorRuntimeTask.class);
-
-  private final List<InputSpec> inputSpecs;
-  private final List<LogicalInput> inputs;
-
-  private final List<OutputSpec> outputSpecs;
-  private final List<LogicalOutput> outputs;
-
-  private List<TezInputContext> inputContexts;
-  private List<TezOutputContext> outputContexts;
-  private TezProcessorContext processorContext;
-  
-  private final ProcessorDescriptor processorDescriptor;
-  private final LogicalIOProcessor processor;
-
-  private final Map<String, ByteBuffer> serviceConsumerMetadata;
-
-  private Map<String, LogicalInput> inputMap;
-  private Map<String, LogicalOutput> outputMap;
-
-  private LinkedBlockingQueue<TezEvent> eventsToBeProcessed;
-  private Thread eventRouterThread = null;
-
-  private final int appAttemptNumber;
-
-  public LogicalIOProcessorRuntimeTask(TaskSpec taskSpec, int appAttemptNumber,
-      Configuration tezConf, TezUmbilical tezUmbilical,
-      Token<JobTokenIdentifier> jobToken) throws IOException {
-    // TODO Remove jobToken from here post TEZ-421
-    super(taskSpec, tezConf, tezUmbilical);
-    LOG.info("Initializing LogicalIOProcessorRuntimeTask with TaskSpec: "
-        + taskSpec);
-    this.inputContexts = new ArrayList<TezInputContext>(taskSpec.getInputs().size());
-    this.outputContexts = new ArrayList<TezOutputContext>(taskSpec.getOutputs().size());
-    this.inputSpecs = taskSpec.getInputs();
-    this.inputs = createInputs(inputSpecs);
-    this.outputSpecs = taskSpec.getOutputs();
-    this.outputs = createOutputs(outputSpecs);
-    this.processorDescriptor = taskSpec.getProcessorDescriptor();
-    this.processor = createProcessor(processorDescriptor);
-    this.serviceConsumerMetadata = new HashMap<String, ByteBuffer>();
-    this.serviceConsumerMetadata.put(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID,
-        ShuffleUtils.convertJobTokenToBytes(jobToken));
-    this.eventsToBeProcessed = new LinkedBlockingQueue<TezEvent>();
-    this.state = State.NEW;
-    this.appAttemptNumber = appAttemptNumber;
-  }
-
-  public void initialize() throws Exception {
-    LOG.info("Initializing LogicalProcessorIORuntimeTask");
-    Preconditions.checkState(this.state == State.NEW, "Already initialized");
-    this.state = State.INITED;
-    inputMap = new LinkedHashMap<String, LogicalInput>(inputs.size());
-    outputMap = new LinkedHashMap<String, LogicalOutput>(outputs.size());
-
-    // TODO Maybe close initialized inputs / outputs in case of failure to
-    // initialize.
-    // Initialize all inputs. TODO: Multi-threaded at some point.
-    for (int i = 0; i < inputs.size(); i++) {
-      String srcVertexName = inputSpecs.get(i).getSourceVertexName();
-      initializeInput(inputs.get(i),
-          inputSpecs.get(i));
-      inputMap.put(srcVertexName, inputs.get(i));
-    }
-
-    // Initialize all outputs. TODO: Multi-threaded at some point.
-    for (int i = 0; i < outputs.size(); i++) {
-      String destVertexName = outputSpecs.get(i).getDestinationVertexName();
-      initializeOutput(outputs.get(i), outputSpecs.get(i));
-      outputMap.put(destVertexName, outputs.get(i));
-    }
-
-    // Initialize processor.
-    initializeLogicalIOProcessor();
-    startRouterThread();
-  }
-
-  public void run() throws Exception {
-    synchronized (this.state) {
-      Preconditions.checkState(this.state == State.INITED,
-          "Can only run while in INITED state. Current: " + this.state);
-      this.state = State.RUNNING;
-    }
-    LogicalIOProcessor lioProcessor = (LogicalIOProcessor) processor;
-    lioProcessor.run(inputMap, outputMap);
-  }
-
-  public void close() throws Exception {
-    try {
-      Preconditions.checkState(this.state == State.RUNNING,
-          "Can only run while in RUNNING state. Current: " + this.state);
-      this.state = State.CLOSED;
-
-      // Close the Inputs.
-      for (int i = 0; i < inputs.size(); i++) {
-        String srcVertexName = inputSpecs.get(i).getSourceVertexName();
-        List<Event> closeInputEvents = inputs.get(i).close();
-        sendTaskGeneratedEvents(closeInputEvents,
-            EventProducerConsumerType.INPUT, taskSpec.getVertexName(),
-            srcVertexName, taskSpec.getTaskAttemptID());
-      }
-
-      // Close the Processor.
-      processor.close();
-
-      // Close the Outputs.
-      for (int i = 0; i < outputs.size(); i++) {
-        String destVertexName = outputSpecs.get(i).getDestinationVertexName();
-        List<Event> closeOutputEvents = outputs.get(i).close();
-        sendTaskGeneratedEvents(closeOutputEvents,
-            EventProducerConsumerType.OUTPUT, taskSpec.getVertexName(),
-            destVertexName, taskSpec.getTaskAttemptID());
-      }
-    } finally {
-      setTaskDone();
-      if (eventRouterThread != null) {
-        eventRouterThread.interrupt();
-      }
-    }
-  }
-
-  private void initializeInput(Input input, InputSpec inputSpec)
-      throws Exception {
-    TezInputContext tezInputContext = createInputContext(inputSpec);
-    inputContexts.add(tezInputContext);
-    if (input instanceof LogicalInput) {
-      ((LogicalInput) input).setNumPhysicalInputs(inputSpec
-          .getPhysicalEdgeCount());
-    }
-    LOG.info("Initializing Input using InputSpec: " + inputSpec);
-    List<Event> events = input.initialize(tezInputContext);
-    sendTaskGeneratedEvents(events, EventProducerConsumerType.INPUT,
-        tezInputContext.getTaskVertexName(),
-        tezInputContext.getSourceVertexName(), taskSpec.getTaskAttemptID());
-  }
-
-  private void initializeOutput(Output output, OutputSpec outputSpec)
-      throws Exception {
-    TezOutputContext tezOutputContext = createOutputContext(outputSpec);
-    outputContexts.add(tezOutputContext);
-    if (output instanceof LogicalOutput) {
-      ((LogicalOutput) output).setNumPhysicalOutputs(outputSpec
-          .getPhysicalEdgeCount());
-    }
-    LOG.info("Initializing Output using OutputSpec: " + outputSpec);
-    List<Event> events = output.initialize(tezOutputContext);
-    sendTaskGeneratedEvents(events, EventProducerConsumerType.OUTPUT,
-        tezOutputContext.getTaskVertexName(),
-        tezOutputContext.getDestinationVertexName(),
-        taskSpec.getTaskAttemptID());
-  }
-
-  private void initializeLogicalIOProcessor() throws Exception {
-    LOG.info("Initializing processor"
-        + ", processorClassName=" + processorDescriptor.getClassName());
-    TezProcessorContext processorContext = createProcessorContext();
-    this.processorContext = processorContext;
-    processor.initialize(processorContext);
-  }
-
-  private TezInputContext createInputContext(InputSpec inputSpec) {
-    TezInputContext inputContext = new TezInputContextImpl(tezConf,
-        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(),
-        inputSpec.getSourceVertexName(), taskSpec.getTaskAttemptID(),
-        tezCounters,
-        inputSpec.getInputDescriptor().getUserPayload() == null ? taskSpec
-            .getProcessorDescriptor().getUserPayload() : inputSpec
-            .getInputDescriptor().getUserPayload(), this,
-        serviceConsumerMetadata);
-    return inputContext;
-  }
-
-  private TezOutputContext createOutputContext(OutputSpec outputSpec) {
-    TezOutputContext outputContext = new TezOutputContextImpl(tezConf,
-        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(),
-        outputSpec.getDestinationVertexName(), taskSpec.getTaskAttemptID(),
-        tezCounters,
-        outputSpec.getOutputDescriptor().getUserPayload() == null ? taskSpec
-            .getProcessorDescriptor().getUserPayload() : outputSpec
-            .getOutputDescriptor().getUserPayload(), this,
-        serviceConsumerMetadata);
-    return outputContext;
-  }
-
-  private TezProcessorContext createProcessorContext() {
-    TezProcessorContext processorContext = new TezProcessorContextImpl(tezConf,
-        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(), taskSpec.getTaskAttemptID(),
-        tezCounters, processorDescriptor.getUserPayload(), this,
-        serviceConsumerMetadata);
-    return processorContext;
-  }
-
-  private List<LogicalInput> createInputs(List<InputSpec> inputSpecs) {
-    List<LogicalInput> inputs = new ArrayList<LogicalInput>(inputSpecs.size());
-    for (InputSpec inputSpec : inputSpecs) {
-      LOG.info("Creating Input from InputSpec: "
-          + inputSpec);
-      Input input = RuntimeUtils.createClazzInstance(inputSpec
-          .getInputDescriptor().getClassName());
-
-      if (input instanceof LogicalInput) {
-        inputs.add((LogicalInput) input);
-      } else {
-        throw new TezUncheckedException(input.getClass().getName()
-            + " is not a sub-type of LogicalInput."
-            + " Only LogicalInput sub-types supported by LogicalIOProcessor.");
-      }
-    }
-    return inputs;
-  }
-
-  private List<LogicalOutput> createOutputs(List<OutputSpec> outputSpecs) {
-    List<LogicalOutput> outputs = new ArrayList<LogicalOutput>(
-        outputSpecs.size());
-    for (OutputSpec outputSpec : outputSpecs) {
-      LOG.info("Creating Output from OutputSpec"
-          + outputSpec);
-      Output output = RuntimeUtils.createClazzInstance(outputSpec
-          .getOutputDescriptor().getClassName());
-      if (output instanceof LogicalOutput) {
-        outputs.add((LogicalOutput) output);
-      } else {
-        throw new TezUncheckedException(output.getClass().getName()
-            + " is not a sub-type of LogicalOutput."
-            + " Only LogicalOutput sub-types supported by LogicalIOProcessor.");
-      }
-    }
-    return outputs;
-  }
-
-  private LogicalIOProcessor createProcessor(
-      ProcessorDescriptor processorDescriptor) {
-    Processor processor = RuntimeUtils.createClazzInstance(processorDescriptor
-        .getClassName());
-    if (!(processor instanceof LogicalIOProcessor)) {
-      throw new TezUncheckedException(processor.getClass().getName()
-          + " is not a sub-type of LogicalIOProcessor."
-          + " Only LogicalOutput sub-types supported by LogicalIOProcessor.");
-    }
-    return (LogicalIOProcessor) processor;
-  }
-
-  private void sendTaskGeneratedEvents(List<Event> events,
-      EventProducerConsumerType generator, String taskVertexName,
-      String edgeVertexName, TezTaskAttemptID taskAttemptID) {
-    if (events == null || events.isEmpty()) {
-      return;
-    }
-    EventMetaData eventMetaData = new EventMetaData(generator,
-        taskVertexName, edgeVertexName, taskAttemptID);
-    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
-    for (Event e : events) {
-      TezEvent te = new TezEvent(e, eventMetaData);
-      tezEvents.add(te);
-    }
-    if (LOG.isDebugEnabled()) {
-      for (TezEvent e : tezEvents) {
-        LOG.debug("Generated event info"
-            + ", eventMetaData=" + eventMetaData.toString()
-            + ", eventType=" + e.getEventType());
-      }
-    }
-    tezUmbilical.addEvents(tezEvents);
-  }
-
-  private boolean handleEvent(TezEvent e) {
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Handling TezEvent in task"
-          + ", taskAttemptId=" + taskSpec.getTaskAttemptID()
-          + ", eventType=" + e.getEventType()
-          + ", eventSourceInfo=" + e.getSourceInfo()
-          + ", eventDestinationInfo=" + e.getDestinationInfo());
-    }
-    try {
-      switch (e.getDestinationInfo().getEventGenerator()) {
-      case INPUT:
-        LogicalInput input = inputMap.get(
-            e.getDestinationInfo().getEdgeVertexName());
-        if (input != null) {
-          input.handleEvents(Collections.singletonList(e.getEvent()));
-        } else {
-          throw new TezUncheckedException("Unhandled event for invalid target: "
-              + e);
-        }
-        break;
-      case OUTPUT:
-        LogicalOutput output = outputMap.get(
-            e.getDestinationInfo().getEdgeVertexName());
-        if (output != null) {
-          output.handleEvents(Collections.singletonList(e.getEvent()));
-        } else {
-          throw new TezUncheckedException("Unhandled event for invalid target: "
-              + e);
-        }
-        break;
-      case PROCESSOR:
-        processor.handleEvents(Collections.singletonList(e.getEvent()));
-        break;
-      case SYSTEM:
-        LOG.warn("Trying to send a System event in a Task: " + e);
-        break;
-      }
-    } catch (Throwable t) {
-      LOG.warn("Failed to handle event", t);
-      setFatalError(t, "Failed to handle event");
-      EventMetaData sourceInfo = new EventMetaData(
-          e.getDestinationInfo().getEventGenerator(),
-          taskSpec.getVertexName(), e.getDestinationInfo().getEdgeVertexName(),
-          getTaskAttemptID());
-      tezUmbilical.signalFatalError(getTaskAttemptID(),
-          StringUtils.stringifyException(t), sourceInfo);
-      return false;
-    }
-    return true;
-  }
-
-  @Override
-  public synchronized void handleEvents(Collection<TezEvent> events) {
-    if (events == null || events.isEmpty()) {
-      return;
-    }
-    eventCounter.addAndGet(events.size());
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Received events to be processed by task"
-          + ", taskAttemptId=" + taskSpec.getTaskAttemptID()
-          + ", eventCount=" + events.size()
-          + ", newEventCounter=" + eventCounter.get());
-    }
-    eventsToBeProcessed.addAll(events);
-  }
-
-  private void startRouterThread() {
-    eventRouterThread = new Thread(new Runnable() {
-      public void run() {
-        while (!isTaskDone() && !Thread.currentThread().isInterrupted()) {
-          try {
-            TezEvent e = eventsToBeProcessed.take();
-            if (e == null) {
-              continue;
-            }
-            // TODO TODONEWTEZ
-            if (!handleEvent(e)) {
-              LOG.warn("Stopping Event Router thread as failed to handle"
-                  + " event: " + e);
-              return;
-            }
-          } catch (InterruptedException e) {
-            if (!isTaskDone()) {
-              LOG.warn("Event Router thread interrupted. Returning.");
-            }
-            return;
-          }
-        }
-      }
-    });
-
-    eventRouterThread.setName("TezTaskEventRouter["
-        + taskSpec.getTaskAttemptID().toString() + "]");
-    eventRouterThread.start();
-  }
-
-  public synchronized void cleanup() {
-    setTaskDone();
-    if (eventRouterThread != null) {
-      eventRouterThread.interrupt();
-    }
-  }
-  
-  @Private
-  @VisibleForTesting
-  public List<TezInputContext> getInputContexts() {
-    return this.inputContexts;
-  }
-  
-  @Private
-  @VisibleForTesting
-  public List<TezOutputContext> getOutputContexts() {
-    return this.outputContexts;
-  }
-
-  @Private
-  @VisibleForTesting
-  public TezProcessorContext getProcessorContext() {
-    return this.processorContext;
-  }
-  
-  @Private
-  @VisibleForTesting
-  public Map<String, LogicalInput> getInputs() {
-    return this.inputMap;
-  }
-  
-  @Private
-  @VisibleForTesting
-  public Map<String, LogicalOutput> getOutputs() {
-    return this.outputMap;
-  }
-  
-  @Private
-  @VisibleForTesting
-  public LogicalIOProcessor getProcessor() {
-    return this.processor;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 22cbc7c..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeTask.java
+++ /dev/null
@@ -1,106 +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.newruntime;
-
-import java.util.Collection;
-import java.util.concurrent.atomic.AtomicBoolean;
-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.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezUmbilical;
-
-public abstract class RuntimeTask {
-
-  protected AtomicBoolean hasFatalError = new AtomicBoolean(false);
-  protected Throwable fatalError = null;
-  protected String fatalErrorMessage = null;
-  protected float progress;
-  protected final TezCounters tezCounters;
-  protected final TaskSpec taskSpec;
-  protected final Configuration tezConf;
-  protected final TezUmbilical tezUmbilical;
-  protected final AtomicInteger eventCounter;
-  private final AtomicBoolean taskDone;
-
-  protected RuntimeTask(TaskSpec taskSpec, Configuration tezConf,
-      TezUmbilical tezUmbilical) {
-    this.taskSpec = taskSpec;
-    this.tezConf = tezConf;
-    this.tezUmbilical = tezUmbilical;
-    this.tezCounters = new TezCounters();
-    this.eventCounter = new AtomicInteger(0);
-    this.progress = 0.0f;
-    this.taskDone = new AtomicBoolean(false);
-  }
-
-  protected enum State {
-    NEW, INITED, RUNNING, CLOSED;
-  }
-
-  protected State state;
-
-  public String getVertexName() {
-    return taskSpec.getVertexName();
-  }
-
-  public void setFatalError(Throwable t, String message) {
-    hasFatalError.set(true);
-    this.fatalError = t;
-    this.fatalErrorMessage = message;
-  }
-
-  public boolean hadFatalError() {
-    return hasFatalError.get();
-  }
-
-  public synchronized void setProgress(float progress) {
-    this.progress = progress;
-  }
-
-  public synchronized float getProgress() {
-    return this.progress;
-  }
-
-  public TezCounters getCounters() {
-    return this.tezCounters;
-  }
-
-  public TezTaskAttemptID getTaskAttemptID() {
-    return taskSpec.getTaskAttemptID();
-  }
-
-  public abstract void handleEvents(Collection<TezEvent> events);
-
-  public int getEventCounter() {
-    return eventCounter.get();
-  }
-
-  public boolean isTaskDone() {
-    return taskDone.get();
-  }
-
-  protected void setTaskDone() {
-    taskDone.set(true);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeUtils.java
deleted file mode 100644
index 20a029e..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/newruntime/RuntimeUtils.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.newruntime;
-
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.tez.dag.api.TezUncheckedException;
-
-public class RuntimeUtils {
-
-  private static final Map<String, Class<?>> CLAZZ_CACHE = new ConcurrentHashMap<String, Class<?>>();
-
-  private static Class<?> getClazz(String className) {
-    Class<?> clazz = CLAZZ_CACHE.get(className);
-    if (clazz == null) {
-      try {
-        clazz = Class.forName(className);
-      } catch (ClassNotFoundException e) {
-        throw new TezUncheckedException("Unable to load class: " + className, e);
-      }
-    }
-    return clazz;
-  }
-
-  private static <T> T getNewInstance(Class<T> clazz) {
-    T instance;
-    try {
-      instance = clazz.newInstance();
-    } catch (InstantiationException e) {
-      throw new TezUncheckedException(
-          "Unable to instantiate class with 0 arguments: " + clazz.getName(), e);
-    } catch (IllegalAccessException e) {
-      throw new TezUncheckedException(
-          "Unable to instantiate class with 0 arguments: " + clazz.getName(), e);
-    }
-    return instance;
-  }
-
-  public static <T> T createClazzInstance(String className) {
-    Class<?> clazz = getClazz(className);
-    @SuppressWarnings("unchecked")
-    T instance = (T) getNewInstance(clazz);
-    return instance;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 531e460..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/DiskFetchedInput.java
+++ /dev/null
@@ -1,111 +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.shuffle.common;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.Path;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-
-import com.google.common.base.Preconditions;
-
-public class DiskFetchedInput extends FetchedInput {
-
-  private static final Log LOG = LogFactory.getLog(DiskFetchedInput.class);
-  
-  private final FileSystem localFS;
-  private final Path tmpOutputPath;
-  private final Path outputPath;
-
-  public DiskFetchedInput(long size,
-      InputAttemptIdentifier inputAttemptIdentifier,
-      FetchedInputCallback callbackHandler, Configuration conf,
-      LocalDirAllocator localDirAllocator, TezTaskOutputFiles filenameAllocator)
-      throws IOException {
-    super(Type.DISK, size, inputAttemptIdentifier, callbackHandler);
-
-    this.localFS = FileSystem.getLocal(conf);
-    this.outputPath = filenameAllocator.getInputFileForWrite(
-        this.inputAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(), size);
-    this.tmpOutputPath = outputPath.suffix(String.valueOf(id));
-  }
-
-  @Override
-  public OutputStream getOutputStream() throws IOException {
-    return localFS.create(tmpOutputPath);
-  }
-
-  @Override
-  public InputStream getInputStream() throws IOException {
-    return localFS.open(outputPath);
-  }
-
-  @Override
-  public void commit() throws IOException {
-    if (state == State.PENDING) {
-      state = State.COMMITTED;
-      localFS.rename(tmpOutputPath, outputPath);
-      notifyFetchComplete();
-    }
-  }
-
-  @Override
-  public void abort() throws IOException {
-    if (state == State.PENDING) {
-      state = State.ABORTED;
-      // TODO NEWTEZ Maybe defer this to container cleanup
-      localFS.delete(tmpOutputPath, false);
-      notifyFetchFailure();
-    }
-  }
-  
-  @Override
-  public void free() {
-    Preconditions.checkState(
-        state == State.COMMITTED || state == State.ABORTED,
-        "FetchedInput can only be freed after it is committed or aborted");
-    if (state == State.COMMITTED) {
-      state = State.FREED;
-      try {
-        // TODO NEWTEZ Maybe defer this to container cleanup
-        localFS.delete(outputPath, false);
-      } catch (IOException e) {
-        // Ignoring the exception, will eventually be cleaned by container
-        // cleanup.
-        LOG.warn("Failed to remvoe file : " + outputPath.toString());
-      }
-      notifyFreedResource();
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "DiskFetchedInput [outputPath=" + outputPath
-        + ", inputAttemptIdentifier=" + inputAttemptIdentifier + ", size="
-        + size + ", type=" + type + ", id=" + id + ", state=" + state + "]";
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchResult.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchResult.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchResult.java
deleted file mode 100644
index fb0b324..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchResult.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.shuffle.common;
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-public class FetchResult {
-
-  private final String host;
-  private final int port;
-  private final int partition;
-  private final Iterable<InputAttemptIdentifier> pendingInputs;
-
-  public FetchResult(String host, int port, int partition,
-      Iterable<InputAttemptIdentifier> pendingInputs) {
-    this.host = host;
-    this.port = port;
-    this.partition = partition;
-    this.pendingInputs = pendingInputs;
-  }
-
-  public String getHost() {
-    return host;
-  }
-
-  public int getPort() {
-    return port;
-  }
-
-  public int getPartition() {
-    return partition;
-  }
-
-  public Iterable<InputAttemptIdentifier> getPendingInputs() {
-    return pendingInputs;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInput.java
deleted file mode 100644
index f5339d3..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInput.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.engine.shuffle.common;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-@Private
-public abstract class FetchedInput {
-  
-  public static enum Type {
-    WAIT, // TODO NEWTEZ Implement this, only if required.
-    MEMORY,
-    DISK,
-  }
-  
-  protected static enum State {
-    PENDING, COMMITTED, ABORTED, FREED
-  }
-
-  private static AtomicInteger ID_GEN = new AtomicInteger(0);
-
-  protected InputAttemptIdentifier inputAttemptIdentifier;
-  protected final long size;
-  protected final Type type;
-  protected final FetchedInputCallback callback;
-  protected final int id;
-  protected State state;
-
-  public FetchedInput(Type type, long size,
-      InputAttemptIdentifier inputAttemptIdentifier,
-      FetchedInputCallback callbackHandler) {
-    this.type = type;
-    this.size = size;
-    this.inputAttemptIdentifier = inputAttemptIdentifier;
-    this.callback = callbackHandler;
-    this.id = ID_GEN.getAndIncrement();
-    this.state = State.PENDING;
-  }
-
-  public Type getType() {
-    return this.type;
-  }
-
-  public long getSize() {
-    return this.size;
-  }
-
-  public InputAttemptIdentifier getInputAttemptIdentifier() {
-    return this.inputAttemptIdentifier;
-  }
-
-  /**
-   * Inform the Allocator about a committed resource.
-   * This should be called by commit
-   */
-  public void notifyFetchComplete() {
-    this.callback.fetchComplete(this);
-  }
-  
-  /**
-   * Inform the Allocator about a failed resource.
-   * This should be called by abort
-   */
-  public void notifyFetchFailure() {
-    this.callback.fetchFailed(this);
-  }
-  
-  /**
-   * Inform the Allocator about a completed resource being released.
-   * This should be called by free
-   */
-  public void notifyFreedResource() {
-    this.callback.freeResources(this);
-  }
-  
-  /**
-   * Returns the output stream to be used to write fetched data. Users are
-   * expected to close the OutputStream when they're done
-   */
-  public abstract OutputStream getOutputStream() throws IOException;
-
-  /**
-   * Return an input stream to be used to read the previously fetched data.
-   * Users are expected to close the InputStream when they're done
-   */
-  public abstract InputStream getInputStream() throws IOException;
-
-  /**
-   * Commit the output. Should be idempotent
-   */
-  public abstract void commit() throws IOException;
-
-  /**
-   * Abort the output. Should be idempotent
-   */
-  public abstract void abort() throws IOException;
-
-  /**
-   * Called when this input has been consumed, so that resources can be
-   * reclaimed.
-   */
-  public abstract void free();
-  
-  @Override
-  public int hashCode() {
-    return id;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    FetchedInput other = (FetchedInput) obj;
-    if (id != other.id)
-      return false;
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputAllocator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputAllocator.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputAllocator.java
deleted file mode 100644
index 7e573f0..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputAllocator.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.engine.shuffle.common;
-
-import java.io.IOException;
-
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-public interface FetchedInputAllocator {
-
-  public FetchedInput allocate(long size,
-      InputAttemptIdentifier inputAttemptIdentifier) throws IOException;
-  
-  
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputCallback.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputCallback.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputCallback.java
deleted file mode 100644
index 2d2d73b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetchedInputCallback.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.shuffle.common;
-
-public interface FetchedInputCallback {
-  
-  public void fetchComplete(FetchedInput fetchedInput);
-  
-  public void fetchFailed(FetchedInput fetchedInput);
-  
-  public void freeResources(FetchedInput fetchedInput);
-  
-}


[10/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/Fetcher.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/Fetcher.java
deleted file mode 100644
index a353416..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/Fetcher.java
+++ /dev/null
@@ -1,608 +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.shuffle.common;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.HttpURLConnection;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLConnection;
-import java.security.GeneralSecurityException;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.crypto.SecretKey;
-import javax.net.ssl.HttpsURLConnection;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.security.ssl.SSLFactory;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.security.SecureShuffleUtils;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
-import org.apache.tez.engine.common.sort.impl.IFileInputStream;
-import org.apache.tez.engine.shuffle.common.FetchedInput.Type;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Responsible for fetching inputs served by the ShuffleHandler for a single
- * host. Construct using {@link FetcherBuilder}
- */
-public class Fetcher implements Callable<FetchResult> {
-
-  private static final Log LOG = LogFactory.getLog(Fetcher.class);
-
-  private static final int UNIT_CONNECT_TIMEOUT = 60 * 1000;
-  private static final AtomicInteger fetcherIdGen = new AtomicInteger(0);
-
-  // Configurable fields.
-  private CompressionCodec codec;
-  private Decompressor decompressor;
-  private int connectionTimeout;
-  private int readTimeout;
-
-  private final SecretKey shuffleSecret;
-  private final Configuration conf;
-
-  private final FetcherCallback fetcherCallback;
-  private final FetchedInputAllocator inputManager;
-  private final ApplicationId appId;
-
-  private static boolean sslShuffle;
-  private static SSLFactory sslFactory;
-  private static boolean sslFactoryInited;
-
-  private final int fetcherIdentifier;
-  
-  // Parameters to track work.
-  private List<InputAttemptIdentifier> srcAttempts;
-  private String host;
-  private int port;
-  private int partition;
-
-  // Maps from the pathComponents (unique per srcTaskId) to the specific taskId
-  private Map<String, InputAttemptIdentifier> pathToAttemptMap;
-  private Set<InputAttemptIdentifier> remaining;
-
-  private URL url;
-  private String encHash;
-  private String msgToEncode;
-
-  private Fetcher(FetcherCallback fetcherCallback,
-      FetchedInputAllocator inputManager, ApplicationId appId, SecretKey shuffleSecret,
-      Configuration conf) {
-    this.fetcherCallback = fetcherCallback;
-    this.inputManager = inputManager;
-    this.shuffleSecret = shuffleSecret;
-    this.appId = appId;
-    this.conf = conf;
-
-    this.fetcherIdentifier = fetcherIdGen.getAndIncrement();
-    
-    // TODO NEWTEZ Ideally, move this out from here into a static initializer block.
-    synchronized (Fetcher.class) {
-      if (!sslFactoryInited) {
-        sslFactoryInited = true;
-        sslShuffle = conf.getBoolean(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL,
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL);
-        if (sslShuffle) {
-          sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
-          try {
-            sslFactory.init();
-          } catch (Exception ex) {
-            sslFactory.destroy();
-            throw new RuntimeException(ex);
-          }
-        }
-      }
-    }
-  }
-
-  @Override
-  public FetchResult call() throws Exception {
-    if (srcAttempts.size() == 0) {
-      return new FetchResult(host, port, partition, srcAttempts);
-    }
-
-    for (InputAttemptIdentifier in : srcAttempts) {
-      pathToAttemptMap.put(in.getPathComponent(), in);
-    }
-
-    remaining = new HashSet<InputAttemptIdentifier>(srcAttempts);
-
-    HttpURLConnection connection;
-    try {
-      connection = connectToShuffleHandler(host, port, partition, srcAttempts);
-    } catch (IOException e) {
-      // ioErrs.increment(1);
-      // If connect did not succeed, just mark all the maps as failed,
-      // indirectly penalizing the host
-      for (Iterator<InputAttemptIdentifier> leftIter = remaining.iterator(); leftIter
-          .hasNext();) {
-        fetcherCallback.fetchFailed(host, leftIter.next(), true);
-        leftIter.remove();
-      }
-      return new FetchResult(host, port, partition, remaining);
-    }
-
-    DataInputStream input;
-
-    try {
-      input = new DataInputStream(connection.getInputStream());
-      validateConnectionResponse(connection, url, msgToEncode, encHash);
-    } catch (IOException e) {
-      // ioErrs.increment(1);
-      // If we got a read error at this stage, it implies there was a problem
-      // with the first map, typically lost map. So, penalize only that map
-      // and add the rest
-      InputAttemptIdentifier firstAttempt = srcAttempts.get(0);
-      fetcherCallback.fetchFailed(host, firstAttempt, false);
-      remaining.remove(firstAttempt);
-      return new FetchResult(host, port, partition, remaining);
-    }
-
-    // By this point, the connection is setup and the response has been
-    // validated.
-
-    // Loop through available map-outputs and fetch them
-    // On any error, faildTasks is not null and we exit
-    // after putting back the remaining maps to the
-    // yet_to_be_fetched list and marking the failed tasks.
-    InputAttemptIdentifier[] failedInputs = null;
-    while (!remaining.isEmpty() && failedInputs == null) {
-      failedInputs = fetchInputs(input);
-    }
-
-    if (failedInputs != null && failedInputs.length > 0) {
-      LOG.warn("copyInputs failed for tasks " + Arrays.toString(failedInputs));
-      for (InputAttemptIdentifier left : failedInputs) {
-        fetcherCallback.fetchFailed(host, left, false);
-        remaining.remove(left);
-      }
-    }
-
-    IOUtils.cleanup(LOG, input);
-
-    // Sanity check
-    if (failedInputs == null && !remaining.isEmpty()) {
-      throw new IOException("server didn't return all expected map outputs: "
-          + remaining.size() + " left.");
-    }
-
-    return new FetchResult(host, port, partition, remaining);
-
-  }
-
-  private InputAttemptIdentifier[] fetchInputs(DataInputStream input) {
-    FetchedInput fetchedInput = null;
-    InputAttemptIdentifier srcAttemptId = null;
-    long decompressedLength = -1;
-    long compressedLength = -1;
-
-    try {
-      long startTime = System.currentTimeMillis();
-      int responsePartition = -1;
-      // Read the shuffle header
-      try {
-        ShuffleHeader header = new ShuffleHeader();
-        header.readFields(input);
-        String pathComponent = header.getMapId();
-
-        srcAttemptId = pathToAttemptMap.get(pathComponent);
-        compressedLength = header.getCompressedLength();
-        decompressedLength = header.getUncompressedLength();
-        responsePartition = header.getPartition();
-      } catch (IllegalArgumentException e) {
-        // badIdErrs.increment(1);
-        LOG.warn("Invalid src id ", e);
-        // Don't know which one was bad, so consider all of them as bad
-        return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
-      }
-
-      // Do some basic sanity verification
-      if (!verifySanity(compressedLength, decompressedLength,
-          responsePartition, srcAttemptId)) {
-        return new InputAttemptIdentifier[] { srcAttemptId };
-      }
-
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("header: " + srcAttemptId + ", len: " + compressedLength
-            + ", decomp len: " + decompressedLength);
-      }
-
-      // Get the location for the map output - either in-memory or on-disk
-      fetchedInput = inputManager.allocate(decompressedLength, srcAttemptId);
-
-      // TODO NEWTEZ No concept of WAIT at the moment.
-      // // Check if we can shuffle *now* ...
-      // if (fetchedInput.getType() == FetchedInput.WAIT) {
-      // LOG.info("fetcher#" + id +
-      // " - MergerManager returned Status.WAIT ...");
-      // //Not an error but wait to process data.
-      // return EMPTY_ATTEMPT_ID_ARRAY;
-      // }
-
-      // Go!
-      LOG.info("fetcher" + " about to shuffle output of srcAttempt "
-          + fetchedInput.getInputAttemptIdentifier() + " decomp: "
-          + decompressedLength + " len: " + compressedLength + " to "
-          + fetchedInput.getType());
-
-      if (fetchedInput.getType() == Type.MEMORY) {
-        shuffleToMemory((MemoryFetchedInput) fetchedInput, input,
-            (int) decompressedLength, (int) compressedLength);
-      } else {
-        shuffleToDisk((DiskFetchedInput) fetchedInput, input, compressedLength);
-      }
-
-      // Inform the shuffle scheduler
-      long endTime = System.currentTimeMillis();
-      fetcherCallback.fetchSucceeded(host, srcAttemptId, fetchedInput,
-          compressedLength, (endTime - startTime));
-
-      // Note successful shuffle
-      remaining.remove(srcAttemptId);
-      // metrics.successFetch();
-      return null;
-    } catch (IOException ioe) {
-      // ioErrs.increment(1);
-      if (srcAttemptId == null || fetchedInput == null) {
-        LOG.info("fetcher" + " failed to read map header" + srcAttemptId
-            + " decomp: " + decompressedLength + ", " + compressedLength, ioe);
-        if (srcAttemptId == null) {
-          return remaining
-              .toArray(new InputAttemptIdentifier[remaining.size()]);
-        } else {
-          return new InputAttemptIdentifier[] { srcAttemptId };
-        }
-      }
-      LOG.warn("Failed to shuffle output of " + srcAttemptId + " from " + host,
-          ioe);
-
-      // Inform the shuffle-scheduler
-      try {
-        fetchedInput.abort();
-      } catch (IOException e) {
-        LOG.info("Failure to cleanup fetchedInput: " + fetchedInput);
-      }
-      // metrics.failedFetch();
-      return new InputAttemptIdentifier[] { srcAttemptId };
-    }
-  }
-
-  @SuppressWarnings("resource")
-  private void shuffleToMemory(MemoryFetchedInput fetchedInput,
-      InputStream input, int decompressedLength, int compressedLength)
-      throws IOException {
-    IFileInputStream checksumIn = new IFileInputStream(input, compressedLength,
-        conf);
-
-    input = checksumIn;
-
-    // Are map-outputs compressed?
-    if (codec != null) {
-      decompressor.reset();
-      input = codec.createInputStream(input, decompressor);
-    }
-    // Copy map-output into an in-memory buffer
-    byte[] shuffleData = fetchedInput.getBytes();
-
-    try {
-      IOUtils.readFully(input, shuffleData, 0, shuffleData.length);
-      // metrics.inputBytes(shuffleData.length);
-      LOG.info("Read " + shuffleData.length + " bytes from input for "
-          + fetchedInput.getInputAttemptIdentifier());
-    } catch (IOException ioe) {
-      // Close the streams
-      IOUtils.cleanup(LOG, input);
-      // Re-throw
-      throw ioe;
-    }
-  }
-
-  private void shuffleToDisk(DiskFetchedInput fetchedInput, InputStream input,
-      long compressedLength) throws IOException {
-    // Copy data to local-disk
-    OutputStream output = fetchedInput.getOutputStream();
-    long bytesLeft = compressedLength;
-    try {
-      final int BYTES_TO_READ = 64 * 1024;
-      byte[] buf = new byte[BYTES_TO_READ];
-      while (bytesLeft > 0) {
-        int n = input.read(buf, 0, (int) Math.min(bytesLeft, BYTES_TO_READ));
-        if (n < 0) {
-          throw new IOException("read past end of stream reading "
-              + fetchedInput.getInputAttemptIdentifier());
-        }
-        output.write(buf, 0, n);
-        bytesLeft -= n;
-        // metrics.inputBytes(n);
-      }
-
-      LOG.info("Read " + (compressedLength - bytesLeft)
-          + " bytes from input for " + fetchedInput.getInputAttemptIdentifier());
-
-      output.close();
-    } catch (IOException ioe) {
-      // Close the streams
-      IOUtils.cleanup(LOG, input, output);
-
-      // Re-throw
-      throw ioe;
-    }
-
-    // Sanity check
-    if (bytesLeft != 0) {
-      throw new IOException("Incomplete input received for "
-          + fetchedInput.getInputAttemptIdentifier() + " from " + host + " ("
-          + bytesLeft + " bytes missing of " + compressedLength + ")");
-    }
-  }
-
-  /**
-   * Do some basic verification on the input received -- Being defensive
-   * 
-   * @param compressedLength
-   * @param decompressedLength
-   * @param fetchPartition
-   * @param remaining
-   * @param mapId
-   * @return true/false, based on if the verification succeeded or not
-   */
-  private boolean verifySanity(long compressedLength, long decompressedLength,
-      int fetchPartition, InputAttemptIdentifier srcAttemptId) {
-    if (compressedLength < 0 || decompressedLength < 0) {
-      // wrongLengthErrs.increment(1);
-      LOG.warn(" invalid lengths in input header: id: " + srcAttemptId
-          + " len: " + compressedLength + ", decomp len: " + decompressedLength);
-      return false;
-    }
-
-    if (fetchPartition != this.partition) {
-      // wrongReduceErrs.increment(1);
-      LOG.warn(" data for the wrong reduce map: " + srcAttemptId + " len: "
-          + compressedLength + " decomp len: " + decompressedLength
-          + " for reduce " + fetchPartition);
-      return false;
-    }
-
-    // Sanity check
-    if (!remaining.contains(srcAttemptId)) {
-      // wrongMapErrs.increment(1);
-      LOG.warn("Invalid input. Received output for " + srcAttemptId);
-      return false;
-    }
-    return true;
-  }
-
-  private HttpURLConnection connectToShuffleHandler(String host, int port,
-      int partition, List<InputAttemptIdentifier> inputs) throws IOException {
-    try {
-      this.url = constructInputURL(host, port, partition, inputs);
-      HttpURLConnection connection = openConnection(url);
-
-      // generate hash of the url
-      this.msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
-      this.encHash = SecureShuffleUtils.hashFromString(msgToEncode,
-          shuffleSecret);
-
-      // put url hash into http header
-      connection.addRequestProperty(SecureShuffleUtils.HTTP_HEADER_URL_HASH,
-          encHash);
-      // set the read timeout
-      connection.setReadTimeout(readTimeout);
-      // put shuffle version into http header
-      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
-      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
-
-      connect(connection, connectionTimeout);
-      return connection;
-    } catch (IOException e) {
-      LOG.warn("Failed to connect to " + host + " with " + srcAttempts.size()
-          + " inputs", e);
-      throw e;
-    }
-  }
-
-  private void validateConnectionResponse(HttpURLConnection connection,
-      URL url, String msgToEncode, String encHash) throws IOException {
-    int rc = connection.getResponseCode();
-    if (rc != HttpURLConnection.HTTP_OK) {
-      throw new IOException("Got invalid response code " + rc + " from " + url
-          + ": " + connection.getResponseMessage());
-    }
-
-    if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(connection
-        .getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
-        || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(connection
-            .getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))) {
-      throw new IOException("Incompatible shuffle response version");
-    }
-
-    // get the replyHash which is HMac of the encHash we sent to the server
-    String replyHash = connection
-        .getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH);
-    if (replyHash == null) {
-      throw new IOException("security validation of TT Map output failed");
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("url=" + msgToEncode + ";encHash=" + encHash + ";replyHash="
-          + replyHash);
-    }
-    // verify that replyHash is HMac of encHash
-    SecureShuffleUtils.verifyReply(replyHash, encHash, shuffleSecret);
-    LOG.info("for url=" + msgToEncode + " sent hash and receievd reply");
-  }
-
-  protected HttpURLConnection openConnection(URL url) throws IOException {
-    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    if (sslShuffle) {
-      HttpsURLConnection httpsConn = (HttpsURLConnection) conn;
-      try {
-        httpsConn.setSSLSocketFactory(sslFactory.createSSLSocketFactory());
-      } catch (GeneralSecurityException ex) {
-        throw new IOException(ex);
-      }
-      httpsConn.setHostnameVerifier(sslFactory.getHostnameVerifier());
-    }
-    return conn;
-  }
-
-  /**
-   * The connection establishment is attempted multiple times and is given up
-   * only on the last failure. Instead of connecting with a timeout of X, we try
-   * connecting with a timeout of x < X but multiple times.
-   */
-  private void connect(URLConnection connection, int connectionTimeout)
-      throws IOException {
-    int unit = 0;
-    if (connectionTimeout < 0) {
-      throw new IOException("Invalid timeout " + "[timeout = "
-          + connectionTimeout + " ms]");
-    } else if (connectionTimeout > 0) {
-      unit = Math.min(UNIT_CONNECT_TIMEOUT, connectionTimeout);
-    }
-    // set the connect timeout to the unit-connect-timeout
-    connection.setConnectTimeout(unit);
-    while (true) {
-      try {
-        connection.connect();
-        break;
-      } catch (IOException ioe) {
-        // update the total remaining connect-timeout
-        connectionTimeout -= unit;
-
-        // throw an exception if we have waited for timeout amount of time
-        // note that the updated value if timeout is used here
-        if (connectionTimeout == 0) {
-          throw ioe;
-        }
-
-        // reset the connect timeout for the last try
-        if (connectionTimeout < unit) {
-          unit = connectionTimeout;
-          // reset the connect time out for the final connect
-          connection.setConnectTimeout(unit);
-        }
-      }
-    }
-  }
-
-  private URL constructInputURL(String host, int port, int partition,
-      List<InputAttemptIdentifier> inputs) throws MalformedURLException {
-    StringBuilder url = ShuffleUtils.constructBaseURIForShuffleHandler(host,
-        port, partition, appId);
-    boolean first = true;
-    for (InputAttemptIdentifier input : inputs) {
-      if (first) {
-        first = false;
-        url.append(input.getPathComponent());
-      } else {
-        url.append(",").append(input.getPathComponent());
-      }
-    }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("InputFetch URL for: " + host + " : " + url.toString());
-    }
-    return new URL(url.toString());
-  }
-
-  /**
-   * Builder for the construction of Fetchers
-   */
-  public static class FetcherBuilder {
-    private Fetcher fetcher;
-    private boolean workAssigned = false;
-
-    public FetcherBuilder(FetcherCallback fetcherCallback,
-        FetchedInputAllocator inputManager, ApplicationId appId,
-        SecretKey shuffleSecret, Configuration conf) {
-      this.fetcher = new Fetcher(fetcherCallback, inputManager, appId,
-          shuffleSecret, conf);
-    }
-
-    public FetcherBuilder setCompressionParameters(CompressionCodec codec,
-        Decompressor decompressor) {
-      fetcher.codec = codec;
-      fetcher.decompressor = decompressor;
-      return this;
-    }
-
-    public FetcherBuilder setConnectionParameters(int connectionTimeout,
-        int readTimeout) {
-      fetcher.connectionTimeout = connectionTimeout;
-      fetcher.readTimeout = readTimeout;
-      return this;
-    }
-
-    public FetcherBuilder assignWork(String host, int port, int partition,
-        List<InputAttemptIdentifier> inputs) {
-      fetcher.host = host;
-      fetcher.port = port;
-      fetcher.partition = partition;
-      fetcher.srcAttempts = inputs;
-      workAssigned = true;
-      return this;
-    }
-
-    public Fetcher build() {
-      Preconditions.checkState(workAssigned == true,
-          "Cannot build a fetcher withot assigning work to it");
-      return fetcher;
-    }
-  }
-
-  @Override
-  public int hashCode() {
-    return fetcherIdentifier;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    Fetcher other = (Fetcher) obj;
-    if (fetcherIdentifier != other.fetcherIdentifier)
-      return false;
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetcherCallback.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetcherCallback.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetcherCallback.java
deleted file mode 100644
index f0b7cd2..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/FetcherCallback.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.engine.shuffle.common;
-
-import java.io.IOException;
-
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-public interface FetcherCallback {
-
-  public void fetchSucceeded(String host, InputAttemptIdentifier srcAttemptIdentifier, FetchedInput fetchedInput, long fetchedBytes, long copyDuration) throws IOException;
-  
-  public void fetchFailed(String host, InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/InputHost.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/InputHost.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/InputHost.java
deleted file mode 100644
index 4862b76..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/InputHost.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.engine.shuffle.common;
-
-import java.util.ArrayList;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-public class InputHost {
-
-  private final String host;
-  private final int port;
-
-  private final List<InputAttemptIdentifier> inputs = new LinkedList<InputAttemptIdentifier>();
-
-  public InputHost(String hostName, int port, ApplicationId appId) {
-    this.host = hostName;
-    this.port = port;
-  }
-
-  public String getHost() {
-    return this.host;
-  }
-
-  public int getPort() {
-    return this.port;
-  }
-
-  public synchronized int getNumPendingInputs() {
-    return inputs.size();
-  }
-  
-  public synchronized void addKnownInput(InputAttemptIdentifier srcAttempt) {
-    inputs.add(srcAttempt);
-  }
-
-  public synchronized List<InputAttemptIdentifier> clearAndGetPendingInputs() {
-    List<InputAttemptIdentifier> inputsCopy = new ArrayList<InputAttemptIdentifier>(
-        inputs);
-    inputs.clear();
-    return inputsCopy;
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((host == null) ? 0 : host.hashCode());
-    result = prime * result + port;
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    InputHost other = (InputHost) obj;
-    if (host == null) {
-      if (other.host != null)
-        return false;
-    } else if (!host.equals(other.host))
-      return false;
-    if (port != other.port)
-      return false;
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/MemoryFetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/MemoryFetchedInput.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/MemoryFetchedInput.java
deleted file mode 100644
index 59d288e..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/MemoryFetchedInput.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.shuffle.common;
-
-import java.io.ByteArrayInputStream;
-import java.io.InputStream;
-import java.io.OutputStream;
-
-import org.apache.hadoop.io.BoundedByteArrayOutputStream;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-import com.google.common.base.Preconditions;
-
-public class MemoryFetchedInput extends FetchedInput {
-
-  private BoundedByteArrayOutputStream byteStream;
-
-  public MemoryFetchedInput(long size,
-      InputAttemptIdentifier inputAttemptIdentifier,
-      FetchedInputCallback callbackHandler) {
-    super(Type.MEMORY, size, inputAttemptIdentifier, callbackHandler);
-    this.byteStream = new BoundedByteArrayOutputStream((int) size);
-  }
-
-  @Override
-  public OutputStream getOutputStream() {
-    return byteStream;
-  }
-
-  @Override
-  public InputStream getInputStream() {
-    return new ByteArrayInputStream(byteStream.getBuffer());
-  }
-
-  public byte[] getBytes() {
-    return byteStream.getBuffer();
-  }
-  
-  @Override
-  public void commit() {
-    if (state == State.PENDING) {
-      state = State.COMMITTED;
-      notifyFetchComplete();
-    }
-  }
-
-  @Override
-  public void abort() {
-    if (state == State.PENDING) {
-      state = State.ABORTED;
-      notifyFetchFailure();
-    }
-  }
-  
-  @Override
-  public void free() {
-    Preconditions.checkState(
-        state == State.COMMITTED || state == State.ABORTED,
-        "FetchedInput can only be freed after it is committed or aborted");
-    if (state == State.COMMITTED) {
-      state = State.FREED;
-      this.byteStream = null;
-      notifyFreedResource();
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "MemoryFetchedInput [inputAttemptIdentifier="
-        + inputAttemptIdentifier + ", size=" + size + ", type=" + type
-        + ", id=" + id + ", state=" + state + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/ShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/ShuffleUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/ShuffleUtils.java
deleted file mode 100644
index 84d270d..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/shuffle/common/ShuffleUtils.java
+++ /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.
- */
-
-package org.apache.tez.engine.shuffle.common;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import javax.crypto.SecretKey;
-
-import org.apache.hadoop.io.DataInputByteBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
-
-public class ShuffleUtils {
-
-  public static String SHUFFLE_HANDLER_SERVICE_ID = "mapreduce.shuffle";
-
-  public static SecretKey getJobTokenSecretFromTokenBytes(ByteBuffer meta)
-      throws IOException {
-    DataInputByteBuffer in = new DataInputByteBuffer();
-    in.reset(meta);
-    Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>();
-    jt.readFields(in);
-    SecretKey sk = JobTokenSecretManager.createSecretKey(jt.getPassword());
-    return sk;
-  }
-
-  public static ByteBuffer convertJobTokenToBytes(
-      Token<JobTokenIdentifier> jobToken) throws IOException {
-    DataOutputBuffer dob = new DataOutputBuffer();
-    jobToken.write(dob);
-    ByteBuffer bb = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
-    return bb;
-  }
-
-  public static int deserializeShuffleProviderMetaData(ByteBuffer meta)
-      throws IOException {
-    DataInputByteBuffer in = new DataInputByteBuffer();
-    try {
-      in.reset(meta);
-      int port = in.readInt();
-      return port;
-    } finally {
-      in.close();
-    }
-  }
-  
-  // TODO NEWTEZ handle ssl shuffle
-  public static StringBuilder constructBaseURIForShuffleHandler(String host, int port, int partition, ApplicationId appId) {
-    StringBuilder sb = new StringBuilder("http://");
-    sb.append(host);
-    sb.append(":");
-    sb.append(String.valueOf(port));
-    sb.append("/");
-    sb.append("mapOutput?job=");
-    sb.append(appId.toString().replace("application", "job"));
-    sb.append("&reduce=");
-    sb.append(String.valueOf(partition));
-    sb.append("&map=");
-    return sb;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/proto/Events.proto
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/proto/Events.proto b/tez-engine/src/main/proto/Events.proto
deleted file mode 100644
index fa9cb2c..0000000
--- a/tez-engine/src/main/proto/Events.proto
+++ /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.
- */
-
-option java_package = "org.apache.tez.engine.api.events";
-option java_outer_classname = "SystemEventProtos";
-option java_generate_equals_and_hash = true;
-
-message TaskAttemptFailedEventProto {
-  optional string diagnostics = 1;
-}
-
-message TaskAttemptCompletedEventProto {
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/proto/ShufflePayloads.proto
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/proto/ShufflePayloads.proto b/tez-engine/src/main/proto/ShufflePayloads.proto
deleted file mode 100644
index f831de2..0000000
--- a/tez-engine/src/main/proto/ShufflePayloads.proto
+++ /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.
- */
-
-option java_package = "org.apache.tez.engine.common.shuffle.newimpl";
-option java_outer_classname = "ShuffleUserPayloads";
-option java_generate_equals_and_hash = true;
-
-message DataMovementEventPayloadProto {
-  optional bool output_generated = 1;
-  optional string host = 2;
-  optional int32 port = 3;
-  optional string path_component = 4;
-  optional int32 run_duration = 5;
-} 
-
-message InputInformationEventPayloadProto {
-  optional int32 partition_range = 1;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/test/java/org/apache/tez/engine/common/objectregistry/TestObjectRegistry.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/test/java/org/apache/tez/engine/common/objectregistry/TestObjectRegistry.java b/tez-engine/src/test/java/org/apache/tez/engine/common/objectregistry/TestObjectRegistry.java
deleted file mode 100644
index 7276782..0000000
--- a/tez-engine/src/test/java/org/apache/tez/engine/common/objectregistry/TestObjectRegistry.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;
-
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-
-public class TestObjectRegistry {
-
-  @SuppressWarnings("unused")
-  @Before
-  public void setup() {
-    Injector injector = Guice.createInjector(new ObjectRegistryModule());
-  }
-
-  @Test
-  public void testBasicCRUD() {
-    ObjectRegistry objectRegistry =
-        ObjectRegistryFactory.getObjectRegistry();
-    Assert.assertNotNull(objectRegistry);
-
-    Assert.assertNull(objectRegistry.get("foo"));
-    Assert.assertFalse(objectRegistry.delete("foo"));
-    Integer one = new Integer(1);
-    Integer two_1 = new Integer(2);
-    Integer two_2 = new Integer(3);
-    Assert.assertNull(objectRegistry.add(ObjectLifeCycle.DAG, "one", one));
-    Assert.assertEquals(one, objectRegistry.get("one"));
-    Assert.assertNull(objectRegistry.add(ObjectLifeCycle.DAG, "two", two_1));
-    Assert.assertNotNull(objectRegistry.add(ObjectLifeCycle.SESSION, "two", two_2));
-    Assert.assertNotEquals(two_1, objectRegistry.get("two"));
-    Assert.assertEquals(two_2, objectRegistry.get("two"));
-    Assert.assertTrue(objectRegistry.delete("one"));
-    Assert.assertFalse(objectRegistry.delete("one"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 05675b5..e98b45f 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
@@ -77,17 +77,17 @@ import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
-import org.apache.tez.engine.common.objectregistry.ObjectLifeCycle;
-import org.apache.tez.engine.common.objectregistry.ObjectRegistry;
-import org.apache.tez.engine.common.objectregistry.ObjectRegistryFactory;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryFactory;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 ec419c1..cad79f5 100644
--- a/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
+++ b/tez-mapreduce-examples/src/main/java/org/apache/tez/mapreduce/examples/OrderedWordCount.java
@@ -70,14 +70,14 @@ import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
 import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 /**
  * An MRR job built on top of word count to return words sorted by

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 7e662cb..7280a1f 100644
--- a/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
+++ b/tez-mapreduce-tests/src/test/java/org/apache/tez/mapreduce/TestMRRJobsDAGApi.java
@@ -68,8 +68,6 @@ import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
 import org.apache.tez.dag.api.client.DAGStatus.State;
-import org.apache.tez.engine.lib.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.examples.MRRSleepJob;
 import org.apache.tez.mapreduce.examples.MRRSleepJob.ISleepReducer;
 import org.apache.tez.mapreduce.examples.MRRSleepJob.MRRSleepJobPartitioner;
@@ -82,6 +80,8 @@ import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
 import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml
index 2c6b78e..aa3d915 100644
--- a/tez-mapreduce/pom.xml
+++ b/tez-mapreduce/pom.xml
@@ -35,7 +35,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-engine</artifactId>
+      <artifactId>tez-runtime-library</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-internals</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 f2b0a38..22d4a75 100644
--- a/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
+++ b/tez-mapreduce/src/main/java/org/apache/hadoop/mapred/LocalJobRunnerTez.java
@@ -853,7 +853,7 @@
 //    LOG.info(TezJobConfig.LOCAL_DIRS + " for child : " + taskAttemptID +
 //        " is " + childMapredLocalDir);
 //    conf.set(TezJobConfig.LOCAL_DIRS, childMapredLocalDir.toString());
-//    conf.setClass(Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+//    conf.setClass(Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
 //        TezLocalTaskOutputFiles.class, TezTaskOutput.class);
 //  }
 //

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 199bbfe..3bc8da2 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/combine/MRCombiner.java
@@ -42,18 +42,18 @@ import org.apache.hadoop.util.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.mapreduce.hadoop.MRConfig;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezTaskContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.ValuesIterator;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
 
 @SuppressWarnings({"rawtypes", "unchecked"})
 public class MRCombiner implements Combiner {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java
index 21a3983..dac92ed 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/DeprecatedKeys.java
@@ -23,9 +23,10 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.runtime.library.common.Constants;
+
 import com.google.common.collect.Maps;
 
 public class DeprecatedKeys {
@@ -50,15 +51,15 @@ public class DeprecatedKeys {
   
   
   /**
-   * Keys used by the engine.
+   * Keys used by the Tez Runtime.
    */
-  private static Map<String, String> mrParamToEngineParamMap =
+  private static Map<String, String> mrParamToTezRuntimeParamMap =
       new HashMap<String, String>();
 
   
  
   static {
-    populateMRToEngineParamMap();
+    populateMRToTezRuntimeParamMap();
     populateMRToDagParamMap();
     populateMultiStageParamMap();
     addDeprecatedKeys();
@@ -70,32 +71,32 @@ public class DeprecatedKeys {
     multiStageParamMap.put(
         MRJobConfig.KEY_COMPARATOR,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_KEY_CLASS,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_CLASS,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_CLASS,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_VALUE_CLASS,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_VALUE_CLASS,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_VALUE_CLASS,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_COMPRESS,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_IS_COMPRESSED,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_IS_COMPRESSED,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS));
     
     multiStageParamMap.put(
         MRJobConfig.MAP_OUTPUT_COMPRESS_CODEC,
         getDeprecationMap(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_COMPRESS_CODEC,
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_COMPRESS_CODEC));
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_COMPRESS_CODEC,
+            TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_COMPRESS_CODEC));
   }
   
   private static Map<MultiStageKeys, String> getDeprecationMap(String inputKey, String outputKey) {
@@ -130,23 +131,23 @@ public class DeprecatedKeys {
   public static void init() {
   }
   
-  private static void populateMRToEngineParamMap() {
+  private static void populateMRToTezRuntimeParamMap() {
     
-    registerMRToEngineKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD, TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD);
+    registerMRToRuntimeKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD, TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD);
 
-    registerMRToEngineKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD_BYTES, TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD_BYTES);
+    registerMRToRuntimeKeyTranslation(MRConfig.MAPRED_IFILE_READAHEAD_BYTES, TezJobConfig.TEZ_RUNTIME_IFILE_READAHEAD_BYTES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.RECORDS_BEFORE_PROGRESS, TezJobConfig.RECORDS_BEFORE_PROGRESS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.RECORDS_BEFORE_PROGRESS, TezJobConfig.RECORDS_BEFORE_PROGRESS);
 
-    registerMRToEngineKeyTranslation(MRJobConfig.IO_SORT_FACTOR, TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.IO_SORT_FACTOR, TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.MAP_SORT_SPILL_PERCENT, TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.MAP_SORT_SPILL_PERCENT, TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.IO_SORT_MB, TezJobConfig.TEZ_ENGINE_IO_SORT_MB);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.IO_SORT_MB, TezJobConfig.TEZ_RUNTIME_IO_SORT_MB);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.INDEX_CACHE_MEMORY_LIMIT, TezJobConfig.TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.INDEX_CACHE_MEMORY_LIMIT, TezJobConfig.TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.MAP_COMBINE_MIN_SPILLS, TezJobConfig.TEZ_ENGINE_COMBINE_MIN_SPILLS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.MAP_COMBINE_MIN_SPILLS, TezJobConfig.TEZ_RUNTIME_COMBINE_MIN_SPILLS);
     
     // Counter replacement will work in this manner, as long as TezCounters
     // extends MRCounters and is used directly by the Mapper/Reducer.
@@ -154,56 +155,56 @@ public class DeprecatedKeys {
     // may break.
     // Framework counters, like FILESYSTEM will likely be incompatible since
     // they enum key belongs to a different package.
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTERS_MAX_KEY, TezJobConfig.COUNTERS_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTERS_MAX_KEY, TezJobConfig.COUNTERS_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTER_GROUP_NAME_MAX_KEY, TezJobConfig.COUNTER_GROUP_NAME_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTER_GROUP_NAME_MAX_KEY, TezJobConfig.COUNTER_GROUP_NAME_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTER_NAME_MAX_KEY, TezJobConfig.COUNTER_NAME_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTER_NAME_MAX_KEY, TezJobConfig.COUNTER_NAME_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.COUNTER_GROUPS_MAX_KEY, TezJobConfig.COUNTER_GROUPS_MAX_KEY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.COUNTER_GROUPS_MAX_KEY, TezJobConfig.COUNTER_GROUPS_MAX_KEY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_MEMORY_TOTAL_BYTES, Constants.TEZ_ENGINE_TASK_MEMORY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_MEMORY_TOTAL_BYTES, Constants.TEZ_RUNTIME_TASK_MEMORY);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_PARALLEL_COPIES, TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_PARALLEL_COPIES, TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_FETCH_FAILURES, TezJobConfig.TEZ_ENGINE_SHUFFLE_FETCH_FAILURES);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_FETCH_FAILURES, TezJobConfig.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_NOTIFY_READERROR, TezJobConfig.TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_NOTIFY_READERROR, TezJobConfig.TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_CONNECT_TIMEOUT, TezJobConfig.TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_CONNECT_TIMEOUT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_READ_TIMEOUT, TezJobConfig.TEZ_ENGINE_SHUFFLE_READ_TIMEOUT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_READ_TIMEOUT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT);
     
-    registerMRToEngineKeyTranslation(MRConfig.SHUFFLE_SSL_ENABLED_KEY, TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL);
+    registerMRToRuntimeKeyTranslation(MRConfig.SHUFFLE_SSL_ENABLED_KEY, TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_MEMORY_LIMIT_PERCENT, TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_MEMORY_LIMIT_PERCENT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.SHUFFLE_MERGE_PERCENT, TezJobConfig.TEZ_ENGINE_SHUFFLE_MERGE_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.SHUFFLE_MERGE_PERCENT, TezJobConfig.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_THRESHOLD, TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_THRESHOLD, TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_ENABLED, TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_MEMTOMEM_ENABLED, TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.REDUCE_INPUT_BUFFER_PERCENT, TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT);
 
-    registerMRToEngineKeyTranslation(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY, TezJobConfig.DAG_CREDENTIALS_BINARY);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY, TezJobConfig.DAG_CREDENTIALS_BINARY);
     
-    registerMRToEngineKeyTranslation("map.sort.class", TezJobConfig.TEZ_ENGINE_INTERNAL_SORTER_CLASS);
+    registerMRToRuntimeKeyTranslation("map.sort.class", TezJobConfig.TEZ_RUNTIME_INTERNAL_SORTER_CLASS);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_ENGINE_GROUP_COMPARATOR_CLASS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_RUNTIME_GROUP_COMPARATOR_CLASS);
     
-    registerMRToEngineKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS);
+    registerMRToRuntimeKeyTranslation(MRJobConfig.GROUP_COMPARATOR_CLASS, TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS);
 
   }
   
   private static void addDeprecatedKeys() {
   }
 
-  private static void registerMRToEngineKeyTranslation(String mrKey,
+  private static void registerMRToRuntimeKeyTranslation(String mrKey,
       String tezKey) {
-    mrParamToEngineParamMap.put(mrKey, tezKey);
+    mrParamToTezRuntimeParamMap.put(mrKey, tezKey);
   }
   
   @SuppressWarnings("unused")
@@ -215,8 +216,8 @@ public class DeprecatedKeys {
     return Collections.unmodifiableMap(mrParamToDAGParamMap);
   }
 
-  public static Map<String, String> getMRToEngineParamMap() {
-    return Collections.unmodifiableMap(mrParamToEngineParamMap);
+  public static Map<String, String> getMRToTezRuntimeParamMap() {
+    return Collections.unmodifiableMap(mrParamToTezRuntimeParamMap);
   }
 
   // TODO Ideally, multi-stage should not be exposed.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 b0ed6ab..c39ca4a 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRHelpers.java
@@ -367,16 +367,16 @@ public class MRHelpers {
     // the AM anyway.
 
     // TODO eventually ACLs
-    conf.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
+    conf.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS, MRPartitioner.class.getName());
     
     boolean useNewApi = conf.getBoolean("mapred.mapper.new-api", false);
     if (useNewApi) {
       if (conf.get(MRJobConfig.COMBINE_CLASS_ATTR) != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     } else {
       if (conf.get("mapred.combiner.class") != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 d888c42..b07b04b 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfToTezTranslator.java
@@ -221,7 +221,7 @@ public class MultiStageMRConfToTezTranslator {
     int numStages = numIntermediateStages + (hasFinalReduceStage ? 2 : 1);
 
     // Setup Tez partitioner class
-    conf.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS,
+    conf.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS,
         MRPartitioner.class.getName());
     
     // Setup Tez Combiner class if required.
@@ -229,11 +229,11 @@ public class MultiStageMRConfToTezTranslator {
     boolean useNewApi = conf.getBoolean("mapred.mapper.new-api", false);
     if (useNewApi) {
       if (conf.get(MRJobConfig.COMBINE_CLASS_ATTR) != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     } else {
       if (conf.get("mapred.combiner.class") != null) {
-        conf.set(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS, MRCombiner.class.getName());
+        conf.set(TezJobConfig.TEZ_RUNTIME_COMBINER_CLASS, MRCombiner.class.getName());
       }
     }
 
@@ -259,7 +259,7 @@ public class MultiStageMRConfToTezTranslator {
   }
 
   private static void processDirectConversion(Configuration conf) {
-    for (Entry<String, String> dep : DeprecatedKeys.getMRToEngineParamMap()
+    for (Entry<String, String> dep : DeprecatedKeys.getMRToTezRuntimeParamMap()
         .entrySet()) {
       if (conf.get(dep.getKey()) != null) {
         // TODO Deprecation reason does not seem to reflect in the config ?
@@ -336,7 +336,7 @@ public class MultiStageMRConfToTezTranslator {
       Configuration baseConf, String stage) {
     JobConf jobConf = new JobConf(baseConf);
     // Don't clobber explicit tez config.
-    if (conf.get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS) == null) {
+    if (conf.get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS) == null) {
       // If this is set, but the comparator is not set, and their types differ -
       // the job will break.
       if (conf.get(MRJobConfig.MAP_OUTPUT_KEY_CLASS) == null) {
@@ -352,7 +352,7 @@ public class MultiStageMRConfToTezTranslator {
       }
     }
 
-    if (conf.get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS) == null) {
+    if (conf.get(TezJobConfig.TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS) == null) {
       if (conf.get(MRJobConfig.MAP_OUTPUT_VALUE_CLASS) == null) {
         conf.set(MRJobConfig.MAP_OUTPUT_VALUE_CLASS, jobConf
             .getMapOutputValueClass().getName());

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java
index 0bcd45e..2378f58 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MultiStageMRConfigUtil.java
@@ -57,7 +57,7 @@ public class MultiStageMRConfigUtil {
   public static Configuration getAndRemoveBasicNonIntermediateStageConf(
       Configuration baseConf) {
     Configuration newConf = new Configuration(false);
-    for (String key : DeprecatedKeys.getMRToEngineParamMap().keySet()) {
+    for (String key : DeprecatedKeys.getMRToTezRuntimeParamMap().keySet()) {
       if (baseConf.get(key) != null) {
         newConf.set(key, baseConf.get(key));
         baseConf.unset(key);

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 2a926d7..635af90 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapred/MRReporter.java
@@ -3,9 +3,9 @@ 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.api.TezProcessorContext;
-import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.common.Utils;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 public class MRReporter implements Reporter {
 

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

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 4035c71..be65be7 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/mapreduce/TaskAttemptContextImpl.java
@@ -24,8 +24,8 @@ import org.apache.hadoop.mapreduce.Counter;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
-import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.common.Utils;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 // NOTE: NEWTEZ: This is a copy of org.apache.tez.mapreduce.hadoop.mapred (not mapreduce). mapred likely does not need it's own copy of this class.
 // Meant for use by the "mapreduce" API

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

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
index 6066d93..b9f2242 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MRInput.java
@@ -46,13 +46,14 @@ import org.apache.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.api.LogicalInput;
-import org.apache.tez.engine.api.TezInputContext;
 import org.apache.tez.mapreduce.common.Utils;
 import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
 import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.Input;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.api.KVReader;
 
 import com.google.common.base.Preconditions;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
index e6bdbe6..11184e4 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
@@ -26,16 +26,16 @@ import org.apache.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.api.LogicalOutput;
-import org.apache.tez.engine.api.TezOutputContext;
 import org.apache.tez.mapreduce.common.Utils;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
 import org.apache.tez.mapreduce.hadoop.mapreduce.TaskAttemptContextImpl;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.api.KVWriter;
 
 public class MROutput implements LogicalOutput {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 d061ad5..224900e 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/partition/MRPartitioner.java
@@ -24,11 +24,11 @@ import org.apache.hadoop.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;
+import org.apache.tez.runtime.library.common.ConfigUtils;
 
 @SuppressWarnings({ "rawtypes", "unchecked" })
-public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
+public class MRPartitioner implements org.apache.tez.runtime.library.api.Partitioner {
 
   static final Log LOG = LogFactory.getLog(MRPartitioner.class);
 
@@ -40,7 +40,7 @@ public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
 
   public MRPartitioner(Configuration conf) {
     this.useNewApi = ConfigUtils.useNewApi(conf);
-    this.partitions = conf.getInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, 1);
+    this.partitions = conf.getInt(TezJobConfig.TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS, 1);
 
     if (useNewApi) {
       if (partitions > 1) {
@@ -85,4 +85,4 @@ public class MRPartitioner implements org.apache.tez.engine.api.Partitioner {
       return oldPartitioner.getPartition(key, value, numPartitions);
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 fac1454..5471c55 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTask.java
@@ -62,7 +62,6 @@ import org.apache.hadoop.util.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;
@@ -70,11 +69,6 @@ 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.mapreduce.hadoop.DeprecatedKeys;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
 import org.apache.tez.mapreduce.hadoop.MRConfig;
@@ -82,6 +76,12 @@ import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.mapred.TaskAttemptContextImpl;
 import org.apache.tez.mapreduce.hadoop.mapreduce.JobContextImpl;
 import org.apache.tez.mapreduce.output.MROutput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.TokenCache;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
 
 @SuppressWarnings("deprecation")
 public abstract class MRTask {
@@ -155,7 +155,7 @@ public abstract class MRTask {
     } else {
       this.jobConf = new JobConf(conf);
     }
-    jobConf.set(Constants.TEZ_ENGINE_TASK_ATTEMPT_ID,
+    jobConf.set(Constants.TEZ_RUNTIME_TASK_ATTEMPT_ID,
         taskAttemptId.toString());
     jobConf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID,
         context.getDAGAttemptNumber());
@@ -687,7 +687,4 @@ public abstract class MRTask {
     return taskAttemptId;
   }
 
-  public TezProcessorContext getTezEngineTaskContext() {
-    return processorContext;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 85139ed..74a34af 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/MRTaskReporter.java
@@ -24,12 +24,12 @@ import org.apache.hadoop.mapred.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.api.TezInputContext;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.api.TezTaskContext;
 import org.apache.tez.mapreduce.hadoop.mapred.MRCounters;
 import org.apache.tez.mapreduce.hadoop.mapred.MRReporter;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.TezTaskContext;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 e4b990a..b7ecddd 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/map/MapProcessor.java
@@ -35,20 +35,20 @@ import org.apache.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.mapreduce.hadoop.mapreduce.MapContextImpl;
 import org.apache.tez.mapreduce.input.MRInput;
 import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalIOProcessor;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 @SuppressWarnings({ "unchecked", "rawtypes" })
 public class MapProcessor extends MRTask implements LogicalIOProcessor {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 19acb39..1ba76f6 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/processor/reduce/ReduceProcessor.java
@@ -37,20 +37,20 @@ import org.apache.hadoop.util.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.input.ShuffledMergedInputLegacy;
-import org.apache.tez.engine.lib.output.OnFileSortedOutput;
 import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.processor.MRTask;
 import org.apache.tez.mapreduce.processor.MRTaskReporter;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.LogicalIOProcessor;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.api.KVWriter;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.input.ShuffledMergedInputLegacy;
+import org.apache.tez.runtime.library.output.OnFileSortedOutput;
 
 
 @SuppressWarnings({ "unchecked", "rawtypes" })

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
index 9de2ed1..08b66eb 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/TestUmbilical.java
@@ -24,9 +24,9 @@ import java.util.Collection;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.api.impl.EventMetaData;
-import org.apache.tez.engine.api.impl.TezEvent;
-import org.apache.tez.engine.api.impl.TezUmbilical;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
 
 public class TestUmbilical implements TezUmbilical {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java
index b75f01e..5e3d201 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestConfigTranslationMRToTez.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.tez.engine.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.ConfigUtils;
 import org.junit.Test;
 
 public class TestConfigTranslationMRToTez {


[13/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 1bf17a3..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/PipelinedSorter.java
+++ /dev/null
@@ -1,932 +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.impl;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.BufferOverflowException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.IntBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.PriorityQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-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.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.HashComparator;
-import org.apache.hadoop.io.RawComparator;
-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;
-
-@SuppressWarnings({"unchecked", "rawtypes"})
-public class PipelinedSorter extends ExternalSorter {
-  
-  private static final Log LOG = LogFactory.getLog(PipelinedSorter.class);
-  
-  /**
-   * The size of each record in the index file for the map-outputs.
-   */
-  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
-
-  private final static int APPROX_HEADER_LENGTH = 150;
-    
-  int partitionBits;
-  
-  private static final int PARTITION = 0;        // partition offset in acct
-  private static final int KEYSTART = 1;         // key offset in acct
-  private static final int VALSTART = 2;         // val offset in acct
-  private static final int VALLEN = 3;           // val len in acct
-  private static final int NMETA = 4;            // num meta ints
-  private static final int METASIZE = NMETA * 4; // size in bytes
-
-  // spill accounting
-  volatile Throwable sortSpillException = null;
-
-  int numSpills = 0;
-  int minSpillsForCombine;
-  private HashComparator hasher;
-  // SortSpans  
-  private SortSpan span;
-  private ByteBuffer largeBuffer;
-  // Merger
-  private SpanMerger merger; 
-  private ExecutorService sortmaster;
-
-  final ArrayList<TezSpillRecord> indexCacheList =
-    new ArrayList<TezSpillRecord>();
-  private int totalIndexCacheMemory;
-  private int indexCacheMemoryLimit;
-
-  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
-    super.initialize(outputContext, conf, numOutputs);
-    
-    partitionBits = bitcount(partitions)+1;
-   
-    //sanity checks
-    final float spillper =
-      this.conf.getFloat(
-          TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT, 
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SORT_SPILL_PERCENT);
-    final int sortmb = 
-        this.conf.getInt(
-            TezJobConfig.TEZ_ENGINE_IO_SORT_MB, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_MB);
-    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES,
-                                       TezJobConfig.DEFAULT_TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES);
-    if (spillper > (float)1.0 || spillper <= (float)0.0) {
-      throw new IOException("Invalid \"" + TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT +
-          "\": " + spillper);
-    }
-    if ((sortmb & 0x7FF) != sortmb) {
-      throw new IOException(
-          "Invalid \"" + TezJobConfig.TEZ_ENGINE_IO_SORT_MB + "\": " + sortmb);
-    }
-    
-    // buffers and accounting
-    int maxMemUsage = sortmb << 20;
-    maxMemUsage -= maxMemUsage % METASIZE;
-    largeBuffer = ByteBuffer.allocate(maxMemUsage);
-    LOG.info(TezJobConfig.TEZ_ENGINE_IO_SORT_MB + " = " + sortmb);
-    // TODO: configurable setting?
-    span = new SortSpan(largeBuffer, 1024*1024, 16);
-    merger = new SpanMerger(comparator);
-    final int sortThreads = 
-            this.conf.getInt(
-                TezJobConfig.TEZ_ENGINE_SORT_THREADS, 
-                TezJobConfig.DEFAULT_TEZ_ENGINE_SORT_THREADS);
-    sortmaster = Executors.newFixedThreadPool(sortThreads);
-
-    // k/v serialization    
-    if(comparator instanceof HashComparator) {
-      hasher = (HashComparator)comparator;
-      LOG.info("Using the HashComparator");
-    } else {
-      hasher = null;
-    }    
-    valSerializer.open(span.out);
-    keySerializer.open(span.out);
-    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_ENGINE_COMBINE_MIN_SPILLS, 3);
-  }
-
-  private int bitcount(int n) {
-    int bit = 0;
-    while(n!=0) {
-      bit++;
-      n >>= 1;
-    }
-    return bit;
-  }
-  
-  public void sort() throws IOException {
-    SortSpan newSpan = span.next();
-
-    if(newSpan == null) {
-      // sort in the same thread, do not wait for the thread pool
-      merger.add(span.sort(sorter, comparator));
-      spill();
-      int items = 1024*1024;
-      int perItem = 16;
-      if(span.length() != 0) {
-        items = span.length();
-        perItem = span.kvbuffer.limit()/items;
-        items = (largeBuffer.capacity())/(METASIZE+perItem);
-        if(items > 1024*1024) {
-            // our goal is to have 1M splits and sort early
-            items = 1024*1024;
-        }
-      }      
-      span = new SortSpan(largeBuffer, items, perItem);
-    } else {
-      // queue up the sort
-      SortTask task = new SortTask(span, sorter, comparator);
-      Future<SpanIterator> future = sortmaster.submit(task);
-      merger.add(future);
-      span = newSpan;
-    }
-    valSerializer.open(span.out);
-    keySerializer.open(span.out);
-  }
-
-  @Override
-  public void write(Object key, Object value) 
-      throws IOException {
-    collect(
-        key, value, partitioner.getPartition(key, value, partitions));
-  }
-
-  /**
-   * Serialize the key, value to intermediate storage.
-   * When this method returns, kvindex must refer to sufficient unused
-   * storage to store one METADATA.
-   */
-  synchronized void collect(Object key, Object value, final int partition
-                                   ) throws IOException {
-    if (key.getClass() != keyClass) {
-      throw new IOException("Type mismatch in key from map: expected "
-                            + keyClass.getName() + ", received "
-                            + key.getClass().getName());
-    }
-    if (value.getClass() != valClass) {
-      throw new IOException("Type mismatch in value from map: expected "
-                            + valClass.getName() + ", received "
-                            + value.getClass().getName());
-    }
-    if (partition < 0 || partition >= partitions) {
-      throw new IOException("Illegal partition for " + key + " (" +
-          partition + ")");
-    }
-    if(span.kvmeta.remaining() < METASIZE) {
-      this.sort();
-    }
-    int keystart = span.kvbuffer.position();
-    int valstart = -1;
-    int valend = -1;
-    try { 
-      keySerializer.serialize(key);
-      valstart = span.kvbuffer.position();      
-      valSerializer.serialize(value);
-      valend = span.kvbuffer.position();
-    } catch(BufferOverflowException overflow) {
-      // restore limit
-      span.kvbuffer.position(keystart);
-      this.sort();
-      // try again
-      this.collect(key, value, partition);
-      return;
-    }
-
-    int prefix = 0;
-
-    if(hasher != null) {
-      prefix = hasher.getHashCode(key);
-    }
-
-    prefix = (partition << (32 - partitionBits)) | (prefix >>> partitionBits);
-
-    /* maintain order as in PARTITION, KEYSTART, VALSTART, VALLEN */
-    span.kvmeta.put(prefix);
-    span.kvmeta.put(keystart);
-    span.kvmeta.put(valstart);
-    span.kvmeta.put(valend - valstart);
-    if((valstart - keystart) > span.keymax) {
-      span.keymax = (valstart - keystart);
-    }
-    if((valend - valstart) > span.valmax) {
-      span.valmax = (valend - valstart);
-    }
-    mapOutputRecordCounter.increment(1);
-    mapOutputByteCounter.increment(valend - keystart);
-  }
-
-  public void spill() throws IOException { 
-    // create spill file
-    final long size = largeBuffer.capacity() + 
-      (partitions * APPROX_HEADER_LENGTH);
-    final TezSpillRecord spillRec = new TezSpillRecord(partitions);
-    final Path filename =
-      mapOutputFile.getSpillFileForWrite(numSpills, size);    
-    FSDataOutputStream out = rfs.create(filename, true, 4096);
-
-    try {
-      merger.ready(); // wait for all the future results from sort threads
-      LOG.info("Spilling to " + filename.toString());
-      for (int i = 0; i < partitions; ++i) {
-        TezRawKeyValueIterator kvIter = merger.filter(i);
-        //write merged output to disk
-        long segmentStart = out.getPos();
-        Writer writer =
-          new Writer(conf, out, keyClass, valClass, codec,
-              spilledRecordsCounter);
-        writer.setRLE(merger.needsRLE());
-        if (combiner == null) {
-          while(kvIter.next()) {
-            writer.append(kvIter.getKey(), kvIter.getValue());
-          }
-        } else {          
-          runCombineProcessor(kvIter, writer);
-        }
-        //close
-        writer.close();
-
-        // record offsets
-        final TezIndexRecord rec = 
-            new TezIndexRecord(
-                segmentStart, 
-                writer.getRawLength(), 
-                writer.getCompressedLength());
-        spillRec.putIndex(rec, i);
-      }
-
-      Path indexFilename =
-        mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
-            * MAP_OUTPUT_INDEX_RECORD_LENGTH);
-      // TODO: cache
-      spillRec.writeToFile(indexFilename, conf);
-      ++numSpills;
-    } catch(InterruptedException ie) {
-      // TODO:the combiner has been interrupted
-    } finally {
-      out.close();
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-    final String uniqueIdentifier = outputContext.getUniqueIdentifier();
-    Path finalOutputFile =
-        mapOutputFile.getOutputFileForWrite(0); //TODO
-    Path finalIndexFile =
-        mapOutputFile.getOutputIndexFileForWrite(0); //TODO
-
-    LOG.info("Starting flush of map output");
-    span.end();
-    merger.add(span.sort(sorter, comparator));
-    spill();
-    sortmaster.shutdown();
-
-    largeBuffer = null;
-
-    if(numSpills == 1) {
-      // someday be able to pass this directly to shuffle
-      // without writing to disk
-      final Path filename =
-          mapOutputFile.getSpillFile(0);
-      Path indexFilename =
-              mapOutputFile.getSpillIndexFile(0);
-      sameVolRename(filename, finalOutputFile);
-      sameVolRename(indexFilename, finalIndexFile);
-      return;
-    }
-    
-    //The output stream for the final single output file
-    FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
-
-    TezMerger.considerFinalMergeForProgress();
-
-    final TezSpillRecord spillRec = new TezSpillRecord(partitions);
-    final ArrayList<TezSpillRecord> indexCacheList = new ArrayList<TezSpillRecord>();
-
-    for(int i = 0; i < numSpills; i++) {
-      // TODO: build this cache before
-      Path indexFilename = mapOutputFile.getSpillIndexFile(i);
-      TezSpillRecord spillIndex = new TezSpillRecord(indexFilename, conf);
-      indexCacheList.add(spillIndex);
-    }
-    
-    for (int parts = 0; parts < partitions; parts++) {
-      //create the segments to be merged
-      List<Segment> segmentList =
-          new ArrayList<Segment>(numSpills);
-      for(int i = 0; i < numSpills; i++) {
-        Path spillFilename = mapOutputFile.getSpillFile(i);
-        TezIndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
-
-        Segment s =
-            new Segment(conf, rfs, spillFilename, indexRecord.getStartOffset(),
-                             indexRecord.getPartLength(), codec, true);
-        segmentList.add(i, s);
-      }
-
-      int mergeFactor = 
-              this.conf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 
-                  TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
-      // sort the segments only if there are intermediate merges
-      boolean sortSegments = segmentList.size() > mergeFactor;
-      //merge
-      TezRawKeyValueIterator kvIter = TezMerger.merge(conf, rfs,
-                     keyClass, valClass, codec,
-                     segmentList, mergeFactor,
-                     new Path(uniqueIdentifier),
-                     (RawComparator)ConfigUtils.getIntermediateOutputKeyComparator(conf), 
-                     nullProgressable, sortSegments,
-                     null, spilledRecordsCounter,
-                     null); // Not using any Progress in TezMerger. Should just work.
-
-      //write merged output to disk
-      long segmentStart = finalOut.getPos();
-      Writer writer =
-          new Writer(conf, finalOut, keyClass, valClass, codec,
-                           spilledRecordsCounter);
-      writer.setRLE(merger.needsRLE());
-      if (combiner == null || numSpills < minSpillsForCombine) {
-        TezMerger.writeFile(kvIter, writer, nullProgressable, conf);
-      } else {
-        runCombineProcessor(kvIter, writer);
-      }
-
-      //close
-      writer.close();
-
-      // record offsets
-      final TezIndexRecord rec = 
-          new TezIndexRecord(
-              segmentStart, 
-              writer.getRawLength(), 
-              writer.getCompressedLength());
-      spillRec.putIndex(rec, parts);
-    }
-
-    spillRec.writeToFile(finalIndexFile, conf);
-    finalOut.close();
-    for(int i = 0; i < numSpills; i++) {
-      Path indexFilename = mapOutputFile.getSpillIndexFile(i);
-      Path spillFilename = mapOutputFile.getSpillFile(i);
-      rfs.delete(indexFilename,true);
-      rfs.delete(spillFilename,true);
-    }
-  }
-
-  public void close() { }
-
-  private interface PartitionedRawKeyValueIterator extends TezRawKeyValueIterator {
-    int getPartition();
-  }
-
-  private class BufferStreamWrapper extends OutputStream 
-  {
-    private final ByteBuffer out;
-    public BufferStreamWrapper(ByteBuffer out) {
-      this.out = out;
-    }
-    
-    @Override
-    public void write(int b) throws IOException { out.put((byte)b); }
-    @Override
-    public void write(byte[] b) throws IOException { out.put(b); }
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException { out.put(b, off, len); }
-  }
-
-  protected class InputByteBuffer extends DataInputBuffer {
-    private byte[] buffer = new byte[256]; 
-    private ByteBuffer wrapped = ByteBuffer.wrap(buffer);
-    private void resize(int length) {
-      if(length > buffer.length) {
-        buffer = new byte[length];
-        wrapped = ByteBuffer.wrap(buffer);
-      }
-      wrapped.limit(length);
-    }
-    public void reset(ByteBuffer b, int start, int length) {
-      resize(length);
-      b.position(start);
-      b.get(buffer, 0, length);
-      super.reset(buffer, 0, length);
-    }
-    // clone-ish function
-    public void reset(DataInputBuffer clone) {
-      byte[] data = clone.getData();
-      int start = clone.getPosition();
-      int length = clone.getLength();
-      resize(length);
-      System.arraycopy(data, start, buffer, 0, length);
-      super.reset(buffer, 0, length);
-    }
-  }
-
-  private class SortSpan  implements IndexedSortable {
-    final IntBuffer kvmeta;
-    final ByteBuffer kvbuffer;
-    final DataOutputStream out;    
-    private RawComparator comparator; 
-    final int imeta[] = new int[NMETA];
-    final int jmeta[] = new int[NMETA];
-    int keymax = 1;
-    int valmax = 1;
-    private int i,j;
-    private byte[] ki;
-    private byte[] kj;
-    private int index = 0;
-    private InputByteBuffer hay = new InputByteBuffer();
-    private long eq = 0;
-
-    public SortSpan(ByteBuffer source, int maxItems, int perItem) {
-      int capacity = source.remaining(); 
-      int metasize = METASIZE*maxItems;
-      int dataSize = maxItems * perItem;
-      if(capacity < (metasize+dataSize)) {
-        // try to allocate less meta space, because we have sample data
-        metasize = METASIZE*(capacity/(perItem+METASIZE));
-      }
-      ByteBuffer reserved = source.duplicate();
-      reserved.mark();
-      LOG.info("reserved.remaining() = "+reserved.remaining());
-      LOG.info("reserved.size = "+metasize);
-      reserved.position(metasize);
-      kvbuffer = reserved.slice();
-      reserved.flip();
-      reserved.limit(metasize);
-      kvmeta = reserved
-                .slice()
-                .order(ByteOrder.nativeOrder())
-               .asIntBuffer();
-      out = new DataOutputStream(
-              new BufferStreamWrapper(kvbuffer));
-    }
-
-    public SpanIterator sort(IndexedSorter sorter, RawComparator comparator) {
-    	this.comparator = comparator;
-      ki = new byte[keymax];
-      kj = new byte[keymax];
-      LOG.info("begin sorting Span"+index + " ("+length()+")");
-      if(length() > 1) {
-        sorter.sort(this, 0, length(), nullProgressable);
-      }
-      LOG.info("done sorting Span"+index);
-      return new SpanIterator(this);
-    }
-
-    int offsetFor(int i) {
-      return (i * NMETA);
-    }
-
-    public void swap(final int mi, final int mj) {
-      final int kvi = offsetFor(mi);
-      final int kvj = offsetFor(mj);
-
-      kvmeta.position(kvi); kvmeta.get(imeta);
-      kvmeta.position(kvj); kvmeta.get(jmeta);
-      kvmeta.position(kvj); kvmeta.put(imeta);
-      kvmeta.position(kvi); kvmeta.put(jmeta);
-
-      if(i == mi || j == mj) i = -1;
-      if(i == mi || j == mj) j = -1;
-    }
-
-    public int compare(final int mi, final int mj) {
-      final int kvi = offsetFor(mi);
-      final int kvj = offsetFor(mj);
-      final int kvip = kvmeta.get(kvi + PARTITION);
-      final int kvjp = kvmeta.get(kvj + PARTITION);
-      // sort by partition      
-      if (kvip != kvjp) {
-        return kvip - kvjp;
-      }
-      
-      final int istart = kvmeta.get(kvi + KEYSTART);
-      final int jstart = kvmeta.get(kvj + KEYSTART);
-      final int ilen   = kvmeta.get(kvi + VALSTART) - istart;
-      final int jlen   = kvmeta.get(kvj + VALSTART) - jstart;
-
-      kvbuffer.position(istart);
-      kvbuffer.get(ki, 0, ilen);
-      kvbuffer.position(jstart);
-      kvbuffer.get(kj, 0, jlen);
-      // sort by key
-      final int cmp = comparator.compare(ki, 0, ilen, kj, 0, jlen);
-      if(cmp == 0) eq++;
-      return cmp;
-    }
-
-    public SortSpan next() {
-      ByteBuffer remaining = end();
-      if(remaining != null) {
-        int items = length();
-        int perItem = kvbuffer.position()/items;
-        SortSpan newSpan = new SortSpan(remaining, items, perItem);
-        newSpan.index = index+1;
-        return newSpan;
-      }
-      return null;
-    }
-
-    public int length() {
-      return kvmeta.limit()/NMETA;
-    }
-
-    public ByteBuffer end() {
-      ByteBuffer remaining = kvbuffer.duplicate();
-      remaining.position(kvbuffer.position());
-      remaining = remaining.slice();
-      kvbuffer.limit(kvbuffer.position());
-      kvmeta.limit(kvmeta.position());
-      int items = length();
-      if(items == 0) {
-        return null;
-      }
-      int perItem = kvbuffer.position()/items;
-      LOG.info(String.format("Span%d.length = %d, perItem = %d", index, length(), perItem));
-      if(remaining.remaining() < NMETA+perItem) {
-        return null;
-      }
-      return remaining;
-    }
-
-    private int compareInternal(DataInputBuffer needle, int needlePart, int index) {
-      int cmp = 0;
-      int keystart;
-      int valstart;
-      int partition;
-      partition = kvmeta.get(span.offsetFor(index) + PARTITION);
-      if(partition != needlePart) {
-          cmp = (partition-needlePart);
-      } else {
-        keystart = kvmeta.get(span.offsetFor(index) + KEYSTART);
-        valstart = kvmeta.get(span.offsetFor(index) + VALSTART);
-        // hay is allocated ahead of time
-        hay.reset(kvbuffer, keystart, valstart - keystart);
-        cmp = comparator.compare(hay.getData(), 
-            hay.getPosition(), hay.getLength(),
-            needle.getData(), 
-            needle.getPosition(), needle.getLength());
-      }
-      return cmp;
-    }
-    
-    public long getEq() {
-      return eq;
-    }
-    
-    @Override
-    public String toString() {
-        return String.format("Span[%d,%d]", NMETA*kvmeta.capacity(), kvbuffer.limit());
-    }
-  }
-
-  private class SpanIterator implements PartitionedRawKeyValueIterator, Comparable<SpanIterator> {
-    private int kvindex = -1;
-    private int maxindex;
-    private IntBuffer kvmeta;
-    private ByteBuffer kvbuffer;
-    private SortSpan span;
-    private InputByteBuffer key = new InputByteBuffer();
-    private InputByteBuffer value = new InputByteBuffer();
-    private Progress progress = new Progress();
-
-    private final int minrun = (1 << 4);
-
-    public SpanIterator(SortSpan span) {
-      this.kvmeta = span.kvmeta;
-      this.kvbuffer = span.kvbuffer;
-      this.span = span;
-      this.maxindex = (kvmeta.limit()/NMETA) - 1;
-    }
-
-    public DataInputBuffer getKey() throws IOException {
-      final int keystart = kvmeta.get(span.offsetFor(kvindex) + KEYSTART);
-      final int valstart = kvmeta.get(span.offsetFor(kvindex) + VALSTART);
-      key.reset(kvbuffer, keystart, valstart - keystart);
-      return key;
-    }
-
-    public DataInputBuffer getValue() throws IOException {
-      final int valstart = kvmeta.get(span.offsetFor(kvindex) + VALSTART);
-      final int vallen = kvmeta.get(span.offsetFor(kvindex) + VALLEN);
-      value.reset(kvbuffer, valstart, vallen);
-      return value;
-    }
-
-    public boolean next() throws IOException {
-      // caveat: since we use this as a comparable in the merger 
-      if(kvindex == maxindex) return false;
-      if(kvindex % 100 == 0) {
-          progress.set((kvindex-maxindex) / maxindex);
-      }
-      kvindex += 1;
-      return true;
-    }
-
-    public void close() throws IOException {
-    }
-
-    public Progress getProgress() { 
-      return progress;
-    }
-
-    public int getPartition() {
-      final int partition = kvmeta.get(span.offsetFor(kvindex) + PARTITION);
-      return partition;
-    }
-
-    public int size() {
-      return (maxindex - kvindex);
-    }
-
-    public int compareTo(SpanIterator other) {
-      try {
-        return span.compareInternal(other.getKey(), other.getPartition(), kvindex);
-      } catch(IOException ie) {
-        // since we're not reading off disk, how could getKey() throw exceptions?
-      }
-      return -1;
-    }
-    
-    @Override
-    public String toString() {
-        return String.format("SpanIterator<%d:%d> (span=%s)", kvindex, maxindex, span.toString());
-    }
-
-    /**
-     * bisect returns the next insertion point for a given raw key, skipping keys
-     * which are <= needle using a binary search instead of a linear comparison.
-     * This is massively efficient when long strings of identical keys occur.
-     * @param needle 
-     * @param needlePart
-     * @return
-     */
-    int bisect(DataInputBuffer needle, int needlePart) {
-      int start = kvindex;
-      int end = maxindex-1;
-      int mid = start;
-      int cmp = 0;
-
-      if(end - start < minrun) {
-        return 0;
-      }
-
-      if(span.compareInternal(needle, needlePart, start) > 0) {
-        return kvindex;
-      }
-      
-      // bail out early if we haven't got a min run 
-      if(span.compareInternal(needle, needlePart, start+minrun) > 0) {
-        return 0;
-      }
-
-      if(span.compareInternal(needle, needlePart, end) < 0) {
-        return end - kvindex;
-      }
-      
-      boolean found = false;
-      
-      // we sort 100k items, the max it can do is 20 loops, but break early
-      for(int i = 0; start < end && i < 16; i++) {
-        mid = start + (end - start)/2;
-        cmp = span.compareInternal(needle, needlePart, mid);
-        if(cmp == 0) {
-          start = mid;
-          found = true;
-        } else if(cmp < 0) {
-          start = mid; 
-          found = true;
-        }
-        if(cmp > 0) {
-          end = mid;
-        }
-      }
-
-      if(found) {
-        return start - kvindex;
-      }
-      return 0;
-    }
-  }
-
-  private class SortTask implements Callable<SpanIterator> {
-    private final SortSpan sortable;
-    private final IndexedSorter sorter;
-    private final RawComparator comparator;
-    
-    public SortTask(SortSpan sortable, 
-              IndexedSorter sorter, RawComparator comparator) {
-        this.sortable = sortable;
-        this.sorter = sorter;
-        this.comparator = comparator;
-    }
-
-    public SpanIterator call() {
-      return sortable.sort(sorter, comparator);
-    }
-  }
-
-  private class PartitionFilter implements TezRawKeyValueIterator {
-    private final PartitionedRawKeyValueIterator iter;
-    private int partition;
-    private boolean dirty = false;
-    public PartitionFilter(PartitionedRawKeyValueIterator iter) {
-      this.iter = iter;
-    }
-    public DataInputBuffer getKey() throws IOException { return iter.getKey(); }
-    public DataInputBuffer getValue() throws IOException { return iter.getValue(); }
-    public void close() throws IOException { }
-    public Progress getProgress() {
-      return new Progress();
-    }
-    public boolean next() throws IOException {
-      if(dirty || iter.next()) { 
-        int prefix = iter.getPartition();
-
-        if((prefix >>> (32 - partitionBits)) == partition) {
-          dirty = false; // we found what we were looking for, good
-          return true;
-        } else if(!dirty) {
-          dirty = true; // we did a lookahead and failed to find partition
-        }
-      }
-      return false;
-    }
-
-    public void reset(int partition) {
-      this.partition = partition;
-    }
-
-    public int getPartition() {
-      return this.partition;
-    }
-  }
-
-  private class SpanHeap extends java.util.PriorityQueue<SpanIterator> {
-    public SpanHeap() {
-      super(256);
-    }
-    /**
-     * {@link PriorityQueue}.poll() by a different name 
-     * @return
-     */
-    public SpanIterator pop() {
-      return this.poll();
-    }
-  }
-
-  private class SpanMerger implements PartitionedRawKeyValueIterator {
-    private final RawComparator comparator;
-    InputByteBuffer key = new InputByteBuffer();
-    InputByteBuffer value = new InputByteBuffer();
-    int partition;
-
-    private ArrayList< Future<SpanIterator>> futures = new ArrayList< Future<SpanIterator>>();
-
-    private SpanHeap heap = new SpanHeap();
-    private PartitionFilter partIter;
-
-    private int gallop = 0;
-    private SpanIterator horse;
-    private long total = 0;
-    private long count = 0;
-    private long eq = 0;
-    
-    public SpanMerger(RawComparator comparator) {
-      this.comparator = comparator;
-      partIter = new PartitionFilter(this);
-    }
-
-    public void add(SpanIterator iter) throws IOException{
-      if(iter.next()) {
-        heap.add(iter);
-      }
-    }
-
-    public void add(Future<SpanIterator> iter) throws IOException{
-      this.futures.add(iter);
-    }
-
-    public boolean ready() throws IOException, InterruptedException {
-      try {
-        SpanIterator iter = null;
-        while(this.futures.size() > 0) {
-          Future<SpanIterator> futureIter = this.futures.remove(0);
-          iter = futureIter.get();
-          this.add(iter);
-        }
-        
-        StringBuilder sb = new StringBuilder();
-        for(SpanIterator sp: heap) {
-            sb.append(sp.toString());
-            sb.append(",");
-            total += sp.span.length();
-            eq += sp.span.getEq();
-        }
-        LOG.info("Heap = " + sb.toString());
-        return true;
-      } catch(Exception e) {
-        LOG.info(e.toString());
-        return false;
-      }
-    }
-
-    private SpanIterator pop() throws IOException {
-      if(gallop > 0) {
-        gallop--;
-        return horse;
-      }
-      SpanIterator current = heap.pop();
-      SpanIterator next = heap.peek();
-      if(next != null && current != null &&
-        ((Object)horse) == ((Object)current)) {
-        // TODO: a better threshold check
-        gallop = current.bisect(next.getKey(), next.getPartition())-1;
-      }
-      horse = current;
-      return current;
-    }
-    
-    public boolean needsRLE() {
-      return (eq > 0.1 * total);
-    }
-    
-    private SpanIterator peek() throws IOException {
-    	if(gallop > 0) {
-            return horse;
-        }
-    	return heap.peek();
-    }
-
-    public boolean next() throws IOException {
-      SpanIterator current = pop();
-
-      if(current != null) {
-        // keep local copies, since add() will move it all out
-        key.reset(current.getKey());
-        value.reset(current.getValue());
-        partition = current.getPartition();
-        if(gallop <= 0) {
-          this.add(current);
-        } else {
-          // galloping
-          current.next();
-        }
-        return true;
-      }
-      return false;
-    }
-
-    public DataInputBuffer getKey() throws IOException { return key; }
-    public DataInputBuffer getValue() throws IOException { return value; }
-    public int getPartition() { return partition; }
-
-    public void close() throws IOException {
-    }
-
-    public Progress getProgress() {
-      // TODO
-      return new Progress();
-    }
-
-    public TezRawKeyValueIterator filter(int partition) {
-      partIter.reset(partition);
-      return partIter;
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezIndexRecord.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezIndexRecord.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezIndexRecord.java
deleted file mode 100644
index ac0267c..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezIndexRecord.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.engine.common.sort.impl;
-
-public class TezIndexRecord {
-  private long startOffset;
-  private long rawLength;
-  private long partLength;
-
-  public TezIndexRecord() { }
-
-  public TezIndexRecord(long startOffset, long rawLength, long partLength) {
-    this.startOffset = startOffset;
-    this.rawLength = rawLength;
-    this.partLength = partLength;
-  }
-
-  public long getStartOffset() {
-    return startOffset;
-  }
-
-  public long getRawLength() {
-    return rawLength;
-  }
-
-  public long getPartLength() {
-    return partLength;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezMerger.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezMerger.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezMerger.java
deleted file mode 100644
index 7815569..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezMerger.java
+++ /dev/null
@@ -1,798 +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.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-
-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.ChecksumFileSystem;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.util.PriorityQueue;
-import org.apache.hadoop.util.Progress;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.common.sort.impl.IFile.Reader;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-
-/**
- * Merger is an utility class used by the Map and Reduce tasks for merging
- * both their memory and disk segments
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-@SuppressWarnings({"unchecked", "rawtypes"})
-public class TezMerger {  
-  private static final Log LOG = LogFactory.getLog(TezMerger.class);
-
-  
-  // Local directories
-  private static LocalDirAllocator lDirAlloc = 
-    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-
-  public static
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                            Class keyClass, Class valueClass, 
-                            CompressionCodec codec,
-                            Path[] inputs, boolean deleteInputs, 
-                            int mergeFactor, Path tmpDir,
-                            RawComparator comparator, Progressable reporter,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            Progress mergePhase)
-  throws IOException {
-    return 
-      new MergeQueue(conf, fs, inputs, deleteInputs, codec, comparator, 
-                           reporter, null).merge(keyClass, valueClass,
-                                           mergeFactor, tmpDir,
-                                           readsCounter, writesCounter, 
-                                           mergePhase);
-  }
-
-  public static 
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                            Class keyClass, Class valueClass, 
-                            CompressionCodec codec,
-                            Path[] inputs, boolean deleteInputs, 
-                            int mergeFactor, Path tmpDir,
-                            RawComparator comparator,
-                            Progressable reporter,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            TezCounter mergedMapOutputsCounter,
-                            Progress mergePhase)
-  throws IOException {
-    return 
-      new MergeQueue(conf, fs, inputs, deleteInputs, codec, comparator, 
-                           reporter, mergedMapOutputsCounter).merge(
-                                           keyClass, valueClass,
-                                           mergeFactor, tmpDir,
-                                           readsCounter, writesCounter,
-                                           mergePhase);
-  }
-  
-  public static
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, 
-                            Class keyClass, Class valueClass, 
-                            List<Segment> segments, 
-                            int mergeFactor, Path tmpDir,
-                            RawComparator comparator, Progressable reporter,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            Progress mergePhase)
-      throws IOException {
-    return merge(conf, fs, keyClass, valueClass, segments, mergeFactor, tmpDir,
-                 comparator, reporter, false, readsCounter, writesCounter,
-                 mergePhase);
-  }
-
-  public static <K extends Object, V extends Object>
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                            Class keyClass, Class valueClass,
-                            List<Segment> segments,
-                            int mergeFactor, Path tmpDir,
-                            RawComparator comparator, Progressable reporter,
-                            boolean sortSegments,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            Progress mergePhase)
-      throws IOException {
-    return new MergeQueue(conf, fs, segments, comparator, reporter,
-                           sortSegments).merge(keyClass, valueClass,
-                                               mergeFactor, tmpDir,
-                                               readsCounter, writesCounter,
-                                               mergePhase);
-  }
-
-  public static <K extends Object, V extends Object>
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                            Class keyClass, Class valueClass,
-                            CompressionCodec codec,
-                            List<Segment> segments,
-                            int mergeFactor, Path tmpDir,
-                            RawComparator comparator, Progressable reporter,
-                            boolean sortSegments,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            Progress mergePhase)
-      throws IOException {
-    return new MergeQueue(conf, fs, segments, comparator, reporter,
-                           sortSegments, codec).merge(keyClass, valueClass,
-                                               mergeFactor, tmpDir,
-                                               readsCounter, writesCounter,
-                                               mergePhase);
-  }
-
-  public static <K extends Object, V extends Object>
-    TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                            Class keyClass, Class valueClass,
-                            List<Segment> segments,
-                            int mergeFactor, int inMemSegments, Path tmpDir,
-                            RawComparator comparator, Progressable reporter,
-                            boolean sortSegments,
-                            TezCounter readsCounter,
-                            TezCounter writesCounter,
-                            Progress mergePhase)
-      throws IOException {
-    return new MergeQueue(conf, fs, segments, comparator, reporter,
-                           sortSegments).merge(keyClass, valueClass,
-                                               mergeFactor, inMemSegments,
-                                               tmpDir,
-                                               readsCounter, writesCounter,
-                                               mergePhase);
-  }
-
-
-  static <K extends Object, V extends Object>
-  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
-                          Class keyClass, Class valueClass,
-                          CompressionCodec codec,
-                          List<Segment> segments,
-                          int mergeFactor, int inMemSegments, Path tmpDir,
-                          RawComparator comparator, Progressable reporter,
-                          boolean sortSegments,
-                          TezCounter readsCounter,
-                          TezCounter writesCounter,
-                          Progress mergePhase)
-    throws IOException {
-  return new MergeQueue(conf, fs, segments, comparator, reporter,
-                         sortSegments, codec).merge(keyClass, valueClass,
-                                             mergeFactor, inMemSegments,
-                                             tmpDir,
-                                             readsCounter, writesCounter,
-                                             mergePhase);
-}
-
-  public static <K extends Object, V extends Object>
-  void writeFile(TezRawKeyValueIterator records, Writer writer, 
-                 Progressable progressable, Configuration conf) 
-  throws IOException {
-    long progressBar = 
-        conf.getLong(TezJobConfig.RECORDS_BEFORE_PROGRESS, 
-            TezJobConfig.DEFAULT_RECORDS_BEFORE_PROGRESS);
-    long recordCtr = 0;
-    while(records.next()) {
-      writer.append(records.getKey(), records.getValue());
-      
-      if (((recordCtr++) % progressBar) == 0) {
-        progressable.progress();
-      }
-    }
-}
-
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public static class Segment<K extends Object, V extends Object> {
-    Reader reader = null;
-    final DataInputBuffer key = new DataInputBuffer();
-    
-    Configuration conf = null;
-    FileSystem fs = null;
-    Path file = null;
-    boolean preserve = false;
-    CompressionCodec codec = null;
-    long segmentOffset = 0;
-    long segmentLength = -1;
-    
-    TezCounter mapOutputsCounter = null;
-
-    public Segment(Configuration conf, FileSystem fs, Path file,
-                   CompressionCodec codec, boolean preserve)
-    throws IOException {
-      this(conf, fs, file, codec, preserve, null);
-    }
-
-    public Segment(Configuration conf, FileSystem fs, Path file,
-                   CompressionCodec codec, boolean preserve,
-                   TezCounter mergedMapOutputsCounter)
-  throws IOException {
-      this(conf, fs, file, 0, fs.getFileStatus(file).getLen(), codec, preserve, 
-           mergedMapOutputsCounter);
-    }
-
-    public Segment(Configuration conf, FileSystem fs, Path file,
-                   long segmentOffset, long segmentLength,
-                   CompressionCodec codec,
-                   boolean preserve) throws IOException {
-      this(conf, fs, file, segmentOffset, segmentLength, codec, preserve, null);
-    }
-
-    public Segment(Configuration conf, FileSystem fs, Path file,
-        long segmentOffset, long segmentLength, CompressionCodec codec,
-        boolean preserve, TezCounter mergedMapOutputsCounter)
-    throws IOException {
-      this.conf = conf;
-      this.fs = fs;
-      this.file = file;
-      this.codec = codec;
-      this.preserve = preserve;
-
-      this.segmentOffset = segmentOffset;
-      this.segmentLength = segmentLength;
-      
-      this.mapOutputsCounter = mergedMapOutputsCounter;
-    }
-    
-    public Segment(Reader reader, boolean preserve) {
-      this(reader, preserve, null);
-    }
-    
-    public Segment(Reader reader, boolean preserve, 
-                   TezCounter mapOutputsCounter) {
-      this.reader = reader;
-      this.preserve = preserve;
-      
-      this.segmentLength = reader.getLength();
-      
-      this.mapOutputsCounter = mapOutputsCounter;
-    }
-
-    void init(TezCounter readsCounter) throws IOException {
-      if (reader == null) {
-        FSDataInputStream in = fs.open(file);
-        in.seek(segmentOffset);
-        reader = new Reader(conf, in, segmentLength, codec, readsCounter);
-      }
-      
-      if (mapOutputsCounter != null) {
-        mapOutputsCounter.increment(1);
-      }
-    }
-    
-    boolean inMemory() {
-      return fs == null;
-    }
-    
-    DataInputBuffer getKey() { return key; }
-
-    DataInputBuffer getValue(DataInputBuffer value) throws IOException {
-      nextRawValue(value);
-      return value;
-    }
-
-    public long getLength() { 
-      return (reader == null) ?
-        segmentLength : reader.getLength();
-    }
-    
-    boolean nextRawKey() throws IOException {
-      return reader.nextRawKey(key);
-    }
-
-    void nextRawValue(DataInputBuffer value) throws IOException {
-      reader.nextRawValue(value);
-    }
-
-    void closeReader() throws IOException {
-      if (reader != null) {
-        reader.close();
-        reader = null;
-      }
-    }
-    
-    void close() throws IOException {
-      closeReader();
-      if (!preserve && fs != null) {
-        fs.delete(file, false);
-      }
-    }
-
-    public long getPosition() throws IOException {
-      return reader.getPosition();
-    }
-
-    // This method is used by BackupStore to extract the 
-    // absolute position after a reset
-    long getActualPosition() throws IOException {
-      return segmentOffset + reader.getPosition();
-    }
-
-    Reader getReader() {
-      return reader;
-    }
-    
-    // This method is used by BackupStore to reinitialize the
-    // reader to start reading from a different segment offset
-    void reinitReader(int offset) throws IOException {
-      if (!inMemory()) {
-        closeReader();
-        segmentOffset = offset;
-        segmentLength = fs.getFileStatus(file).getLen() - segmentOffset;
-        init(null);
-      }
-    }
-  }
-  
-  // Boolean variable for including/considering final merge as part of sort
-  // phase or not. This is true in map task, false in reduce task. It is
-  // used in calculating mergeProgress.
-  static boolean includeFinalMerge = false;
-  
-  /**
-   * Sets the boolean variable includeFinalMerge to true. Called from
-   * map task before calling merge() so that final merge of map task
-   * is also considered as part of sort phase.
-   */
-  public static void considerFinalMergeForProgress() {
-    includeFinalMerge = true;
-  }
-  
-  private static class MergeQueue<K extends Object, V extends Object> 
-  extends PriorityQueue<Segment> implements TezRawKeyValueIterator {
-    Configuration conf;
-    FileSystem fs;
-    CompressionCodec codec;
-    
-    List<Segment> segments = new ArrayList<Segment>();
-    
-    RawComparator comparator;
-    
-    private long totalBytesProcessed;
-    private float progPerByte;
-    private Progress mergeProgress = new Progress();
-    
-    Progressable reporter;
-    
-    DataInputBuffer key;
-    final DataInputBuffer value = new DataInputBuffer();
-    final DataInputBuffer diskIFileValue = new DataInputBuffer();
-    
-    Segment minSegment;
-    Comparator<Segment> segmentComparator =   
-      new Comparator<Segment>() {
-      public int compare(Segment o1, Segment o2) {
-        if (o1.getLength() == o2.getLength()) {
-          return 0;
-        }
-
-        return o1.getLength() < o2.getLength() ? -1 : 1;
-      }
-    };
-
-    public MergeQueue(Configuration conf, FileSystem fs, 
-                      Path[] inputs, boolean deleteInputs, 
-                      CompressionCodec codec, RawComparator comparator,
-                      Progressable reporter, 
-                      TezCounter mergedMapOutputsCounter) 
-    throws IOException {
-      this.conf = conf;
-      this.fs = fs;
-      this.codec = codec;
-      this.comparator = comparator;
-      this.reporter = reporter;
-      
-      for (Path file : inputs) {
-        LOG.debug("MergeQ: adding: " + file);
-        segments.add(new Segment(conf, fs, file, codec, !deleteInputs, 
-                                       (file.toString().endsWith(
-                                           Constants.MERGED_OUTPUT_PREFIX) ? 
-                                        null : mergedMapOutputsCounter)));
-      }
-      
-      // Sort segments on file-lengths
-      Collections.sort(segments, segmentComparator); 
-    }
-    
-    public MergeQueue(Configuration conf, FileSystem fs, 
-        List<Segment> segments, RawComparator comparator,
-        Progressable reporter, boolean sortSegments) {
-      this.conf = conf;
-      this.fs = fs;
-      this.comparator = comparator;
-      this.segments = segments;
-      this.reporter = reporter;
-      if (sortSegments) {
-        Collections.sort(segments, segmentComparator);
-      }
-    }
-
-    public MergeQueue(Configuration conf, FileSystem fs,
-        List<Segment> segments, RawComparator comparator,
-        Progressable reporter, boolean sortSegments, CompressionCodec codec) {
-      this(conf, fs, segments, comparator, reporter, sortSegments);
-      this.codec = codec;
-    }
-
-    public void close() throws IOException {
-      Segment segment;
-      while((segment = pop()) != null) {
-        segment.close();
-      }
-    }
-
-    public DataInputBuffer getKey() throws IOException {
-      return key;
-    }
-
-    public DataInputBuffer getValue() throws IOException {
-      return value;
-    }
-
-    private void adjustPriorityQueue(Segment reader) throws IOException{
-      long startPos = reader.getPosition();
-      boolean hasNext = reader.nextRawKey();
-      long endPos = reader.getPosition();
-      totalBytesProcessed += endPos - startPos;
-      mergeProgress.set(totalBytesProcessed * progPerByte);
-      if (hasNext) {
-        adjustTop();
-      } else {
-        pop();
-        reader.close();
-      }
-    }
-
-    public boolean next() throws IOException {
-      if (size() == 0)
-        return false;
-
-      if (minSegment != null) {
-        //minSegment is non-null for all invocations of next except the first
-        //one. For the first invocation, the priority queue is ready for use
-        //but for the subsequent invocations, first adjust the queue 
-        adjustPriorityQueue(minSegment);
-        if (size() == 0) {
-          minSegment = null;
-          return false;
-        }
-      }
-      minSegment = top();
-      if (!minSegment.inMemory()) {
-        //When we load the value from an inmemory segment, we reset
-        //the "value" DIB in this class to the inmem segment's byte[].
-        //When we load the value bytes from disk, we shouldn't use
-        //the same byte[] since it would corrupt the data in the inmem
-        //segment. So we maintain an explicit DIB for value bytes
-        //obtained from disk, and if the current segment is a disk
-        //segment, we reset the "value" DIB to the byte[] in that (so 
-        //we reuse the disk segment DIB whenever we consider
-        //a disk segment).
-        value.reset(diskIFileValue.getData(), diskIFileValue.getLength());
-      }
-      long startPos = minSegment.getPosition();
-      key = minSegment.getKey();
-      minSegment.getValue(value);
-      long endPos = minSegment.getPosition();
-      totalBytesProcessed += endPos - startPos;
-      mergeProgress.set(totalBytesProcessed * progPerByte);
-      return true;
-    }
-
-    protected boolean lessThan(Object a, Object b) {
-      DataInputBuffer key1 = ((Segment)a).getKey();
-      DataInputBuffer key2 = ((Segment)b).getKey();
-      int s1 = key1.getPosition();
-      int l1 = key1.getLength() - s1;
-      int s2 = key2.getPosition();
-      int l2 = key2.getLength() - s2;
-
-      return comparator.compare(key1.getData(), s1, l1, key2.getData(), s2, l2) < 0;
-    }
-    
-    public TezRawKeyValueIterator merge(Class keyClass, Class valueClass,
-                                     int factor, Path tmpDir,
-                                     TezCounter readsCounter,
-                                     TezCounter writesCounter,
-                                     Progress mergePhase)
-        throws IOException {
-      return merge(keyClass, valueClass, factor, 0, tmpDir,
-                   readsCounter, writesCounter, mergePhase);
-    }
-
-    TezRawKeyValueIterator merge(Class keyClass, Class valueClass,
-                                     int factor, int inMem, Path tmpDir,
-                                     TezCounter readsCounter,
-                                     TezCounter writesCounter,
-                                     Progress mergePhase)
-        throws IOException {
-      LOG.info("Merging " + segments.size() + " sorted segments");
-
-      /*
-       * If there are inMemory segments, then they come first in the segments
-       * list and then the sorted disk segments. Otherwise(if there are only
-       * disk segments), then they are sorted segments if there are more than
-       * factor segments in the segments list.
-       */
-      int numSegments = segments.size();
-      int origFactor = factor;
-      int passNo = 1;
-      if (mergePhase != null) {
-        mergeProgress = mergePhase;
-      }
-
-      long totalBytes = computeBytesInMerges(factor, inMem);
-      if (totalBytes != 0) {
-        progPerByte = 1.0f / (float)totalBytes;
-      }
-      
-      //create the MergeStreams from the sorted map created in the constructor
-      //and dump the final output to a file
-      do {
-        //get the factor for this pass of merge. We assume in-memory segments
-        //are the first entries in the segment list and that the pass factor
-        //doesn't apply to them
-        factor = getPassFactor(factor, passNo, numSegments - inMem);
-        if (1 == passNo) {
-          factor += inMem;
-        }
-        List<Segment> segmentsToMerge =
-          new ArrayList<Segment>();
-        int segmentsConsidered = 0;
-        int numSegmentsToConsider = factor;
-        long startBytes = 0; // starting bytes of segments of this merge
-        while (true) {
-          //extract the smallest 'factor' number of segments  
-          //Call cleanup on the empty segments (no key/value data)
-          List<Segment> mStream = 
-            getSegmentDescriptors(numSegmentsToConsider);
-          for (Segment segment : mStream) {
-            // Initialize the segment at the last possible moment;
-            // this helps in ensuring we don't use buffers until we need them
-            segment.init(readsCounter);
-            long startPos = segment.getPosition();
-            boolean hasNext = segment.nextRawKey();
-            long endPos = segment.getPosition();
-            
-            if (hasNext) {
-              startBytes += endPos - startPos;
-              segmentsToMerge.add(segment);
-              segmentsConsidered++;
-            }
-            else {
-              segment.close();
-              numSegments--; //we ignore this segment for the merge
-            }
-          }
-          //if we have the desired number of segments
-          //or looked at all available segments, we break
-          if (segmentsConsidered == factor || 
-              segments.size() == 0) {
-            break;
-          }
-            
-          numSegmentsToConsider = factor - segmentsConsidered;
-        }
-        
-        //feed the streams to the priority queue
-        initialize(segmentsToMerge.size());
-        clear();
-        for (Segment segment : segmentsToMerge) {
-          put(segment);
-        }
-        
-        //if we have lesser number of segments remaining, then just return the
-        //iterator, else do another single level merge
-        if (numSegments <= factor) {
-          if (!includeFinalMerge) { // for reduce task
-
-            // Reset totalBytesProcessed and recalculate totalBytes from the
-            // remaining segments to track the progress of the final merge.
-            // Final merge is considered as the progress of the reducePhase,
-            // the 3rd phase of reduce task.
-            totalBytesProcessed = 0;
-            totalBytes = 0;
-            for (int i = 0; i < segmentsToMerge.size(); i++) {
-              totalBytes += segmentsToMerge.get(i).getLength();
-            }
-          }
-          if (totalBytes != 0) //being paranoid
-            progPerByte = 1.0f / (float)totalBytes;
-          
-          totalBytesProcessed += startBytes;         
-          if (totalBytes != 0)
-            mergeProgress.set(totalBytesProcessed * progPerByte);
-          else
-            mergeProgress.set(1.0f); // Last pass and no segments left - we're done
-          
-          LOG.info("Down to the last merge-pass, with " + numSegments + 
-                   " segments left of total size: " +
-                   (totalBytes - totalBytesProcessed) + " bytes");
-          return this;
-        } else {
-          LOG.info("Merging " + segmentsToMerge.size() + 
-                   " intermediate segments out of a total of " + 
-                   (segments.size()+segmentsToMerge.size()));
-          
-          long bytesProcessedInPrevMerges = totalBytesProcessed;
-          totalBytesProcessed += startBytes;
-
-          //we want to spread the creation of temp files on multiple disks if 
-          //available under the space constraints
-          long approxOutputSize = 0; 
-          for (Segment s : segmentsToMerge) {
-            approxOutputSize += s.getLength() + 
-                                ChecksumFileSystem.getApproxChkSumLength(
-                                s.getLength());
-          }
-          Path tmpFilename = 
-            new Path(tmpDir, "intermediate").suffix("." + passNo);
-
-          Path outputFile =  lDirAlloc.getLocalPathForWrite(
-                                              tmpFilename.toString(),
-                                              approxOutputSize, conf);
-
-          Writer writer = 
-            new Writer(conf, fs, outputFile, keyClass, valueClass, codec,
-                             writesCounter);
-          writeFile(this, writer, reporter, conf);
-          writer.close();
-          
-          //we finished one single level merge; now clean up the priority 
-          //queue
-          this.close();
-
-          // Add the newly create segment to the list of segments to be merged
-          Segment tempSegment = 
-            new Segment(conf, fs, outputFile, codec, false);
-
-          // Insert new merged segment into the sorted list
-          int pos = Collections.binarySearch(segments, tempSegment,
-                                             segmentComparator);
-          if (pos < 0) {
-            // binary search failed. So position to be inserted at is -pos-1
-            pos = -pos-1;
-          }
-          segments.add(pos, tempSegment);
-          numSegments = segments.size();
-          
-          // Subtract the difference between expected size of new segment and 
-          // actual size of new segment(Expected size of new segment is
-          // inputBytesOfThisMerge) from totalBytes. Expected size and actual
-          // size will match(almost) if combiner is not called in merge.
-          long inputBytesOfThisMerge = totalBytesProcessed -
-                                       bytesProcessedInPrevMerges;
-          totalBytes -= inputBytesOfThisMerge - tempSegment.getLength();
-          if (totalBytes != 0) {
-            progPerByte = 1.0f / (float)totalBytes;
-          }
-          
-          passNo++;
-        }
-        //we are worried about only the first pass merge factor. So reset the 
-        //factor to what it originally was
-        factor = origFactor;
-      } while(true);
-    }
-    
-    /**
-     * Determine the number of segments to merge in a given pass. Assuming more
-     * than factor segments, the first pass should attempt to bring the total
-     * number of segments - 1 to be divisible by the factor - 1 (each pass
-     * takes X segments and produces 1) to minimize the number of merges.
-     */
-    private int getPassFactor(int factor, int passNo, int numSegments) {
-      if (passNo > 1 || numSegments <= factor || factor == 1) 
-        return factor;
-      int mod = (numSegments - 1) % (factor - 1);
-      if (mod == 0)
-        return factor;
-      return mod + 1;
-    }
-    
-    /** Return (& remove) the requested number of segment descriptors from the
-     * sorted map.
-     */
-    private List<Segment> getSegmentDescriptors(int numDescriptors) {
-      if (numDescriptors > segments.size()) {
-        List<Segment> subList = new ArrayList<Segment>(segments);
-        segments.clear();
-        return subList;
-      }
-      
-      List<Segment> subList = 
-        new ArrayList<Segment>(segments.subList(0, numDescriptors));
-      for (int i=0; i < numDescriptors; ++i) {
-        segments.remove(0);
-      }
-      return subList;
-    }
-    
-    /**
-     * Compute expected size of input bytes to merges, will be used in
-     * calculating mergeProgress. This simulates the above merge() method and
-     * tries to obtain the number of bytes that are going to be merged in all
-     * merges(assuming that there is no combiner called while merging).
-     * @param factor mapreduce.task.io.sort.factor
-     * @param inMem  number of segments in memory to be merged
-     */
-    long computeBytesInMerges(int factor, int inMem) {
-      int numSegments = segments.size();
-      List<Long> segmentSizes = new ArrayList<Long>(numSegments);
-      long totalBytes = 0;
-      int n = numSegments - inMem;
-      // factor for 1st pass
-      int f = getPassFactor(factor, 1, n) + inMem;
-      n = numSegments;
- 
-      for (int i = 0; i < numSegments; i++) {
-        // Not handling empty segments here assuming that it would not affect
-        // much in calculation of mergeProgress.
-        segmentSizes.add(segments.get(i).getLength());
-      }
-      
-      // If includeFinalMerge is true, allow the following while loop iterate
-      // for 1 more iteration. This is to include final merge as part of the
-      // computation of expected input bytes of merges
-      boolean considerFinalMerge = includeFinalMerge;
-      
-      while (n > f || considerFinalMerge) {
-        if (n <=f ) {
-          considerFinalMerge = false;
-        }
-        long mergedSize = 0;
-        f = Math.min(f, segmentSizes.size());
-        for (int j = 0; j < f; j++) {
-          mergedSize += segmentSizes.remove(0);
-        }
-        totalBytes += mergedSize;
-        
-        // insert new size into the sorted list
-        int pos = Collections.binarySearch(segmentSizes, mergedSize);
-        if (pos < 0) {
-          pos = -pos-1;
-        }
-        segmentSizes.add(pos, mergedSize);
-        
-        n -= (f-1);
-        f = factor;
-      }
-
-      return totalBytes;
-    }
-
-    public Progress getProgress() {
-      return mergeProgress;
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezRawKeyValueIterator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezRawKeyValueIterator.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezRawKeyValueIterator.java
deleted file mode 100644
index 39cffcb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezRawKeyValueIterator.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.common.sort.impl;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.util.Progress;
-
-/**
- * <code>TezRawKeyValueIterator</code> is an iterator used to iterate over
- * the raw keys and values during sort/merge of intermediate data. 
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public interface TezRawKeyValueIterator {
-  /** 
-   * Gets the current raw key.
-   * 
-   * @return Gets the current raw key as a DataInputBuffer
-   * @throws IOException
-   */
-  DataInputBuffer getKey() throws IOException;
-  
-  /** 
-   * Gets the current raw value.
-   * 
-   * @return Gets the current raw value as a DataInputBuffer 
-   * @throws IOException
-   */
-  DataInputBuffer getValue() throws IOException;
-  
-  /** 
-   * Sets up the current key and value (for getKey and getValue).
-   * 
-   * @return <code>true</code> if there exists a key/value, 
-   *         <code>false</code> otherwise. 
-   * @throws IOException
-   */
-  boolean next() throws IOException;
-  
-  /** 
-   * Closes the iterator so that the underlying streams can be closed.
-   * 
-   * @throws IOException
-   */
-  void close() throws IOException;
-  
-  /** Gets the Progress object; this has a float (0.0 - 1.0) 
-   * indicating the bytes processed by the iterator so far
-   */
-  Progress getProgress();
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezSpillRecord.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezSpillRecord.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezSpillRecord.java
deleted file mode 100644
index 19fbd7f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/TezSpillRecord.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.common.sort.impl;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.LongBuffer;
-import java.util.zip.CheckedInputStream;
-import java.util.zip.CheckedOutputStream;
-import java.util.zip.Checksum;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ChecksumException;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.tez.common.Constants;
-
-public class TezSpillRecord {
-
-  /** Backing store */
-  private final ByteBuffer buf;
-  /** View of backing storage as longs */
-  private final LongBuffer entries;
-
-  public TezSpillRecord(int numPartitions) {
-    buf = ByteBuffer.allocate(
-        numPartitions * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH);
-    entries = buf.asLongBuffer();
-  }
-
-  public TezSpillRecord(Path indexFileName, Configuration job) throws IOException {
-    this(indexFileName, job, null);
-  }
-
-  public TezSpillRecord(Path indexFileName, Configuration job, String expectedIndexOwner)
-    throws IOException {
-    this(indexFileName, job, new PureJavaCrc32(), expectedIndexOwner);
-  }
-
-  public TezSpillRecord(Path indexFileName, Configuration job, Checksum crc,
-                     String expectedIndexOwner)
-      throws IOException {
-
-    final FileSystem rfs = FileSystem.getLocal(job).getRaw();
-    final FSDataInputStream in = rfs.open(indexFileName);
-    try {
-      final long length = rfs.getFileStatus(indexFileName).getLen();
-      final int partitions = 
-          (int) length / Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH;
-      final int size = partitions * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH;
-
-      buf = ByteBuffer.allocate(size);
-      if (crc != null) {
-        crc.reset();
-        CheckedInputStream chk = new CheckedInputStream(in, crc);
-        IOUtils.readFully(chk, buf.array(), 0, size);
-        if (chk.getChecksum().getValue() != in.readLong()) {
-          throw new ChecksumException("Checksum error reading spill index: " +
-                                indexFileName, -1);
-        }
-      } else {
-        IOUtils.readFully(in, buf.array(), 0, size);
-      }
-      entries = buf.asLongBuffer();
-    } finally {
-      in.close();
-    }
-  }
-
-  /**
-   * Return number of IndexRecord entries in this spill.
-   */
-  public int size() {
-    return entries.capacity() / (Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8);
-  }
-
-  /**
-   * Get spill offsets for given partition.
-   */
-  public TezIndexRecord getIndex(int partition) {
-    final int pos = partition * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8;
-    return new TezIndexRecord(entries.get(pos), entries.get(pos + 1),
-                           entries.get(pos + 2));
-  }
-
-  /**
-   * Set spill offsets for given partition.
-   */
-  public void putIndex(TezIndexRecord rec, int partition) {
-    final int pos = partition * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8;
-    entries.put(pos, rec.getStartOffset());
-    entries.put(pos + 1, rec.getRawLength());
-    entries.put(pos + 2, rec.getPartLength());
-  }
-
-  /**
-   * Write this spill record to the location provided.
-   */
-  public void writeToFile(Path loc, Configuration job)
-      throws IOException {
-    writeToFile(loc, job, new PureJavaCrc32());
-  }
-
-  public void writeToFile(Path loc, Configuration job, Checksum crc)
-      throws IOException {
-    final FileSystem rfs = FileSystem.getLocal(job).getRaw();
-    CheckedOutputStream chk = null;
-    final FSDataOutputStream out = rfs.create(loc);
-    try {
-      if (crc != null) {
-        crc.reset();
-        chk = new CheckedOutputStream(out, crc);
-        chk.write(buf.array());
-        out.writeLong(chk.getChecksum().getValue());
-      } else {
-        out.write(buf.array());
-      }
-    } finally {
-      if (chk != null) {
-        chk.close();
-      } else {
-        out.close();
-      }
-    }
-  }
-
-}


[16/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java b/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
deleted file mode 100644
index b7867aa..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/ValuesIterator.java
+++ /dev/null
@@ -1,194 +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;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Iterates values while keys match in sorted input.
- * 
- * This class is not thread safe. Accessing methods from multiple threads will
- * lead to corrupt data.
- * 
- */
-public class ValuesIterator<KEY,VALUE> {
-  protected TezRawKeyValueIterator in; //input iterator
-  private KEY key;               // current key
-  private KEY nextKey;
-  private VALUE value;             // current value
-  //private boolean hasNext;                      // more w/ this key
-  private boolean more;                         // more in file
-  private RawComparator<KEY> comparator;
-  private Deserializer<KEY> keyDeserializer;
-  private Deserializer<VALUE> valDeserializer;
-  private DataInputBuffer keyIn = new DataInputBuffer();
-  private DataInputBuffer valueIn = new DataInputBuffer();
-  private TezCounter inputKeyCounter;
-  private TezCounter inputValueCounter;
-  
-  private int keyCtr = 0;
-  private boolean hasMoreValues; // For the current key.
-  private boolean isFirstRecord = true;
-  
-  public ValuesIterator (TezRawKeyValueIterator in, 
-                         RawComparator<KEY> comparator, 
-                         Class<KEY> keyClass,
-                         Class<VALUE> valClass, Configuration conf,
-                         TezCounter inputKeyCounter,
-                         TezCounter inputValueCounter)
-    throws IOException {
-    this.in = in;
-    this.comparator = comparator;
-    this.inputKeyCounter = inputKeyCounter;
-    this.inputValueCounter = inputValueCounter;
-    SerializationFactory serializationFactory = new SerializationFactory(conf);
-    this.keyDeserializer = serializationFactory.getDeserializer(keyClass);
-    this.keyDeserializer.open(keyIn);
-    this.valDeserializer = serializationFactory.getDeserializer(valClass);
-    this.valDeserializer.open(this.valueIn);
-  }
-
-  TezRawKeyValueIterator getRawIterator() { return in; }
-
-  /**
-   * Move to the next K-Vs pair
-   * @return true if another pair exists, otherwise false.
-   * @throws IOException 
-   */
-  public boolean moveToNext() throws IOException {
-    if (isFirstRecord) {
-      readNextKey();
-      key = nextKey;
-      nextKey = null;
-      hasMoreValues = more;
-      isFirstRecord = false;
-    } else {
-      nextKey();
-    }
-    return more;
-  }
-  
-  /** The current key. */
-  public KEY getKey() { 
-    return key; 
-  }
-  
-  // TODO NEWTEZ Maybe add another method which returns an iterator instead of iterable
-  
-  public Iterable<VALUE> getValues() {
-    return new Iterable<VALUE>() {
-
-      @Override
-      public Iterator<VALUE> iterator() {
-        
-        return new Iterator<VALUE>() {
-
-          private final int keyNumber = keyCtr;
-          
-          @Override
-          public boolean hasNext() {
-            return hasMoreValues;
-          }
-
-          @Override
-          public VALUE next() {
-            if (!hasMoreValues) {
-              throw new NoSuchElementException("iterate past last value");
-            }
-            Preconditions
-                .checkState(
-                    keyNumber == keyCtr,
-                    "Cannot use values iterator on the previous K-V pair after moveToNext has been invoked to move to the next K-V pair");
-            
-            try {
-              readNextValue();
-              readNextKey();
-            } catch (IOException ie) {
-              throw new RuntimeException("problem advancing post rec#"+keyCtr, ie);
-            }
-            inputValueCounter.increment(1);
-            return value;
-          }
-
-          @Override
-          public void remove() {
-            throw new UnsupportedOperationException("Cannot remove elements");
-          }
-        };
-      }
-    };
-  }
-  
-  
-
-  /** Start processing next unique key. */
-  private void nextKey() throws IOException {
-    // read until we find a new key
-    while (hasMoreValues) { 
-      readNextKey();
-    }
-    if (more) {
-      inputKeyCounter.increment(1);
-      ++keyCtr;
-    }
-    
-    // move the next key to the current one
-    KEY tmpKey = key;
-    key = nextKey;
-    nextKey = tmpKey;
-    hasMoreValues = more;
-  }
-
-  /** 
-   * read the next key - which may be the same as the current key.
-   */
-  private void readNextKey() throws IOException {
-    more = in.next();
-    if (more) {      
-      DataInputBuffer nextKeyBytes = in.getKey();
-      keyIn.reset(nextKeyBytes.getData(), nextKeyBytes.getPosition(), nextKeyBytes.getLength());
-      nextKey = keyDeserializer.deserialize(nextKey);
-      hasMoreValues = key != null && (comparator.compare(key, nextKey) == 0);
-    } else {
-      hasMoreValues = false;
-    }
-  }
-
-  /**
-   * Read the next value
-   * @throws IOException
-   */
-  private void readNextValue() throws IOException {
-    DataInputBuffer nextValueBytes = in.getValue();
-    valueIn.reset(nextValueBytes.getData(), nextValueBytes.getPosition(), nextValueBytes.getLength());
-    value = valDeserializer.deserialize(value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/YARNMaster.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/YARNMaster.java b/tez-engine/src/main/java/org/apache/tez/engine/common/YARNMaster.java
deleted file mode 100644
index 48ad639..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/YARNMaster.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.engine.common;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-
-@Private
-@Unstable
-public class YARNMaster {
-  
-  public enum State {
-    INITIALIZING, RUNNING;
-  }
-
-  public static String getMasterUserName(Configuration conf) {
-    return conf.get(YarnConfiguration.RM_PRINCIPAL);
-  }
-  
-  public static InetSocketAddress getMasterAddress(Configuration conf) {
-    return conf.getSocketAddr(
-        YarnConfiguration.RM_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_ADDRESS,
-        YarnConfiguration.DEFAULT_RM_PORT);
-  }
-
-  public static String getMasterPrincipal(Configuration conf) 
-  throws IOException {
-    String masterHostname = getMasterAddress(conf).getHostName();
-    // get kerberos principal for use as delegation token renewer
-    return SecurityUtil.getServerPrincipal(
-        getMasterUserName(conf), masterHostname);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index b387b36..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/combine/Combiner.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.engine.common.combine;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-/**
- *<b>Combiner Initialization</b></p> The Combiner class is picked up
- * using the TEZ_ENGINE_COMBINER_CLASS attribute in {@link TezJobConfig}
- * 
- * 
- * Partitioners need to provide a single argument ({@link TezTaskContext})
- * constructor.
- */
-@Unstable
-@LimitedPrivate("mapreduce")
-public interface Combiner {
-  public void combine(TezRawKeyValueIterator rawIter, Writer writer)
-      throws InterruptedException, IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 546151f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/localshuffle/LocalShuffle.java
+++ /dev/null
@@ -1,120 +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.localshuffle;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-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;
-
-@SuppressWarnings({"rawtypes"})
-public class LocalShuffle {
-
-  // TODO NEWTEZ This is broken.
-
-  private final TezInputContext inputContext;
-  private final Configuration conf;
-  private final int numInputs;
-
-  private final Class keyClass;
-  private final Class valClass;
-  private final RawComparator comparator;
-
-  private final FileSystem rfs;
-  private final int sortFactor;
-  
-  private final TezCounter spilledRecordsCounter;
-  private final CompressionCodec codec;
-  private final TezTaskOutput mapOutputFile;
-
-  public LocalShuffle(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
-    this.inputContext = inputContext;
-    this.conf = conf;
-    this.numInputs = numInputs;
-    
-    this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
-    this.valClass = ConfigUtils.getIntermediateInputValueClass(conf);
-    this.comparator = ConfigUtils.getIntermediateInputKeyComparator(conf);
-    
-    this.sortFactor =
-        conf.getInt(
-            TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
-    
-    this.rfs = FileSystem.getLocal(conf).getRaw();
-
-    this.spilledRecordsCounter = inputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
-    
- // compression
-    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
-      Class<? extends CompressionCodec> codecClass =
-          ConfigUtils.getIntermediateInputCompressorClass(conf, DefaultCodec.class);
-      this.codec = ReflectionUtils.newInstance(codecClass, conf);
-    } else {
-      this.codec = null;
-    }
-    
-    // Always local
-    this.mapOutputFile = new TezLocalTaskOutputFiles(conf, inputContext.getUniqueIdentifier());
-  }
- 
-  
-  public TezRawKeyValueIterator run() throws IOException {
-    // Copy is complete, obviously! 
-
-    
-    // Merge
-    return TezMerger.merge(conf, rfs, 
-        keyClass, valClass,
-        codec, 
-        getMapFiles(),
-        false, 
-        sortFactor,
-        new Path(inputContext.getUniqueIdentifier()), // TODO NEWTEZ This is likely broken 
-        comparator,
-        null, spilledRecordsCounter, null, null);
-  }
-  
-  private Path[] getMapFiles() 
-  throws IOException {
-    List<Path> fileList = new ArrayList<Path>();
-      // for local jobs
-      for(int i = 0; i < numInputs; ++i) {
-        //fileList.add(mapOutputFile.getInputFile(i));
-      }
-      
-    return fileList.toArray(new Path[0]);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryImpl.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryImpl.java b/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryImpl.java
deleted file mode 100644
index 351e01c..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryImpl.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.common.objectregistry;
-
-import java.util.AbstractMap;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import com.google.inject.Singleton;
-
-@Singleton
-public class ObjectRegistryImpl implements ObjectRegistry {
-
-  private Map<String, Map.Entry<Object, ObjectLifeCycle>> objectCache =
-      new HashMap<String, Map.Entry<Object, ObjectLifeCycle>>();
-
-  @Override
-  public synchronized Object add(ObjectLifeCycle lifeCycle,
-      String key, Object value) {
-    Map.Entry<Object, ObjectLifeCycle> oldEntry =
-        objectCache.put(key,
-            new AbstractMap.SimpleImmutableEntry<Object, ObjectLifeCycle>(
-                value, lifeCycle));
-    return oldEntry != null ? oldEntry.getKey() : null;
-  }
-
-  @Override
-  public synchronized Object get(String key) {
-    Map.Entry<Object, ObjectLifeCycle> entry =
-        objectCache.get(key);
-    return entry != null ? entry.getKey() : null;
-  }
-
-  @Override
-  public synchronized boolean delete(String key) {
-    return (null != objectCache.remove(key));
-  }
-
-  public synchronized void clearCache(ObjectLifeCycle lifeCycle) {
-    for (Entry<String, Entry<Object, ObjectLifeCycle>> entry :
-      objectCache.entrySet()) {
-      if (entry.getValue().getValue().equals(lifeCycle)) {
-        objectCache.remove(entry.getKey());
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryModule.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryModule.java b/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryModule.java
deleted file mode 100644
index ab346fd..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/objectregistry/ObjectRegistryModule.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.common.objectregistry;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.inject.AbstractModule;
-
-public class ObjectRegistryModule extends AbstractModule {
-
-  private final ObjectRegistry objectRegistry;
-
-  public ObjectRegistryModule(ObjectRegistry objectRegistry) {
-    this.objectRegistry = objectRegistry;
-  }
-
-  @VisibleForTesting
-  public ObjectRegistryModule() {
-    objectRegistry = new ObjectRegistryImpl();
-  }
-
-  @Override
-  protected void configure() {
-    bind(ObjectRegistry.class).toInstance(this.objectRegistry);
-    requestStaticInjection(ObjectRegistryFactory.class);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenIdentifier.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenIdentifier.java
deleted file mode 100644
index 827001b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenIdentifier.java
+++ /dev/null
@@ -1,98 +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.security;
-
-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.Text;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-
-/**
- * The token identifier for job token
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class JobTokenIdentifier extends TokenIdentifier {
-  private Text jobid;
-  public final static Text KIND_NAME = new Text("mapreduce.job");
-  
-  /**
-   * Default constructor
-   */
-  public JobTokenIdentifier() {
-    this.jobid = new Text();
-  }
-
-  /**
-   * Create a job token identifier from a jobid
-   * @param jobid the jobid to use
-   */
-  public JobTokenIdentifier(Text jobid) {
-    this.jobid = jobid;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public Text getKind() {
-    return KIND_NAME;
-  }
-  
-  /** {@inheritDoc} */
-  @Override
-  public UserGroupInformation getUser() {
-    if (jobid == null || "".equals(jobid.toString())) {
-      return null;
-    }
-    return UserGroupInformation.createRemoteUser(jobid.toString());
-  }
-  
-  /**
-   * Get the jobid
-   * @return the jobid
-   */
-  public Text getJobId() {
-    return jobid;
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    jobid.readFields(in);
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public void write(DataOutput out) throws IOException {
-    jobid.write(out);
-  }
-
-  @InterfaceAudience.Private
-  public static class Renewer extends Token.TrivialRenewer {
-    @Override
-    protected Text getKind() {
-      return KIND_NAME;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSecretManager.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSecretManager.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSecretManager.java
deleted file mode 100644
index d957b8b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSecretManager.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.common.security;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-import javax.crypto.SecretKey;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.security.token.SecretManager;
-import org.apache.hadoop.security.token.Token;
-
-/**
- * SecretManager for job token. It can be used to cache generated job tokens.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class JobTokenSecretManager extends SecretManager<JobTokenIdentifier> {
-  private final SecretKey masterKey;
-  private final Map<String, SecretKey> currentJobTokens;
-
-  /**
-   * Convert the byte[] to a secret key
-   * @param key the byte[] to create the secret key from
-   * @return the secret key
-   */
-  public static SecretKey createSecretKey(byte[] key) {
-    return SecretManager.createSecretKey(key);
-  }
-  
-  /**
-   * Compute the HMAC hash of the message using the key
-   * @param msg the message to hash
-   * @param key the key to use
-   * @return the computed hash
-   */
-  public static byte[] computeHash(byte[] msg, SecretKey key) {
-    return createPassword(msg, key);
-  }
-  
-  /**
-   * Default constructor
-   */
-  public JobTokenSecretManager() {
-    this.masterKey = generateSecret();
-    this.currentJobTokens = new TreeMap<String, SecretKey>();
-  }
-  
-  /**
-   * Create a new password/secret for the given job token identifier.
-   * @param identifier the job token identifier
-   * @return token password/secret
-   */
-  @Override
-  public byte[] createPassword(JobTokenIdentifier identifier) {
-    byte[] result = createPassword(identifier.getBytes(), masterKey);
-    return result;
-  }
-
-  /**
-   * Add the job token of a job to cache
-   * @param jobId the job that owns the token
-   * @param token the job token
-   */
-  public void addTokenForJob(String jobId, Token<JobTokenIdentifier> token) {
-    SecretKey tokenSecret = createSecretKey(token.getPassword());
-    synchronized (currentJobTokens) {
-      currentJobTokens.put(jobId, tokenSecret);
-    }
-  }
-
-  /**
-   * Remove the cached job token of a job from cache
-   * @param jobId the job whose token is to be removed
-   */
-  public void removeTokenForJob(String jobId) {
-    synchronized (currentJobTokens) {
-      currentJobTokens.remove(jobId);
-    }
-  }
-  
-  /**
-   * Look up the token password/secret for the given jobId.
-   * @param jobId the jobId to look up
-   * @return token password/secret as SecretKey
-   * @throws InvalidToken
-   */
-  public SecretKey retrieveTokenSecret(String jobId) throws InvalidToken {
-    SecretKey tokenSecret = null;
-    synchronized (currentJobTokens) {
-      tokenSecret = currentJobTokens.get(jobId);
-    }
-    if (tokenSecret == null) {
-      throw new InvalidToken("Can't find job token for job " + jobId + " !!");
-    }
-    return tokenSecret;
-  }
-  
-  /**
-   * Look up the token password/secret for the given job token identifier.
-   * @param identifier the job token identifier to look up
-   * @return token password/secret as byte[]
-   * @throws InvalidToken
-   */
-  @Override
-  public byte[] retrievePassword(JobTokenIdentifier identifier)
-      throws InvalidToken {
-    return retrieveTokenSecret(identifier.getJobId().toString()).getEncoded();
-  }
-
-  /**
-   * Create an empty job token identifier
-   * @return a newly created empty job token identifier
-   */
-  @Override
-  public JobTokenIdentifier createIdentifier() {
-    return new JobTokenIdentifier();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSelector.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSelector.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSelector.java
deleted file mode 100644
index 080b9e2..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/JobTokenSelector.java
+++ /dev/null
@@ -1,53 +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.security;
-
-import java.util.Collection;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.TokenSelector;
-
-/**
- * Look through tokens to find the first job token that matches the service
- * and return it.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class JobTokenSelector implements TokenSelector<JobTokenIdentifier> {
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public Token<JobTokenIdentifier> selectToken(Text service,
-      Collection<Token<? extends TokenIdentifier>> tokens) {
-    if (service == null) {
-      return null;
-    }
-    for (Token<? extends TokenIdentifier> token : tokens) {
-      if (JobTokenIdentifier.KIND_NAME.equals(token.getKind())
-          && service.equals(token.getService())) {
-        return (Token<JobTokenIdentifier>) token;
-      }
-    }
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/Master.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/Master.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/Master.java
deleted file mode 100644
index 20cad0a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/Master.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.engine.common.security;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-
-@Private
-@Unstable
-public class Master {
-
-  public enum State {
-    INITIALIZING, RUNNING;
-  }
-
-  public static String getMasterUserName(Configuration conf) {
-    return conf.get(YarnConfiguration.RM_PRINCIPAL);
-  }
-
-  public static InetSocketAddress getMasterAddress(Configuration conf) {
-    return conf
-        .getSocketAddr(YarnConfiguration.RM_ADDRESS,
-            YarnConfiguration.DEFAULT_RM_ADDRESS,
-            YarnConfiguration.DEFAULT_RM_PORT);
-  }
-
-  public static String getMasterPrincipal(Configuration conf)
-      throws IOException {
-    String masterHostname = getMasterAddress(conf).getHostName();
-    // get kerberos principal for use as delegation token renewer
-    return SecurityUtil.getServerPrincipal(getMasterUserName(conf),
-        masterHostname);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/SecureShuffleUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/SecureShuffleUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/SecureShuffleUtils.java
deleted file mode 100644
index 71d990a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/SecureShuffleUtils.java
+++ /dev/null
@@ -1,142 +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.security;
-
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.URL;
-
-import javax.crypto.SecretKey;
-import javax.servlet.http.HttpServletRequest;
-
-import org.apache.commons.codec.binary.Base64;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.WritableComparator;
-
-/**
- * 
- * utilities for generating kyes, hashes and verifying them for shuffle
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class SecureShuffleUtils {
-  public static final String HTTP_HEADER_URL_HASH = "UrlHash";
-  public static final String HTTP_HEADER_REPLY_URL_HASH = "ReplyHash";
-  
-  /**
-   * Base64 encoded hash of msg
-   * @param msg
-   */
-  public static String generateHash(byte[] msg, SecretKey key) {
-    return new String(Base64.encodeBase64(generateByteHash(msg, key)));
-  }
-  
-  /**
-   * calculate hash of msg
-   * @param msg
-   * @return
-   */
-  private static byte[] generateByteHash(byte[] msg, SecretKey key) {
-    return JobTokenSecretManager.computeHash(msg, key);
-  }
-  
-  /**
-   * verify that hash equals to HMacHash(msg)
-   * @param newHash
-   * @return true if is the same
-   */
-  private static boolean verifyHash(byte[] hash, byte[] msg, SecretKey key) {
-    byte[] msg_hash = generateByteHash(msg, key);
-    return WritableComparator.compareBytes(msg_hash, 0, msg_hash.length, hash, 0, hash.length) == 0;
-  }
-  
-  /**
-   * Aux util to calculate hash of a String
-   * @param enc_str
-   * @param key
-   * @return Base64 encodedHash
-   * @throws IOException
-   */
-  public static String hashFromString(String enc_str, SecretKey key) 
-  throws IOException {
-    return generateHash(enc_str.getBytes(), key); 
-  }
-  
-  /**
-   * verify that base64Hash is same as HMacHash(msg)  
-   * @param base64Hash (Base64 encoded hash)
-   * @param msg
-   * @throws IOException if not the same
-   */
-  public static void verifyReply(String base64Hash, String msg, SecretKey key)
-  throws IOException {
-    byte[] hash = Base64.decodeBase64(base64Hash.getBytes());
-    
-    boolean res = verifyHash(hash, msg.getBytes(), key);
-    
-    if(res != true) {
-      throw new IOException("Verification of the hashReply failed");
-    }
-  }
-  
-  /**
-   * Shuffle specific utils - build string for encoding from URL
-   * @param url
-   * @return string for encoding
-   */
-  public static String buildMsgFrom(URL url) {
-    return buildMsgFrom(url.getPath(), url.getQuery(), url.getPort());
-  }
-  /**
-   * Shuffle specific utils - build string for encoding from URL
-   * @param request
-   * @return string for encoding
-   */
-  public static String buildMsgFrom(HttpServletRequest request ) {
-    return buildMsgFrom(request.getRequestURI(), request.getQueryString(),
-        request.getLocalPort());
-  }
-  /**
-   * Shuffle specific utils - build string for encoding from URL
-   * @param uri_path
-   * @param uri_query
-   * @return string for encoding
-   */
-  private static String buildMsgFrom(String uri_path, String uri_query, int port) {
-    return String.valueOf(port) + uri_path + "?" + uri_query;
-  }
-  
-  
-  /**
-   * byte array to Hex String
-   * @param ba
-   * @return string with HEX value of the key
-   */
-  public static String toHex(byte[] ba) {
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    PrintStream ps = new PrintStream(baos);
-    for(byte b: ba) {
-      ps.printf("%x", b);
-    }
-    return baos.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/security/TokenCache.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/security/TokenCache.java b/tez-engine/src/main/java/org/apache/tez/engine/common/security/TokenCache.java
deleted file mode 100644
index 5c42d22..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/security/TokenCache.java
+++ /dev/null
@@ -1,205 +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.security;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.Set;
-
-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.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.tez.common.TezJobConfig;
-
-
-/**
- * This class provides user facing APIs for transferring secrets from
- * the job client to the tasks.
- * The secrets can be stored just before submission of jobs and read during
- * the task execution.  
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class TokenCache {
-  
-  private static final Log LOG = LogFactory.getLog(TokenCache.class);
-
-  
-  /**
-   * auxiliary method to get user's secret keys..
-   * @param alias
-   * @return secret key from the storage
-   */
-  public static byte[] getSecretKey(Credentials credentials, Text alias) {
-    if(credentials == null)
-      return null;
-    return credentials.getSecretKey(alias);
-  }
-  
-  /**
-   * Convenience method to obtain delegation tokens from namenodes 
-   * corresponding to the paths passed.
-   * @param credentials
-   * @param ps array of paths
-   * @param conf configuration
-   * @throws IOException
-   */
-  public static void obtainTokensForNamenodes(Credentials credentials,
-      Path[] ps, Configuration conf) throws IOException {
-    if (!UserGroupInformation.isSecurityEnabled()) {
-      return;
-    }
-    obtainTokensForNamenodesInternal(credentials, ps, conf);
-  }
-
-  /**
-   * Remove jobtoken referrals which don't make sense in the context
-   * of the task execution.
-   *
-   * @param conf
-   */
-  public static void cleanUpTokenReferral(Configuration conf) {
-    conf.unset(TezJobConfig.DAG_CREDENTIALS_BINARY);
-  }
-
-  static void obtainTokensForNamenodesInternal(Credentials credentials,
-      Path[] ps, Configuration conf) throws IOException {
-    Set<FileSystem> fsSet = new HashSet<FileSystem>();
-    for(Path p: ps) {
-      fsSet.add(p.getFileSystem(conf));
-    }
-    for (FileSystem fs : fsSet) {
-      obtainTokensForNamenodesInternal(fs, credentials, conf);
-    }
-  }
-
-  /**
-   * get delegation token for a specific FS
-   * @param fs
-   * @param credentials
-   * @param p
-   * @param conf
-   * @throws IOException
-   */
-  static void obtainTokensForNamenodesInternal(FileSystem fs, 
-      Credentials credentials, Configuration conf) throws IOException {
-    String delegTokenRenewer = Master.getMasterPrincipal(conf);
-    if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
-      throw new IOException(
-          "Can't get Master Kerberos principal for use as renewer");
-    }
-    mergeBinaryTokens(credentials, conf);
-
-    final Token<?> tokens[] = fs.addDelegationTokens(delegTokenRenewer,
-                                                     credentials);
-    if (tokens != null) {
-      for (Token<?> token : tokens) {
-        LOG.info("Got dt for " + fs.getUri() + "; "+token);
-      }
-    }
-  }
-
-  private static void mergeBinaryTokens(Credentials creds, Configuration conf) {
-    String binaryTokenFilename =
-        conf.get(TezJobConfig.DAG_CREDENTIALS_BINARY);
-    if (binaryTokenFilename != null) {
-      Credentials binary;
-      try {
-        binary = Credentials.readTokenStorageFile(
-            new Path("file:///" +  binaryTokenFilename), conf);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      // supplement existing tokens with the tokens in the binary file
-      creds.mergeAll(binary);
-    }
-  }
-  
-  /**
-   * file name used on HDFS for generated job token
-   */
-  @InterfaceAudience.Private
-  public static final String JOB_TOKEN_HDFS_FILE = "jobToken";
-
-  /**
-   * conf setting for job tokens cache file name
-   */
-  @InterfaceAudience.Private
-  public static final String JOB_TOKENS_FILENAME = "mapreduce.job.jobTokenFile";
-  private static final Text JOB_TOKEN = new Text("JobToken");
-  private static final Text SHUFFLE_TOKEN = new Text("MapReduceShuffleToken");
-  
-  /**
-   * load job token from a file
-   * @param conf
-   * @throws IOException
-   */
-  @InterfaceAudience.Private
-  public static Credentials loadTokens(String jobTokenFile, Configuration conf) 
-  throws IOException {
-    Path localJobTokenFile = new Path ("file:///" + jobTokenFile);
-
-    Credentials ts = Credentials.readTokenStorageFile(localJobTokenFile, conf);
-
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Task: Loaded jobTokenFile from: "+
-          localJobTokenFile.toUri().getPath() 
-          +"; num of sec keys  = " + ts.numberOfSecretKeys() +
-          " Number of tokens " +  ts.numberOfTokens());
-    }
-    return ts;
-  }
-  /**
-   * store job token
-   * @param t
-   */
-  @InterfaceAudience.Private
-  public static void setJobToken(Token<? extends TokenIdentifier> t, 
-      Credentials credentials) {
-    credentials.addToken(JOB_TOKEN, t);
-  }
-  /**
-   * 
-   * @return job token
-   */
-  @SuppressWarnings("unchecked")
-  @InterfaceAudience.Private
-  public static Token<JobTokenIdentifier> getJobToken(Credentials credentials) {
-    return (Token<JobTokenIdentifier>) credentials.getToken(JOB_TOKEN);
-  }
-
-  @InterfaceAudience.Private
-  public static void setShuffleSecretKey(byte[] key, Credentials credentials) {
-    credentials.addSecretKey(SHUFFLE_TOKEN, key);
-  }
-
-  @InterfaceAudience.Private
-  public static byte[] getShuffleSecretKey(Credentials credentials) {
-    return getSecretKey(credentials, SHUFFLE_TOKEN);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ExceptionReporter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ExceptionReporter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ExceptionReporter.java
deleted file mode 100644
index 89a5888..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ExceptionReporter.java
+++ /dev/null
@@ -1,25 +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.shuffle.impl;
-
-/**
- * An interface for reporting exceptions to other threads
- */
-interface ExceptionReporter {
-  void reportException(Throwable t);
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index b48bb0b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Fetcher.java
+++ /dev/null
@@ -1,624 +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.shuffle.impl;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.HttpURLConnection;
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.net.URLConnection;
-import java.security.GeneralSecurityException;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import javax.crypto.SecretKey;
-import javax.net.ssl.HttpsURLConnection;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.compress.CodecPool;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.io.compress.DefaultCodec;
-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 com.google.common.annotations.VisibleForTesting;
-
-class Fetcher extends Thread {
-  
-  private static final Log LOG = LogFactory.getLog(Fetcher.class);
-  
-  /** Basic/unit connection timeout (in milliseconds) */
-  private final static int UNIT_CONNECT_TIMEOUT = 60 * 1000;
-
-  private static enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
-                                    CONNECTION, WRONG_REDUCE}
-  
-  private final static String SHUFFLE_ERR_GRP_NAME = "Shuffle Errors";
-  private final TezCounter connectionErrs;
-  private final TezCounter ioErrs;
-  private final TezCounter wrongLengthErrs;
-  private final TezCounter badIdErrs;
-  private final TezCounter wrongMapErrs;
-  private final TezCounter wrongReduceErrs;
-  private final MergeManager merger;
-  private final ShuffleScheduler scheduler;
-  private final ShuffleClientMetrics metrics;
-  private final Shuffle shuffle;
-  private final int id;
-  private static int nextId = 0;
-  
-  private final int connectionTimeout;
-  private final int readTimeout;
-  
-  // Decompression of map-outputs
-  private final CompressionCodec codec;
-  private final Decompressor decompressor;
-  private final SecretKey jobTokenSecret;
-
-  private volatile boolean stopped = false;
-
-  private Configuration job;
-
-  private static boolean sslShuffle;
-  private static SSLFactory sslFactory;
-
-  public Fetcher(Configuration job, 
-      ShuffleScheduler scheduler, MergeManager merger,
-      ShuffleClientMetrics metrics,
-      Shuffle shuffle, SecretKey jobTokenSecret, TezInputContext inputContext) throws IOException {
-    this.job = job;
-    this.scheduler = scheduler;
-    this.merger = merger;
-    this.metrics = metrics;
-    this.shuffle = shuffle;
-    this.id = ++nextId;
-    this.jobTokenSecret = jobTokenSecret;
-    ioErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.IO_ERROR.toString());
-    wrongLengthErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.WRONG_LENGTH.toString());
-    badIdErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.BAD_ID.toString());
-    wrongMapErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.WRONG_MAP.toString());
-    connectionErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.CONNECTION.toString());
-    wrongReduceErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
-        ShuffleErrors.WRONG_REDUCE.toString());
-
-    if (ConfigUtils.isIntermediateInputCompressed(job)) {
-      Class<? extends CompressionCodec> codecClass =
-          ConfigUtils.getIntermediateInputCompressorClass(job, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, job);
-      decompressor = CodecPool.getDecompressor(codec);
-    } else {
-      codec = null;
-      decompressor = null;
-    }
-
-    this.connectionTimeout = 
-        job.getInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT,
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_STALLED_COPY_TIMEOUT);
-    this.readTimeout = 
-        job.getInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_READ_TIMEOUT, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_READ_TIMEOUT);
-
-    setName("fetcher#" + id);
-    setDaemon(true);
-
-    synchronized (Fetcher.class) {
-      sslShuffle = job.getBoolean(TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL,
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL);
-      if (sslShuffle && sslFactory == null) {
-        sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, job);
-        try {
-          sslFactory.init();
-        } catch (Exception ex) {
-          sslFactory.destroy();
-          throw new RuntimeException(ex);
-        }
-      }
-    }
-  }
-  
-  public void run() {
-    try {
-      while (!stopped && !Thread.currentThread().isInterrupted()) {
-        MapHost host = null;
-        try {
-          // If merge is on, block
-          merger.waitForInMemoryMerge();
-
-          // Get a host to shuffle from
-          host = scheduler.getHost();
-          metrics.threadBusy();
-
-          // Shuffle
-          copyFromHost(host);
-        } finally {
-          if (host != null) {
-            scheduler.freeHost(host);
-            metrics.threadFree();            
-          }
-        }
-      }
-    } catch (InterruptedException ie) {
-      return;
-    } catch (Throwable t) {
-      shuffle.reportException(t);
-    }
-  }
-
-  public void shutDown() throws InterruptedException {
-    this.stopped = true;
-    interrupt();
-    try {
-      join(5000);
-    } catch (InterruptedException ie) {
-      LOG.warn("Got interrupt while joining " + getName(), ie);
-    }
-    if (sslFactory != null) {
-      sslFactory.destroy();
-    }
-  }
-
-  @VisibleForTesting
-  protected HttpURLConnection openConnection(URL url) throws IOException {
-    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    if (sslShuffle) {
-      HttpsURLConnection httpsConn = (HttpsURLConnection) conn;
-      try {
-        httpsConn.setSSLSocketFactory(sslFactory.createSSLSocketFactory());
-      } catch (GeneralSecurityException ex) {
-        throw new IOException(ex);
-      }
-      httpsConn.setHostnameVerifier(sslFactory.getHostnameVerifier());
-    }
-    return conn;
-  }
-  
-  /**
-   * The crux of the matter...
-   * 
-   * @param host {@link MapHost} from which we need to  
-   *              shuffle available map-outputs.
-   */
-  @VisibleForTesting
-  protected void copyFromHost(MapHost host) throws IOException {
-    // Get completed maps on 'host'
-    List<InputAttemptIdentifier> srcAttempts = scheduler.getMapsForHost(host);
-    
-    // Sanity check to catch hosts with only 'OBSOLETE' maps, 
-    // especially at the tail of large jobs
-    if (srcAttempts.size() == 0) {
-      return;
-    }
-    
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: "
-        + srcAttempts);
-    }
-    
-    // List of maps to be fetched yet
-    Set<InputAttemptIdentifier> remaining = new HashSet<InputAttemptIdentifier>(srcAttempts);
-    
-    // Construct the url and connect
-    DataInputStream input;
-    boolean connectSucceeded = false;
-    
-    try {
-      URL url = getMapOutputURL(host, srcAttempts);
-      HttpURLConnection connection = openConnection(url);
-      
-      // generate hash of the url
-      String msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
-      String encHash = SecureShuffleUtils.hashFromString(msgToEncode, jobTokenSecret);
-      
-      // put url hash into http header
-      connection.addRequestProperty(
-          SecureShuffleUtils.HTTP_HEADER_URL_HASH, encHash);
-      // set the read timeout
-      connection.setReadTimeout(readTimeout);
-      // put shuffle version into http header
-      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
-      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
-      connect(connection, connectionTimeout);
-      connectSucceeded = true;
-      input = new DataInputStream(connection.getInputStream());
-
-      // Validate response code
-      int rc = connection.getResponseCode();
-      if (rc != HttpURLConnection.HTTP_OK) {
-        throw new IOException(
-            "Got invalid response code " + rc + " from " + url +
-            ": " + connection.getResponseMessage());
-      }
-      // get the shuffle version
-      if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(
-          connection.getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
-          || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(
-              connection.getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))) {
-        throw new IOException("Incompatible shuffle response version");
-      }
-      // get the replyHash which is HMac of the encHash we sent to the server
-      String replyHash = connection.getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH);
-      if(replyHash==null) {
-        throw new IOException("security validation of TT Map output failed");
-      }
-      LOG.debug("url="+msgToEncode+";encHash="+encHash+";replyHash="+replyHash);
-      // verify that replyHash is HMac of encHash
-      SecureShuffleUtils.verifyReply(replyHash, encHash, jobTokenSecret);
-      LOG.info("for url="+msgToEncode+" sent hash and receievd reply");
-    } catch (IOException ie) {
-      ioErrs.increment(1);
-      LOG.warn("Failed to connect to " + host + " with " + remaining.size() + 
-               " map outputs", ie);
-
-      // If connect did not succeed, just mark all the maps as failed,
-      // indirectly penalizing the host
-      if (!connectSucceeded) {
-        for(InputAttemptIdentifier left: remaining) {
-          scheduler.copyFailed(left, host, connectSucceeded);
-        }
-      } else {
-        // If we got a read error at this stage, it implies there was a problem
-        // with the first map, typically lost map. So, penalize only that map
-        // and add the rest
-        InputAttemptIdentifier firstMap = srcAttempts.get(0);
-        scheduler.copyFailed(firstMap, host, connectSucceeded);
-      }
-      
-      // Add back all the remaining maps, WITHOUT marking them as failed
-      for(InputAttemptIdentifier left: remaining) {
-        // TODO Should the first one be skipped ?
-        scheduler.putBackKnownMapOutput(host, left);
-      }
-      
-      return;
-    }
-    
-    try {
-      // Loop through available map-outputs and fetch them
-      // On any error, faildTasks is not null and we exit
-      // after putting back the remaining maps to the 
-      // yet_to_be_fetched list and marking the failed tasks.
-      InputAttemptIdentifier[] failedTasks = null;
-      while (!remaining.isEmpty() && failedTasks == null) {
-        failedTasks = copyMapOutput(host, input, remaining);
-      }
-      
-      if(failedTasks != null && failedTasks.length > 0) {
-        LOG.warn("copyMapOutput failed for tasks "+Arrays.toString(failedTasks));
-        for(InputAttemptIdentifier left: failedTasks) {
-          scheduler.copyFailed(left, host, true);
-        }
-      }
-      
-      IOUtils.cleanup(LOG, input);
-      
-      // Sanity check
-      if (failedTasks == null && !remaining.isEmpty()) {
-        throw new IOException("server didn't return all expected map outputs: "
-            + remaining.size() + " left.");
-      }
-    } finally {
-      for (InputAttemptIdentifier left : remaining) {
-        scheduler.putBackKnownMapOutput(host, left);
-      }
-    }
-  }
-  
-  private static InputAttemptIdentifier[] EMPTY_ATTEMPT_ID_ARRAY = new InputAttemptIdentifier[0];
-  
-  private InputAttemptIdentifier[] copyMapOutput(MapHost host,
-                                DataInputStream input,
-                                Set<InputAttemptIdentifier> remaining) {
-    MapOutput mapOutput = null;
-    InputAttemptIdentifier srcAttemptId = null;
-    long decompressedLength = -1;
-    long compressedLength = -1;
-    
-    try {
-      long startTime = System.currentTimeMillis();
-      int forReduce = -1;
-      //Read the shuffle header
-      try {
-        ShuffleHeader header = new ShuffleHeader();
-        header.readFields(input);
-        String pathComponent = header.mapId;
-        srcAttemptId = scheduler.getIdentifierForPathComponent(pathComponent);
-        compressedLength = header.compressedLength;
-        decompressedLength = header.uncompressedLength;
-        forReduce = header.forReduce;
-      } catch (IllegalArgumentException e) {
-        badIdErrs.increment(1);
-        LOG.warn("Invalid map id ", e);
-        //Don't know which one was bad, so consider all of them as bad
-        return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
-      }
-
- 
-      // Do some basic sanity verification
-      if (!verifySanity(compressedLength, decompressedLength, forReduce,
-          remaining, srcAttemptId)) {
-        return new InputAttemptIdentifier[] {srcAttemptId};
-      }
-      
-      if(LOG.isDebugEnabled()) {
-        LOG.debug("header: " + srcAttemptId + ", len: " + compressedLength + 
-            ", decomp len: " + decompressedLength);
-      }
-      
-      // Get the location for the map output - either in-memory or on-disk
-      mapOutput = merger.reserve(srcAttemptId, decompressedLength, id);
-      
-      // Check if we can shuffle *now* ...
-      if (mapOutput.getType() == Type.WAIT) {
-        LOG.info("fetcher#" + id + " - MergerManager returned Status.WAIT ...");
-        //Not an error but wait to process data.
-        return EMPTY_ATTEMPT_ID_ARRAY;
-      } 
-      
-      // Go!
-      LOG.info("fetcher#" + id + " about to shuffle output of map " + 
-               mapOutput.getAttemptIdentifier() + " decomp: " +
-               decompressedLength + " len: " + compressedLength + " to " +
-               mapOutput.getType());
-      if (mapOutput.getType() == Type.MEMORY) {
-        shuffleToMemory(host, mapOutput, input, 
-                        (int) decompressedLength, (int) compressedLength);
-      } else {
-        shuffleToDisk(host, mapOutput, input, compressedLength);
-      }
-      
-      // Inform the shuffle scheduler
-      long endTime = System.currentTimeMillis();
-      scheduler.copySucceeded(srcAttemptId, host, compressedLength, 
-                              endTime - startTime, mapOutput);
-      // Note successful shuffle
-      remaining.remove(srcAttemptId);
-      metrics.successFetch();
-      return null;
-    } catch (IOException ioe) {
-      ioErrs.increment(1);
-      if (srcAttemptId == null || mapOutput == null) {
-        LOG.info("fetcher#" + id + " failed to read map header" + 
-                 srcAttemptId + " decomp: " + 
-                 decompressedLength + ", " + compressedLength, ioe);
-        if(srcAttemptId == null) {
-          return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
-        } else {
-          return new InputAttemptIdentifier[] {srcAttemptId};
-        }
-      }
-      
-      LOG.warn("Failed to shuffle output of " + srcAttemptId + 
-               " from " + host.getHostName(), ioe); 
-
-      // Inform the shuffle-scheduler
-      mapOutput.abort();
-      metrics.failedFetch();
-      return new InputAttemptIdentifier[] {srcAttemptId};
-    }
-
-  }
-  
-  /**
-   * Do some basic verification on the input received -- Being defensive
-   * @param compressedLength
-   * @param decompressedLength
-   * @param forReduce
-   * @param remaining
-   * @param mapId
-   * @return true/false, based on if the verification succeeded or not
-   */
-  private boolean verifySanity(long compressedLength, long decompressedLength,
-      int forReduce, Set<InputAttemptIdentifier> remaining, InputAttemptIdentifier srcAttemptId) {
-    if (compressedLength < 0 || decompressedLength < 0) {
-      wrongLengthErrs.increment(1);
-      LOG.warn(getName() + " invalid lengths in map output header: id: " +
-          srcAttemptId + " len: " + compressedLength + ", decomp len: " + 
-               decompressedLength);
-      return false;
-    }
-    
-    int reduceStartId = shuffle.getReduceStartId();
-    int reduceRange = shuffle.getReduceRange();
-    if (forReduce < reduceStartId || forReduce >= reduceStartId+reduceRange) {
-      wrongReduceErrs.increment(1);
-      LOG.warn(getName() + " data for the wrong reduce map: " +
-               srcAttemptId + " len: " + compressedLength + " decomp len: " +
-               decompressedLength + " for reduce " + forReduce);
-      return false;
-    }
-
-    // Sanity check
-    if (!remaining.contains(srcAttemptId)) {
-      wrongMapErrs.increment(1);
-      LOG.warn("Invalid map-output! Received output for " + srcAttemptId);
-      return false;
-    }
-    
-    return true;
-  }
-
-  /**
-   * Create the map-output-url. This will contain all the map ids
-   * separated by commas
-   * @param host
-   * @param maps
-   * @return
-   * @throws MalformedURLException
-   */
-  private URL getMapOutputURL(MapHost host, List<InputAttemptIdentifier> srcAttempts
-                              )  throws MalformedURLException {
-    // Get the base url
-    StringBuffer url = new StringBuffer(host.getBaseUrl());
-    
-    boolean first = true;
-    for (InputAttemptIdentifier mapId : srcAttempts) {
-      if (!first) {
-        url.append(",");
-      }
-      url.append(mapId.getPathComponent());
-      first = false;
-    }
-   
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("MapOutput URL for " + host + " -> " + url.toString());
-    }
-    return new URL(url.toString());
-  }
-  
-  /** 
-   * The connection establishment is attempted multiple times and is given up 
-   * only on the last failure. Instead of connecting with a timeout of 
-   * X, we try connecting with a timeout of x < X but multiple times. 
-   */
-  private void connect(URLConnection connection, int connectionTimeout)
-  throws IOException {
-    int unit = 0;
-    if (connectionTimeout < 0) {
-      throw new IOException("Invalid timeout "
-                            + "[timeout = " + connectionTimeout + " ms]");
-    } else if (connectionTimeout > 0) {
-      unit = Math.min(UNIT_CONNECT_TIMEOUT, connectionTimeout);
-    }
-    // set the connect timeout to the unit-connect-timeout
-    connection.setConnectTimeout(unit);
-    while (true) {
-      try {
-        connection.connect();
-        break;
-      } catch (IOException ioe) {
-        // update the total remaining connect-timeout
-        connectionTimeout -= unit;
-
-        // throw an exception if we have waited for timeout amount of time
-        // note that the updated value if timeout is used here
-        if (connectionTimeout == 0) {
-          throw ioe;
-        }
-
-        // reset the connect timeout for the last try
-        if (connectionTimeout < unit) {
-          unit = connectionTimeout;
-          // reset the connect time out for the final connect
-          connection.setConnectTimeout(unit);
-        }
-      }
-    }
-  }
-
-  private void shuffleToMemory(MapHost host, MapOutput mapOutput, 
-                               InputStream input, 
-                               int decompressedLength, 
-                               int compressedLength) throws IOException {    
-    IFileInputStream checksumIn = 
-      new IFileInputStream(input, compressedLength, job);
-
-    input = checksumIn;       
-  
-    // Are map-outputs compressed?
-    if (codec != null) {
-      decompressor.reset();
-      input = codec.createInputStream(input, decompressor);
-    }
-  
-    // Copy map-output into an in-memory buffer
-    byte[] shuffleData = mapOutput.getMemory();
-    
-    try {
-      IOUtils.readFully(input, shuffleData, 0, shuffleData.length);
-      metrics.inputBytes(shuffleData.length);
-      LOG.info("Read " + shuffleData.length + " bytes from map-output for " +
-               mapOutput.getAttemptIdentifier());
-    } catch (IOException ioe) {      
-      // Close the streams
-      IOUtils.cleanup(LOG, input);
-
-      // Re-throw
-      throw ioe;
-    }
-
-  }
-  
-  private void shuffleToDisk(MapHost host, MapOutput mapOutput, 
-                             InputStream input, 
-                             long compressedLength) 
-  throws IOException {
-    // Copy data to local-disk
-    OutputStream output = mapOutput.getDisk();
-    long bytesLeft = compressedLength;
-    try {
-      final int BYTES_TO_READ = 64 * 1024;
-      byte[] buf = new byte[BYTES_TO_READ];
-      while (bytesLeft > 0) {
-        int n = input.read(buf, 0, (int) Math.min(bytesLeft, BYTES_TO_READ));
-        if (n < 0) {
-          throw new IOException("read past end of stream reading " + 
-                                mapOutput.getAttemptIdentifier());
-        }
-        output.write(buf, 0, n);
-        bytesLeft -= n;
-        metrics.inputBytes(n);
-      }
-
-      LOG.info("Read " + (compressedLength - bytesLeft) + 
-               " bytes from map-output for " +
-               mapOutput.getAttemptIdentifier());
-
-      output.close();
-    } catch (IOException ioe) {
-      // Close the streams
-      IOUtils.cleanup(LOG, input, output);
-
-      // Re-throw
-      throw ioe;
-    }
-
-    // Sanity check
-    if (bytesLeft != 0) {
-      throw new IOException("Incomplete map output received for " +
-                            mapOutput.getAttemptIdentifier() + " from " +
-                            host.getHostName() + " (" + 
-                            bytesLeft + " bytes missing of " + 
-                            compressedLength + ")"
-      );
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryReader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryReader.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryReader.java
deleted file mode 100644
index 1beed44..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryReader.java
+++ /dev/null
@@ -1,156 +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.shuffle.impl;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.IFile.Reader;
-
-/**
- * <code>IFile.InMemoryReader</code> to read map-outputs present in-memory.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class InMemoryReader extends Reader {
-  private final InputAttemptIdentifier taskAttemptId;
-  private final MergeManager merger;
-  DataInputBuffer memDataIn = new DataInputBuffer();
-  private int start;
-  private int length;
-  private int prevKeyPos;
-
-  public InMemoryReader(MergeManager merger, InputAttemptIdentifier taskAttemptId,
-                        byte[] data, int start, int length)
-  throws IOException {
-    super(null, null, length - start, null, null);
-    this.merger = merger;
-    this.taskAttemptId = taskAttemptId;
-
-    buffer = data;
-    bufferSize = (int)fileLength;
-    memDataIn.reset(buffer, start, length);
-    this.start = start;
-    this.length = length;
-  }
-
-  @Override
-  public void reset(int offset) {
-    memDataIn.reset(buffer, start + offset, length);
-    bytesRead = offset;
-    eof = false;
-  }
-
-  @Override
-  public long getPosition() throws IOException {
-    // InMemoryReader does not initialize streams like Reader, so in.getPos()
-    // would not work. Instead, return the number of uncompressed bytes read,
-    // which will be correct since in-memory data is not compressed.
-    return bytesRead;
-  }
-  
-  @Override
-  public long getLength() { 
-    return fileLength;
-  }
-  
-  private void dumpOnError() {
-    File dumpFile = new File("../output/" + taskAttemptId + ".dump");
-    System.err.println("Dumping corrupt map-output of " + taskAttemptId + 
-                       " to " + dumpFile.getAbsolutePath());
-    try {
-      FileOutputStream fos = new FileOutputStream(dumpFile);
-      fos.write(buffer, 0, bufferSize);
-      fos.close();
-    } catch (IOException ioe) {
-      System.err.println("Failed to dump map-output of " + taskAttemptId);
-    }
-  }
-  
-  public KeyState readRawKey(DataInputBuffer key) throws IOException {
-    try {
-      if (!positionToNextRecord(memDataIn)) {
-        return KeyState.NO_KEY;
-      }
-      // Setup the key
-      int pos = memDataIn.getPosition();
-      byte[] data = memDataIn.getData();      
-      if(currentKeyLength == IFile.RLE_MARKER) {
-        key.reset(data, prevKeyPos, prevKeyLength);
-        currentKeyLength = prevKeyLength;
-        return KeyState.SAME_KEY;
-      }      
-      key.reset(data, pos, currentKeyLength);
-      prevKeyPos = pos;
-      // Position for the next value
-      long skipped = memDataIn.skip(currentKeyLength);
-      if (skipped != currentKeyLength) {
-        throw new IOException("Rec# " + recNo + 
-            ": Failed to skip past key of length: " + 
-            currentKeyLength);
-      }
-
-      // Record the byte
-      bytesRead += currentKeyLength;
-      return KeyState.NEW_KEY;
-    } catch (IOException ioe) {
-      dumpOnError();
-      throw ioe;
-    }
-  }
-  
-  public void nextRawValue(DataInputBuffer value) throws IOException {
-    try {
-      int pos = memDataIn.getPosition();
-      byte[] data = memDataIn.getData();
-      value.reset(data, pos, currentValueLength);
-
-      // Position for the next record
-      long skipped = memDataIn.skip(currentValueLength);
-      if (skipped != currentValueLength) {
-        throw new IOException("Rec# " + recNo + 
-            ": Failed to skip past value of length: " + 
-            currentValueLength);
-      }
-      // Record the byte
-      bytesRead += currentValueLength;
-
-      ++recNo;
-    } catch (IOException ioe) {
-      dumpOnError();
-      throw ioe;
-    }
-  }
-    
-  public void close() {
-    // Release
-    dataIn = null;
-    buffer = null;
-      // Inform the MergeManager
-    if (merger != null) {
-      merger.unreserve(bufferSize);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryWriter.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryWriter.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryWriter.java
deleted file mode 100644
index b3ebb8b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/InMemoryWriter.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.engine.common.shuffle.impl;
-
-import java.io.DataOutputStream;
-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.io.BoundedByteArrayOutputStream;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.IFileOutputStream;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class InMemoryWriter extends Writer {
-  private static final Log LOG = LogFactory.getLog(InMemoryWriter.class);
-
-  private DataOutputStream out;
-
-  public InMemoryWriter(BoundedByteArrayOutputStream arrayStream) {
-    super(null);
-    this.out =
-      new DataOutputStream(new IFileOutputStream(arrayStream));
-  }
-
-  public void append(Object key, Object value) throws IOException {
-    throw new UnsupportedOperationException
-    ("InMemoryWriter.append(K key, V value");
-  }
-
-  public void append(DataInputBuffer key, DataInputBuffer value)
-  throws IOException {
-    int keyLength = key.getLength() - key.getPosition();
-    if (keyLength < 0) {
-      throw new IOException("Negative key-length not allowed: " + keyLength +
-                            " for " + key);
-    }
-
-    boolean sameKey = (key == IFile.REPEAT_KEY);
-
-    int valueLength = value.getLength() - value.getPosition();
-    if (valueLength < 0) {
-      throw new IOException("Negative value-length not allowed: " +
-                            valueLength + " for " + value);
-    }
-
-    if(sameKey) {
-      WritableUtils.writeVInt(out, IFile.RLE_MARKER);
-      WritableUtils.writeVInt(out, valueLength);
-      out.write(value.getData(), value.getPosition(), valueLength);
-    } else {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("InMemWriter.append" +
-            " key.data=" + key.getData() +
-            " key.pos=" + key.getPosition() +
-            " key.len=" +key.getLength() +
-            " val.data=" + value.getData() +
-            " val.pos=" + value.getPosition() +
-            " val.len=" + value.getLength());
-      }
-      WritableUtils.writeVInt(out, keyLength);
-      WritableUtils.writeVInt(out, valueLength);
-      out.write(key.getData(), key.getPosition(), keyLength);
-      out.write(value.getData(), value.getPosition(), valueLength);
-    }
-
-  }
-
-  public void close() throws IOException {
-    // Write EOF_MARKER for key/value length
-    WritableUtils.writeVInt(out, IFile.EOF_MARKER);
-    WritableUtils.writeVInt(out, IFile.EOF_MARKER);
-
-    // Close the stream
-    out.close();
-    out = null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapHost.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapHost.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapHost.java
deleted file mode 100644
index 20ec472..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapHost.java
+++ /dev/null
@@ -1,124 +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.shuffle.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-
-@Private
-class MapHost {
-  
-  public static enum State {
-    IDLE,               // No map outputs available
-    BUSY,               // Map outputs are being fetched
-    PENDING,            // Known map outputs which need to be fetched
-    PENALIZED           // Host penalized due to shuffle failures
-  }
-  
-  private State state = State.IDLE;
-  private final String hostName;
-  private final int partitionId;
-  private final String baseUrl;
-  private final String identifier;
-  // Tracks attempt IDs
-  private List<InputAttemptIdentifier> maps = new ArrayList<InputAttemptIdentifier>();
-  
-  public MapHost(int partitionId, String hostName, String baseUrl) {
-    this.partitionId = partitionId;
-    this.hostName = hostName;
-    this.baseUrl = baseUrl;
-    this.identifier = createIdentifier(hostName, partitionId);
-  }
-  
-  public static String createIdentifier(String hostName, int partitionId) {
-    return hostName + ":" + Integer.toString(partitionId);
-  }
-  
-  public String getIdentifier() {
-    return identifier;
-  }
-  
-  public int getPartitionId() {
-    return partitionId;
-  }
-
-  public State getState() {
-    return state;
-  }
-
-  public String getHostName() {
-    return hostName;
-  }
-
-  public String getBaseUrl() {
-    return baseUrl;
-  }
-
-  public synchronized void addKnownMap(InputAttemptIdentifier srcAttempt) {
-    maps.add(srcAttempt);
-    if (state == State.IDLE) {
-      state = State.PENDING;
-    }
-  }
-
-  public synchronized List<InputAttemptIdentifier> getAndClearKnownMaps() {
-    List<InputAttemptIdentifier> currentKnownMaps = maps;
-    maps = new ArrayList<InputAttemptIdentifier>();
-    return currentKnownMaps;
-  }
-  
-  public synchronized void markBusy() {
-    state = State.BUSY;
-  }
-  
-  public synchronized void markPenalized() {
-    state = State.PENALIZED;
-  }
-  
-  public synchronized int getNumKnownMapOutputs() {
-    return maps.size();
-  }
-
-  /**
-   * Called when the node is done with its penalty or done copying.
-   * @return the host's new state
-   */
-  public synchronized State markAvailable() {
-    if (maps.isEmpty()) {
-      state = State.IDLE;
-    } else {
-      state = State.PENDING;
-    }
-    return state;
-  }
-  
-  @Override
-  public String toString() {
-    return hostName;
-  }
-  
-  /**
-   * Mark the host as penalized
-   */
-  public synchronized void penalize() {
-    state = State.PENALIZED;
-  }
-}


[09/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java
index e5bd108..12a3740 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestDeprecatedKeys.java
@@ -21,8 +21,8 @@ package org.apache.tez.mapreduce.hadoop;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.library.common.Constants;
 import org.junit.Test;
 
 public class TestDeprecatedKeys {
@@ -43,22 +43,22 @@ public class TestDeprecatedKeys {
     MultiStageMRConfToTezTranslator.translateVertexConfToTez(jobConf, null);
 
     assertEquals(0.4f, jobConf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT, 0f), 0.01f);
-    assertEquals(20000l, jobConf.getLong(Constants.TEZ_ENGINE_TASK_MEMORY, 0));
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT, 0f), 0.01f);
+    assertEquals(20000l, jobConf.getLong(Constants.TEZ_RUNTIME_TASK_MEMORY, 0));
     assertEquals(2000,
-        jobConf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 0));
+        jobConf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 0));
     assertEquals(0.55f, jobConf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT, 0), 0.01f);
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT, 0), 0.01f);
     assertEquals(0.60f,
-        jobConf.getFloat(TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS, 0),
+        jobConf.getFloat(TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS, 0),
         0.01f);
     assertEquals(0.22f,
-        jobConf.getFloat(TezJobConfig.TEZ_ENGINE_SHUFFLE_MERGE_PERCENT, 0),
+        jobConf.getFloat(TezJobConfig.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT, 0),
         0.01f);
     assertEquals(true, jobConf.getBoolean(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM, false));
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM, false));
     assertEquals(0.33f,
-        jobConf.getFloat(TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT, 0),
+        jobConf.getFloat(TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT, 0),
         0.01f);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 4b2c0e8..9590e72 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/MapUtils.java
@@ -48,15 +48,15 @@ import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.api.impl.TezUmbilical;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
 
 public class MapUtils {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 89292ab..5b8eedf 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/map/TestMapProcessor.java
@@ -31,19 +31,9 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.MRConfig;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.OutputDescriptor;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
-import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TestUmbilical;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.mapreduce.hadoop.MultiStageMRConfToTezTranslator;
@@ -51,6 +41,16 @@ import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
 import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.partition.MRPartitioner;
 import org.apache.tez.mapreduce.processor.MapUtils;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.task.local.output.TezLocalTaskOutputFiles;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.output.LocalOnFileSorterOutput;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -85,10 +85,10 @@ public class TestMapProcessor {
     job.set(TezJobConfig.LOCAL_DIRS, workDir.toString());
     job.set(MRConfig.LOCAL_DIR, workDir.toString());
     job.setClass(
-        Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+        Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
         TezLocalTaskOutputFiles.class, 
         TezTaskOutput.class);
-    job.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
+    job.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS, MRPartitioner.class.getName());
     job.setNumReduceTasks(1);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 274c353..d2c7952 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/processor/reduce/TestReduceProcessor.java
@@ -33,21 +33,11 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.SequenceFileOutputFormat;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.security.token.Token;
-import org.apache.tez.common.Constants;
 import org.apache.tez.common.TezJobConfig;
 import org.apache.tez.common.TezUtils;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.OutputDescriptor;
 import org.apache.tez.dag.api.ProcessorDescriptor;
-import org.apache.tez.engine.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezLocalTaskOutputFiles;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.lib.input.LocalMergedInput;
-import org.apache.tez.engine.lib.output.LocalOnFileSorterOutput;
-import org.apache.tez.engine.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.TestUmbilical;
 import org.apache.tez.mapreduce.TezTestUtils;
 import org.apache.tez.mapreduce.hadoop.IDConverter;
@@ -58,6 +48,16 @@ import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.output.MROutput;
 import org.apache.tez.mapreduce.partition.MRPartitioner;
 import org.apache.tez.mapreduce.processor.MapUtils;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.task.local.output.TezLocalTaskOutputFiles;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.input.LocalMergedInput;
+import org.apache.tez.runtime.library.output.LocalOnFileSorterOutput;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -90,10 +90,10 @@ public class TestReduceProcessor {
     job.set(TezJobConfig.LOCAL_DIRS, workDir.toString());
     job.set(MRConfig.LOCAL_DIR, workDir.toString());
     job.setClass(
-        Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
+        Constants.TEZ_RUNTIME_TASK_OUTPUT_MANAGER,
         TezLocalTaskOutputFiles.class, 
         TezTaskOutput.class);
-    job.set(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS, MRPartitioner.class.getName());
+    job.set(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS, MRPartitioner.class.getName());
     job.setNumReduceTasks(1);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/findbugs-exclude.xml b/tez-runtime-internals/findbugs-exclude.xml
new file mode 100644
index 0000000..5b11308
--- /dev/null
+++ b/tez-runtime-internals/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/b212ca1d/tez-runtime-internals/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml
new file mode 100644
index 0000000..4f64701
--- /dev/null
+++ b/tez-runtime-internals/pom.xml
@@ -0,0 +1,95 @@
+<?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-runtime-internals</artifactId>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-runtime-library</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-common</artifactId>
+    </dependency>
+    <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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerContext.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerContext.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerContext.java
new file mode 100644
index 0000000..df92bdc
--- /dev/null
+++ b/tez-runtime-internals/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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerTask.java
new file mode 100644
index 0000000..c865631
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/ContainerTask.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.tez.common;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+
+public class ContainerTask implements Writable {
+
+  TaskSpec taskSpec;
+  boolean shouldDie;
+
+  public ContainerTask() {
+  }
+
+  public ContainerTask(TaskSpec taskSpec, boolean shouldDie) {
+    this.taskSpec = taskSpec;
+    this.shouldDie = shouldDie;
+  }
+
+  public TaskSpec getTaskSpec() {
+    return taskSpec;
+  }
+
+  public boolean shouldDie() {
+    return shouldDie;
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeBoolean(shouldDie);
+    if (taskSpec != null) {
+      out.writeBoolean(true);
+      taskSpec.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    shouldDie = in.readBoolean();
+    boolean taskComing = in.readBoolean();
+    if (taskComing) {
+      taskSpec = new TaskSpec();
+      taskSpec.readFields(in);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "shouldDie: " + shouldDie + ", TaskSpec: "
+        + taskSpec;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
new file mode 100644
index 0000000..1966790
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.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 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.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+
+/** Protocol that task child process uses to contact its parent process.  The
+ * parent is a daemon which which polls the central master for a new map or
+ * reduce task and runs it as a child process.  All communication between child
+ * and parent is via this protocol. */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+@ProtocolInfo(protocolName = "TezTaskUmbilicalProtocol", protocolVersion = 1)
+public interface TezTaskUmbilicalProtocol extends VersionedProtocol {
+
+  public static final long versionID = 19L;
+
+  ContainerTask getTask(ContainerContext containerContext) throws IOException;
+
+  boolean canCommit(TezTaskAttemptID taskid) throws IOException;
+
+  ProceedToCompletionResponse
+      proceedToCompletion(TezTaskAttemptID taskAttemptId) throws IOException;
+
+  /// Copies from TezUmbilical until complete re-factor is done
+  // TODONEWTEZ
+
+  public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
+      throws IOException, TezException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
new file mode 100644
index 0000000..8aff6d1
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/LogicalIOProcessorRuntimeTask.java
@@ -0,0 +1,475 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.token.Token;
+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.runtime.api.Event;
+import org.apache.tez.runtime.api.Input;
+import org.apache.tez.runtime.api.LogicalIOProcessor;
+import org.apache.tez.runtime.api.LogicalInput;
+import org.apache.tez.runtime.api.LogicalOutput;
+import org.apache.tez.runtime.api.Output;
+import org.apache.tez.runtime.api.Processor;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezInputContextImpl;
+import org.apache.tez.runtime.api.impl.TezOutputContextImpl;
+import org.apache.tez.runtime.api.impl.TezProcessorContextImpl;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+@Private
+public class LogicalIOProcessorRuntimeTask extends RuntimeTask {
+
+  private static final Log LOG = LogFactory
+      .getLog(LogicalIOProcessorRuntimeTask.class);
+
+  private final List<InputSpec> inputSpecs;
+  private final List<LogicalInput> inputs;
+
+  private final List<OutputSpec> outputSpecs;
+  private final List<LogicalOutput> outputs;
+
+  private List<TezInputContext> inputContexts;
+  private List<TezOutputContext> outputContexts;
+  private TezProcessorContext processorContext;
+  
+  private final ProcessorDescriptor processorDescriptor;
+  private final LogicalIOProcessor processor;
+
+  private final Map<String, ByteBuffer> serviceConsumerMetadata;
+
+  private Map<String, LogicalInput> inputMap;
+  private Map<String, LogicalOutput> outputMap;
+
+  private LinkedBlockingQueue<TezEvent> eventsToBeProcessed;
+  private Thread eventRouterThread = null;
+
+  private final int appAttemptNumber;
+
+  public LogicalIOProcessorRuntimeTask(TaskSpec taskSpec, int appAttemptNumber,
+      Configuration tezConf, TezUmbilical tezUmbilical,
+      Token<JobTokenIdentifier> jobToken) throws IOException {
+    // TODO Remove jobToken from here post TEZ-421
+    super(taskSpec, tezConf, tezUmbilical);
+    LOG.info("Initializing LogicalIOProcessorRuntimeTask with TaskSpec: "
+        + taskSpec);
+    this.inputContexts = new ArrayList<TezInputContext>(taskSpec.getInputs().size());
+    this.outputContexts = new ArrayList<TezOutputContext>(taskSpec.getOutputs().size());
+    this.inputSpecs = taskSpec.getInputs();
+    this.inputs = createInputs(inputSpecs);
+    this.outputSpecs = taskSpec.getOutputs();
+    this.outputs = createOutputs(outputSpecs);
+    this.processorDescriptor = taskSpec.getProcessorDescriptor();
+    this.processor = createProcessor(processorDescriptor);
+    this.serviceConsumerMetadata = new HashMap<String, ByteBuffer>();
+    this.serviceConsumerMetadata.put(ShuffleUtils.SHUFFLE_HANDLER_SERVICE_ID,
+        ShuffleUtils.convertJobTokenToBytes(jobToken));
+    this.eventsToBeProcessed = new LinkedBlockingQueue<TezEvent>();
+    this.state = State.NEW;
+    this.appAttemptNumber = appAttemptNumber;
+  }
+
+  public void initialize() throws Exception {
+    LOG.info("Initializing LogicalProcessorIORuntimeTask");
+    Preconditions.checkState(this.state == State.NEW, "Already initialized");
+    this.state = State.INITED;
+    inputMap = new LinkedHashMap<String, LogicalInput>(inputs.size());
+    outputMap = new LinkedHashMap<String, LogicalOutput>(outputs.size());
+
+    // TODO Maybe close initialized inputs / outputs in case of failure to
+    // initialize.
+    // Initialize all inputs. TODO: Multi-threaded at some point.
+    for (int i = 0; i < inputs.size(); i++) {
+      String srcVertexName = inputSpecs.get(i).getSourceVertexName();
+      initializeInput(inputs.get(i),
+          inputSpecs.get(i));
+      inputMap.put(srcVertexName, inputs.get(i));
+    }
+
+    // Initialize all outputs. TODO: Multi-threaded at some point.
+    for (int i = 0; i < outputs.size(); i++) {
+      String destVertexName = outputSpecs.get(i).getDestinationVertexName();
+      initializeOutput(outputs.get(i), outputSpecs.get(i));
+      outputMap.put(destVertexName, outputs.get(i));
+    }
+
+    // Initialize processor.
+    initializeLogicalIOProcessor();
+    startRouterThread();
+  }
+
+  public void run() throws Exception {
+    synchronized (this.state) {
+      Preconditions.checkState(this.state == State.INITED,
+          "Can only run while in INITED state. Current: " + this.state);
+      this.state = State.RUNNING;
+    }
+    LogicalIOProcessor lioProcessor = (LogicalIOProcessor) processor;
+    lioProcessor.run(inputMap, outputMap);
+  }
+
+  public void close() throws Exception {
+    try {
+      Preconditions.checkState(this.state == State.RUNNING,
+          "Can only run while in RUNNING state. Current: " + this.state);
+      this.state = State.CLOSED;
+
+      // Close the Inputs.
+      for (int i = 0; i < inputs.size(); i++) {
+        String srcVertexName = inputSpecs.get(i).getSourceVertexName();
+        List<Event> closeInputEvents = inputs.get(i).close();
+        sendTaskGeneratedEvents(closeInputEvents,
+            EventProducerConsumerType.INPUT, taskSpec.getVertexName(),
+            srcVertexName, taskSpec.getTaskAttemptID());
+      }
+
+      // Close the Processor.
+      processor.close();
+
+      // Close the Outputs.
+      for (int i = 0; i < outputs.size(); i++) {
+        String destVertexName = outputSpecs.get(i).getDestinationVertexName();
+        List<Event> closeOutputEvents = outputs.get(i).close();
+        sendTaskGeneratedEvents(closeOutputEvents,
+            EventProducerConsumerType.OUTPUT, taskSpec.getVertexName(),
+            destVertexName, taskSpec.getTaskAttemptID());
+      }
+    } finally {
+      setTaskDone();
+      if (eventRouterThread != null) {
+        eventRouterThread.interrupt();
+      }
+    }
+  }
+
+  private void initializeInput(Input input, InputSpec inputSpec)
+      throws Exception {
+    TezInputContext tezInputContext = createInputContext(inputSpec);
+    inputContexts.add(tezInputContext);
+    if (input instanceof LogicalInput) {
+      ((LogicalInput) input).setNumPhysicalInputs(inputSpec
+          .getPhysicalEdgeCount());
+    }
+    LOG.info("Initializing Input using InputSpec: " + inputSpec);
+    List<Event> events = input.initialize(tezInputContext);
+    sendTaskGeneratedEvents(events, EventProducerConsumerType.INPUT,
+        tezInputContext.getTaskVertexName(),
+        tezInputContext.getSourceVertexName(), taskSpec.getTaskAttemptID());
+  }
+
+  private void initializeOutput(Output output, OutputSpec outputSpec)
+      throws Exception {
+    TezOutputContext tezOutputContext = createOutputContext(outputSpec);
+    outputContexts.add(tezOutputContext);
+    if (output instanceof LogicalOutput) {
+      ((LogicalOutput) output).setNumPhysicalOutputs(outputSpec
+          .getPhysicalEdgeCount());
+    }
+    LOG.info("Initializing Output using OutputSpec: " + outputSpec);
+    List<Event> events = output.initialize(tezOutputContext);
+    sendTaskGeneratedEvents(events, EventProducerConsumerType.OUTPUT,
+        tezOutputContext.getTaskVertexName(),
+        tezOutputContext.getDestinationVertexName(),
+        taskSpec.getTaskAttemptID());
+  }
+
+  private void initializeLogicalIOProcessor() throws Exception {
+    LOG.info("Initializing processor"
+        + ", processorClassName=" + processorDescriptor.getClassName());
+    TezProcessorContext processorContext = createProcessorContext();
+    this.processorContext = processorContext;
+    processor.initialize(processorContext);
+  }
+
+  private TezInputContext createInputContext(InputSpec inputSpec) {
+    TezInputContext inputContext = new TezInputContextImpl(tezConf,
+        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(),
+        inputSpec.getSourceVertexName(), taskSpec.getTaskAttemptID(),
+        tezCounters,
+        inputSpec.getInputDescriptor().getUserPayload() == null ? taskSpec
+            .getProcessorDescriptor().getUserPayload() : inputSpec
+            .getInputDescriptor().getUserPayload(), this,
+        serviceConsumerMetadata);
+    return inputContext;
+  }
+
+  private TezOutputContext createOutputContext(OutputSpec outputSpec) {
+    TezOutputContext outputContext = new TezOutputContextImpl(tezConf,
+        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(),
+        outputSpec.getDestinationVertexName(), taskSpec.getTaskAttemptID(),
+        tezCounters,
+        outputSpec.getOutputDescriptor().getUserPayload() == null ? taskSpec
+            .getProcessorDescriptor().getUserPayload() : outputSpec
+            .getOutputDescriptor().getUserPayload(), this,
+        serviceConsumerMetadata);
+    return outputContext;
+  }
+
+  private TezProcessorContext createProcessorContext() {
+    TezProcessorContext processorContext = new TezProcessorContextImpl(tezConf,
+        appAttemptNumber, tezUmbilical, taskSpec.getVertexName(), taskSpec.getTaskAttemptID(),
+        tezCounters, processorDescriptor.getUserPayload(), this,
+        serviceConsumerMetadata);
+    return processorContext;
+  }
+
+  private List<LogicalInput> createInputs(List<InputSpec> inputSpecs) {
+    List<LogicalInput> inputs = new ArrayList<LogicalInput>(inputSpecs.size());
+    for (InputSpec inputSpec : inputSpecs) {
+      LOG.info("Creating Input from InputSpec: "
+          + inputSpec);
+      Input input = RuntimeUtils.createClazzInstance(inputSpec
+          .getInputDescriptor().getClassName());
+
+      if (input instanceof LogicalInput) {
+        inputs.add((LogicalInput) input);
+      } else {
+        throw new TezUncheckedException(input.getClass().getName()
+            + " is not a sub-type of LogicalInput."
+            + " Only LogicalInput sub-types supported by LogicalIOProcessor.");
+      }
+    }
+    return inputs;
+  }
+
+  private List<LogicalOutput> createOutputs(List<OutputSpec> outputSpecs) {
+    List<LogicalOutput> outputs = new ArrayList<LogicalOutput>(
+        outputSpecs.size());
+    for (OutputSpec outputSpec : outputSpecs) {
+      LOG.info("Creating Output from OutputSpec"
+          + outputSpec);
+      Output output = RuntimeUtils.createClazzInstance(outputSpec
+          .getOutputDescriptor().getClassName());
+      if (output instanceof LogicalOutput) {
+        outputs.add((LogicalOutput) output);
+      } else {
+        throw new TezUncheckedException(output.getClass().getName()
+            + " is not a sub-type of LogicalOutput."
+            + " Only LogicalOutput sub-types supported by LogicalIOProcessor.");
+      }
+    }
+    return outputs;
+  }
+
+  private LogicalIOProcessor createProcessor(
+      ProcessorDescriptor processorDescriptor) {
+    Processor processor = RuntimeUtils.createClazzInstance(processorDescriptor
+        .getClassName());
+    if (!(processor instanceof LogicalIOProcessor)) {
+      throw new TezUncheckedException(processor.getClass().getName()
+          + " is not a sub-type of LogicalIOProcessor."
+          + " Only LogicalOutput sub-types supported by LogicalIOProcessor.");
+    }
+    return (LogicalIOProcessor) processor;
+  }
+
+  private void sendTaskGeneratedEvents(List<Event> events,
+      EventProducerConsumerType generator, String taskVertexName,
+      String edgeVertexName, TezTaskAttemptID taskAttemptID) {
+    if (events == null || events.isEmpty()) {
+      return;
+    }
+    EventMetaData eventMetaData = new EventMetaData(generator,
+        taskVertexName, edgeVertexName, taskAttemptID);
+    List<TezEvent> tezEvents = new ArrayList<TezEvent>(events.size());
+    for (Event e : events) {
+      TezEvent te = new TezEvent(e, eventMetaData);
+      tezEvents.add(te);
+    }
+    if (LOG.isDebugEnabled()) {
+      for (TezEvent e : tezEvents) {
+        LOG.debug("Generated event info"
+            + ", eventMetaData=" + eventMetaData.toString()
+            + ", eventType=" + e.getEventType());
+      }
+    }
+    tezUmbilical.addEvents(tezEvents);
+  }
+
+  private boolean handleEvent(TezEvent e) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Handling TezEvent in task"
+          + ", taskAttemptId=" + taskSpec.getTaskAttemptID()
+          + ", eventType=" + e.getEventType()
+          + ", eventSourceInfo=" + e.getSourceInfo()
+          + ", eventDestinationInfo=" + e.getDestinationInfo());
+    }
+    try {
+      switch (e.getDestinationInfo().getEventGenerator()) {
+      case INPUT:
+        LogicalInput input = inputMap.get(
+            e.getDestinationInfo().getEdgeVertexName());
+        if (input != null) {
+          input.handleEvents(Collections.singletonList(e.getEvent()));
+        } else {
+          throw new TezUncheckedException("Unhandled event for invalid target: "
+              + e);
+        }
+        break;
+      case OUTPUT:
+        LogicalOutput output = outputMap.get(
+            e.getDestinationInfo().getEdgeVertexName());
+        if (output != null) {
+          output.handleEvents(Collections.singletonList(e.getEvent()));
+        } else {
+          throw new TezUncheckedException("Unhandled event for invalid target: "
+              + e);
+        }
+        break;
+      case PROCESSOR:
+        processor.handleEvents(Collections.singletonList(e.getEvent()));
+        break;
+      case SYSTEM:
+        LOG.warn("Trying to send a System event in a Task: " + e);
+        break;
+      }
+    } catch (Throwable t) {
+      LOG.warn("Failed to handle event", t);
+      setFatalError(t, "Failed to handle event");
+      EventMetaData sourceInfo = new EventMetaData(
+          e.getDestinationInfo().getEventGenerator(),
+          taskSpec.getVertexName(), e.getDestinationInfo().getEdgeVertexName(),
+          getTaskAttemptID());
+      tezUmbilical.signalFatalError(getTaskAttemptID(),
+          StringUtils.stringifyException(t), sourceInfo);
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public synchronized void handleEvents(Collection<TezEvent> events) {
+    if (events == null || events.isEmpty()) {
+      return;
+    }
+    eventCounter.addAndGet(events.size());
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Received events to be processed by task"
+          + ", taskAttemptId=" + taskSpec.getTaskAttemptID()
+          + ", eventCount=" + events.size()
+          + ", newEventCounter=" + eventCounter.get());
+    }
+    eventsToBeProcessed.addAll(events);
+  }
+
+  private void startRouterThread() {
+    eventRouterThread = new Thread(new Runnable() {
+      public void run() {
+        while (!isTaskDone() && !Thread.currentThread().isInterrupted()) {
+          try {
+            TezEvent e = eventsToBeProcessed.take();
+            if (e == null) {
+              continue;
+            }
+            // TODO TODONEWTEZ
+            if (!handleEvent(e)) {
+              LOG.warn("Stopping Event Router thread as failed to handle"
+                  + " event: " + e);
+              return;
+            }
+          } catch (InterruptedException e) {
+            if (!isTaskDone()) {
+              LOG.warn("Event Router thread interrupted. Returning.");
+            }
+            return;
+          }
+        }
+      }
+    });
+
+    eventRouterThread.setName("TezTaskEventRouter["
+        + taskSpec.getTaskAttemptID().toString() + "]");
+    eventRouterThread.start();
+  }
+
+  public synchronized void cleanup() {
+    setTaskDone();
+    if (eventRouterThread != null) {
+      eventRouterThread.interrupt();
+    }
+  }
+  
+  @Private
+  @VisibleForTesting
+  public List<TezInputContext> getInputContexts() {
+    return this.inputContexts;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public List<TezOutputContext> getOutputContexts() {
+    return this.outputContexts;
+  }
+
+  @Private
+  @VisibleForTesting
+  public TezProcessorContext getProcessorContext() {
+    return this.processorContext;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public Map<String, LogicalInput> getInputs() {
+    return this.inputMap;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public Map<String, LogicalOutput> getOutputs() {
+    return this.outputMap;
+  }
+  
+  @Private
+  @VisibleForTesting
+  public LogicalIOProcessor getProcessor() {
+    return this.processor;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
new file mode 100644
index 0000000..f018333
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeTask.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime;
+
+import java.util.Collection;
+import java.util.concurrent.atomic.AtomicBoolean;
+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.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
+
+public abstract class RuntimeTask {
+
+  protected AtomicBoolean hasFatalError = new AtomicBoolean(false);
+  protected Throwable fatalError = null;
+  protected String fatalErrorMessage = null;
+  protected float progress;
+  protected final TezCounters tezCounters;
+  protected final TaskSpec taskSpec;
+  protected final Configuration tezConf;
+  protected final TezUmbilical tezUmbilical;
+  protected final AtomicInteger eventCounter;
+  private final AtomicBoolean taskDone;
+
+  protected RuntimeTask(TaskSpec taskSpec, Configuration tezConf,
+      TezUmbilical tezUmbilical) {
+    this.taskSpec = taskSpec;
+    this.tezConf = tezConf;
+    this.tezUmbilical = tezUmbilical;
+    this.tezCounters = new TezCounters();
+    this.eventCounter = new AtomicInteger(0);
+    this.progress = 0.0f;
+    this.taskDone = new AtomicBoolean(false);
+  }
+
+  protected enum State {
+    NEW, INITED, RUNNING, CLOSED;
+  }
+
+  protected State state;
+
+  public String getVertexName() {
+    return taskSpec.getVertexName();
+  }
+
+  public void setFatalError(Throwable t, String message) {
+    hasFatalError.set(true);
+    this.fatalError = t;
+    this.fatalErrorMessage = message;
+  }
+
+  public boolean hadFatalError() {
+    return hasFatalError.get();
+  }
+
+  public synchronized void setProgress(float progress) {
+    this.progress = progress;
+  }
+
+  public synchronized float getProgress() {
+    return this.progress;
+  }
+
+  public TezCounters getCounters() {
+    return this.tezCounters;
+  }
+
+  public TezTaskAttemptID getTaskAttemptID() {
+    return taskSpec.getTaskAttemptID();
+  }
+
+  public abstract void handleEvents(Collection<TezEvent> events);
+
+  public int getEventCounter() {
+    return eventCounter.get();
+  }
+
+  public boolean isTaskDone() {
+    return taskDone.get();
+  }
+
+  protected void setTaskDone() {
+    taskDone.set(true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeUtils.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeUtils.java
new file mode 100644
index 0000000..8a1b550
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/RuntimeUtils.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.tez.dag.api.TezUncheckedException;
+
+public class RuntimeUtils {
+
+  private static final Map<String, Class<?>> CLAZZ_CACHE = new ConcurrentHashMap<String, Class<?>>();
+
+  private static Class<?> getClazz(String className) {
+    Class<?> clazz = CLAZZ_CACHE.get(className);
+    if (clazz == null) {
+      try {
+        clazz = Class.forName(className);
+      } catch (ClassNotFoundException e) {
+        throw new TezUncheckedException("Unable to load class: " + className, e);
+      }
+    }
+    return clazz;
+  }
+
+  private static <T> T getNewInstance(Class<T> clazz) {
+    T instance;
+    try {
+      instance = clazz.newInstance();
+    } catch (InstantiationException e) {
+      throw new TezUncheckedException(
+          "Unable to instantiate class with 0 arguments: " + clazz.getName(), e);
+    } catch (IllegalAccessException e) {
+      throw new TezUncheckedException(
+          "Unable to instantiate class with 0 arguments: " + clazz.getName(), e);
+    }
+    return instance;
+  }
+
+  public static <T> T createClazzInstance(String className) {
+    Class<?> clazz = getClazz(className);
+    @SuppressWarnings("unchecked")
+    T instance = (T) getNewInstance(clazz);
+    return instance;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptCompletedEvent.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptCompletedEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptCompletedEvent.java
new file mode 100644
index 0000000..597718f
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.api.events;
+
+import org.apache.tez.runtime.api.Event;
+
+public class TaskAttemptCompletedEvent extends Event {
+
+  public TaskAttemptCompletedEvent() {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptFailedEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskAttemptFailedEvent.java
new file mode 100644
index 0000000..935fdbb
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.api.events;
+
+import org.apache.tez.runtime.api.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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskStatusUpdateEvent.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskStatusUpdateEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/events/TaskStatusUpdateEvent.java
new file mode 100644
index 0000000..47c2998
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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.runtime.api.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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventMetaData.java
new file mode 100644
index 0000000..d650fa3
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/EventType.java
new file mode 100644
index 0000000..81ff5fc
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/InputSpec.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/InputSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/InputSpec.java
new file mode 100644
index 0000000..78ed886
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/OutputSpec.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/OutputSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/OutputSpec.java
new file mode 100644
index 0000000..4034cdd
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TaskSpec.java
new file mode 100644
index 0000000..6e0995a
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
new file mode 100644
index 0000000..e195cf9
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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.runtime.api.Event;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.EventProtos.DataMovementEventProto;
+import org.apache.tez.runtime.api.events.EventProtos.InputFailedEventProto;
+import org.apache.tez.runtime.api.events.EventProtos.InputInformationEventProto;
+import org.apache.tez.runtime.api.events.EventProtos.InputReadErrorEventProto;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputInformationEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.internals.api.events.SystemEventProtos.TaskAttemptCompletedEventProto;
+import org.apache.tez.runtime.internals.api.events.SystemEventProtos.TaskAttemptFailedEventProto;
+
+import com.google.protobuf.ByteString;
+
+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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatRequest.java
new file mode 100644
index 0000000..af7cebb
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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)
+        + " }";
+  }
+}


[17/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index dc1a447..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 22ae7eb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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)
-        + " }";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 245cd3b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.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/b212ca1d/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
deleted file mode 100644
index 6b42e13..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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.Event;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.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/b212ca1d/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
deleted file mode 100644
index 7ffcfd6..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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.Event;
-import org.apache.tez.engine.api.TezProcessorContext;
-import org.apache.tez.engine.api.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/b212ca1d/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
deleted file mode 100644
index d5a4037..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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.api.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/b212ca1d/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
deleted file mode 100644
index 925d87b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 1211598..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastInputManager.java
+++ /dev/null
@@ -1,138 +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.broadcast.input;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-import org.apache.tez.engine.shuffle.common.DiskFetchedInput;
-import org.apache.tez.engine.shuffle.common.FetchedInput;
-import org.apache.tez.engine.shuffle.common.FetchedInputAllocator;
-import org.apache.tez.engine.shuffle.common.FetchedInputCallback;
-import org.apache.tez.engine.shuffle.common.MemoryFetchedInput;
-
-public class BroadcastInputManager implements FetchedInputAllocator,
-    FetchedInputCallback {
-
-  private final Configuration conf;
-
-  private final TezTaskOutputFiles fileNameAllocator;
-  private final LocalDirAllocator localDirAllocator;
-
-  // Configuration parameters
-  private final long memoryLimit;
-  private final long maxSingleShuffleLimit;
-
-  private long usedMemory = 0;
-
-  public BroadcastInputManager(TezInputContext inputContext, Configuration conf) {
-    this.conf = conf;
-
-    this.fileNameAllocator = new TezTaskOutputFiles(conf,
-        inputContext.getUniqueIdentifier());
-    this.localDirAllocator = new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-
-    // Setup configuration
-    final float maxInMemCopyUse = conf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT);
-    if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0) {
-      throw new IllegalArgumentException("Invalid value for "
-          + TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT + ": "
-          + maxInMemCopyUse);
-    }
-
-    // Allow unit tests to fix Runtime memory
-    this.memoryLimit = (long) (conf.getLong(Constants.TEZ_ENGINE_TASK_MEMORY,
-        Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE)) * maxInMemCopyUse);
-
-    final float singleShuffleMemoryLimitPercent = conf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT);
-    if (singleShuffleMemoryLimitPercent <= 0.0f
-        || singleShuffleMemoryLimitPercent > 1.0f) {
-      throw new IllegalArgumentException("Invalid value for "
-          + TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
-          + singleShuffleMemoryLimitPercent);
-    }
-
-    this.maxSingleShuffleLimit = (long) (memoryLimit * singleShuffleMemoryLimitPercent);
-  }
-
-  @Override
-  public synchronized FetchedInput allocate(long size,
-      InputAttemptIdentifier inputAttemptIdentifier) throws IOException {
-    if (size > maxSingleShuffleLimit
-        || this.usedMemory + size > this.memoryLimit) {
-      return new DiskFetchedInput(size, inputAttemptIdentifier, this, conf,
-          localDirAllocator, fileNameAllocator);
-    } else {
-      this.usedMemory += size;
-      return new MemoryFetchedInput(size, inputAttemptIdentifier, this);
-    }
-  }
-
-  @Override
-  public void fetchComplete(FetchedInput fetchedInput) {
-    switch (fetchedInput.getType()) {
-    // Not tracking anything here.
-    case DISK:
-    case MEMORY:
-      break;
-    default:
-      throw new TezUncheckedException("InputType: " + fetchedInput.getType()
-          + " not expected for Broadcast fetch");
-    }
-  }
-
-  @Override
-  public void fetchFailed(FetchedInput fetchedInput) {
-    cleanup(fetchedInput);
-  }
-
-  @Override
-  public void freeResources(FetchedInput fetchedInput) {
-    cleanup(fetchedInput);
-  }
-
-  private void cleanup(FetchedInput fetchedInput) {
-    switch (fetchedInput.getType()) {
-    case DISK:
-      break;
-    case MEMORY:
-      unreserve(fetchedInput.getSize());
-      break;
-    default:
-      throw new TezUncheckedException("InputType: " + fetchedInput.getType()
-          + " not expected for Broadcast fetch");
-    }
-  }
-
-  private synchronized void unreserve(long size) {
-    this.usedMemory -= size;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 2c53e75..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastKVReader.java
+++ /dev/null
@@ -1,225 +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.broadcast.input;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-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.compress.CompressionCodec;
-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.shuffle.common.FetchedInput;
-import org.apache.tez.engine.shuffle.common.FetchedInput.Type;
-import org.apache.tez.engine.shuffle.common.MemoryFetchedInput;
-
-public class BroadcastKVReader<K, V> implements KVReader {
-
-  private static final Log LOG = LogFactory.getLog(BroadcastKVReader.class);
-  
-  private final BroadcastShuffleManager shuffleManager;
-  private final Configuration conf;
-  private final CompressionCodec codec;
-  
-  private final Class<K> keyClass;
-  private final Class<V> valClass;
-  private final Deserializer<K> keyDeserializer;
-  private final Deserializer<V> valDeserializer;
-  private final DataInputBuffer keyIn;
-  private final DataInputBuffer valIn;
-
-  private final SimpleValueIterator valueIterator;
-  private final SimpleIterable valueIterable;
-  
-  private K key;
-  private V value;
-  
-  private FetchedInput currentFetchedInput;
-  private IFile.Reader currentReader;
-  
-  
-  public BroadcastKVReader(BroadcastShuffleManager shuffleManager,
-      Configuration conf) {
-    this.shuffleManager = shuffleManager;
-    this.conf = conf;
-
-    if (ConfigUtils.isIntermediateInputCompressed(this.conf)) {
-      Class<? extends CompressionCodec> codecClass = ConfigUtils
-          .getIntermediateInputCompressorClass(conf, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, conf);
-    } else {
-      codec = null;
-    }
-
-    this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
-    this.valClass = ConfigUtils.getIntermediateInputKeyClass(conf);
-
-    this.keyIn = new DataInputBuffer();
-    this.valIn = new DataInputBuffer();
-
-    SerializationFactory serializationFactory = new SerializationFactory(conf);
-
-    this.keyDeserializer = serializationFactory.getDeserializer(keyClass); 
-    this.valDeserializer = serializationFactory.getDeserializer(valClass);
-    
-    this.valueIterator = new SimpleValueIterator();
-    this.valueIterable = new SimpleIterable(this.valueIterator);
-  }
-
-  // TODO NEWTEZ Maybe add an interface to check whether next will block.
-  
-  /**
-   * 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
-   */
-  @Override  
-  public boolean next() throws IOException {
-    if (readNextFromCurrentReader()) {
-      return true;
-    } else {
-      boolean nextInputExists = moveToNextInput();
-      while (nextInputExists) {
-        if(readNextFromCurrentReader()) {
-          return true;
-        }
-        nextInputExists = moveToNextInput();
-      }
-      return false;
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public KVRecord getCurrentKV() throws IOException {
-    this.valueIterator.setValue(value);
-    return new KVRecord((Object)key, (Iterable<Object>)this.valueIterable);
-  }
-
-  /**
-   * Tries reading the next key and value from the current reader.
-   * @return true if the current reader has more records
-   * @throws IOException
-   */
-  private boolean readNextFromCurrentReader() throws IOException {
-    // Initial reader.
-    if (this.currentReader == null) {
-      return false;
-    } else {
-      boolean hasMore = this.currentReader.nextRawKey(keyIn);
-      if (hasMore) {
-        this.currentReader.nextRawValue(valIn);
-        this.key = keyDeserializer.deserialize(this.key);
-        this.value = valDeserializer.deserialize(this.value);
-        return true;
-      }
-      return false;
-    }
-  }
-  
-  /**
-   * Moves to the next available input. This method may block if the input is not ready yet.
-   * Also takes care of closing the previous input.
-   * 
-   * @return true if the next input exists, false otherwise
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  private boolean moveToNextInput() throws IOException {
-    if (currentReader != null) { // Close the current reader.
-      currentReader.close();
-      currentFetchedInput.free();
-    }
-    try {
-      currentFetchedInput = shuffleManager.getNextInput();
-    } catch (InterruptedException e) {
-      LOG.warn("Interrupted while waiting for next available input", e);
-      throw new IOException(e);
-    }
-    if (currentFetchedInput == null) {
-      return false; // No more inputs
-    } else {
-      currentReader = openIFileReader(currentFetchedInput);
-      return true;
-    }
-  }
-
-  public IFile.Reader openIFileReader(FetchedInput fetchedInput)
-      throws IOException {
-    if (fetchedInput.getType() == Type.MEMORY) {
-      MemoryFetchedInput mfi = (MemoryFetchedInput) fetchedInput;
-
-      return new InMemoryReader(null, mfi.getInputAttemptIdentifier(),
-          mfi.getBytes(), 0, (int) mfi.getSize());
-    } else {
-      return new IFile.Reader(conf, fetchedInput.getInputStream(),
-          fetchedInput.getSize(), codec, null);
-    }
-  }
-
-  
-  
-  // TODO NEWTEZ Move this into a common class. Also used in MRInput
-  private class SimpleValueIterator implements Iterator<V> {
-
-    private V value;
-
-    public void setValue(V value) {
-      this.value = value;
-    }
-
-    public boolean hasNext() {
-      return value != null;
-    }
-
-    public V next() {
-      V value = this.value;
-      this.value = null;
-      return value;
-    }
-
-    public void remove() {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  private class SimpleIterable implements Iterable<V> {
-    private final Iterator<V> iterator;
-    public SimpleIterable(Iterator<V> iterator) {
-      this.iterator = iterator;
-    }
-
-    @Override
-    public Iterator<V> iterator() {
-      return iterator;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index e89e892..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleInputEventHandler.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.broadcast.input;
-
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleInputEventHandler;
-import org.apache.tez.engine.common.shuffle.newimpl.ShuffleUserPayloads.DataMovementEventPayloadProto;
-
-import com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
-
-public class BroadcastShuffleInputEventHandler {
-
-  private static final Log LOG = LogFactory.getLog(ShuffleInputEventHandler.class);
-  
-  private final BroadcastShuffleManager shuffleManager;
-  
-  public BroadcastShuffleInputEventHandler(TezInputContext inputContext, BroadcastShuffleManager shuffleManager) {
-    this.shuffleManager = shuffleManager;
-  }
-
-  public void handleEvents(List<Event> events) {
-    for (Event event : events) {
-      handleEvent(event);
-    }
-  }
-  
-  private void handleEvent(Event event) {
-    if (event instanceof DataMovementEvent) {
-      processDataMovementEvent((DataMovementEvent)event);
-    } else if (event instanceof InputFailedEvent) {
-      processInputFailedEvent((InputFailedEvent)event);
-    } else {
-      throw new TezUncheckedException("Unexpected event type: " + event.getClass().getName());
-    }
-  }
-  
-  
-  private void processDataMovementEvent(DataMovementEvent dme) {
-    Preconditions.checkArgument(dme.getSourceIndex() == 0,
-        "Unexpected srcIndex: " + dme.getSourceIndex()
-            + " on DataMovementEvent. Can only be 0");
-    DataMovementEventPayloadProto shufflePayload;
-    try {
-      shufflePayload = DataMovementEventPayloadProto.parseFrom(dme.getUserPayload());
-    } catch (InvalidProtocolBufferException e) {
-      throw new TezUncheckedException("Unable to parse DataMovementEvent payload", e);
-    }
-    if (shufflePayload.getOutputGenerated()) {
-      InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dme.getTargetIndex(), dme.getVersion(), shufflePayload.getPathComponent());
-      shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), srcAttemptIdentifier, 0);
-    } else {
-      shuffleManager.addCompletedInputWithNoData(new InputAttemptIdentifier(dme.getTargetIndex(), dme.getVersion()));
-    }
-  }
-  
-  private void processInputFailedEvent(InputFailedEvent ife) {
-    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(ife.getTargetIndex(), ife.getVersion());
-    shuffleManager.obsoleteKnownInput(srcAttemptIdentifier);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 7b205fa..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/input/BroadcastShuffleManager.java
+++ /dev/null
@@ -1,489 +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.broadcast.input;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
-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.io.compress.CodecPool;
-import org.apache.hadoop.io.compress.CompressionCodec;
-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.shuffle.common.FetchResult;
-import org.apache.tez.engine.shuffle.common.FetchedInput;
-import org.apache.tez.engine.shuffle.common.Fetcher;
-import org.apache.tez.engine.shuffle.common.FetcherCallback;
-import org.apache.tez.engine.shuffle.common.InputHost;
-import org.apache.tez.engine.shuffle.common.ShuffleUtils;
-import org.apache.tez.engine.shuffle.common.Fetcher.FetcherBuilder;
-import org.apache.tez.engine.shuffle.common.FetchedInputAllocator;
-
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.common.util.concurrent.MoreExecutors;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-public class BroadcastShuffleManager implements FetcherCallback {
-
-  private static final Log LOG = LogFactory.getLog(BroadcastShuffleManager.class);
-  
-  private TezInputContext inputContext;
-  private int numInputs;
-  private Configuration conf;
-  
-  private final BroadcastShuffleInputEventHandler inputEventHandler;
-  private final FetchedInputAllocator inputManager;
-  
-  private final ExecutorService fetcherRawExecutor;
-  private final ListeningExecutorService fetcherExecutor;
-
-  private final BlockingQueue<FetchedInput> completedInputs;
-  private final Set<InputIdentifier> completedInputSet;
-  private final Set<InputIdentifier> pendingInputs;
-  private final ConcurrentMap<String, InputHost> knownSrcHosts;
-  private final Set<InputHost> pendingHosts;
-  private final Set<InputAttemptIdentifier> obsoletedInputs;
-  
-  private final AtomicInteger numCompletedInputs = new AtomicInteger(0);
-  
-  private final long startTime;
-  private long lastProgressTime;
-  
-  private FutureTask<Void> runShuffleFuture;
-  
-  // Required to be held when manipulating pendingHosts
-  private ReentrantLock lock = new ReentrantLock();
-  private Condition wakeLoop = lock.newCondition();
-  
-  private final int numFetchers;
-  private final AtomicInteger numRunningFetchers = new AtomicInteger(0);
-  
-  // Parameters required by Fetchers
-  private final SecretKey shuffleSecret;
-  private final int connectionTimeout;
-  private final int readTimeout;
-  private final CompressionCodec codec;
-  private final Decompressor decompressor;
-  
-  private final FetchFutureCallback fetchFutureCallback = new FetchFutureCallback();
-  
-  private volatile Throwable shuffleError;
-  
-  // TODO NEWTEZ Add counters.
-  
-  public BroadcastShuffleManager(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
-    this.inputContext = inputContext;
-    this.conf = conf;
-    this.numInputs = numInputs;
-    
-    this.inputEventHandler = new BroadcastShuffleInputEventHandler(inputContext, this);
-    this.inputManager = new BroadcastInputManager(inputContext, conf);
-
-    pendingInputs = Collections.newSetFromMap(new ConcurrentHashMap<InputIdentifier, Boolean>(numInputs));
-    completedInputSet = Collections.newSetFromMap(new ConcurrentHashMap<InputIdentifier, Boolean>(numInputs));
-    completedInputs = new LinkedBlockingQueue<FetchedInput>(numInputs);
-    knownSrcHosts = new ConcurrentHashMap<String, InputHost>();
-    pendingHosts = Collections.newSetFromMap(new ConcurrentHashMap<InputHost, Boolean>());
-    obsoletedInputs = Collections.newSetFromMap(new ConcurrentHashMap<InputAttemptIdentifier, Boolean>());
-    
-    int maxConfiguredFetchers = 
-        conf.getInt(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
-    
-    this.numFetchers = Math.min(maxConfiguredFetchers, numInputs);
-    
-    this.fetcherRawExecutor = Executors.newFixedThreadPool(numFetchers,
-        new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Fetcher #%d")
-            .build());
-    this.fetcherExecutor = MoreExecutors.listeningDecorator(fetcherRawExecutor);
-    
-    this.startTime = System.currentTimeMillis();
-    this.lastProgressTime = startTime;
-    
-    this.shuffleSecret = ShuffleUtils
-        .getJobTokenSecretFromTokenBytes(inputContext
-            .getServiceConsumerMetaData(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID));
-    
-    this.connectionTimeout = conf.getInt(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_STALLED_COPY_TIMEOUT);
-    this.readTimeout = conf.getInt(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_READ_TIMEOUT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_READ_TIMEOUT);
-    
-    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
-      Class<? extends CompressionCodec> codecClass = ConfigUtils
-          .getIntermediateInputCompressorClass(conf, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, conf);
-      decompressor = CodecPool.getDecompressor(codec);
-    } else {
-      codec = null;
-      decompressor = null;
-    }
-  }
-  
-  public void run() {
-    RunBroadcastShuffleCallable callable = new RunBroadcastShuffleCallable();
-    runShuffleFuture = new FutureTask<Void>(callable);
-    new Thread(runShuffleFuture, "ShuffleRunner");
-  }
-  
-  private class RunBroadcastShuffleCallable implements Callable<Void> {
-
-    @Override
-    public Void call() throws Exception {
-      while (numCompletedInputs.get() < numInputs) {
-        if (numRunningFetchers.get() >= numFetchers || pendingHosts.size() == 0) {
-          synchronized(lock) {
-            wakeLoop.await();
-          }
-          if (shuffleError != null) {
-            // InputContext has already been informed of a fatal error.
-            // Initiate shutdown.
-            break;
-          }
-          
-          if (numCompletedInputs.get() < numInputs) {
-            synchronized (lock) {
-              int numFetchersToRun = Math.min(pendingHosts.size(), numFetchers - numRunningFetchers.get());
-              int count = 0;
-              for (Iterator<InputHost> inputHostIter = pendingHosts.iterator() ; inputHostIter.hasNext() ; ) {
-                InputHost inputHost = inputHostIter.next();
-                inputHostIter.remove();
-                if (inputHost.getNumPendingInputs() > 0) {
-                  Fetcher fetcher = constructFetcherForHost(inputHost);
-                  numRunningFetchers.incrementAndGet();
-                  ListenableFuture<FetchResult> future = fetcherExecutor
-                      .submit(fetcher);
-                  Futures.addCallback(future, fetchFutureCallback);
-                  if (++count >= numFetchersToRun) {
-                    break;
-                  }
-                }
-              }
-            }
-          }
-        }
-      }
-      // TODO NEWTEZ Maybe clean up inputs.
-      if (!fetcherExecutor.isShutdown()) {
-        fetcherExecutor.shutdownNow();
-      }
-      return null;
-    }
-  }
-  
-  private Fetcher constructFetcherForHost(InputHost inputHost) {
-    FetcherBuilder fetcherBuilder = new FetcherBuilder(
-        BroadcastShuffleManager.this, inputManager,
-        inputContext.getApplicationId(), shuffleSecret, conf);
-    fetcherBuilder.setConnectionParameters(connectionTimeout, readTimeout);
-    fetcherBuilder.setCompressionParameters(codec, decompressor);
-
-    // Remove obsolete inputs from the list being given to the fetcher. Also
-    // remove from the obsolete list.
-    List<InputAttemptIdentifier> pendingInputsForHost = inputHost
-        .clearAndGetPendingInputs();
-    for (Iterator<InputAttemptIdentifier> inputIter = pendingInputsForHost
-        .iterator(); inputIter.hasNext();) {
-      InputAttemptIdentifier input = inputIter.next();
-      // Avoid adding attempts which have already completed.
-      if (completedInputSet.contains(input.getInputIdentifier())) {
-        inputIter.remove();
-      }
-      // Avoid adding attempts which have been marked as OBSOLETE 
-      if (obsoletedInputs.contains(input)) {
-        inputIter.remove();
-        obsoletedInputs.remove(input);
-      }
-    }
-    // TODO NEWTEZ Maybe limit the number of inputs being given to a single
-    // fetcher, especially in the case where #hosts < #fetchers
-    fetcherBuilder.assignWork(inputHost.getHost(), inputHost.getPort(), 0,
-        inputHost.clearAndGetPendingInputs());
-    return fetcherBuilder.build();
-  }
-  
-  /////////////////// Methods for InputEventHandler
-  
-  public void addKnownInput(String hostName, int port,
-      InputAttemptIdentifier srcAttemptIdentifier, int partition) {
-    InputHost host = knownSrcHosts.get(hostName);
-    if (host == null) {
-      host = new InputHost(hostName, port, inputContext.getApplicationId());
-      InputHost old = knownSrcHosts.putIfAbsent(hostName, host);
-      if (old != null) {
-        host = old;
-      }
-    }
-    host.addKnownInput(srcAttemptIdentifier);
-    synchronized(lock) {
-      pendingHosts.add(host);
-      wakeLoop.signal();
-    }
-  }
-
-  public void addCompletedInputWithNoData(
-      InputAttemptIdentifier srcAttemptIdentifier) {
-    InputIdentifier inputIdentifier = srcAttemptIdentifier.getInputIdentifier();
-    LOG.info("No input data exists for SrcTask: " + inputIdentifier + ". Marking as complete.");
-    if (pendingInputs.remove(inputIdentifier)) {
-      completedInputSet.add(inputIdentifier);
-      completedInputs.add(new NullFetchedInput(srcAttemptIdentifier));
-      numCompletedInputs.incrementAndGet();
-    }
-
-    // Awake the loop to check for termination.
-    synchronized (lock) {
-      wakeLoop.signal();
-    } 
-  }
-
-  public synchronized void obsoleteKnownInput(InputAttemptIdentifier srcAttemptIdentifier) {
-    obsoletedInputs.add(srcAttemptIdentifier);
-    // TODO NEWTEZ Maybe inform the fetcher about this. For now, this is used during the initial fetch list construction.
-  }
-  
-  
-  public void handleEvents(List<Event> events) {
-    inputEventHandler.handleEvents(events);
-  }
-
-  /////////////////// End of Methods for InputEventHandler
-  /////////////////// Methods from FetcherCallbackHandler
-  
-  @Override
-  public void fetchSucceeded(String host,
-      InputAttemptIdentifier srcAttemptIdentifier, FetchedInput fetchedInput, long fetchedBytes,
-      long copyDuration) throws IOException {
-    InputIdentifier inputIdentifier = srcAttemptIdentifier.getInputIdentifier();
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Complete fetch for attempt: " + srcAttemptIdentifier + " to " + fetchedInput.getType());
-    }
-    
-    // Count irrespective of whether this is a copy of an already fetched input
-    synchronized(lock) {
-      lastProgressTime = System.currentTimeMillis();
-    }
-    
-    boolean committed = false;
-    if (!completedInputSet.contains(inputIdentifier)) {
-      synchronized (completedInputSet) {
-        if (!completedInputSet.contains(inputIdentifier)) {
-          fetchedInput.commit();
-          committed = true;
-          pendingInputs.remove(inputIdentifier);
-          completedInputSet.add(inputIdentifier);
-          completedInputs.add(fetchedInput);
-          numCompletedInputs.incrementAndGet();
-        }
-      }
-    }
-    if (!committed) {
-      fetchedInput.abort(); // If this fails, the fetcher may attempt another abort.
-    } else {
-      synchronized(lock) {
-        // Signal the wakeLoop to check for termination.
-        wakeLoop.signal();
-      }
-    }
-    // TODO NEWTEZ Maybe inform fetchers, in case they have an alternate attempt of the same task in their queue.
-  }
-
-  @Override
-  public void fetchFailed(String host,
-      InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed) {
-    // TODO NEWTEZ. Implement logic to report fetch failures after a threshold.
-    // For now, reporting immediately.
-    InputReadErrorEvent readError = new InputReadErrorEvent(
-        "Fetch failure while fetching from "
-            + TezEngineUtils.getTaskAttemptIdentifier(
-                inputContext.getSourceVertexName(),
-                srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
-                srcAttemptIdentifier.getAttemptNumber()),
-        srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
-        srcAttemptIdentifier.getAttemptNumber());
-    
-    List<Event> failedEvents = Lists.newArrayListWithCapacity(1);
-    failedEvents.add(readError);
-    inputContext.sendEvents(failedEvents);
-  }
-  /////////////////// End of Methods from FetcherCallbackHandler
-
-  public void shutdown() throws InterruptedException {
-    if (this.fetcherExecutor != null && !this.fetcherExecutor.isShutdown()) {
-      this.fetcherExecutor.shutdown();
-      this.fetcherExecutor.awaitTermination(2000l, TimeUnit.MILLISECONDS);
-      if (!this.fetcherExecutor.isShutdown()) {
-        this.fetcherExecutor.shutdownNow();
-      }
-    }
-  }
-  
-  /////////////////// Methods for walking the available inputs
-  
-  /**
-   * @return true if there is another input ready for consumption.
-   */
-  public boolean newInputAvailable() {
-    FetchedInput head = completedInputs.peek();
-    if (head == null || head instanceof NullFetchedInput) {
-      return false;
-    } else {
-      return true;
-    }
-  }
-
-  /**
-   * @return true if all of the required inputs have been fetched.
-   */
-  public boolean allInputsFetched() {
-    return numCompletedInputs.get() == numInputs;
-  }
-
-  /**
-   * @return the next available input, or null if there are no available inputs.
-   *         This method will block if there are currently no available inputs,
-   *         but more may become available.
-   */
-  public FetchedInput getNextInput() throws InterruptedException {
-    FetchedInput input = null;
-    do {
-      input = completedInputs.peek();
-      if (input == null) {
-        if (allInputsFetched()) {
-          break;
-        } else {
-          input = completedInputs.take(); // block
-        }
-      } else {
-        input = completedInputs.poll();
-      }
-    } while (input instanceof NullFetchedInput);
-    return input;
-  }
-
-  /////////////////// End of methods for walking the available inputs
-  
-  
-  /**
-   * Fake input that is added to the completed input list in case an input does not have any data.
-   *
-   */
-  private class NullFetchedInput extends FetchedInput {
-
-    public NullFetchedInput(InputAttemptIdentifier inputAttemptIdentifier) {
-      super(Type.MEMORY, -1, inputAttemptIdentifier, null);
-    }
-
-    @Override
-    public OutputStream getOutputStream() throws IOException {
-      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
-    }
-
-    @Override
-    public InputStream getInputStream() throws IOException {
-      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
-    }
-
-    @Override
-    public void commit() throws IOException {
-      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
-    }
-
-    @Override
-    public void abort() throws IOException {
-      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
-    }
-
-    @Override
-    public void free() {
-      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
-    }
-  }
-  
-  
-  private class FetchFutureCallback implements FutureCallback<FetchResult> {
-
-    private void doBookKeepingForFetcherComplete() {
-      numRunningFetchers.decrementAndGet();
-      synchronized(lock) {
-        wakeLoop.signal();
-      }
-    }
-    
-    @Override
-    public void onSuccess(FetchResult result) {
-      Iterable<InputAttemptIdentifier> pendingInputs = result.getPendingInputs();
-      if (pendingInputs != null && pendingInputs.iterator().hasNext()) {
-        InputHost inputHost = knownSrcHosts.get(result.getHost());
-        assert inputHost != null;
-        for (InputAttemptIdentifier input : pendingInputs) {
-          inputHost.addKnownInput(input);
-        }
-        pendingHosts.add(inputHost);
-      }
-      doBookKeepingForFetcherComplete();
-    }
-
-    @Override
-    public void onFailure(Throwable t) {
-      LOG.error("Fetcher failed with error: " + t);
-      shuffleError = t;
-      inputContext.fatalError(t, "Fetched failed");
-      doBookKeepingForFetcherComplete();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 474d1cd..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/broadcast/output/FileBasedKVWriter.java
+++ /dev/null
@@ -1,125 +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.broadcast.output;
-
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.compress.CompressionCodec;
-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.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;
-
-public class FileBasedKVWriter implements KVWriter {
-
-  public static final int INDEX_RECORD_LENGTH = 24;
-
-  private final Configuration conf;
-  private int numRecords = 0;
-
-  @SuppressWarnings("rawtypes")
-  private Class keyClass;
-  @SuppressWarnings("rawtypes")
-  private Class valClass;
-  private CompressionCodec codec;
-  private FileSystem rfs;
-  private IFile.Writer writer;
-
-  private TezTaskOutput ouputFileManager;
-
-  // TODO NEWTEZ Define Counters
-  // Number of records
-  // Time waiting for a write to complete, if that's possible.
-  // Size of key-value pairs written.
-
-  public FileBasedKVWriter(TezOutputContext outputContext) throws IOException {
-    this.conf = TezUtils.createConfFromUserPayload(outputContext
-        .getUserPayload());
-    this.conf.setStrings(TezJobConfig.LOCAL_DIRS,
-        outputContext.getWorkDirs());
-
-    this.rfs = ((LocalFileSystem) FileSystem.getLocal(this.conf)).getRaw();
-
-    // Setup serialization
-    keyClass = ConfigUtils.getIntermediateOutputKeyClass(this.conf);
-    valClass = ConfigUtils.getIntermediateOutputValueClass(this.conf);
-
-    // Setup compression
-    if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) {
-      Class<? extends CompressionCodec> codecClass = ConfigUtils
-          .getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, this.conf);
-    } else {
-      codec = null;
-    }
-
-    this.ouputFileManager = TezEngineUtils.instantiateTaskOutputManager(conf,
-        outputContext);
-
-    initWriter();
-  }
-
-  /**
-   * @return true if any output was generated. false otherwise
-   * @throws IOException
-   */
-  public boolean close() throws IOException {
-    this.writer.close();
-    TezIndexRecord rec = new TezIndexRecord(0, writer.getRawLength(),
-        writer.getCompressedLength());
-    TezSpillRecord sr = new TezSpillRecord(1);
-    sr.putIndex(rec, 0);
-
-    Path indexFile = ouputFileManager
-        .getOutputIndexFileForWrite(INDEX_RECORD_LENGTH);
-    sr.writeToFile(indexFile, conf);
-    return numRecords > 0;
-  }
-
-  @Override
-  public void write(Object key, Object value) throws IOException {
-    this.writer.append(key, value);
-    numRecords++;
-  }
-
-  public void initWriter() throws IOException {
-    Path outputFile = ouputFileManager.getOutputFileForWrite();
-
-    // TODO NEWTEZ Consider making the buffer size configurable. Also consider
-    // setting up an in-memory buffer which is occasionally flushed to disk so
-    // that the output does not block.
-
-    // TODO NEWTEZ maybe use appropriate counter
-    this.writer = new IFile.Writer(conf, rfs, outputFile, keyClass, valClass,
-        codec, null);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java b/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
deleted file mode 100644
index f73adfd..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/ConfigUtils.java
+++ /dev/null
@@ -1,148 +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;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.TezJobConfig;
-
-@SuppressWarnings({"unchecked", "rawtypes"})
-public class ConfigUtils {
-
-  public static Class<? extends CompressionCodec> getIntermediateOutputCompressorClass(
-      Configuration conf, Class<DefaultCodec> defaultValue) {
-    Class<? extends CompressionCodec> codecClass = defaultValue;
-    String name = conf
-        .get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_COMPRESS_CODEC);
-    if (name != null) {
-      try {
-        codecClass = conf.getClassByName(name).asSubclass(
-            CompressionCodec.class);
-      } catch (ClassNotFoundException e) {
-        throw new IllegalArgumentException("Compression codec " + name
-            + " was not found.", e);
-      }
-    }
-    return codecClass;
-  }
-  
-  public static Class<? extends CompressionCodec> getIntermediateInputCompressorClass(
-      Configuration conf, Class<DefaultCodec> defaultValue) {
-    Class<? extends CompressionCodec> codecClass = defaultValue;
-    String name = conf
-        .get(TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_COMPRESS_CODEC);
-    if (name != null) {
-      try {
-        codecClass = conf.getClassByName(name).asSubclass(
-            CompressionCodec.class);
-      } catch (ClassNotFoundException e) {
-        throw new IllegalArgumentException("Compression codec " + name
-            + " was not found.", e);
-      }
-    }
-    return codecClass;
-  }
-
-
-  // TODO Move defaults over to a constants file.
-  
-  public static boolean shouldCompressIntermediateOutput(Configuration conf) {
-    return conf.getBoolean(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS, false);
-  }
-
-  public static boolean isIntermediateInputCompressed(Configuration conf) {
-    return conf.getBoolean(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_IS_COMPRESSED, false);
-  }
-
-  public static <V> Class<V> getIntermediateOutputValueClass(Configuration conf) {
-    Class<V> retv = (Class<V>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS, null,
-        Object.class);
-    return retv;
-  }
-  
-  public static <V> Class<V> getIntermediateInputValueClass(Configuration conf) {
-    Class<V> retv = (Class<V>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_VALUE_CLASS, null,
-        Object.class);
-    return retv;
-  }
-
-  public static <K> Class<K> getIntermediateOutputKeyClass(Configuration conf) {
-    Class<K> retv = (Class<K>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS, null,
-        Object.class);
-    return retv;
-  }
-
-  public static <K> Class<K> getIntermediateInputKeyClass(Configuration conf) {
-    Class<K> retv = (Class<K>) conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_CLASS, null,
-        Object.class);
-    return retv;
-  }
-
-  public static <K> RawComparator<K> getIntermediateOutputKeyComparator(Configuration conf) {
-    Class<? extends RawComparator> theClass = conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS, null,
-        RawComparator.class);
-    if (theClass != null)
-      return ReflectionUtils.newInstance(theClass, conf);
-    return WritableComparator.get(getIntermediateOutputKeyClass(conf).asSubclass(
-        WritableComparable.class));
-  }
-
-  public static <K> RawComparator<K> getIntermediateInputKeyComparator(Configuration conf) {
-    Class<? extends RawComparator> theClass = conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS, null,
-        RawComparator.class);
-    if (theClass != null)
-      return ReflectionUtils.newInstance(theClass, conf);
-    return WritableComparator.get(getIntermediateInputKeyClass(conf).asSubclass(
-        WritableComparable.class));
-  }
-
-  
-  
-  // TODO Fix name
-  public static <V> RawComparator<V> getInputKeySecondaryGroupingComparator(
-      Configuration conf) {
-    Class<? extends RawComparator> theClass = conf
-        .getClass(
-            TezJobConfig.TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS,
-            null, RawComparator.class);
-    if (theClass == null) {
-      return getIntermediateInputKeyComparator(conf);
-    }
-
-    return ReflectionUtils.newInstance(theClass, conf);
-  }
-  
-  public static boolean useNewApi(Configuration conf) {
-    return conf.getBoolean("mapred.mapper.new-api", false);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/InputAttemptIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/InputAttemptIdentifier.java b/tez-engine/src/main/java/org/apache/tez/engine/common/InputAttemptIdentifier.java
deleted file mode 100644
index 076807e..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/InputAttemptIdentifier.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.engine.common;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-
-/**
- * Container for a task number and an attempt number for the task.
- */
-@Private
-public class InputAttemptIdentifier {
-
-  private final InputIdentifier inputIdentifier;
-  private final int attemptNumber;
-  private String pathComponent;
-  
-  public InputAttemptIdentifier(int taskIndex, int attemptNumber) {
-    this(new InputIdentifier(taskIndex), attemptNumber, null);
-  }
-  
-  public InputAttemptIdentifier(InputIdentifier inputIdentifier, int attemptNumber, String pathComponent) {
-    this.inputIdentifier = inputIdentifier;
-    this.attemptNumber = attemptNumber;
-    this.pathComponent = pathComponent;
-  }
-  
-  public InputAttemptIdentifier(int taskIndex, int attemptNumber, String pathComponent) {
-    this(new InputIdentifier(taskIndex), attemptNumber, pathComponent);
-  }
-
-  public InputIdentifier getInputIdentifier() {
-    return this.inputIdentifier;
-  }
-
-  public int getAttemptNumber() {
-    return attemptNumber;
-  }
-  
-  public String getPathComponent() {
-    return pathComponent;
-  }
-
-  // PathComponent does not need to be part of the hashCode and equals computation.
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + attemptNumber;
-    result = prime * result
-        + ((inputIdentifier == null) ? 0 : inputIdentifier.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;
-    InputAttemptIdentifier other = (InputAttemptIdentifier) obj;
-    if (attemptNumber != other.attemptNumber)
-      return false;
-    if (inputIdentifier == null) {
-      if (other.inputIdentifier != null)
-        return false;
-    } else if (!inputIdentifier.equals(other.inputIdentifier))
-      return false;
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    return "InputAttemptIdentifier [inputIdentifier=" + inputIdentifier
-        + ", attemptNumber=" + attemptNumber + ", pathComponent="
-        + pathComponent + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/InputIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/InputIdentifier.java b/tez-engine/src/main/java/org/apache/tez/engine/common/InputIdentifier.java
deleted file mode 100644
index b694530..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/InputIdentifier.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;
-
-public class InputIdentifier {
-
-  private final int srcTaskIndex;
-  
-  public InputIdentifier(int srcTaskIndex) {
-    this.srcTaskIndex = srcTaskIndex;
-  }
-
-  public int getSrcTaskIndex() {
-    return this.srcTaskIndex;
-  }
-
-  @Override
-  public int hashCode() {
-    return srcTaskIndex;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (obj == null)
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    InputIdentifier other = (InputIdentifier) obj;
-    if (srcTaskIndex != other.srcTaskIndex)
-      return false;
-    return true;
-  }
-
-  @Override
-  public String toString() {
-    return "InputIdentifier [srcTaskIndex=" + srcTaskIndex + "]";
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index cc29e94..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/TezEngineUtils.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.engine.common;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.engine.api.Partitioner;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.api.TezTaskContext;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-
-public class TezEngineUtils {
-
-  private static final Log LOG = LogFactory
-      .getLog(TezEngineUtils.class);
-  
-  public static String getTaskIdentifier(String vertexName, int taskIndex) {
-    return String.format("%s_%06d", vertexName, taskIndex);
-  }
-
-  public static String getTaskAttemptIdentifier(int taskIndex,
-      int taskAttemptNumber) {
-    return String.format("%d_%d", taskIndex, taskAttemptNumber);
-  }
-
-  // TODO Maybe include a dag name in this.
-  public static String getTaskAttemptIdentifier(String vertexName,
-      int taskIndex, int taskAttemptNumber) {
-    return String.format("%s_%06d_%02d", vertexName, taskIndex,
-        taskAttemptNumber);
-  }
-
-  @SuppressWarnings("unchecked")
-  public static Combiner instantiateCombiner(Configuration conf, TezTaskContext taskContext) throws IOException {
-    Class<? extends Combiner> clazz;
-    String className = conf.get(TezJobConfig.TEZ_ENGINE_COMBINER_CLASS);
-    if (className == null) {
-      LOG.info("No combiner specified via " + TezJobConfig.TEZ_ENGINE_COMBINER_CLASS + ". Combiner will not be used");
-      return null;
-    }
-    LOG.info("Using Combiner class: " + className);
-    try {
-      clazz = (Class<? extends Combiner>) conf.getClassByName(className);
-    } catch (ClassNotFoundException e) {
-      throw new IOException("Unable to load combiner class: " + className);
-    }
-    
-    Combiner combiner = null;
-    
-      Constructor<? extends Combiner> ctor;
-      try {
-        ctor = clazz.getConstructor(TezTaskContext.class);
-        combiner = ctor.newInstance(taskContext);
-      } catch (SecurityException e) {
-        throw new IOException(e);
-      } catch (NoSuchMethodException e) {
-        throw new IOException(e);
-      } catch (IllegalArgumentException e) {
-        throw new IOException(e);
-      } catch (InstantiationException e) {
-        throw new IOException(e);
-      } catch (IllegalAccessException e) {
-        throw new IOException(e);
-      } catch (InvocationTargetException e) {
-        throw new IOException(e);
-      }
-      return combiner;
-  }
-  
-  @SuppressWarnings("unchecked")
-  public static Partitioner instantiatePartitioner(Configuration conf)
-      throws IOException {
-    Class<? extends Partitioner> clazz;
-    try {
-      clazz = (Class<? extends Partitioner>) conf
-          .getClassByName(conf.get(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS));
-    } catch (ClassNotFoundException e) {
-      throw new IOException("Unable to find Partitioner class in config", e);
-    }
-
-    LOG.info("Using partitioner class: " + clazz.getName());
-
-    Partitioner partitioner = null;
-
-    try {
-      Constructor<? extends Partitioner> ctorWithConf = clazz
-          .getConstructor(Configuration.class);
-      partitioner = ctorWithConf.newInstance(conf);
-    } catch (SecurityException e) {
-      throw new IOException(e);
-    } catch (NoSuchMethodException e) {
-      try {
-        // Try a 0 argument constructor.
-        partitioner = clazz.newInstance();
-      } catch (InstantiationException e1) {
-        throw new IOException(e1);
-      } catch (IllegalAccessException e1) {
-        throw new IOException(e1);
-      }
-    } catch (IllegalArgumentException e) {
-      throw new IOException(e);
-    } catch (InstantiationException e) {
-      throw new IOException(e);
-    } catch (IllegalAccessException e) {
-      throw new IOException(e);
-    } catch (InvocationTargetException e) {
-      throw new IOException(e);
-    }
-    return partitioner;
-  }
-  
-  public static TezTaskOutput instantiateTaskOutputManager(Configuration conf, TezOutputContext outputContext) {
-    Class<?> clazz = conf.getClass(Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
-        TezTaskOutputFiles.class);
-    try {
-      Constructor<?> ctor = clazz.getConstructor(Configuration.class, String.class);
-      ctor.setAccessible(true);
-      TezTaskOutput instance = (TezTaskOutput) ctor.newInstance(conf, outputContext.getUniqueIdentifier());
-      return instance;
-    } catch (Exception e) {
-      throw new TezUncheckedException(
-          "Unable to instantiate configured TezOutputFileManager: "
-              + conf.get(Constants.TEZ_ENGINE_TASK_OUTPUT_MANAGER,
-                  TezTaskOutputFiles.class.getName()), e);
-    }
-  }
-}


[19/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalInput.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalInput.java b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalInput.java
new file mode 100644
index 0000000..3382d12
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalOutput.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalOutput.java b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalOutput.java
new file mode 100644
index 0000000..475eaef
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Output.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Output.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Output.java
new file mode 100644
index 0000000..22f8d66
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Processor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Processor.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Processor.java
new file mode 100644
index 0000000..17c2d05
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Reader.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Reader.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Reader.java
new file mode 100644
index 0000000..dd006bc
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/TezInputContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/TezInputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezInputContext.java
new file mode 100644
index 0000000..b07e92c
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/TezOutputContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/TezOutputContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezOutputContext.java
new file mode 100644
index 0000000..fda30ca
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/TezProcessorContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/TezProcessorContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezProcessorContext.java
new file mode 100644
index 0000000..001461b
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/TezTaskContext.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/TezTaskContext.java b/tez-api/src/main/java/org/apache/tez/runtime/api/TezTaskContext.java
new file mode 100644
index 0000000..ef652b6
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Writer.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Writer.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Writer.java
new file mode 100644
index 0000000..9604e59
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/events/DataMovementEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/DataMovementEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/DataMovementEvent.java
new file mode 100644
index 0000000..b384676
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.api.events;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputFailedEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputFailedEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputFailedEvent.java
new file mode 100644
index 0000000..dedc8d9
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.api.events;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInformationEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInformationEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInformationEvent.java
new file mode 100644
index 0000000..0322b75
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.api.events;
+
+import org.apache.tez.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputReadErrorEvent.java
new file mode 100644
index 0000000..612a1e3
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.api.events;
+
+import org.apache.tez.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectLifeCycle.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectLifeCycle.java b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectLifeCycle.java
new file mode 100644
index 0000000..9dae043
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistry.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistry.java b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistry.java
new file mode 100644
index 0000000..03f0424
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryFactory.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryFactory.java b/tez-api/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryFactory.java
new file mode 100644
index 0000000..3e409cd
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/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
index 21cacf6..b91125d 100644
--- a/tez-api/src/main/proto/Events.proto
+++ b/tez-api/src/main/proto/Events.proto
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-option java_package = "org.apache.tez.engine.api.events";
+option java_package = "org.apache.tez.runtime.api.events";
 option java_outer_classname = "EventProtos";
 option java_generate_equals_and_hash = true;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/pom.xml
----------------------------------------------------------------------
diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml
index bc6aeef..9bf3fe4 100644
--- a/tez-dag/pom.xml
+++ b/tez-dag/pom.xml
@@ -41,7 +41,7 @@
     </dependency>
     <dependency>
       <groupId>org.apache.tez</groupId>
-      <artifactId>tez-engine</artifactId>
+      <artifactId>tez-runtime-internals</artifactId>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 1967462..4938d9e 100644
--- a/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
+++ b/tez-dag/src/main/java/org/apache/hadoop/mapred/YarnTezDagChild.java
@@ -67,26 +67,26 @@ import org.apache.tez.dag.api.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.newruntime.LogicalIOProcessorRuntimeTask;
 import org.apache.tez.mapreduce.input.MRInputLegacy;
 import org.apache.tez.mapreduce.output.MROutput;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.apache.tez.runtime.api.impl.TezUmbilical;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryModule;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.TokenCache;
 
 import com.google.inject.Guice;
 import com.google.inject.Injector;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 56c89b2..67fe763 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -108,7 +108,7 @@ import org.apache.tez.dag.history.HistoryEventHandler;
 import org.apache.tez.dag.history.avro.HistoryEventType;
 import org.apache.tez.dag.history.events.AMStartedEvent;
 import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
 
 /**
  * The Map-Reduce Application Master.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 7a143a5..582d274 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskAttemptListenerImpTezDag.java
@@ -51,10 +51,10 @@ import org.apache.tez.dag.app.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.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
 
 @SuppressWarnings("unchecked")
 public class TaskAttemptListenerImpTezDag extends AbstractService implements

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 2779faf..674d18e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/EdgeManager.java
@@ -20,9 +20,9 @@ package org.apache.tez.dag.app.dag;
 
 import java.util.List;
 
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public abstract class EdgeManager {
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 088a195..293e4c5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
@@ -26,7 +26,7 @@ import org.apache.tez.dag.api.oldrecords.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.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 /**
  * Read only view of Task.

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 42ff8de..76964a3 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
@@ -32,9 +32,9 @@ import org.apache.tez.dag.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.api.impl.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
index 4c79712..4a1a7a6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexScheduler.java
@@ -20,7 +20,7 @@ package org.apache.tez.dag.app.dag;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 // Rename to VertexManager TEZ-364 and move to DAG API. Make abstract class.
 public interface VertexScheduler {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 0b8db76..30aefde 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStatusUpdate.java
@@ -25,7 +25,7 @@ import org.apache.tez.common.counters.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.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
 
 public class TaskAttemptEventStatusUpdate extends TaskAttemptEvent {
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 4154bd0..1fbe609 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventAddTezEvent.java
@@ -19,7 +19,7 @@
 package org.apache.tez.dag.app.dag.event;
 
 import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.engine.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 public class TaskEventAddTezEvent extends TaskEvent {
 

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

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
index 27d9a02..388beba 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceTaskAttemptCompleted.java
@@ -19,7 +19,7 @@
 package org.apache.tez.dag.app.dag.event;
 
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 public class VertexEventSourceTaskAttemptCompleted extends VertexEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
index ef6de96..c9d3f7d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventTaskAttemptCompleted.java
@@ -18,7 +18,7 @@
 
 package org.apache.tez.dag.app.dag.event;
 
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 public class VertexEventTaskAttemptCompleted extends VertexEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 b05a6f1..55a2c86 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/BroadcastEdgeManager.java
@@ -22,9 +22,9 @@ import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public class BroadcastEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
index d30d178..ee12221 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
@@ -88,10 +88,10 @@ import org.apache.tez.dag.history.events.DAGStartedEvent;
 import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TezBuilderUtils;
-import org.apache.tez.engine.common.security.JobTokenIdentifier;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
-import org.apache.tez.engine.common.security.TokenCache;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.TokenCache;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 060a112..aaca662 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/Edge.java
@@ -31,14 +31,14 @@ import org.apache.tez.dag.app.dag.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.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
 
 public class Edge {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
index 8ee7e55..a4e5f3b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexScheduler.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.VertexScheduler;
 import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 /**
  * Starts all tasks immediately on vertex start

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 a916ad2..29abfac 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/OneToOneEdgeManager.java
@@ -22,9 +22,9 @@ import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public class OneToOneEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 1d4df5b..3d1d289 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ScatterGatherEdgeManager.java
@@ -22,9 +22,9 @@ import java.util.List;
 
 import org.apache.tez.dag.app.dag.EdgeManager;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 
 public class ScatterGatherEdgeManager extends EdgeManager {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 a0ed329..a4dd555 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ShuffleVertexManager.java
@@ -39,11 +39,11 @@ import org.apache.tez.dag.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.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 /**
  * Starts scheduling tasks when number of completed source tasks crosses 
@@ -264,14 +264,14 @@ public class ShuffleVertexManager implements VertexScheduler {
       List<byte[]> taskConfs = new ArrayList<byte[]>(finalTaskParallelism);
       try {
         Configuration taskConf = new Configuration(false);
-        taskConf.setInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE,
+        taskConf.setInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE,
             basePartitionRange);
         // create event user payload to inform the task
         for (int i = 0; i < numShufflersWithBaseRange; ++i) {
           taskConfs.add(MRHelpers.createUserPayloadFromConf(taskConf));
         }
         if(finalTaskParallelism > numShufflersWithBaseRange) {
-          taskConf.setInt(TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE,
+          taskConf.setInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE,
               remainderRangeForLastShuffler);
           taskConfs.add(MRHelpers.createUserPayloadFromConf(taskConf));
         }

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 f2b2776..b79f856 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
@@ -89,8 +89,8 @@ import org.apache.tez.dag.records.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.api.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 13fa915..c1a9415 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
@@ -75,8 +75,8 @@ import org.apache.tez.dag.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.api.impl.TezEvent;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 74005b7..520473d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
@@ -106,15 +106,15 @@ import org.apache.tez.dag.records.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.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.HashMultiset;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 14edd96..7a0b6f7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventTALaunchRequest.java
@@ -22,7 +22,7 @@ import org.apache.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.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class AMSchedulerEventTALaunchRequest extends AMSchedulerEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 76e80f5..4adc302 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerEventAssignTA.java
@@ -19,7 +19,7 @@ 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.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class AMContainerEventAssignTA extends AMContainerEvent {
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java
index 1855dbd..11e8aeb 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerHelpers.java
@@ -44,9 +44,9 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.TaskAttemptListener;
-import org.apache.tez.dag.utils.TezEngineChildJVM;
-import org.apache.tez.engine.common.security.TokenCache;
-import org.apache.tez.engine.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.dag.utils.TezRuntimeChildJVM;
+import org.apache.tez.runtime.library.common.security.TokenCache;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -163,7 +163,7 @@ public class AMContainerHelpers {
     myEnv.putAll(vertexEnv);
 
     // Set up the launch command
-    List<String> commands = TezEngineChildJVM.getVMCommand(
+    List<String> commands = TezRuntimeChildJVM.getVMCommand(
         taskAttemptListener.getAddress(), containerId.toString(),
         appContext.getApplicationID().toString(),
         appContext.getApplicationAttemptId().getAttemptId(),

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 94dd580..4731a24 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -54,7 +54,7 @@ import org.apache.tez.dag.app.rm.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.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 @SuppressWarnings("rawtypes")
 public class AMContainerImpl implements AMContainer {

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 c0ef524..2015505 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerTask.java
@@ -18,7 +18,7 @@
 
 package org.apache.tez.dag.app.rm.container;
 
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 
 public class AMContainerTask {
   private final boolean shouldDie;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/utils/TezEngineChildJVM.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezEngineChildJVM.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezEngineChildJVM.java
deleted file mode 100644
index 8919698..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezEngineChildJVM.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.dag.utils;
-
-import java.net.InetSocketAddress;
-import java.util.List;
-import java.util.Vector;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.YarnTezDagChild;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-
-public class TezEngineChildJVM {
-
-    // FIXME
-  public static enum LogName {
-    /** Log on the stdout of the task. */
-    STDOUT ("stdout"),
-
-    /** Log on the stderr of the task. */
-    STDERR ("stderr"),
-
-    /** Log on the map-reduce system logs of the task. */
-    SYSLOG ("syslog"),
-
-    /** The java profiler information. */
-    PROFILE ("profile.out"),
-
-    /** Log the debug script's stdout  */
-    DEBUGOUT ("debugout");
-
-    private String prefix;
-
-    private LogName(String prefix) {
-      this.prefix = prefix;
-    }
-
-    @Override
-    public String toString() {
-      return prefix;
-    }
-  }
-
-  private static String getTaskLogFile(LogName filter) {
-    return ApplicationConstants.LOG_DIR_EXPANSION_VAR + Path.SEPARATOR +
-        filter.toString();
-  }
-
-  public static List<String> getVMCommand(
-      InetSocketAddress taskAttemptListenerAddr,
-      String containerIdentifier,
-      String tokenIdentifier,
-      int applicationAttemptNumber,
-      boolean shouldProfile,
-      String javaOpts) {
-
-    Vector<String> vargs = new Vector<String>(9);
-
-    vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
-
-    //set custom javaOpts
-    vargs.add(javaOpts);
-
-//[Debug Task] Current simplest way to attach debugger to  Tez Child Task
-// Uncomment the following, then launch a regular job
-// Works best on one-box configured with a single container (hence one task at a time).
-//    LOG.error(" !!!!!!!!! Launching Child-Task in debug/suspend mode.  Attach to port 8003 !!!!!!!!");
-//    vargs.add("-Xdebug -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,address=8003,server=y,suspend=y");
-
-    Path childTmpDir = new Path(Environment.PWD.$(),
-        YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR);
-    vargs.add("-Djava.io.tmpdir=" + childTmpDir);
-
-    // FIXME Setup the log4j properties
-
-    // Decision to profile needs to be made in the scheduler.
-    if (shouldProfile) {
-      // FIXME add support for profiling
-    }
-
-    // Add main class and its arguments
-    vargs.add(YarnTezDagChild.class.getName());  // main of Child
-
-    // pass TaskAttemptListener's address
-    vargs.add(taskAttemptListenerAddr.getAddress().getHostAddress());
-    vargs.add(Integer.toString(taskAttemptListenerAddr.getPort()));
-    vargs.add(containerIdentifier);
-    vargs.add(tokenIdentifier);
-    vargs.add(Integer.toString(applicationAttemptNumber));
-
-    vargs.add("1>" + getTaskLogFile(LogName.STDOUT));
-    vargs.add("2>" + getTaskLogFile(LogName.STDERR));
-
-    // Final commmand
-    StringBuilder mergedCommand = new StringBuilder();
-    for (CharSequence str : vargs) {
-      mergedCommand.append(str).append(" ");
-    }
-    Vector<String> vargsFinal = new Vector<String>(1);
-    vargsFinal.add(mergedCommand.toString());
-    return vargsFinal;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java
new file mode 100644
index 0000000..e1219c1
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/utils/TezRuntimeChildJVM.java
@@ -0,0 +1,122 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS 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.utils;
+
+import java.net.InetSocketAddress;
+import java.util.List;
+import java.util.Vector;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.YarnTezDagChild;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+
+public class TezRuntimeChildJVM {
+
+    // FIXME
+  public static enum LogName {
+    /** Log on the stdout of the task. */
+    STDOUT ("stdout"),
+
+    /** Log on the stderr of the task. */
+    STDERR ("stderr"),
+
+    /** Log on the map-reduce system logs of the task. */
+    SYSLOG ("syslog"),
+
+    /** The java profiler information. */
+    PROFILE ("profile.out"),
+
+    /** Log the debug script's stdout  */
+    DEBUGOUT ("debugout");
+
+    private String prefix;
+
+    private LogName(String prefix) {
+      this.prefix = prefix;
+    }
+
+    @Override
+    public String toString() {
+      return prefix;
+    }
+  }
+
+  private static String getTaskLogFile(LogName filter) {
+    return ApplicationConstants.LOG_DIR_EXPANSION_VAR + Path.SEPARATOR +
+        filter.toString();
+  }
+
+  public static List<String> getVMCommand(
+      InetSocketAddress taskAttemptListenerAddr,
+      String containerIdentifier,
+      String tokenIdentifier,
+      int applicationAttemptNumber,
+      boolean shouldProfile,
+      String javaOpts) {
+
+    Vector<String> vargs = new Vector<String>(9);
+
+    vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
+
+    //set custom javaOpts
+    vargs.add(javaOpts);
+
+//[Debug Task] Current simplest way to attach debugger to  Tez Child Task
+// Uncomment the following, then launch a regular job
+// Works best on one-box configured with a single container (hence one task at a time).
+//    LOG.error(" !!!!!!!!! Launching Child-Task in debug/suspend mode.  Attach to port 8003 !!!!!!!!");
+//    vargs.add("-Xdebug -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,address=8003,server=y,suspend=y");
+
+    Path childTmpDir = new Path(Environment.PWD.$(),
+        YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR);
+    vargs.add("-Djava.io.tmpdir=" + childTmpDir);
+
+    // FIXME Setup the log4j properties
+
+    // Decision to profile needs to be made in the scheduler.
+    if (shouldProfile) {
+      // FIXME add support for profiling
+    }
+
+    // Add main class and its arguments
+    vargs.add(YarnTezDagChild.class.getName());  // main of Child
+
+    // pass TaskAttemptListener's address
+    vargs.add(taskAttemptListenerAddr.getAddress().getHostAddress());
+    vargs.add(Integer.toString(taskAttemptListenerAddr.getPort()));
+    vargs.add(containerIdentifier);
+    vargs.add(tokenIdentifier);
+    vargs.add(Integer.toString(applicationAttemptNumber));
+
+    vargs.add("1>" + getTaskLogFile(LogName.STDOUT));
+    vargs.add("2>" + getTaskLogFile(LogName.STDERR));
+
+    // Final commmand
+    StringBuilder mergedCommand = new StringBuilder();
+    for (CharSequence str : vargs) {
+      mergedCommand.append(str).append(" ");
+    }
+    Vector<String> vargsFinal = new Vector<String>(1);
+    vargsFinal.add(mergedCommand.toString());
+    return vargsFinal;
+  }
+
+}


[12/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 6b48270..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/DefaultSorter.java
+++ /dev/null
@@ -1,1108 +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.impl.dflt;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.nio.IntBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.ReentrantLock;
-
-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.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-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;
-import org.apache.tez.engine.common.sort.impl.TezIndexRecord;
-import org.apache.tez.engine.common.sort.impl.TezMerger;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.common.sort.impl.TezSpillRecord;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
-
-@SuppressWarnings({"unchecked", "rawtypes"})
-public class DefaultSorter extends ExternalSorter implements IndexedSortable {
-  
-  private static final Log LOG = LogFactory.getLog(DefaultSorter.class);
-
-  // TODO NEWTEZ Progress reporting to Tez framework. (making progress vs %complete)
-  
-  /**
-   * The size of each record in the index file for the map-outputs.
-   */
-  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
-
-  private final static int APPROX_HEADER_LENGTH = 150;
-
-  // k/v accounting
-  IntBuffer kvmeta; // metadata overlay on backing store
-  int kvstart;            // marks origin of spill metadata
-  int kvend;              // marks end of spill metadata
-  int kvindex;            // marks end of fully serialized records
-
-  int equator;            // marks origin of meta/serialization
-  int bufstart;           // marks beginning of spill
-  int bufend;             // marks beginning of collectable
-  int bufmark;            // marks end of record
-  int bufindex;           // marks end of collected
-  int bufvoid;            // marks the point where we should stop
-                          // reading at the end of the buffer
-
-  byte[] kvbuffer;        // main output buffer
-  private final byte[] b0 = new byte[0];
-
-  protected static final int INDEX = 0;            // index offset in acct
-  protected static final int VALSTART = 1;         // val offset in acct
-  protected static final int KEYSTART = 2;         // key offset in acct
-  protected static final int PARTITION = 3;        // partition offset in acct
-  protected static final int NMETA = 4;            // num meta ints
-  protected static final int METASIZE = NMETA * 4; // size in bytes
-
-  // spill accounting
-  int maxRec;
-  int softLimit;
-  boolean spillInProgress;
-  int bufferRemaining;
-  volatile Throwable sortSpillException = null;
-
-  int numSpills = 0;
-  int minSpillsForCombine;
-  final ReentrantLock spillLock = new ReentrantLock();
-  final Condition spillDone = spillLock.newCondition();
-  final Condition spillReady = spillLock.newCondition();
-  final BlockingBuffer bb = new BlockingBuffer();
-  volatile boolean spillThreadRunning = false;
-  final SpillThread spillThread = new SpillThread();
-
-  final ArrayList<TezSpillRecord> indexCacheList =
-    new ArrayList<TezSpillRecord>();
-  private int totalIndexCacheMemory;
-  private int indexCacheMemoryLimit;
-
-  @Override
-  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException { 
-    super.initialize(outputContext, conf, numOutputs);
-
-    // sanity checks
-    final float spillper = this.conf.getFloat(
-        TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_SORT_SPILL_PERCENT);
-    final int sortmb = this.conf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_MB,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_MB);
-    if (spillper > (float) 1.0 || spillper <= (float) 0.0) {
-      throw new IOException("Invalid \""
-          + TezJobConfig.TEZ_ENGINE_SORT_SPILL_PERCENT + "\": " + spillper);
-    }
-    if ((sortmb & 0x7FF) != sortmb) {
-      throw new IOException("Invalid \"" + TezJobConfig.TEZ_ENGINE_IO_SORT_MB
-          + "\": " + sortmb);
-    }
-
-    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES,
-                                       TezJobConfig.DEFAULT_TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES);
-
-    // buffers and accounting
-    int maxMemUsage = sortmb << 20;
-    maxMemUsage -= maxMemUsage % METASIZE;
-    kvbuffer = new byte[maxMemUsage];
-    bufvoid = kvbuffer.length;
-    kvmeta = ByteBuffer.wrap(kvbuffer)
-       .order(ByteOrder.nativeOrder())
-       .asIntBuffer();
-    setEquator(0);
-    bufstart = bufend = bufindex = equator;
-    kvstart = kvend = kvindex;
-
-    maxRec = kvmeta.capacity() / NMETA;
-    softLimit = (int)(kvbuffer.length * spillper);
-    bufferRemaining = softLimit;
-    if (LOG.isInfoEnabled()) {
-      LOG.info(TezJobConfig.TEZ_ENGINE_IO_SORT_MB + ": " + sortmb);
-      LOG.info("soft limit at " + softLimit);
-      LOG.info("bufstart = " + bufstart + "; bufvoid = " + bufvoid);
-      LOG.info("kvstart = " + kvstart + "; length = " + maxRec);
-    }
-
-    // k/v serialization
-    valSerializer.open(bb);
-    keySerializer.open(bb);
-
-    spillInProgress = false;
-    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_ENGINE_COMBINE_MIN_SPILLS, 3);
-    spillThread.setDaemon(true);
-    spillThread.setName("SpillThread");
-    spillLock.lock();
-    try {
-      spillThread.start();
-      while (!spillThreadRunning) {
-        spillDone.await();
-      }
-    } catch (InterruptedException e) {
-      throw new IOException("Spill thread failed to initialize", e);
-    } finally {
-      spillLock.unlock();
-    }
-    if (sortSpillException != null) {
-      throw new IOException("Spill thread failed to initialize",
-          sortSpillException);
-    }
-  }
-
-  @Override
-  public void write(Object key, Object value)
-      throws IOException {
-    collect(
-        key, value, partitioner.getPartition(key, value, partitions));
-  }
-
-  /**
-   * Serialize the key, value to intermediate storage.
-   * When this method returns, kvindex must refer to sufficient unused
-   * storage to store one METADATA.
-   */
-  synchronized void collect(Object key, Object value, final int partition
-                                   ) throws IOException {
-
-    if (key.getClass() != keyClass) {
-      throw new IOException("Type mismatch in key from map: expected "
-                            + keyClass.getName() + ", received "
-                            + key.getClass().getName());
-    }
-    if (value.getClass() != valClass) {
-      throw new IOException("Type mismatch in value from map: expected "
-                            + valClass.getName() + ", received "
-                            + value.getClass().getName());
-    }
-    if (partition < 0 || partition >= partitions) {
-      throw new IOException("Illegal partition for " + key + " (" +
-          partition + ")" + ", TotalPartitions: " + partitions);
-    }
-    checkSpillException();
-    bufferRemaining -= METASIZE;
-    if (bufferRemaining <= 0) {
-      // start spill if the thread is not running and the soft limit has been
-      // reached
-      spillLock.lock();
-      try {
-        do {
-          if (!spillInProgress) {
-            final int kvbidx = 4 * kvindex;
-            final int kvbend = 4 * kvend;
-            // serialized, unspilled bytes always lie between kvindex and
-            // bufindex, crossing the equator. Note that any void space
-            // created by a reset must be included in "used" bytes
-            final int bUsed = distanceTo(kvbidx, bufindex);
-            final boolean bufsoftlimit = bUsed >= softLimit;
-            if ((kvbend + METASIZE) % kvbuffer.length !=
-                equator - (equator % METASIZE)) {
-              // spill finished, reclaim space
-              resetSpill();
-              bufferRemaining = Math.min(
-                  distanceTo(bufindex, kvbidx) - 2 * METASIZE,
-                  softLimit - bUsed) - METASIZE;
-              continue;
-            } else if (bufsoftlimit && kvindex != kvend) {
-              // spill records, if any collected; check latter, as it may
-              // be possible for metadata alignment to hit spill pcnt
-              startSpill();
-              final int avgRec = (int)
-                (mapOutputByteCounter.getValue() /
-                mapOutputRecordCounter.getValue());
-              // leave at least half the split buffer for serialization data
-              // ensure that kvindex >= bufindex
-              final int distkvi = distanceTo(bufindex, kvbidx);
-              final int newPos = (bufindex +
-                Math.max(2 * METASIZE - 1,
-                        Math.min(distkvi / 2,
-                                 distkvi / (METASIZE + avgRec) * METASIZE)))
-                % kvbuffer.length;
-              setEquator(newPos);
-              bufmark = bufindex = newPos;
-              final int serBound = 4 * kvend;
-              // bytes remaining before the lock must be held and limits
-              // checked is the minimum of three arcs: the metadata space, the
-              // serialization space, and the soft limit
-              bufferRemaining = Math.min(
-                  // metadata max
-                  distanceTo(bufend, newPos),
-                  Math.min(
-                    // serialization max
-                    distanceTo(newPos, serBound),
-                    // soft limit
-                    softLimit)) - 2 * METASIZE;
-            }
-          }
-        } while (false);
-      } finally {
-        spillLock.unlock();
-      }
-    }
-
-    try {
-      // serialize key bytes into buffer
-      int keystart = bufindex;
-      keySerializer.serialize(key);
-      if (bufindex < keystart) {
-        // wrapped the key; must make contiguous
-        bb.shiftBufferedKey();
-        keystart = 0;
-      }
-      // serialize value bytes into buffer
-      final int valstart = bufindex;
-      valSerializer.serialize(value);
-      // It's possible for records to have zero length, i.e. the serializer
-      // will perform no writes. To ensure that the boundary conditions are
-      // checked and that the kvindex invariant is maintained, perform a
-      // zero-length write into the buffer. The logic monitoring this could be
-      // moved into collect, but this is cleaner and inexpensive. For now, it
-      // is acceptable.
-      bb.write(b0, 0, 0);
-
-      // the record must be marked after the preceding write, as the metadata
-      // for this record are not yet written
-      int valend = bb.markRecord();
-
-      mapOutputRecordCounter.increment(1);
-      mapOutputByteCounter.increment(
-          distanceTo(keystart, valend, bufvoid));
-
-      // write accounting info
-      kvmeta.put(kvindex + INDEX, kvindex);
-      kvmeta.put(kvindex + PARTITION, partition);
-      kvmeta.put(kvindex + KEYSTART, keystart);
-      kvmeta.put(kvindex + VALSTART, valstart);
-      // advance kvindex
-      kvindex = (kvindex - NMETA + kvmeta.capacity()) % kvmeta.capacity();
-    } catch (MapBufferTooSmallException e) {
-      LOG.info("Record too large for in-memory buffer: " + e.getMessage());
-      spillSingleRecord(key, value, partition);
-      mapOutputRecordCounter.increment(1);
-      return;
-    }
-  }
-
-  /**
-   * Set the point from which meta and serialization data expand. The meta
-   * indices are aligned with the buffer, so metadata never spans the ends of
-   * the circular buffer.
-   */
-  private void setEquator(int pos) {
-    equator = pos;
-    // set index prior to first entry, aligned at meta boundary
-    final int aligned = pos - (pos % METASIZE);
-    kvindex =
-      ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
-    if (LOG.isInfoEnabled()) {
-      LOG.info("(EQUATOR) " + pos + " kvi " + kvindex +
-          "(" + (kvindex * 4) + ")");
-    }
-  }
-
-  /**
-   * The spill is complete, so set the buffer and meta indices to be equal to
-   * the new equator to free space for continuing collection. Note that when
-   * kvindex == kvend == kvstart, the buffer is empty.
-   */
-  private void resetSpill() {
-    final int e = equator;
-    bufstart = bufend = e;
-    final int aligned = e - (e % METASIZE);
-    // set start/end to point to first meta record
-    kvstart = kvend =
-      ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
-    if (LOG.isInfoEnabled()) {
-      LOG.info("(RESET) equator " + e + " kv " + kvstart + "(" +
-        (kvstart * 4) + ")" + " kvi " + kvindex + "(" + (kvindex * 4) + ")");
-    }
-  }
-
-  /**
-   * Compute the distance in bytes between two indices in the serialization
-   * buffer.
-   * @see #distanceTo(int,int,int)
-   */
-  final int distanceTo(final int i, final int j) {
-    return distanceTo(i, j, kvbuffer.length);
-  }
-
-  /**
-   * Compute the distance between two indices in the circular buffer given the
-   * max distance.
-   */
-  int distanceTo(final int i, final int j, final int mod) {
-    return i <= j
-      ? j - i
-      : mod - i + j;
-  }
-
-  /**
-   * For the given meta position, return the dereferenced position in the
-   * integer array. Each meta block contains several integers describing
-   * record data in its serialized form, but the INDEX is not necessarily
-   * related to the proximate metadata. The index value at the referenced int
-   * position is the start offset of the associated metadata block. So the
-   * metadata INDEX at metapos may point to the metadata described by the
-   * metadata block at metapos + k, which contains information about that
-   * serialized record.
-   */
-  int offsetFor(int metapos) {
-    return kvmeta.get((metapos % maxRec) * NMETA + INDEX);
-  }
-
-  /**
-   * Compare logical range, st i, j MOD offset capacity.
-   * Compare by partition, then by key.
-   * @see IndexedSortable#compare
-   */
-  public int compare(final int mi, final int mj) {
-    final int kvi = offsetFor(mi);
-    final int kvj = offsetFor(mj);
-    final int kvip = kvmeta.get(kvi + PARTITION);
-    final int kvjp = kvmeta.get(kvj + PARTITION);
-    // sort by partition
-    if (kvip != kvjp) {
-      return kvip - kvjp;
-    }
-    // sort by key
-    return comparator.compare(kvbuffer,
-        kvmeta.get(kvi + KEYSTART),
-        kvmeta.get(kvi + VALSTART) - kvmeta.get(kvi + KEYSTART),
-        kvbuffer,
-        kvmeta.get(kvj + KEYSTART),
-        kvmeta.get(kvj + VALSTART) - kvmeta.get(kvj + KEYSTART));
-  }
-
-  /**
-   * Swap logical indices st i, j MOD offset capacity.
-   * @see IndexedSortable#swap
-   */
-  public void swap(final int mi, final int mj) {
-    final int kvi = (mi % maxRec) * NMETA + INDEX;
-    final int kvj = (mj % maxRec) * NMETA + INDEX;
-    int tmp = kvmeta.get(kvi);
-    kvmeta.put(kvi, kvmeta.get(kvj));
-    kvmeta.put(kvj, tmp);
-  }
-
-  /**
-   * Inner class managing the spill of serialized records to disk.
-   */
-  protected class BlockingBuffer extends DataOutputStream {
-
-    public BlockingBuffer() {
-      super(new Buffer());
-    }
-
-    /**
-     * Mark end of record. Note that this is required if the buffer is to
-     * cut the spill in the proper place.
-     */
-    public int markRecord() {
-      bufmark = bufindex;
-      return bufindex;
-    }
-
-    /**
-     * Set position from last mark to end of writable buffer, then rewrite
-     * the data between last mark and kvindex.
-     * This handles a special case where the key wraps around the buffer.
-     * If the key is to be passed to a RawComparator, then it must be
-     * contiguous in the buffer. This recopies the data in the buffer back
-     * into itself, but starting at the beginning of the buffer. Note that
-     * this method should <b>only</b> be called immediately after detecting
-     * this condition. To call it at any other time is undefined and would
-     * likely result in data loss or corruption.
-     * @see #markRecord()
-     */
-    protected void shiftBufferedKey() throws IOException {
-      // spillLock unnecessary; both kvend and kvindex are current
-      int headbytelen = bufvoid - bufmark;
-      bufvoid = bufmark;
-      final int kvbidx = 4 * kvindex;
-      final int kvbend = 4 * kvend;
-      final int avail =
-        Math.min(distanceTo(0, kvbidx), distanceTo(0, kvbend));
-      if (bufindex + headbytelen < avail) {
-        System.arraycopy(kvbuffer, 0, kvbuffer, headbytelen, bufindex);
-        System.arraycopy(kvbuffer, bufvoid, kvbuffer, 0, headbytelen);
-        bufindex += headbytelen;
-        bufferRemaining -= kvbuffer.length - bufvoid;
-      } else {
-        byte[] keytmp = new byte[bufindex];
-        System.arraycopy(kvbuffer, 0, keytmp, 0, bufindex);
-        bufindex = 0;
-        out.write(kvbuffer, bufmark, headbytelen);
-        out.write(keytmp);
-      }
-    }
-  }
-
-  public class Buffer extends OutputStream {
-    private final byte[] scratch = new byte[1];
-
-    @Override
-    public void write(int v)
-        throws IOException {
-      scratch[0] = (byte)v;
-      write(scratch, 0, 1);
-    }
-
-    /**
-     * Attempt to write a sequence of bytes to the collection buffer.
-     * This method will block if the spill thread is running and it
-     * cannot write.
-     * @throws MapBufferTooSmallException if record is too large to
-     *    deserialize into the collection buffer.
-     */
-    @Override
-    public void write(byte b[], int off, int len)
-        throws IOException {
-      // must always verify the invariant that at least METASIZE bytes are
-      // available beyond kvindex, even when len == 0
-      bufferRemaining -= len;
-      if (bufferRemaining <= 0) {
-        // writing these bytes could exhaust available buffer space or fill
-        // the buffer to soft limit. check if spill or blocking are necessary
-        boolean blockwrite = false;
-        spillLock.lock();
-        try {
-          do {
-            checkSpillException();
-
-            final int kvbidx = 4 * kvindex;
-            final int kvbend = 4 * kvend;
-            // ser distance to key index
-            final int distkvi = distanceTo(bufindex, kvbidx);
-            // ser distance to spill end index
-            final int distkve = distanceTo(bufindex, kvbend);
-
-            // if kvindex is closer than kvend, then a spill is neither in
-            // progress nor complete and reset since the lock was held. The
-            // write should block only if there is insufficient space to
-            // complete the current write, write the metadata for this record,
-            // and write the metadata for the next record. If kvend is closer,
-            // then the write should block if there is too little space for
-            // either the metadata or the current write. Note that collect
-            // ensures its metadata requirement with a zero-length write
-            blockwrite = distkvi <= distkve
-              ? distkvi <= len + 2 * METASIZE
-              : distkve <= len || distanceTo(bufend, kvbidx) < 2 * METASIZE;
-
-            if (!spillInProgress) {
-              if (blockwrite) {
-                if ((kvbend + METASIZE) % kvbuffer.length !=
-                    equator - (equator % METASIZE)) {
-                  // spill finished, reclaim space
-                  // need to use meta exclusively; zero-len rec & 100% spill
-                  // pcnt would fail
-                  resetSpill(); // resetSpill doesn't move bufindex, kvindex
-                  bufferRemaining = Math.min(
-                      distkvi - 2 * METASIZE,
-                      softLimit - distanceTo(kvbidx, bufindex)) - len;
-                  continue;
-                }
-                // we have records we can spill; only spill if blocked
-                if (kvindex != kvend) {
-                  startSpill();
-                  // Blocked on this write, waiting for the spill just
-                  // initiated to finish. Instead of repositioning the marker
-                  // and copying the partial record, we set the record start
-                  // to be the new equator
-                  setEquator(bufmark);
-                } else {
-                  // We have no buffered records, and this record is too large
-                  // to write into kvbuffer. We must spill it directly from
-                  // collect
-                  final int size = distanceTo(bufstart, bufindex) + len;
-                  setEquator(0);
-                  bufstart = bufend = bufindex = equator;
-                  kvstart = kvend = kvindex;
-                  bufvoid = kvbuffer.length;
-                  throw new MapBufferTooSmallException(size + " bytes");
-                }
-              }
-            }
-
-            if (blockwrite) {
-              // wait for spill
-              try {
-                while (spillInProgress) {
-                  spillDone.await();
-                }
-              } catch (InterruptedException e) {
-                  throw new IOException(
-                      "Buffer interrupted while waiting for the writer", e);
-              }
-            }
-          } while (blockwrite);
-        } finally {
-          spillLock.unlock();
-        }
-      }
-      // here, we know that we have sufficient space to write
-      if (bufindex + len > bufvoid) {
-        final int gaplen = bufvoid - bufindex;
-        System.arraycopy(b, off, kvbuffer, bufindex, gaplen);
-        len -= gaplen;
-        off += gaplen;
-        bufindex = 0;
-      }
-      System.arraycopy(b, off, kvbuffer, bufindex, len);
-      bufindex += len;
-    }
-  }
-
-  @Override
-  public void flush() throws IOException {
-    LOG.info("Starting flush of map output");
-    spillLock.lock();
-    try {
-      while (spillInProgress) {
-        spillDone.await();
-      }
-      checkSpillException();
-
-      final int kvbend = 4 * kvend;
-      if ((kvbend + METASIZE) % kvbuffer.length !=
-          equator - (equator % METASIZE)) {
-        // spill finished
-        resetSpill();
-      }
-      if (kvindex != kvend) {
-        kvend = (kvindex + NMETA) % kvmeta.capacity();
-        bufend = bufmark;
-        if (LOG.isInfoEnabled()) {
-          LOG.info("Sorting & Spilling map output");
-          LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
-                   "; bufvoid = " + bufvoid);
-          LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
-                   "); kvend = " + kvend + "(" + (kvend * 4) +
-                   "); length = " + (distanceTo(kvend, kvstart,
-                         kvmeta.capacity()) + 1) + "/" + maxRec);
-        }
-        sortAndSpill();
-      }
-    } catch (InterruptedException e) {
-      throw new IOException("Interrupted while waiting for the writer", e);
-    } finally {
-      spillLock.unlock();
-    }
-    assert !spillLock.isHeldByCurrentThread();
-    // shut down spill thread and wait for it to exit. Since the preceding
-    // ensures that it is finished with its work (and sortAndSpill did not
-    // throw), we elect to use an interrupt instead of setting a flag.
-    // Spilling simultaneously from this thread while the spill thread
-    // finishes its work might be both a useful way to extend this and also
-    // sufficient motivation for the latter approach.
-    try {
-      spillThread.interrupt();
-      spillThread.join();
-    } catch (InterruptedException e) {
-      throw new IOException("Spill failed", e);
-    }
-    // release sort buffer before the merge
-    //FIXME
-    //kvbuffer = null;
-    mergeParts();
-    Path outputPath = mapOutputFile.getOutputFile();
-    fileOutputByteCounter.increment(rfs.getFileStatus(outputPath).getLen());
-  }
-
-  @Override
-  public void close() throws IOException { }
-
-  protected class SpillThread extends Thread {
-
-    @Override
-    public void run() {
-      spillLock.lock();
-      spillThreadRunning = true;
-      try {
-        while (true) {
-          spillDone.signal();
-          while (!spillInProgress) {
-            spillReady.await();
-          }
-          try {
-            spillLock.unlock();
-            sortAndSpill();
-          } catch (Throwable t) {
-            LOG.warn("Got an exception in sortAndSpill", t);
-            sortSpillException = t;
-          } finally {
-            spillLock.lock();
-            if (bufend < bufstart) {
-              bufvoid = kvbuffer.length;
-            }
-            kvstart = kvend;
-            bufstart = bufend;
-            spillInProgress = false;
-          }
-        }
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-      } finally {
-        spillLock.unlock();
-        spillThreadRunning = false;
-      }
-    }
-  }
-
-  private void checkSpillException() throws IOException {
-    final Throwable lspillException = sortSpillException;
-    if (lspillException != null) {
-      if (lspillException instanceof Error) {
-        final String logMsg = "Task " + outputContext.getUniqueIdentifier()
-            + " failed : " + StringUtils.stringifyException(lspillException);
-        outputContext.fatalError(lspillException, logMsg);
-      }
-      throw new IOException("Spill failed", lspillException);
-    }
-  }
-
-  private void startSpill() {
-    assert !spillInProgress;
-    kvend = (kvindex + NMETA) % kvmeta.capacity();
-    bufend = bufmark;
-    spillInProgress = true;
-    if (LOG.isInfoEnabled()) {
-      LOG.info("Spilling map output");
-      LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
-               "; bufvoid = " + bufvoid);
-      LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
-               "); kvend = " + kvend + "(" + (kvend * 4) +
-               "); length = " + (distanceTo(kvend, kvstart,
-                     kvmeta.capacity()) + 1) + "/" + maxRec);
-    }
-    spillReady.signal();
-  }
-
-  int getMetaStart() {
-    return kvend / NMETA;
-  }
-
-  int getMetaEnd() {
-    return 1 + // kvend is a valid record
-        (kvstart >= kvend
-        ? kvstart
-        : kvmeta.capacity() + kvstart) / NMETA;
-  }
-
-  protected void sortAndSpill()
-      throws IOException, InterruptedException {
-    final int mstart = getMetaStart();
-    final int mend = getMetaEnd();
-    sorter.sort(this, mstart, mend, nullProgressable);
-    spill(mstart, mend);
-  }
-
-  protected void spill(int mstart, int mend)
-      throws IOException, InterruptedException {
-
-    //approximate the length of the output file to be the length of the
-    //buffer + header lengths for the partitions
-    final long size = (bufend >= bufstart
-        ? bufend - bufstart
-        : (bufvoid - bufend) + bufstart) +
-                partitions * APPROX_HEADER_LENGTH;
-    FSDataOutputStream out = null;
-    try {
-      // create spill file
-      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
-      final Path filename =
-          mapOutputFile.getSpillFileForWrite(numSpills, size);
-      out = rfs.create(filename);
-
-      int spindex = mstart;
-      final InMemValBytes value = createInMemValBytes();
-      for (int i = 0; i < partitions; ++i) {
-        IFile.Writer writer = null;
-        try {
-          long segmentStart = out.getPos();
-          writer = new Writer(conf, out, keyClass, valClass, codec,
-                                    spilledRecordsCounter);
-          if (combiner == null) {
-            // spill directly
-            DataInputBuffer key = new DataInputBuffer();
-            while (spindex < mend &&
-                kvmeta.get(offsetFor(spindex) + PARTITION) == i) {
-              final int kvoff = offsetFor(spindex);
-              key.reset(
-                  kvbuffer,
-                  kvmeta.get(kvoff + KEYSTART),
-                  (kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART))
-                  );
-              getVBytesForOffset(kvoff, value);
-              writer.append(key, value);
-              ++spindex;
-            }
-          } else {
-            int spstart = spindex;
-            while (spindex < mend &&
-                kvmeta.get(offsetFor(spindex)
-                          + PARTITION) == i) {
-              ++spindex;
-            }
-            // Note: we would like to avoid the combiner if we've fewer
-            // than some threshold of records for a partition
-            if (spstart != spindex) {
-              TezRawKeyValueIterator kvIter =
-                new MRResultIterator(spstart, spindex);
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Running combine processor");
-              }
-              runCombineProcessor(kvIter, writer);
-            }
-          }
-
-          // close the writer
-          writer.close();
-
-          // record offsets
-          final TezIndexRecord rec =
-              new TezIndexRecord(
-                  segmentStart,
-                  writer.getRawLength(),
-                  writer.getCompressedLength());
-          spillRec.putIndex(rec, i);
-
-          writer = null;
-        } finally {
-          if (null != writer) writer.close();
-        }
-      }
-
-      if (totalIndexCacheMemory >= indexCacheMemoryLimit) {
-        // create spill index file
-        Path indexFilename =
-            mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
-                * MAP_OUTPUT_INDEX_RECORD_LENGTH);
-        spillRec.writeToFile(indexFilename, conf);
-      } else {
-        indexCacheList.add(spillRec);
-        totalIndexCacheMemory +=
-          spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH;
-      }
-      LOG.info("Finished spill " + numSpills);
-      ++numSpills;
-    } finally {
-      if (out != null) out.close();
-    }
-  }
-
-  /**
-   * Handles the degenerate case where serialization fails to fit in
-   * the in-memory buffer, so we must spill the record from collect
-   * directly to a spill file. Consider this "losing".
-   */
-  private void spillSingleRecord(final Object key, final Object value,
-                                 int partition) throws IOException {
-    long size = kvbuffer.length + partitions * APPROX_HEADER_LENGTH;
-    FSDataOutputStream out = null;
-    try {
-      // create spill file
-      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
-      final Path filename =
-          mapOutputFile.getSpillFileForWrite(numSpills, size);
-      out = rfs.create(filename);
-
-      // we don't run the combiner for a single record
-      for (int i = 0; i < partitions; ++i) {
-        IFile.Writer writer = null;
-        try {
-          long segmentStart = out.getPos();
-          // Create a new codec, don't care!
-          writer = new IFile.Writer(conf, out, keyClass, valClass, codec,
-                                          spilledRecordsCounter);
-
-          if (i == partition) {
-            final long recordStart = out.getPos();
-            writer.append(key, value);
-            // Note that our map byte count will not be accurate with
-            // compression
-            mapOutputByteCounter.increment(out.getPos() - recordStart);
-          }
-          writer.close();
-
-          // record offsets
-          TezIndexRecord rec =
-              new TezIndexRecord(
-                  segmentStart,
-                  writer.getRawLength(),
-                  writer.getCompressedLength());
-          spillRec.putIndex(rec, i);
-
-          writer = null;
-        } catch (IOException e) {
-          if (null != writer) writer.close();
-          throw e;
-        }
-      }
-      if (totalIndexCacheMemory >= indexCacheMemoryLimit) {
-        // create spill index file
-        Path indexFilename =
-            mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
-                * MAP_OUTPUT_INDEX_RECORD_LENGTH);
-        spillRec.writeToFile(indexFilename, conf);
-      } else {
-        indexCacheList.add(spillRec);
-        totalIndexCacheMemory +=
-          spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH;
-      }
-      ++numSpills;
-    } finally {
-      if (out != null) out.close();
-    }
-  }
-
-  protected int getInMemVBytesLength(int kvoff) {
-    // get the keystart for the next serialized value to be the end
-    // of this value. If this is the last value in the buffer, use bufend
-    final int nextindex = kvoff == kvend
-      ? bufend
-      : kvmeta.get(
-          (kvoff - NMETA + kvmeta.capacity() + KEYSTART) % kvmeta.capacity());
-    // calculate the length of the value
-    int vallen = (nextindex >= kvmeta.get(kvoff + VALSTART))
-      ? nextindex - kvmeta.get(kvoff + VALSTART)
-      : (bufvoid - kvmeta.get(kvoff + VALSTART)) + nextindex;
-      return vallen;
-  }
-
-  /**
-   * Given an offset, populate vbytes with the associated set of
-   * deserialized value bytes. Should only be called during a spill.
-   */
-  int getVBytesForOffset(int kvoff, InMemValBytes vbytes) {
-    int vallen = getInMemVBytesLength(kvoff);
-    vbytes.reset(kvbuffer, kvmeta.get(kvoff + VALSTART), vallen);
-    return vallen;
-  }
-
-  /**
-   * Inner class wrapping valuebytes, used for appendRaw.
-   */
-  static class InMemValBytes extends DataInputBuffer {
-    private byte[] buffer;
-    private int start;
-    private int length;
-    private final int bufvoid;
-
-    public InMemValBytes(int bufvoid) {
-      this.bufvoid = bufvoid;
-    }
-
-    public void reset(byte[] buffer, int start, int length) {
-      this.buffer = buffer;
-      this.start = start;
-      this.length = length;
-
-      if (start + length > bufvoid) {
-        this.buffer = new byte[this.length];
-        final int taillen = bufvoid - start;
-        System.arraycopy(buffer, start, this.buffer, 0, taillen);
-        System.arraycopy(buffer, 0, this.buffer, taillen, length-taillen);
-        this.start = 0;
-      }
-
-      super.reset(this.buffer, this.start, this.length);
-    }
-  }
-
-  InMemValBytes createInMemValBytes() {
-    return new InMemValBytes(bufvoid);
-  }
-
-  protected class MRResultIterator implements TezRawKeyValueIterator {
-    private final DataInputBuffer keybuf = new DataInputBuffer();
-    private final InMemValBytes vbytes = createInMemValBytes();
-    private final int end;
-    private int current;
-    public MRResultIterator(int start, int end) {
-      this.end = end;
-      current = start - 1;
-    }
-    public boolean next() throws IOException {
-      return ++current < end;
-    }
-    public DataInputBuffer getKey() throws IOException {
-      final int kvoff = offsetFor(current);
-      keybuf.reset(kvbuffer, kvmeta.get(kvoff + KEYSTART),
-          kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART));
-      return keybuf;
-    }
-    public DataInputBuffer getValue() throws IOException {
-      getVBytesForOffset(offsetFor(current), vbytes);
-      return vbytes;
-    }
-    public Progress getProgress() {
-      return null;
-    }
-    public void close() { }
-  }
-
-  private void mergeParts() throws IOException {
-    // get the approximate size of the final output/index files
-    long finalOutFileSize = 0;
-    long finalIndexFileSize = 0;
-    final Path[] filename = new Path[numSpills];
-    final String taskIdentifier = outputContext.getUniqueIdentifier();
-
-    for(int i = 0; i < numSpills; i++) {
-      filename[i] = mapOutputFile.getSpillFile(i);
-      finalOutFileSize += rfs.getFileStatus(filename[i]).getLen();
-    }
-    if (numSpills == 1) { //the spill is the final output
-      sameVolRename(filename[0],
-          mapOutputFile.getOutputFileForWriteInVolume(filename[0]));
-      if (indexCacheList.size() == 0) {
-        sameVolRename(mapOutputFile.getSpillIndexFile(0),
-          mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]));
-      } else {
-        indexCacheList.get(0).writeToFile(
-          mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]), conf);
-      }
-      return;
-    }
-
-    // read in paged indices
-    for (int i = indexCacheList.size(); i < numSpills; ++i) {
-      Path indexFileName = mapOutputFile.getSpillIndexFile(i);
-      indexCacheList.add(new TezSpillRecord(indexFileName, conf));
-    }
-
-    //make correction in the length to include the sequence file header
-    //lengths for each partition
-    finalOutFileSize += partitions * APPROX_HEADER_LENGTH;
-    finalIndexFileSize = partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH;
-    Path finalOutputFile =
-        mapOutputFile.getOutputFileForWrite(finalOutFileSize);
-    Path finalIndexFile =
-        mapOutputFile.getOutputIndexFileForWrite(finalIndexFileSize);
-
-    //The output stream for the final single output file
-    FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
-
-    if (numSpills == 0) {
-      //create dummy files
-
-      TezSpillRecord sr = new TezSpillRecord(partitions);
-      try {
-        for (int i = 0; i < partitions; i++) {
-          long segmentStart = finalOut.getPos();
-          Writer writer =
-            new Writer(conf, finalOut, keyClass, valClass, codec, null);
-          writer.close();
-
-          TezIndexRecord rec =
-              new TezIndexRecord(
-                  segmentStart,
-                  writer.getRawLength(),
-                  writer.getCompressedLength());
-          sr.putIndex(rec, i);
-        }
-        sr.writeToFile(finalIndexFile, conf);
-      } finally {
-        finalOut.close();
-      }
-      return;
-    }
-    else {
-      TezMerger.considerFinalMergeForProgress();
-
-      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
-      for (int parts = 0; parts < partitions; parts++) {
-        //create the segments to be merged
-        List<Segment> segmentList =
-          new ArrayList<Segment>(numSpills);
-        for(int i = 0; i < numSpills; i++) {
-          TezIndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
-
-          Segment s =
-            new Segment(conf, rfs, filename[i], indexRecord.getStartOffset(),
-                             indexRecord.getPartLength(), codec, true);
-          segmentList.add(i, s);
-
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("TaskIdentifier=" + taskIdentifier + " Partition=" + parts +
-                "Spill =" + i + "(" + indexRecord.getStartOffset() + "," +
-                indexRecord.getRawLength() + ", " +
-                indexRecord.getPartLength() + ")");
-          }
-        }
-
-        int mergeFactor =
-            this.conf.getInt(TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR,
-                TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
-        // sort the segments only if there are intermediate merges
-        boolean sortSegments = segmentList.size() > mergeFactor;
-        //merge
-        TezRawKeyValueIterator kvIter = TezMerger.merge(conf, rfs,
-                       keyClass, valClass, codec,
-                       segmentList, mergeFactor,
-                       new Path(taskIdentifier),
-                       (RawComparator)ConfigUtils.getIntermediateOutputKeyComparator(conf),
-                       nullProgressable, sortSegments,
-                       null, spilledRecordsCounter,
-                       null); // Not using any Progress in TezMerger. Should just work.
-
-        //write merged output to disk
-        long segmentStart = finalOut.getPos();
-        Writer writer =
-            new Writer(conf, finalOut, keyClass, valClass, codec,
-                spilledRecordsCounter);
-        if (combiner == null || numSpills < minSpillsForCombine) {
-          TezMerger.writeFile(kvIter, writer,
-              nullProgressable, conf);
-        } else {
-          runCombineProcessor(kvIter, writer);
-        }
-        writer.close();
-
-        // record offsets
-        final TezIndexRecord rec =
-            new TezIndexRecord(
-                segmentStart,
-                writer.getRawLength(),
-                writer.getCompressedLength());
-        spillRec.putIndex(rec, parts);
-      }
-      spillRec.writeToFile(finalIndexFile, conf);
-      finalOut.close();
-      for(int i = 0; i < numSpills; i++) {
-        rfs.delete(filename[i],true);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index e2b3315..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/InMemoryShuffleSorter.java
+++ /dev/null
@@ -1,126 +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.impl.dflt;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.IntBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.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;
-
-public class InMemoryShuffleSorter extends DefaultSorter {
-
-  private static final Log LOG = LogFactory.getLog(InMemoryShuffleSorter.class);
-  
-  static final int IFILE_EOF_LENGTH = 
-      2 * WritableUtils.getVIntSize(IFile.EOF_MARKER);
-  static final int IFILE_CHECKSUM_LENGTH = DataChecksum.Type.CRC32.size;
-  
-  private List<Integer> spillIndices = new ArrayList<Integer>();
-  private List<ShuffleHeader> shuffleHeaders = new ArrayList<ShuffleHeader>();
-
-  ShuffleHandler shuffleHandler = new ShuffleHandler(this);
-  
-  byte[] kvbuffer;
-  IntBuffer kvmeta;
-
-  @Override
-  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
-    super.initialize(outputContext, conf, numOutputs);
-    shuffleHandler.initialize(outputContext, conf);
-  }
-
-  @Override
-  protected void spill(int mstart, int mend) 
-      throws IOException, InterruptedException {
-    // Start the shuffleHandler
-    shuffleHandler.start();
-
-    // Don't spill!
-    
-    // Make a copy
-    this.kvbuffer = super.kvbuffer;
-    this.kvmeta = super.kvmeta;
-
-    // Just save spill-indices for serving later
-    int spindex = mstart;
-    for (int i = 0; i < partitions; ++i) {
-      spillIndices.add(spindex);
-      
-      int length = 0;
-      while (spindex < mend &&
-          kvmeta.get(offsetFor(spindex) + PARTITION) == i) {
-
-        final int kvoff = offsetFor(spindex);
-        int keyLen = 
-            kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART);
-        int valLen = getInMemVBytesLength(kvoff);
-        length += 
-            (keyLen + WritableUtils.getVIntSize(keyLen)) + 
-            (valLen + WritableUtils.getVIntSize(valLen));
-
-        ++spindex;
-      }
-      length += IFILE_EOF_LENGTH;
-      
-      shuffleHeaders.add( 
-          new ShuffleHeader(
-              outputContext.getUniqueIdentifier(), // TODO Verify that this is correct. 
-              length + IFILE_CHECKSUM_LENGTH, length, i)
-          );
-      LOG.info("shuffleHeader[" + i + "]:" +
-      		" rawLen=" + length + " partLen=" + (length + IFILE_CHECKSUM_LENGTH) + 
-          " spillIndex=" + spillIndices.get(i));
-    }
-    
-    LOG.info("Saved " + spillIndices.size() + " spill-indices and " + 
-        shuffleHeaders.size() + " shuffle headers");
-  }
-
-  @Override
-  public InputStream getSortedStream(int partition) {
-    return new SortBufferInputStream(this, partition);
-  }
-
-  @Override
-  public void close() throws IOException {
-    // FIXME
-    //shuffleHandler.stop();
-  }
-
-  @Override
-  public ShuffleHeader getShuffleHeader(int reduce) {
-    return shuffleHeaders.get(reduce);
-  }
-
-  public int getSpillIndex(int partition) {
-    return spillIndices.get(partition);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/SortBufferInputStream.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/SortBufferInputStream.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/SortBufferInputStream.java
deleted file mode 100644
index d74e159..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/dflt/SortBufferInputStream.java
+++ /dev/null
@@ -1,271 +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.impl.dflt;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.IntBuffer;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.io.BoundedByteArrayOutputStream;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.tez.engine.common.shuffle.impl.InMemoryWriter;
-import org.apache.tez.engine.common.sort.impl.dflt.DefaultSorter.InMemValBytes;
-
-  public class SortBufferInputStream extends InputStream {
-
-  private static final Log LOG = LogFactory.getLog(SortBufferInputStream.class);
-  
-  private final InMemoryShuffleSorter sorter;
-  private InMemoryWriter sortOutput;
-  
-  private int mend;
-  private int recIndex;
-  private final byte[] kvbuffer;       
-  private final IntBuffer kvmeta;
-  private final int partitionBytes;
-  private final int partition;
-  
-  byte[] dualBuf = new byte[8192];
-  DualBufferOutputStream out;
-  private int readBytes = 0;
-  
-  public SortBufferInputStream(
-      InMemoryShuffleSorter sorter, int partition) {
-    this.sorter = sorter;
-    this.partitionBytes = 
-        (int)sorter.getShuffleHeader(partition).getCompressedLength();
-    this.partition = partition;
-    this.mend = sorter.getMetaEnd();
-    this.recIndex = sorter.getSpillIndex(partition);
-    this.kvbuffer = sorter.kvbuffer;
-    this.kvmeta = sorter.kvmeta;
-    out = new DualBufferOutputStream(null, 0, 0, dualBuf);
-    sortOutput = new InMemoryWriter(out);
-  }
-  
-  byte[] one = new byte[1];
-  
-  @Override
-  public int read() throws IOException {
-    int b = read(one, 0, 1);
-    return (b == -1) ? b : one[0]; 
-  }
-
-  @Override
-  public int read(byte[] b) throws IOException {
-    return read(b, 0, b.length);
-  }
-
-  @Override
-  public int read(byte[] b, int off, int len) throws IOException {
-    if (available() == 0) {
-      return -1;
-    }
-    
-    int currentOffset = off;
-    int currentLength = len;
-    int currentReadBytes = 0;
-    
-    // Check if there is residual data in the dualBuf
-    int residualLen = out.getCurrent();
-    if (residualLen > 0) {
-      int readable = Math.min(currentLength, residualLen);
-      System.arraycopy(dualBuf, 0, b, currentOffset, readable);
-      currentOffset += readable;
-      currentReadBytes += readable;
-      out.setCurrentPointer(-readable);
-      
-      // buffer has less capacity
-      currentLength -= readable;
-      
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("XXX read_residual:" +
-            " readable=" + readable +
-            " readBytes=" + readBytes);
-      }
-    }
-    
-    // Now, use the provided buffer
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("XXX read: out.reset" +
-          " b=" + b + 
-          " currentOffset=" + currentOffset + 
-          " currentLength=" + currentLength +
-          " recIndex=" + recIndex);
-    }
-    out.reset(b, currentOffset, currentLength);
-    
-    // Read from sort-buffer into the provided buffer, space permitting
-    DataInputBuffer key = new DataInputBuffer();
-    final InMemValBytes value = sorter.createInMemValBytes();
-    
-    int kvPartition = 0;
-    int numRec = 0;
-    for (;
-         currentLength > 0 && recIndex < mend && 
-             (kvPartition = getKVPartition(recIndex)) == partition;
-        ++recIndex) {
-      
-      final int kvoff = sorter.offsetFor(recIndex);
-      
-      int keyLen = 
-          (kvmeta.get(kvoff + InMemoryShuffleSorter.VALSTART) - 
-              kvmeta.get(kvoff + InMemoryShuffleSorter.KEYSTART));
-      key.reset(
-          kvbuffer, 
-          kvmeta.get(kvoff + InMemoryShuffleSorter.KEYSTART),
-          keyLen
-          );
-      
-      int valLen = sorter.getVBytesForOffset(kvoff, value);
-
-      int recLen = 
-          (keyLen + WritableUtils.getVIntSize(keyLen)) + 
-          (valLen + WritableUtils.getVIntSize(valLen));
-      
-      currentReadBytes += recLen;
-      currentOffset += recLen;
-      currentLength -= recLen;
-
-      // Write out key/value into the in-mem ifile
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("XXX read: sortOutput.append" +
-            " #rec=" + ++numRec +
-            " recIndex=" + recIndex + " kvoff=" + kvoff + 
-            " keyLen=" + keyLen + " valLen=" + valLen + " recLen=" + recLen +
-            " readBytes=" + readBytes +
-            " currentReadBytes="  + currentReadBytes +
-            " currentLength=" + currentLength);
-      }
-      sortOutput.append(key, value);
-    }
-
-    // If we are at the end of the segment, close the ifile
-    if (currentLength > 0 && 
-        (recIndex == mend || kvPartition != partition)) {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("XXX About to call close:" +
-            " currentLength=" + currentLength + 
-            " recIndex=" + recIndex + " mend=" + mend + 
-            " kvPartition=" + kvPartition + " partitino=" + partition);
-      }
-      sortOutput.close();
-      currentReadBytes += 
-          (InMemoryShuffleSorter.IFILE_EOF_LENGTH + 
-              InMemoryShuffleSorter.IFILE_CHECKSUM_LENGTH);
-    } else {
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("XXX Hmm..." +
-            " currentLength=" + currentLength + 
-            " recIndex=" + recIndex + " mend=" + mend + 
-            " kvPartition=" + kvPartition + " partitino=" + partition);
-      }
-    }
-    
-    int retVal = Math.min(currentReadBytes, len);
-    readBytes += retVal;
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("XXX read: done" +
-          " retVal=" + retVal + 
-          " currentReadBytes=" + currentReadBytes +
-          " len=" + len + 
-          " readBytes=" + readBytes +
-          " partitionBytes=" + partitionBytes +
-          " residualBytes=" + out.getCurrent());
-    }
-    return retVal;
-  }
-
-  private int getKVPartition(int recIndex) {
-    return kvmeta.get(
-        sorter.offsetFor(recIndex) + InMemoryShuffleSorter.PARTITION);
-  }
-  
-  @Override
-  public int available() throws IOException {
-    return (partitionBytes - readBytes);
-  }
-
-  @Override
-  public void close() throws IOException {
-    super.close();
-  }
-
-  @Override
-  public boolean markSupported() {
-    return false;
-  }
-  
-  static class DualBufferOutputStream extends BoundedByteArrayOutputStream {
-
-    byte[] dualBuf;
-    int currentPointer = 0;
-    byte[] one = new byte[1];
-    
-    public DualBufferOutputStream(
-        byte[] buf, int offset, int length, 
-        byte[] altBuf) {
-      super(buf, offset, length);
-      this.dualBuf = altBuf;
-    }
-    
-    public void reset(byte[] b, int off, int len) {
-      super.resetBuffer(b, off, len);
-    }
-
-    @Override
-    public void write(int b) throws IOException {
-      one[0] = (byte)b;
-      write(one, 0, 1);
-    }
-
-    @Override
-    public void write(byte[] b) throws IOException {
-      write(b, 0, b.length);
-    }
-
-    @Override
-    public void write(byte[] b, int off, int len) throws IOException {
-      int available = super.available();
-      if (available >= len) {
-        super.write(b, off, len);
-      } else {
-        super.write(b, off, available);
-        System.arraycopy(b, off+available, dualBuf, currentPointer, len-available);
-        currentPointer += (len - available);
-      }
-    }
-    
-    int getCurrent() {
-      return currentPointer;
-    }
-    
-    void setCurrentPointer(int delta) {
-      if ((currentPointer + delta) > dualBuf.length) {
-        throw new IndexOutOfBoundsException("Trying to set dualBuf 'current'" +
-        		" marker to " + (currentPointer+delta) + " when " +
-        		" dualBuf.length is " + dualBuf.length);
-      }
-      currentPointer = (currentPointer + delta) % dualBuf.length;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java b/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
deleted file mode 100644
index 841e54d..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/impl/ValuesIterator.java
+++ /dev/null
@@ -1,149 +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 java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.serializer.Deserializer;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.util.Progressable;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-
-
-/**
- * Iterates values while keys match in sorted input.
- *
- * Usage: Call moveToNext to move to the next k, v pair. This returns true if another exists,
- * followed by getKey() and getValues() to get the current key and list of values.
- * 
- */
-public class ValuesIterator<KEY,VALUE> implements Iterator<VALUE> {
-  protected TezRawKeyValueIterator in; //input iterator
-  private KEY key;               // current key
-  private KEY nextKey;
-  private VALUE value;             // current value
-  private boolean hasNext;                      // more w/ this key
-  private boolean more;                         // more in file
-  private RawComparator<KEY> comparator;
-  protected Progressable reporter;
-  private Deserializer<KEY> keyDeserializer;
-  private Deserializer<VALUE> valDeserializer;
-  private DataInputBuffer keyIn = new DataInputBuffer();
-  private DataInputBuffer valueIn = new DataInputBuffer();
-  
-  public ValuesIterator (TezRawKeyValueIterator in, 
-                         RawComparator<KEY> comparator, 
-                         Class<KEY> keyClass,
-                         Class<VALUE> valClass, Configuration conf, 
-                         Progressable reporter)
-    throws IOException {
-    this.in = in;
-    this.comparator = comparator;
-    this.reporter = reporter;
-    SerializationFactory serializationFactory = new SerializationFactory(conf);
-    this.keyDeserializer = serializationFactory.getDeserializer(keyClass);
-    this.keyDeserializer.open(keyIn);
-    this.valDeserializer = serializationFactory.getDeserializer(valClass);
-    this.valDeserializer.open(this.valueIn);
-    readNextKey();
-    key = nextKey;
-    nextKey = null; // force new instance creation
-    hasNext = more;
-  }
-
-  TezRawKeyValueIterator getRawIterator() { return in; }
-  
-  /// Iterator methods
-
-  public boolean hasNext() { return hasNext; }
-
-  private int ctr = 0;
-  public VALUE next() {
-    if (!hasNext) {
-      throw new NoSuchElementException("iterate past last value");
-    }
-    try {
-      readNextValue();
-      readNextKey();
-    } catch (IOException ie) {
-      throw new RuntimeException("problem advancing post rec#"+ctr, ie);
-    }
-    reporter.progress();
-    return value;
-  }
-
-  public void remove() { throw new RuntimeException("not implemented"); }
-
-  /// Auxiliary methods
-
-  /** Start processing next unique key. */
-  public void nextKey() throws IOException {
-    // read until we find a new key
-    while (hasNext) { 
-      readNextKey();
-    }
-    ++ctr;
-    
-    // move the next key to the current one
-    KEY tmpKey = key;
-    key = nextKey;
-    nextKey = tmpKey;
-    hasNext = more;
-  }
-
-  /** True iff more keys remain. */
-  public boolean more() { 
-    return more; 
-  }
-
-  /** The current key. */
-  public KEY getKey() { 
-    return key; 
-  }
-
-  /** 
-   * read the next key 
-   */
-  private void readNextKey() throws IOException {
-    more = in.next();
-    if (more) {
-      DataInputBuffer nextKeyBytes = in.getKey();
-      keyIn.reset(nextKeyBytes.getData(), nextKeyBytes.getPosition(), nextKeyBytes.getLength());
-      nextKey = keyDeserializer.deserialize(nextKey);
-      hasNext = key != null && (comparator.compare(key, nextKey) == 0);
-    } else {
-      hasNext = false;
-    }
-  }
-
-  /**
-   * Read the next value
-   * @throws IOException
-   */
-  private void readNextValue() throws IOException {
-    DataInputBuffer nextValueBytes = in.getValue();
-    valueIn.reset(nextValueBytes.getData(), nextValueBytes.getPosition(), nextValueBytes.getLength());
-    value = valDeserializer.deserialize(value);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 40e6b1a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/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.output;
-
-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/b212ca1d/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
deleted file mode 100644
index e1d83ad..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/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.output;
-
-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/b212ca1d/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
deleted file mode 100644
index b8f051b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/task/local/output/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.output;
-
-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/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/hadoop/compat/NullProgressable.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/hadoop/compat/NullProgressable.java b/tez-engine/src/main/java/org/apache/tez/engine/hadoop/compat/NullProgressable.java
deleted file mode 100644
index 5071dd2..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/hadoop/compat/NullProgressable.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.hadoop.compat;
-
-import org.apache.hadoop.util.Progressable;
-
-public class NullProgressable implements Progressable {
-
-  public NullProgressable() {
-    // TODO Auto-generated constructor stub
-  }
-
-  @Override
-  public void progress() {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 6371787..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/lib/input/LocalMergedInput.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.engine.lib.input;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.tez.common.TezUtils;
-import org.apache.tez.engine.api.Event;
-import org.apache.tez.engine.api.LogicalInput;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.localshuffle.LocalShuffle;
-
-/**
- * <code>LocalMergedInput</code> in an {@link LogicalInput} which shuffles intermediate
- * sorted data, merges them and provides key/<values> to the consumer. 
- */
-public class LocalMergedInput extends ShuffledMergedInputLegacy {
-
-  @Override
-  public List<Event> initialize(TezInputContext inputContext) throws IOException {
-    this.inputContext = inputContext;
-    this.conf = TezUtils.createConfFromUserPayload(inputContext.getUserPayload());
-
-    LocalShuffle localShuffle = new LocalShuffle(inputContext, conf, numInputs);
-    rawIter = localShuffle.run();
-    createValuesIterator();
-    return Collections.emptyList();
-  }
-
-  @Override
-  public List<Event> close() throws IOException {
-    rawIter.close();
-    return Collections.emptyList();
-  }
-}


[03/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
new file mode 100644
index 0000000..bb4b4a2
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezMerger.java
@@ -0,0 +1,798 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.common.sort.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+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.ChecksumFileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.util.PriorityQueue;
+import org.apache.hadoop.util.Progress;
+import org.apache.hadoop.util.Progressable;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Reader;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+
+/**
+ * Merger is an utility class used by the Map and Reduce tasks for merging
+ * both their memory and disk segments
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class TezMerger {  
+  private static final Log LOG = LogFactory.getLog(TezMerger.class);
+
+  
+  // Local directories
+  private static LocalDirAllocator lDirAlloc = 
+    new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+
+  public static
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                            Class keyClass, Class valueClass, 
+                            CompressionCodec codec,
+                            Path[] inputs, boolean deleteInputs, 
+                            int mergeFactor, Path tmpDir,
+                            RawComparator comparator, Progressable reporter,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            Progress mergePhase)
+  throws IOException {
+    return 
+      new MergeQueue(conf, fs, inputs, deleteInputs, codec, comparator, 
+                           reporter, null).merge(keyClass, valueClass,
+                                           mergeFactor, tmpDir,
+                                           readsCounter, writesCounter, 
+                                           mergePhase);
+  }
+
+  public static 
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                            Class keyClass, Class valueClass, 
+                            CompressionCodec codec,
+                            Path[] inputs, boolean deleteInputs, 
+                            int mergeFactor, Path tmpDir,
+                            RawComparator comparator,
+                            Progressable reporter,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            TezCounter mergedMapOutputsCounter,
+                            Progress mergePhase)
+  throws IOException {
+    return 
+      new MergeQueue(conf, fs, inputs, deleteInputs, codec, comparator, 
+                           reporter, mergedMapOutputsCounter).merge(
+                                           keyClass, valueClass,
+                                           mergeFactor, tmpDir,
+                                           readsCounter, writesCounter,
+                                           mergePhase);
+  }
+  
+  public static
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs, 
+                            Class keyClass, Class valueClass, 
+                            List<Segment> segments, 
+                            int mergeFactor, Path tmpDir,
+                            RawComparator comparator, Progressable reporter,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            Progress mergePhase)
+      throws IOException {
+    return merge(conf, fs, keyClass, valueClass, segments, mergeFactor, tmpDir,
+                 comparator, reporter, false, readsCounter, writesCounter,
+                 mergePhase);
+  }
+
+  public static <K extends Object, V extends Object>
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                            Class keyClass, Class valueClass,
+                            List<Segment> segments,
+                            int mergeFactor, Path tmpDir,
+                            RawComparator comparator, Progressable reporter,
+                            boolean sortSegments,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            Progress mergePhase)
+      throws IOException {
+    return new MergeQueue(conf, fs, segments, comparator, reporter,
+                           sortSegments).merge(keyClass, valueClass,
+                                               mergeFactor, tmpDir,
+                                               readsCounter, writesCounter,
+                                               mergePhase);
+  }
+
+  public static <K extends Object, V extends Object>
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                            Class keyClass, Class valueClass,
+                            CompressionCodec codec,
+                            List<Segment> segments,
+                            int mergeFactor, Path tmpDir,
+                            RawComparator comparator, Progressable reporter,
+                            boolean sortSegments,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            Progress mergePhase)
+      throws IOException {
+    return new MergeQueue(conf, fs, segments, comparator, reporter,
+                           sortSegments, codec).merge(keyClass, valueClass,
+                                               mergeFactor, tmpDir,
+                                               readsCounter, writesCounter,
+                                               mergePhase);
+  }
+
+  public static <K extends Object, V extends Object>
+    TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                            Class keyClass, Class valueClass,
+                            List<Segment> segments,
+                            int mergeFactor, int inMemSegments, Path tmpDir,
+                            RawComparator comparator, Progressable reporter,
+                            boolean sortSegments,
+                            TezCounter readsCounter,
+                            TezCounter writesCounter,
+                            Progress mergePhase)
+      throws IOException {
+    return new MergeQueue(conf, fs, segments, comparator, reporter,
+                           sortSegments).merge(keyClass, valueClass,
+                                               mergeFactor, inMemSegments,
+                                               tmpDir,
+                                               readsCounter, writesCounter,
+                                               mergePhase);
+  }
+
+
+  static <K extends Object, V extends Object>
+  TezRawKeyValueIterator merge(Configuration conf, FileSystem fs,
+                          Class keyClass, Class valueClass,
+                          CompressionCodec codec,
+                          List<Segment> segments,
+                          int mergeFactor, int inMemSegments, Path tmpDir,
+                          RawComparator comparator, Progressable reporter,
+                          boolean sortSegments,
+                          TezCounter readsCounter,
+                          TezCounter writesCounter,
+                          Progress mergePhase)
+    throws IOException {
+  return new MergeQueue(conf, fs, segments, comparator, reporter,
+                         sortSegments, codec).merge(keyClass, valueClass,
+                                             mergeFactor, inMemSegments,
+                                             tmpDir,
+                                             readsCounter, writesCounter,
+                                             mergePhase);
+}
+
+  public static <K extends Object, V extends Object>
+  void writeFile(TezRawKeyValueIterator records, Writer writer, 
+                 Progressable progressable, Configuration conf) 
+  throws IOException {
+    long progressBar = 
+        conf.getLong(TezJobConfig.RECORDS_BEFORE_PROGRESS, 
+            TezJobConfig.DEFAULT_RECORDS_BEFORE_PROGRESS);
+    long recordCtr = 0;
+    while(records.next()) {
+      writer.append(records.getKey(), records.getValue());
+      
+      if (((recordCtr++) % progressBar) == 0) {
+        progressable.progress();
+      }
+    }
+}
+
+  @InterfaceAudience.Private
+  @InterfaceStability.Unstable
+  public static class Segment<K extends Object, V extends Object> {
+    Reader reader = null;
+    final DataInputBuffer key = new DataInputBuffer();
+    
+    Configuration conf = null;
+    FileSystem fs = null;
+    Path file = null;
+    boolean preserve = false;
+    CompressionCodec codec = null;
+    long segmentOffset = 0;
+    long segmentLength = -1;
+    
+    TezCounter mapOutputsCounter = null;
+
+    public Segment(Configuration conf, FileSystem fs, Path file,
+                   CompressionCodec codec, boolean preserve)
+    throws IOException {
+      this(conf, fs, file, codec, preserve, null);
+    }
+
+    public Segment(Configuration conf, FileSystem fs, Path file,
+                   CompressionCodec codec, boolean preserve,
+                   TezCounter mergedMapOutputsCounter)
+  throws IOException {
+      this(conf, fs, file, 0, fs.getFileStatus(file).getLen(), codec, preserve, 
+           mergedMapOutputsCounter);
+    }
+
+    public Segment(Configuration conf, FileSystem fs, Path file,
+                   long segmentOffset, long segmentLength,
+                   CompressionCodec codec,
+                   boolean preserve) throws IOException {
+      this(conf, fs, file, segmentOffset, segmentLength, codec, preserve, null);
+    }
+
+    public Segment(Configuration conf, FileSystem fs, Path file,
+        long segmentOffset, long segmentLength, CompressionCodec codec,
+        boolean preserve, TezCounter mergedMapOutputsCounter)
+    throws IOException {
+      this.conf = conf;
+      this.fs = fs;
+      this.file = file;
+      this.codec = codec;
+      this.preserve = preserve;
+
+      this.segmentOffset = segmentOffset;
+      this.segmentLength = segmentLength;
+      
+      this.mapOutputsCounter = mergedMapOutputsCounter;
+    }
+    
+    public Segment(Reader reader, boolean preserve) {
+      this(reader, preserve, null);
+    }
+    
+    public Segment(Reader reader, boolean preserve, 
+                   TezCounter mapOutputsCounter) {
+      this.reader = reader;
+      this.preserve = preserve;
+      
+      this.segmentLength = reader.getLength();
+      
+      this.mapOutputsCounter = mapOutputsCounter;
+    }
+
+    void init(TezCounter readsCounter) throws IOException {
+      if (reader == null) {
+        FSDataInputStream in = fs.open(file);
+        in.seek(segmentOffset);
+        reader = new Reader(conf, in, segmentLength, codec, readsCounter);
+      }
+      
+      if (mapOutputsCounter != null) {
+        mapOutputsCounter.increment(1);
+      }
+    }
+    
+    boolean inMemory() {
+      return fs == null;
+    }
+    
+    DataInputBuffer getKey() { return key; }
+
+    DataInputBuffer getValue(DataInputBuffer value) throws IOException {
+      nextRawValue(value);
+      return value;
+    }
+
+    public long getLength() { 
+      return (reader == null) ?
+        segmentLength : reader.getLength();
+    }
+    
+    boolean nextRawKey() throws IOException {
+      return reader.nextRawKey(key);
+    }
+
+    void nextRawValue(DataInputBuffer value) throws IOException {
+      reader.nextRawValue(value);
+    }
+
+    void closeReader() throws IOException {
+      if (reader != null) {
+        reader.close();
+        reader = null;
+      }
+    }
+    
+    void close() throws IOException {
+      closeReader();
+      if (!preserve && fs != null) {
+        fs.delete(file, false);
+      }
+    }
+
+    public long getPosition() throws IOException {
+      return reader.getPosition();
+    }
+
+    // This method is used by BackupStore to extract the 
+    // absolute position after a reset
+    long getActualPosition() throws IOException {
+      return segmentOffset + reader.getPosition();
+    }
+
+    Reader getReader() {
+      return reader;
+    }
+    
+    // This method is used by BackupStore to reinitialize the
+    // reader to start reading from a different segment offset
+    void reinitReader(int offset) throws IOException {
+      if (!inMemory()) {
+        closeReader();
+        segmentOffset = offset;
+        segmentLength = fs.getFileStatus(file).getLen() - segmentOffset;
+        init(null);
+      }
+    }
+  }
+  
+  // Boolean variable for including/considering final merge as part of sort
+  // phase or not. This is true in map task, false in reduce task. It is
+  // used in calculating mergeProgress.
+  static boolean includeFinalMerge = false;
+  
+  /**
+   * Sets the boolean variable includeFinalMerge to true. Called from
+   * map task before calling merge() so that final merge of map task
+   * is also considered as part of sort phase.
+   */
+  public static void considerFinalMergeForProgress() {
+    includeFinalMerge = true;
+  }
+  
+  private static class MergeQueue<K extends Object, V extends Object> 
+  extends PriorityQueue<Segment> implements TezRawKeyValueIterator {
+    Configuration conf;
+    FileSystem fs;
+    CompressionCodec codec;
+    
+    List<Segment> segments = new ArrayList<Segment>();
+    
+    RawComparator comparator;
+    
+    private long totalBytesProcessed;
+    private float progPerByte;
+    private Progress mergeProgress = new Progress();
+    
+    Progressable reporter;
+    
+    DataInputBuffer key;
+    final DataInputBuffer value = new DataInputBuffer();
+    final DataInputBuffer diskIFileValue = new DataInputBuffer();
+    
+    Segment minSegment;
+    Comparator<Segment> segmentComparator =   
+      new Comparator<Segment>() {
+      public int compare(Segment o1, Segment o2) {
+        if (o1.getLength() == o2.getLength()) {
+          return 0;
+        }
+
+        return o1.getLength() < o2.getLength() ? -1 : 1;
+      }
+    };
+
+    public MergeQueue(Configuration conf, FileSystem fs, 
+                      Path[] inputs, boolean deleteInputs, 
+                      CompressionCodec codec, RawComparator comparator,
+                      Progressable reporter, 
+                      TezCounter mergedMapOutputsCounter) 
+    throws IOException {
+      this.conf = conf;
+      this.fs = fs;
+      this.codec = codec;
+      this.comparator = comparator;
+      this.reporter = reporter;
+      
+      for (Path file : inputs) {
+        LOG.debug("MergeQ: adding: " + file);
+        segments.add(new Segment(conf, fs, file, codec, !deleteInputs, 
+                                       (file.toString().endsWith(
+                                           Constants.MERGED_OUTPUT_PREFIX) ? 
+                                        null : mergedMapOutputsCounter)));
+      }
+      
+      // Sort segments on file-lengths
+      Collections.sort(segments, segmentComparator); 
+    }
+    
+    public MergeQueue(Configuration conf, FileSystem fs, 
+        List<Segment> segments, RawComparator comparator,
+        Progressable reporter, boolean sortSegments) {
+      this.conf = conf;
+      this.fs = fs;
+      this.comparator = comparator;
+      this.segments = segments;
+      this.reporter = reporter;
+      if (sortSegments) {
+        Collections.sort(segments, segmentComparator);
+      }
+    }
+
+    public MergeQueue(Configuration conf, FileSystem fs,
+        List<Segment> segments, RawComparator comparator,
+        Progressable reporter, boolean sortSegments, CompressionCodec codec) {
+      this(conf, fs, segments, comparator, reporter, sortSegments);
+      this.codec = codec;
+    }
+
+    public void close() throws IOException {
+      Segment segment;
+      while((segment = pop()) != null) {
+        segment.close();
+      }
+    }
+
+    public DataInputBuffer getKey() throws IOException {
+      return key;
+    }
+
+    public DataInputBuffer getValue() throws IOException {
+      return value;
+    }
+
+    private void adjustPriorityQueue(Segment reader) throws IOException{
+      long startPos = reader.getPosition();
+      boolean hasNext = reader.nextRawKey();
+      long endPos = reader.getPosition();
+      totalBytesProcessed += endPos - startPos;
+      mergeProgress.set(totalBytesProcessed * progPerByte);
+      if (hasNext) {
+        adjustTop();
+      } else {
+        pop();
+        reader.close();
+      }
+    }
+
+    public boolean next() throws IOException {
+      if (size() == 0)
+        return false;
+
+      if (minSegment != null) {
+        //minSegment is non-null for all invocations of next except the first
+        //one. For the first invocation, the priority queue is ready for use
+        //but for the subsequent invocations, first adjust the queue 
+        adjustPriorityQueue(minSegment);
+        if (size() == 0) {
+          minSegment = null;
+          return false;
+        }
+      }
+      minSegment = top();
+      if (!minSegment.inMemory()) {
+        //When we load the value from an inmemory segment, we reset
+        //the "value" DIB in this class to the inmem segment's byte[].
+        //When we load the value bytes from disk, we shouldn't use
+        //the same byte[] since it would corrupt the data in the inmem
+        //segment. So we maintain an explicit DIB for value bytes
+        //obtained from disk, and if the current segment is a disk
+        //segment, we reset the "value" DIB to the byte[] in that (so 
+        //we reuse the disk segment DIB whenever we consider
+        //a disk segment).
+        value.reset(diskIFileValue.getData(), diskIFileValue.getLength());
+      }
+      long startPos = minSegment.getPosition();
+      key = minSegment.getKey();
+      minSegment.getValue(value);
+      long endPos = minSegment.getPosition();
+      totalBytesProcessed += endPos - startPos;
+      mergeProgress.set(totalBytesProcessed * progPerByte);
+      return true;
+    }
+
+    protected boolean lessThan(Object a, Object b) {
+      DataInputBuffer key1 = ((Segment)a).getKey();
+      DataInputBuffer key2 = ((Segment)b).getKey();
+      int s1 = key1.getPosition();
+      int l1 = key1.getLength() - s1;
+      int s2 = key2.getPosition();
+      int l2 = key2.getLength() - s2;
+
+      return comparator.compare(key1.getData(), s1, l1, key2.getData(), s2, l2) < 0;
+    }
+    
+    public TezRawKeyValueIterator merge(Class keyClass, Class valueClass,
+                                     int factor, Path tmpDir,
+                                     TezCounter readsCounter,
+                                     TezCounter writesCounter,
+                                     Progress mergePhase)
+        throws IOException {
+      return merge(keyClass, valueClass, factor, 0, tmpDir,
+                   readsCounter, writesCounter, mergePhase);
+    }
+
+    TezRawKeyValueIterator merge(Class keyClass, Class valueClass,
+                                     int factor, int inMem, Path tmpDir,
+                                     TezCounter readsCounter,
+                                     TezCounter writesCounter,
+                                     Progress mergePhase)
+        throws IOException {
+      LOG.info("Merging " + segments.size() + " sorted segments");
+
+      /*
+       * If there are inMemory segments, then they come first in the segments
+       * list and then the sorted disk segments. Otherwise(if there are only
+       * disk segments), then they are sorted segments if there are more than
+       * factor segments in the segments list.
+       */
+      int numSegments = segments.size();
+      int origFactor = factor;
+      int passNo = 1;
+      if (mergePhase != null) {
+        mergeProgress = mergePhase;
+      }
+
+      long totalBytes = computeBytesInMerges(factor, inMem);
+      if (totalBytes != 0) {
+        progPerByte = 1.0f / (float)totalBytes;
+      }
+      
+      //create the MergeStreams from the sorted map created in the constructor
+      //and dump the final output to a file
+      do {
+        //get the factor for this pass of merge. We assume in-memory segments
+        //are the first entries in the segment list and that the pass factor
+        //doesn't apply to them
+        factor = getPassFactor(factor, passNo, numSegments - inMem);
+        if (1 == passNo) {
+          factor += inMem;
+        }
+        List<Segment> segmentsToMerge =
+          new ArrayList<Segment>();
+        int segmentsConsidered = 0;
+        int numSegmentsToConsider = factor;
+        long startBytes = 0; // starting bytes of segments of this merge
+        while (true) {
+          //extract the smallest 'factor' number of segments  
+          //Call cleanup on the empty segments (no key/value data)
+          List<Segment> mStream = 
+            getSegmentDescriptors(numSegmentsToConsider);
+          for (Segment segment : mStream) {
+            // Initialize the segment at the last possible moment;
+            // this helps in ensuring we don't use buffers until we need them
+            segment.init(readsCounter);
+            long startPos = segment.getPosition();
+            boolean hasNext = segment.nextRawKey();
+            long endPos = segment.getPosition();
+            
+            if (hasNext) {
+              startBytes += endPos - startPos;
+              segmentsToMerge.add(segment);
+              segmentsConsidered++;
+            }
+            else {
+              segment.close();
+              numSegments--; //we ignore this segment for the merge
+            }
+          }
+          //if we have the desired number of segments
+          //or looked at all available segments, we break
+          if (segmentsConsidered == factor || 
+              segments.size() == 0) {
+            break;
+          }
+            
+          numSegmentsToConsider = factor - segmentsConsidered;
+        }
+        
+        //feed the streams to the priority queue
+        initialize(segmentsToMerge.size());
+        clear();
+        for (Segment segment : segmentsToMerge) {
+          put(segment);
+        }
+        
+        //if we have lesser number of segments remaining, then just return the
+        //iterator, else do another single level merge
+        if (numSegments <= factor) {
+          if (!includeFinalMerge) { // for reduce task
+
+            // Reset totalBytesProcessed and recalculate totalBytes from the
+            // remaining segments to track the progress of the final merge.
+            // Final merge is considered as the progress of the reducePhase,
+            // the 3rd phase of reduce task.
+            totalBytesProcessed = 0;
+            totalBytes = 0;
+            for (int i = 0; i < segmentsToMerge.size(); i++) {
+              totalBytes += segmentsToMerge.get(i).getLength();
+            }
+          }
+          if (totalBytes != 0) //being paranoid
+            progPerByte = 1.0f / (float)totalBytes;
+          
+          totalBytesProcessed += startBytes;         
+          if (totalBytes != 0)
+            mergeProgress.set(totalBytesProcessed * progPerByte);
+          else
+            mergeProgress.set(1.0f); // Last pass and no segments left - we're done
+          
+          LOG.info("Down to the last merge-pass, with " + numSegments + 
+                   " segments left of total size: " +
+                   (totalBytes - totalBytesProcessed) + " bytes");
+          return this;
+        } else {
+          LOG.info("Merging " + segmentsToMerge.size() + 
+                   " intermediate segments out of a total of " + 
+                   (segments.size()+segmentsToMerge.size()));
+          
+          long bytesProcessedInPrevMerges = totalBytesProcessed;
+          totalBytesProcessed += startBytes;
+
+          //we want to spread the creation of temp files on multiple disks if 
+          //available under the space constraints
+          long approxOutputSize = 0; 
+          for (Segment s : segmentsToMerge) {
+            approxOutputSize += s.getLength() + 
+                                ChecksumFileSystem.getApproxChkSumLength(
+                                s.getLength());
+          }
+          Path tmpFilename = 
+            new Path(tmpDir, "intermediate").suffix("." + passNo);
+
+          Path outputFile =  lDirAlloc.getLocalPathForWrite(
+                                              tmpFilename.toString(),
+                                              approxOutputSize, conf);
+
+          Writer writer = 
+            new Writer(conf, fs, outputFile, keyClass, valueClass, codec,
+                             writesCounter);
+          writeFile(this, writer, reporter, conf);
+          writer.close();
+          
+          //we finished one single level merge; now clean up the priority 
+          //queue
+          this.close();
+
+          // Add the newly create segment to the list of segments to be merged
+          Segment tempSegment = 
+            new Segment(conf, fs, outputFile, codec, false);
+
+          // Insert new merged segment into the sorted list
+          int pos = Collections.binarySearch(segments, tempSegment,
+                                             segmentComparator);
+          if (pos < 0) {
+            // binary search failed. So position to be inserted at is -pos-1
+            pos = -pos-1;
+          }
+          segments.add(pos, tempSegment);
+          numSegments = segments.size();
+          
+          // Subtract the difference between expected size of new segment and 
+          // actual size of new segment(Expected size of new segment is
+          // inputBytesOfThisMerge) from totalBytes. Expected size and actual
+          // size will match(almost) if combiner is not called in merge.
+          long inputBytesOfThisMerge = totalBytesProcessed -
+                                       bytesProcessedInPrevMerges;
+          totalBytes -= inputBytesOfThisMerge - tempSegment.getLength();
+          if (totalBytes != 0) {
+            progPerByte = 1.0f / (float)totalBytes;
+          }
+          
+          passNo++;
+        }
+        //we are worried about only the first pass merge factor. So reset the 
+        //factor to what it originally was
+        factor = origFactor;
+      } while(true);
+    }
+    
+    /**
+     * Determine the number of segments to merge in a given pass. Assuming more
+     * than factor segments, the first pass should attempt to bring the total
+     * number of segments - 1 to be divisible by the factor - 1 (each pass
+     * takes X segments and produces 1) to minimize the number of merges.
+     */
+    private int getPassFactor(int factor, int passNo, int numSegments) {
+      if (passNo > 1 || numSegments <= factor || factor == 1) 
+        return factor;
+      int mod = (numSegments - 1) % (factor - 1);
+      if (mod == 0)
+        return factor;
+      return mod + 1;
+    }
+    
+    /** Return (& remove) the requested number of segment descriptors from the
+     * sorted map.
+     */
+    private List<Segment> getSegmentDescriptors(int numDescriptors) {
+      if (numDescriptors > segments.size()) {
+        List<Segment> subList = new ArrayList<Segment>(segments);
+        segments.clear();
+        return subList;
+      }
+      
+      List<Segment> subList = 
+        new ArrayList<Segment>(segments.subList(0, numDescriptors));
+      for (int i=0; i < numDescriptors; ++i) {
+        segments.remove(0);
+      }
+      return subList;
+    }
+    
+    /**
+     * Compute expected size of input bytes to merges, will be used in
+     * calculating mergeProgress. This simulates the above merge() method and
+     * tries to obtain the number of bytes that are going to be merged in all
+     * merges(assuming that there is no combiner called while merging).
+     * @param factor mapreduce.task.io.sort.factor
+     * @param inMem  number of segments in memory to be merged
+     */
+    long computeBytesInMerges(int factor, int inMem) {
+      int numSegments = segments.size();
+      List<Long> segmentSizes = new ArrayList<Long>(numSegments);
+      long totalBytes = 0;
+      int n = numSegments - inMem;
+      // factor for 1st pass
+      int f = getPassFactor(factor, 1, n) + inMem;
+      n = numSegments;
+ 
+      for (int i = 0; i < numSegments; i++) {
+        // Not handling empty segments here assuming that it would not affect
+        // much in calculation of mergeProgress.
+        segmentSizes.add(segments.get(i).getLength());
+      }
+      
+      // If includeFinalMerge is true, allow the following while loop iterate
+      // for 1 more iteration. This is to include final merge as part of the
+      // computation of expected input bytes of merges
+      boolean considerFinalMerge = includeFinalMerge;
+      
+      while (n > f || considerFinalMerge) {
+        if (n <=f ) {
+          considerFinalMerge = false;
+        }
+        long mergedSize = 0;
+        f = Math.min(f, segmentSizes.size());
+        for (int j = 0; j < f; j++) {
+          mergedSize += segmentSizes.remove(0);
+        }
+        totalBytes += mergedSize;
+        
+        // insert new size into the sorted list
+        int pos = Collections.binarySearch(segmentSizes, mergedSize);
+        if (pos < 0) {
+          pos = -pos-1;
+        }
+        segmentSizes.add(pos, mergedSize);
+        
+        n -= (f-1);
+        f = factor;
+      }
+
+      return totalBytes;
+    }
+
+    public Progress getProgress() {
+      return mergeProgress;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.java
new file mode 100644
index 0000000..3a2c2bf
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezRawKeyValueIterator.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.runtime.library.common.sort.impl;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.util.Progress;
+
+/**
+ * <code>TezRawKeyValueIterator</code> is an iterator used to iterate over
+ * the raw keys and values during sort/merge of intermediate data. 
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface TezRawKeyValueIterator {
+  /** 
+   * Gets the current raw key.
+   * 
+   * @return Gets the current raw key as a DataInputBuffer
+   * @throws IOException
+   */
+  DataInputBuffer getKey() throws IOException;
+  
+  /** 
+   * Gets the current raw value.
+   * 
+   * @return Gets the current raw value as a DataInputBuffer 
+   * @throws IOException
+   */
+  DataInputBuffer getValue() throws IOException;
+  
+  /** 
+   * Sets up the current key and value (for getKey and getValue).
+   * 
+   * @return <code>true</code> if there exists a key/value, 
+   *         <code>false</code> otherwise. 
+   * @throws IOException
+   */
+  boolean next() throws IOException;
+  
+  /** 
+   * Closes the iterator so that the underlying streams can be closed.
+   * 
+   * @throws IOException
+   */
+  void close() throws IOException;
+  
+  /** Gets the Progress object; this has a float (0.0 - 1.0) 
+   * indicating the bytes processed by the iterator so far
+   */
+  Progress getProgress();
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.java
new file mode 100644
index 0000000..ab4142b
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/TezSpillRecord.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.runtime.library.common.sort.impl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.LongBuffer;
+import java.util.zip.CheckedInputStream;
+import java.util.zip.CheckedOutputStream;
+import java.util.zip.Checksum;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.PureJavaCrc32;
+import org.apache.tez.runtime.library.common.Constants;
+
+public class TezSpillRecord {
+
+  /** Backing store */
+  private final ByteBuffer buf;
+  /** View of backing storage as longs */
+  private final LongBuffer entries;
+
+  public TezSpillRecord(int numPartitions) {
+    buf = ByteBuffer.allocate(
+        numPartitions * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH);
+    entries = buf.asLongBuffer();
+  }
+
+  public TezSpillRecord(Path indexFileName, Configuration job) throws IOException {
+    this(indexFileName, job, null);
+  }
+
+  public TezSpillRecord(Path indexFileName, Configuration job, String expectedIndexOwner)
+    throws IOException {
+    this(indexFileName, job, new PureJavaCrc32(), expectedIndexOwner);
+  }
+
+  public TezSpillRecord(Path indexFileName, Configuration job, Checksum crc,
+                     String expectedIndexOwner)
+      throws IOException {
+
+    final FileSystem rfs = FileSystem.getLocal(job).getRaw();
+    final FSDataInputStream in = rfs.open(indexFileName);
+    try {
+      final long length = rfs.getFileStatus(indexFileName).getLen();
+      final int partitions = 
+          (int) length / Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH;
+      final int size = partitions * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH;
+
+      buf = ByteBuffer.allocate(size);
+      if (crc != null) {
+        crc.reset();
+        CheckedInputStream chk = new CheckedInputStream(in, crc);
+        IOUtils.readFully(chk, buf.array(), 0, size);
+        if (chk.getChecksum().getValue() != in.readLong()) {
+          throw new ChecksumException("Checksum error reading spill index: " +
+                                indexFileName, -1);
+        }
+      } else {
+        IOUtils.readFully(in, buf.array(), 0, size);
+      }
+      entries = buf.asLongBuffer();
+    } finally {
+      in.close();
+    }
+  }
+
+  /**
+   * Return number of IndexRecord entries in this spill.
+   */
+  public int size() {
+    return entries.capacity() / (Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8);
+  }
+
+  /**
+   * Get spill offsets for given partition.
+   */
+  public TezIndexRecord getIndex(int partition) {
+    final int pos = partition * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8;
+    return new TezIndexRecord(entries.get(pos), entries.get(pos + 1),
+                           entries.get(pos + 2));
+  }
+
+  /**
+   * Set spill offsets for given partition.
+   */
+  public void putIndex(TezIndexRecord rec, int partition) {
+    final int pos = partition * Constants.MAP_OUTPUT_INDEX_RECORD_LENGTH / 8;
+    entries.put(pos, rec.getStartOffset());
+    entries.put(pos + 1, rec.getRawLength());
+    entries.put(pos + 2, rec.getPartLength());
+  }
+
+  /**
+   * Write this spill record to the location provided.
+   */
+  public void writeToFile(Path loc, Configuration job)
+      throws IOException {
+    writeToFile(loc, job, new PureJavaCrc32());
+  }
+
+  public void writeToFile(Path loc, Configuration job, Checksum crc)
+      throws IOException {
+    final FileSystem rfs = FileSystem.getLocal(job).getRaw();
+    CheckedOutputStream chk = null;
+    final FSDataOutputStream out = rfs.create(loc);
+    try {
+      if (crc != null) {
+        crc.reset();
+        chk = new CheckedOutputStream(out, crc);
+        chk.write(buf.array());
+        out.writeLong(chk.getChecksum().getValue());
+      } else {
+        out.write(buf.array());
+      }
+    } finally {
+      if (chk != null) {
+        chk.close();
+      } else {
+        out.close();
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
new file mode 100644
index 0000000..1ff486f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/DefaultSorter.java
@@ -0,0 +1,1108 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.common.sort.impl.dflt;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.IntBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+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.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+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.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.sort.impl.ExternalSorter;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.TezIndexRecord;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.TezSpillRecord;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger.Segment;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public class DefaultSorter extends ExternalSorter implements IndexedSortable {
+  
+  private static final Log LOG = LogFactory.getLog(DefaultSorter.class);
+
+  // TODO NEWTEZ Progress reporting to Tez framework. (making progress vs %complete)
+  
+  /**
+   * The size of each record in the index file for the map-outputs.
+   */
+  public static final int MAP_OUTPUT_INDEX_RECORD_LENGTH = 24;
+
+  private final static int APPROX_HEADER_LENGTH = 150;
+
+  // k/v accounting
+  IntBuffer kvmeta; // metadata overlay on backing store
+  int kvstart;            // marks origin of spill metadata
+  int kvend;              // marks end of spill metadata
+  int kvindex;            // marks end of fully serialized records
+
+  int equator;            // marks origin of meta/serialization
+  int bufstart;           // marks beginning of spill
+  int bufend;             // marks beginning of collectable
+  int bufmark;            // marks end of record
+  int bufindex;           // marks end of collected
+  int bufvoid;            // marks the point where we should stop
+                          // reading at the end of the buffer
+
+  byte[] kvbuffer;        // main output buffer
+  private final byte[] b0 = new byte[0];
+
+  protected static final int INDEX = 0;            // index offset in acct
+  protected static final int VALSTART = 1;         // val offset in acct
+  protected static final int KEYSTART = 2;         // key offset in acct
+  protected static final int PARTITION = 3;        // partition offset in acct
+  protected static final int NMETA = 4;            // num meta ints
+  protected static final int METASIZE = NMETA * 4; // size in bytes
+
+  // spill accounting
+  int maxRec;
+  int softLimit;
+  boolean spillInProgress;
+  int bufferRemaining;
+  volatile Throwable sortSpillException = null;
+
+  int numSpills = 0;
+  int minSpillsForCombine;
+  final ReentrantLock spillLock = new ReentrantLock();
+  final Condition spillDone = spillLock.newCondition();
+  final Condition spillReady = spillLock.newCondition();
+  final BlockingBuffer bb = new BlockingBuffer();
+  volatile boolean spillThreadRunning = false;
+  final SpillThread spillThread = new SpillThread();
+
+  final ArrayList<TezSpillRecord> indexCacheList =
+    new ArrayList<TezSpillRecord>();
+  private int totalIndexCacheMemory;
+  private int indexCacheMemoryLimit;
+
+  @Override
+  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException { 
+    super.initialize(outputContext, conf, numOutputs);
+
+    // sanity checks
+    final float spillper = this.conf.getFloat(
+        TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SORT_SPILL_PERCENT);
+    final int sortmb = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_MB,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_MB);
+    if (spillper > (float) 1.0 || spillper <= (float) 0.0) {
+      throw new IOException("Invalid \""
+          + TezJobConfig.TEZ_RUNTIME_SORT_SPILL_PERCENT + "\": " + spillper);
+    }
+    if ((sortmb & 0x7FF) != sortmb) {
+      throw new IOException("Invalid \"" + TezJobConfig.TEZ_RUNTIME_IO_SORT_MB
+          + "\": " + sortmb);
+    }
+
+    indexCacheMemoryLimit = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES,
+                                       TezJobConfig.DEFAULT_TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES);
+
+    // buffers and accounting
+    int maxMemUsage = sortmb << 20;
+    maxMemUsage -= maxMemUsage % METASIZE;
+    kvbuffer = new byte[maxMemUsage];
+    bufvoid = kvbuffer.length;
+    kvmeta = ByteBuffer.wrap(kvbuffer)
+       .order(ByteOrder.nativeOrder())
+       .asIntBuffer();
+    setEquator(0);
+    bufstart = bufend = bufindex = equator;
+    kvstart = kvend = kvindex;
+
+    maxRec = kvmeta.capacity() / NMETA;
+    softLimit = (int)(kvbuffer.length * spillper);
+    bufferRemaining = softLimit;
+    if (LOG.isInfoEnabled()) {
+      LOG.info(TezJobConfig.TEZ_RUNTIME_IO_SORT_MB + ": " + sortmb);
+      LOG.info("soft limit at " + softLimit);
+      LOG.info("bufstart = " + bufstart + "; bufvoid = " + bufvoid);
+      LOG.info("kvstart = " + kvstart + "; length = " + maxRec);
+    }
+
+    // k/v serialization
+    valSerializer.open(bb);
+    keySerializer.open(bb);
+
+    spillInProgress = false;
+    minSpillsForCombine = this.conf.getInt(TezJobConfig.TEZ_RUNTIME_COMBINE_MIN_SPILLS, 3);
+    spillThread.setDaemon(true);
+    spillThread.setName("SpillThread");
+    spillLock.lock();
+    try {
+      spillThread.start();
+      while (!spillThreadRunning) {
+        spillDone.await();
+      }
+    } catch (InterruptedException e) {
+      throw new IOException("Spill thread failed to initialize", e);
+    } finally {
+      spillLock.unlock();
+    }
+    if (sortSpillException != null) {
+      throw new IOException("Spill thread failed to initialize",
+          sortSpillException);
+    }
+  }
+
+  @Override
+  public void write(Object key, Object value)
+      throws IOException {
+    collect(
+        key, value, partitioner.getPartition(key, value, partitions));
+  }
+
+  /**
+   * Serialize the key, value to intermediate storage.
+   * When this method returns, kvindex must refer to sufficient unused
+   * storage to store one METADATA.
+   */
+  synchronized void collect(Object key, Object value, final int partition
+                                   ) throws IOException {
+
+    if (key.getClass() != keyClass) {
+      throw new IOException("Type mismatch in key from map: expected "
+                            + keyClass.getName() + ", received "
+                            + key.getClass().getName());
+    }
+    if (value.getClass() != valClass) {
+      throw new IOException("Type mismatch in value from map: expected "
+                            + valClass.getName() + ", received "
+                            + value.getClass().getName());
+    }
+    if (partition < 0 || partition >= partitions) {
+      throw new IOException("Illegal partition for " + key + " (" +
+          partition + ")" + ", TotalPartitions: " + partitions);
+    }
+    checkSpillException();
+    bufferRemaining -= METASIZE;
+    if (bufferRemaining <= 0) {
+      // start spill if the thread is not running and the soft limit has been
+      // reached
+      spillLock.lock();
+      try {
+        do {
+          if (!spillInProgress) {
+            final int kvbidx = 4 * kvindex;
+            final int kvbend = 4 * kvend;
+            // serialized, unspilled bytes always lie between kvindex and
+            // bufindex, crossing the equator. Note that any void space
+            // created by a reset must be included in "used" bytes
+            final int bUsed = distanceTo(kvbidx, bufindex);
+            final boolean bufsoftlimit = bUsed >= softLimit;
+            if ((kvbend + METASIZE) % kvbuffer.length !=
+                equator - (equator % METASIZE)) {
+              // spill finished, reclaim space
+              resetSpill();
+              bufferRemaining = Math.min(
+                  distanceTo(bufindex, kvbidx) - 2 * METASIZE,
+                  softLimit - bUsed) - METASIZE;
+              continue;
+            } else if (bufsoftlimit && kvindex != kvend) {
+              // spill records, if any collected; check latter, as it may
+              // be possible for metadata alignment to hit spill pcnt
+              startSpill();
+              final int avgRec = (int)
+                (mapOutputByteCounter.getValue() /
+                mapOutputRecordCounter.getValue());
+              // leave at least half the split buffer for serialization data
+              // ensure that kvindex >= bufindex
+              final int distkvi = distanceTo(bufindex, kvbidx);
+              final int newPos = (bufindex +
+                Math.max(2 * METASIZE - 1,
+                        Math.min(distkvi / 2,
+                                 distkvi / (METASIZE + avgRec) * METASIZE)))
+                % kvbuffer.length;
+              setEquator(newPos);
+              bufmark = bufindex = newPos;
+              final int serBound = 4 * kvend;
+              // bytes remaining before the lock must be held and limits
+              // checked is the minimum of three arcs: the metadata space, the
+              // serialization space, and the soft limit
+              bufferRemaining = Math.min(
+                  // metadata max
+                  distanceTo(bufend, newPos),
+                  Math.min(
+                    // serialization max
+                    distanceTo(newPos, serBound),
+                    // soft limit
+                    softLimit)) - 2 * METASIZE;
+            }
+          }
+        } while (false);
+      } finally {
+        spillLock.unlock();
+      }
+    }
+
+    try {
+      // serialize key bytes into buffer
+      int keystart = bufindex;
+      keySerializer.serialize(key);
+      if (bufindex < keystart) {
+        // wrapped the key; must make contiguous
+        bb.shiftBufferedKey();
+        keystart = 0;
+      }
+      // serialize value bytes into buffer
+      final int valstart = bufindex;
+      valSerializer.serialize(value);
+      // It's possible for records to have zero length, i.e. the serializer
+      // will perform no writes. To ensure that the boundary conditions are
+      // checked and that the kvindex invariant is maintained, perform a
+      // zero-length write into the buffer. The logic monitoring this could be
+      // moved into collect, but this is cleaner and inexpensive. For now, it
+      // is acceptable.
+      bb.write(b0, 0, 0);
+
+      // the record must be marked after the preceding write, as the metadata
+      // for this record are not yet written
+      int valend = bb.markRecord();
+
+      mapOutputRecordCounter.increment(1);
+      mapOutputByteCounter.increment(
+          distanceTo(keystart, valend, bufvoid));
+
+      // write accounting info
+      kvmeta.put(kvindex + INDEX, kvindex);
+      kvmeta.put(kvindex + PARTITION, partition);
+      kvmeta.put(kvindex + KEYSTART, keystart);
+      kvmeta.put(kvindex + VALSTART, valstart);
+      // advance kvindex
+      kvindex = (kvindex - NMETA + kvmeta.capacity()) % kvmeta.capacity();
+    } catch (MapBufferTooSmallException e) {
+      LOG.info("Record too large for in-memory buffer: " + e.getMessage());
+      spillSingleRecord(key, value, partition);
+      mapOutputRecordCounter.increment(1);
+      return;
+    }
+  }
+
+  /**
+   * Set the point from which meta and serialization data expand. The meta
+   * indices are aligned with the buffer, so metadata never spans the ends of
+   * the circular buffer.
+   */
+  private void setEquator(int pos) {
+    equator = pos;
+    // set index prior to first entry, aligned at meta boundary
+    final int aligned = pos - (pos % METASIZE);
+    kvindex =
+      ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
+    if (LOG.isInfoEnabled()) {
+      LOG.info("(EQUATOR) " + pos + " kvi " + kvindex +
+          "(" + (kvindex * 4) + ")");
+    }
+  }
+
+  /**
+   * The spill is complete, so set the buffer and meta indices to be equal to
+   * the new equator to free space for continuing collection. Note that when
+   * kvindex == kvend == kvstart, the buffer is empty.
+   */
+  private void resetSpill() {
+    final int e = equator;
+    bufstart = bufend = e;
+    final int aligned = e - (e % METASIZE);
+    // set start/end to point to first meta record
+    kvstart = kvend =
+      ((aligned - METASIZE + kvbuffer.length) % kvbuffer.length) / 4;
+    if (LOG.isInfoEnabled()) {
+      LOG.info("(RESET) equator " + e + " kv " + kvstart + "(" +
+        (kvstart * 4) + ")" + " kvi " + kvindex + "(" + (kvindex * 4) + ")");
+    }
+  }
+
+  /**
+   * Compute the distance in bytes between two indices in the serialization
+   * buffer.
+   * @see #distanceTo(int,int,int)
+   */
+  final int distanceTo(final int i, final int j) {
+    return distanceTo(i, j, kvbuffer.length);
+  }
+
+  /**
+   * Compute the distance between two indices in the circular buffer given the
+   * max distance.
+   */
+  int distanceTo(final int i, final int j, final int mod) {
+    return i <= j
+      ? j - i
+      : mod - i + j;
+  }
+
+  /**
+   * For the given meta position, return the dereferenced position in the
+   * integer array. Each meta block contains several integers describing
+   * record data in its serialized form, but the INDEX is not necessarily
+   * related to the proximate metadata. The index value at the referenced int
+   * position is the start offset of the associated metadata block. So the
+   * metadata INDEX at metapos may point to the metadata described by the
+   * metadata block at metapos + k, which contains information about that
+   * serialized record.
+   */
+  int offsetFor(int metapos) {
+    return kvmeta.get((metapos % maxRec) * NMETA + INDEX);
+  }
+
+  /**
+   * Compare logical range, st i, j MOD offset capacity.
+   * Compare by partition, then by key.
+   * @see IndexedSortable#compare
+   */
+  public int compare(final int mi, final int mj) {
+    final int kvi = offsetFor(mi);
+    final int kvj = offsetFor(mj);
+    final int kvip = kvmeta.get(kvi + PARTITION);
+    final int kvjp = kvmeta.get(kvj + PARTITION);
+    // sort by partition
+    if (kvip != kvjp) {
+      return kvip - kvjp;
+    }
+    // sort by key
+    return comparator.compare(kvbuffer,
+        kvmeta.get(kvi + KEYSTART),
+        kvmeta.get(kvi + VALSTART) - kvmeta.get(kvi + KEYSTART),
+        kvbuffer,
+        kvmeta.get(kvj + KEYSTART),
+        kvmeta.get(kvj + VALSTART) - kvmeta.get(kvj + KEYSTART));
+  }
+
+  /**
+   * Swap logical indices st i, j MOD offset capacity.
+   * @see IndexedSortable#swap
+   */
+  public void swap(final int mi, final int mj) {
+    final int kvi = (mi % maxRec) * NMETA + INDEX;
+    final int kvj = (mj % maxRec) * NMETA + INDEX;
+    int tmp = kvmeta.get(kvi);
+    kvmeta.put(kvi, kvmeta.get(kvj));
+    kvmeta.put(kvj, tmp);
+  }
+
+  /**
+   * Inner class managing the spill of serialized records to disk.
+   */
+  protected class BlockingBuffer extends DataOutputStream {
+
+    public BlockingBuffer() {
+      super(new Buffer());
+    }
+
+    /**
+     * Mark end of record. Note that this is required if the buffer is to
+     * cut the spill in the proper place.
+     */
+    public int markRecord() {
+      bufmark = bufindex;
+      return bufindex;
+    }
+
+    /**
+     * Set position from last mark to end of writable buffer, then rewrite
+     * the data between last mark and kvindex.
+     * This handles a special case where the key wraps around the buffer.
+     * If the key is to be passed to a RawComparator, then it must be
+     * contiguous in the buffer. This recopies the data in the buffer back
+     * into itself, but starting at the beginning of the buffer. Note that
+     * this method should <b>only</b> be called immediately after detecting
+     * this condition. To call it at any other time is undefined and would
+     * likely result in data loss or corruption.
+     * @see #markRecord()
+     */
+    protected void shiftBufferedKey() throws IOException {
+      // spillLock unnecessary; both kvend and kvindex are current
+      int headbytelen = bufvoid - bufmark;
+      bufvoid = bufmark;
+      final int kvbidx = 4 * kvindex;
+      final int kvbend = 4 * kvend;
+      final int avail =
+        Math.min(distanceTo(0, kvbidx), distanceTo(0, kvbend));
+      if (bufindex + headbytelen < avail) {
+        System.arraycopy(kvbuffer, 0, kvbuffer, headbytelen, bufindex);
+        System.arraycopy(kvbuffer, bufvoid, kvbuffer, 0, headbytelen);
+        bufindex += headbytelen;
+        bufferRemaining -= kvbuffer.length - bufvoid;
+      } else {
+        byte[] keytmp = new byte[bufindex];
+        System.arraycopy(kvbuffer, 0, keytmp, 0, bufindex);
+        bufindex = 0;
+        out.write(kvbuffer, bufmark, headbytelen);
+        out.write(keytmp);
+      }
+    }
+  }
+
+  public class Buffer extends OutputStream {
+    private final byte[] scratch = new byte[1];
+
+    @Override
+    public void write(int v)
+        throws IOException {
+      scratch[0] = (byte)v;
+      write(scratch, 0, 1);
+    }
+
+    /**
+     * Attempt to write a sequence of bytes to the collection buffer.
+     * This method will block if the spill thread is running and it
+     * cannot write.
+     * @throws MapBufferTooSmallException if record is too large to
+     *    deserialize into the collection buffer.
+     */
+    @Override
+    public void write(byte b[], int off, int len)
+        throws IOException {
+      // must always verify the invariant that at least METASIZE bytes are
+      // available beyond kvindex, even when len == 0
+      bufferRemaining -= len;
+      if (bufferRemaining <= 0) {
+        // writing these bytes could exhaust available buffer space or fill
+        // the buffer to soft limit. check if spill or blocking are necessary
+        boolean blockwrite = false;
+        spillLock.lock();
+        try {
+          do {
+            checkSpillException();
+
+            final int kvbidx = 4 * kvindex;
+            final int kvbend = 4 * kvend;
+            // ser distance to key index
+            final int distkvi = distanceTo(bufindex, kvbidx);
+            // ser distance to spill end index
+            final int distkve = distanceTo(bufindex, kvbend);
+
+            // if kvindex is closer than kvend, then a spill is neither in
+            // progress nor complete and reset since the lock was held. The
+            // write should block only if there is insufficient space to
+            // complete the current write, write the metadata for this record,
+            // and write the metadata for the next record. If kvend is closer,
+            // then the write should block if there is too little space for
+            // either the metadata or the current write. Note that collect
+            // ensures its metadata requirement with a zero-length write
+            blockwrite = distkvi <= distkve
+              ? distkvi <= len + 2 * METASIZE
+              : distkve <= len || distanceTo(bufend, kvbidx) < 2 * METASIZE;
+
+            if (!spillInProgress) {
+              if (blockwrite) {
+                if ((kvbend + METASIZE) % kvbuffer.length !=
+                    equator - (equator % METASIZE)) {
+                  // spill finished, reclaim space
+                  // need to use meta exclusively; zero-len rec & 100% spill
+                  // pcnt would fail
+                  resetSpill(); // resetSpill doesn't move bufindex, kvindex
+                  bufferRemaining = Math.min(
+                      distkvi - 2 * METASIZE,
+                      softLimit - distanceTo(kvbidx, bufindex)) - len;
+                  continue;
+                }
+                // we have records we can spill; only spill if blocked
+                if (kvindex != kvend) {
+                  startSpill();
+                  // Blocked on this write, waiting for the spill just
+                  // initiated to finish. Instead of repositioning the marker
+                  // and copying the partial record, we set the record start
+                  // to be the new equator
+                  setEquator(bufmark);
+                } else {
+                  // We have no buffered records, and this record is too large
+                  // to write into kvbuffer. We must spill it directly from
+                  // collect
+                  final int size = distanceTo(bufstart, bufindex) + len;
+                  setEquator(0);
+                  bufstart = bufend = bufindex = equator;
+                  kvstart = kvend = kvindex;
+                  bufvoid = kvbuffer.length;
+                  throw new MapBufferTooSmallException(size + " bytes");
+                }
+              }
+            }
+
+            if (blockwrite) {
+              // wait for spill
+              try {
+                while (spillInProgress) {
+                  spillDone.await();
+                }
+              } catch (InterruptedException e) {
+                  throw new IOException(
+                      "Buffer interrupted while waiting for the writer", e);
+              }
+            }
+          } while (blockwrite);
+        } finally {
+          spillLock.unlock();
+        }
+      }
+      // here, we know that we have sufficient space to write
+      if (bufindex + len > bufvoid) {
+        final int gaplen = bufvoid - bufindex;
+        System.arraycopy(b, off, kvbuffer, bufindex, gaplen);
+        len -= gaplen;
+        off += gaplen;
+        bufindex = 0;
+      }
+      System.arraycopy(b, off, kvbuffer, bufindex, len);
+      bufindex += len;
+    }
+  }
+
+  @Override
+  public void flush() throws IOException {
+    LOG.info("Starting flush of map output");
+    spillLock.lock();
+    try {
+      while (spillInProgress) {
+        spillDone.await();
+      }
+      checkSpillException();
+
+      final int kvbend = 4 * kvend;
+      if ((kvbend + METASIZE) % kvbuffer.length !=
+          equator - (equator % METASIZE)) {
+        // spill finished
+        resetSpill();
+      }
+      if (kvindex != kvend) {
+        kvend = (kvindex + NMETA) % kvmeta.capacity();
+        bufend = bufmark;
+        if (LOG.isInfoEnabled()) {
+          LOG.info("Sorting & Spilling map output");
+          LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
+                   "; bufvoid = " + bufvoid);
+          LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
+                   "); kvend = " + kvend + "(" + (kvend * 4) +
+                   "); length = " + (distanceTo(kvend, kvstart,
+                         kvmeta.capacity()) + 1) + "/" + maxRec);
+        }
+        sortAndSpill();
+      }
+    } catch (InterruptedException e) {
+      throw new IOException("Interrupted while waiting for the writer", e);
+    } finally {
+      spillLock.unlock();
+    }
+    assert !spillLock.isHeldByCurrentThread();
+    // shut down spill thread and wait for it to exit. Since the preceding
+    // ensures that it is finished with its work (and sortAndSpill did not
+    // throw), we elect to use an interrupt instead of setting a flag.
+    // Spilling simultaneously from this thread while the spill thread
+    // finishes its work might be both a useful way to extend this and also
+    // sufficient motivation for the latter approach.
+    try {
+      spillThread.interrupt();
+      spillThread.join();
+    } catch (InterruptedException e) {
+      throw new IOException("Spill failed", e);
+    }
+    // release sort buffer before the merge
+    //FIXME
+    //kvbuffer = null;
+    mergeParts();
+    Path outputPath = mapOutputFile.getOutputFile();
+    fileOutputByteCounter.increment(rfs.getFileStatus(outputPath).getLen());
+  }
+
+  @Override
+  public void close() throws IOException { }
+
+  protected class SpillThread extends Thread {
+
+    @Override
+    public void run() {
+      spillLock.lock();
+      spillThreadRunning = true;
+      try {
+        while (true) {
+          spillDone.signal();
+          while (!spillInProgress) {
+            spillReady.await();
+          }
+          try {
+            spillLock.unlock();
+            sortAndSpill();
+          } catch (Throwable t) {
+            LOG.warn("Got an exception in sortAndSpill", t);
+            sortSpillException = t;
+          } finally {
+            spillLock.lock();
+            if (bufend < bufstart) {
+              bufvoid = kvbuffer.length;
+            }
+            kvstart = kvend;
+            bufstart = bufend;
+            spillInProgress = false;
+          }
+        }
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      } finally {
+        spillLock.unlock();
+        spillThreadRunning = false;
+      }
+    }
+  }
+
+  private void checkSpillException() throws IOException {
+    final Throwable lspillException = sortSpillException;
+    if (lspillException != null) {
+      if (lspillException instanceof Error) {
+        final String logMsg = "Task " + outputContext.getUniqueIdentifier()
+            + " failed : " + StringUtils.stringifyException(lspillException);
+        outputContext.fatalError(lspillException, logMsg);
+      }
+      throw new IOException("Spill failed", lspillException);
+    }
+  }
+
+  private void startSpill() {
+    assert !spillInProgress;
+    kvend = (kvindex + NMETA) % kvmeta.capacity();
+    bufend = bufmark;
+    spillInProgress = true;
+    if (LOG.isInfoEnabled()) {
+      LOG.info("Spilling map output");
+      LOG.info("bufstart = " + bufstart + "; bufend = " + bufmark +
+               "; bufvoid = " + bufvoid);
+      LOG.info("kvstart = " + kvstart + "(" + (kvstart * 4) +
+               "); kvend = " + kvend + "(" + (kvend * 4) +
+               "); length = " + (distanceTo(kvend, kvstart,
+                     kvmeta.capacity()) + 1) + "/" + maxRec);
+    }
+    spillReady.signal();
+  }
+
+  int getMetaStart() {
+    return kvend / NMETA;
+  }
+
+  int getMetaEnd() {
+    return 1 + // kvend is a valid record
+        (kvstart >= kvend
+        ? kvstart
+        : kvmeta.capacity() + kvstart) / NMETA;
+  }
+
+  protected void sortAndSpill()
+      throws IOException, InterruptedException {
+    final int mstart = getMetaStart();
+    final int mend = getMetaEnd();
+    sorter.sort(this, mstart, mend, nullProgressable);
+    spill(mstart, mend);
+  }
+
+  protected void spill(int mstart, int mend)
+      throws IOException, InterruptedException {
+
+    //approximate the length of the output file to be the length of the
+    //buffer + header lengths for the partitions
+    final long size = (bufend >= bufstart
+        ? bufend - bufstart
+        : (bufvoid - bufend) + bufstart) +
+                partitions * APPROX_HEADER_LENGTH;
+    FSDataOutputStream out = null;
+    try {
+      // create spill file
+      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
+      final Path filename =
+          mapOutputFile.getSpillFileForWrite(numSpills, size);
+      out = rfs.create(filename);
+
+      int spindex = mstart;
+      final InMemValBytes value = createInMemValBytes();
+      for (int i = 0; i < partitions; ++i) {
+        IFile.Writer writer = null;
+        try {
+          long segmentStart = out.getPos();
+          writer = new Writer(conf, out, keyClass, valClass, codec,
+                                    spilledRecordsCounter);
+          if (combiner == null) {
+            // spill directly
+            DataInputBuffer key = new DataInputBuffer();
+            while (spindex < mend &&
+                kvmeta.get(offsetFor(spindex) + PARTITION) == i) {
+              final int kvoff = offsetFor(spindex);
+              key.reset(
+                  kvbuffer,
+                  kvmeta.get(kvoff + KEYSTART),
+                  (kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART))
+                  );
+              getVBytesForOffset(kvoff, value);
+              writer.append(key, value);
+              ++spindex;
+            }
+          } else {
+            int spstart = spindex;
+            while (spindex < mend &&
+                kvmeta.get(offsetFor(spindex)
+                          + PARTITION) == i) {
+              ++spindex;
+            }
+            // Note: we would like to avoid the combiner if we've fewer
+            // than some threshold of records for a partition
+            if (spstart != spindex) {
+              TezRawKeyValueIterator kvIter =
+                new MRResultIterator(spstart, spindex);
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Running combine processor");
+              }
+              runCombineProcessor(kvIter, writer);
+            }
+          }
+
+          // close the writer
+          writer.close();
+
+          // record offsets
+          final TezIndexRecord rec =
+              new TezIndexRecord(
+                  segmentStart,
+                  writer.getRawLength(),
+                  writer.getCompressedLength());
+          spillRec.putIndex(rec, i);
+
+          writer = null;
+        } finally {
+          if (null != writer) writer.close();
+        }
+      }
+
+      if (totalIndexCacheMemory >= indexCacheMemoryLimit) {
+        // create spill index file
+        Path indexFilename =
+            mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
+                * MAP_OUTPUT_INDEX_RECORD_LENGTH);
+        spillRec.writeToFile(indexFilename, conf);
+      } else {
+        indexCacheList.add(spillRec);
+        totalIndexCacheMemory +=
+          spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH;
+      }
+      LOG.info("Finished spill " + numSpills);
+      ++numSpills;
+    } finally {
+      if (out != null) out.close();
+    }
+  }
+
+  /**
+   * Handles the degenerate case where serialization fails to fit in
+   * the in-memory buffer, so we must spill the record from collect
+   * directly to a spill file. Consider this "losing".
+   */
+  private void spillSingleRecord(final Object key, final Object value,
+                                 int partition) throws IOException {
+    long size = kvbuffer.length + partitions * APPROX_HEADER_LENGTH;
+    FSDataOutputStream out = null;
+    try {
+      // create spill file
+      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
+      final Path filename =
+          mapOutputFile.getSpillFileForWrite(numSpills, size);
+      out = rfs.create(filename);
+
+      // we don't run the combiner for a single record
+      for (int i = 0; i < partitions; ++i) {
+        IFile.Writer writer = null;
+        try {
+          long segmentStart = out.getPos();
+          // Create a new codec, don't care!
+          writer = new IFile.Writer(conf, out, keyClass, valClass, codec,
+                                          spilledRecordsCounter);
+
+          if (i == partition) {
+            final long recordStart = out.getPos();
+            writer.append(key, value);
+            // Note that our map byte count will not be accurate with
+            // compression
+            mapOutputByteCounter.increment(out.getPos() - recordStart);
+          }
+          writer.close();
+
+          // record offsets
+          TezIndexRecord rec =
+              new TezIndexRecord(
+                  segmentStart,
+                  writer.getRawLength(),
+                  writer.getCompressedLength());
+          spillRec.putIndex(rec, i);
+
+          writer = null;
+        } catch (IOException e) {
+          if (null != writer) writer.close();
+          throw e;
+        }
+      }
+      if (totalIndexCacheMemory >= indexCacheMemoryLimit) {
+        // create spill index file
+        Path indexFilename =
+            mapOutputFile.getSpillIndexFileForWrite(numSpills, partitions
+                * MAP_OUTPUT_INDEX_RECORD_LENGTH);
+        spillRec.writeToFile(indexFilename, conf);
+      } else {
+        indexCacheList.add(spillRec);
+        totalIndexCacheMemory +=
+          spillRec.size() * MAP_OUTPUT_INDEX_RECORD_LENGTH;
+      }
+      ++numSpills;
+    } finally {
+      if (out != null) out.close();
+    }
+  }
+
+  protected int getInMemVBytesLength(int kvoff) {
+    // get the keystart for the next serialized value to be the end
+    // of this value. If this is the last value in the buffer, use bufend
+    final int nextindex = kvoff == kvend
+      ? bufend
+      : kvmeta.get(
+          (kvoff - NMETA + kvmeta.capacity() + KEYSTART) % kvmeta.capacity());
+    // calculate the length of the value
+    int vallen = (nextindex >= kvmeta.get(kvoff + VALSTART))
+      ? nextindex - kvmeta.get(kvoff + VALSTART)
+      : (bufvoid - kvmeta.get(kvoff + VALSTART)) + nextindex;
+      return vallen;
+  }
+
+  /**
+   * Given an offset, populate vbytes with the associated set of
+   * deserialized value bytes. Should only be called during a spill.
+   */
+  int getVBytesForOffset(int kvoff, InMemValBytes vbytes) {
+    int vallen = getInMemVBytesLength(kvoff);
+    vbytes.reset(kvbuffer, kvmeta.get(kvoff + VALSTART), vallen);
+    return vallen;
+  }
+
+  /**
+   * Inner class wrapping valuebytes, used for appendRaw.
+   */
+  static class InMemValBytes extends DataInputBuffer {
+    private byte[] buffer;
+    private int start;
+    private int length;
+    private final int bufvoid;
+
+    public InMemValBytes(int bufvoid) {
+      this.bufvoid = bufvoid;
+    }
+
+    public void reset(byte[] buffer, int start, int length) {
+      this.buffer = buffer;
+      this.start = start;
+      this.length = length;
+
+      if (start + length > bufvoid) {
+        this.buffer = new byte[this.length];
+        final int taillen = bufvoid - start;
+        System.arraycopy(buffer, start, this.buffer, 0, taillen);
+        System.arraycopy(buffer, 0, this.buffer, taillen, length-taillen);
+        this.start = 0;
+      }
+
+      super.reset(this.buffer, this.start, this.length);
+    }
+  }
+
+  InMemValBytes createInMemValBytes() {
+    return new InMemValBytes(bufvoid);
+  }
+
+  protected class MRResultIterator implements TezRawKeyValueIterator {
+    private final DataInputBuffer keybuf = new DataInputBuffer();
+    private final InMemValBytes vbytes = createInMemValBytes();
+    private final int end;
+    private int current;
+    public MRResultIterator(int start, int end) {
+      this.end = end;
+      current = start - 1;
+    }
+    public boolean next() throws IOException {
+      return ++current < end;
+    }
+    public DataInputBuffer getKey() throws IOException {
+      final int kvoff = offsetFor(current);
+      keybuf.reset(kvbuffer, kvmeta.get(kvoff + KEYSTART),
+          kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART));
+      return keybuf;
+    }
+    public DataInputBuffer getValue() throws IOException {
+      getVBytesForOffset(offsetFor(current), vbytes);
+      return vbytes;
+    }
+    public Progress getProgress() {
+      return null;
+    }
+    public void close() { }
+  }
+
+  private void mergeParts() throws IOException {
+    // get the approximate size of the final output/index files
+    long finalOutFileSize = 0;
+    long finalIndexFileSize = 0;
+    final Path[] filename = new Path[numSpills];
+    final String taskIdentifier = outputContext.getUniqueIdentifier();
+
+    for(int i = 0; i < numSpills; i++) {
+      filename[i] = mapOutputFile.getSpillFile(i);
+      finalOutFileSize += rfs.getFileStatus(filename[i]).getLen();
+    }
+    if (numSpills == 1) { //the spill is the final output
+      sameVolRename(filename[0],
+          mapOutputFile.getOutputFileForWriteInVolume(filename[0]));
+      if (indexCacheList.size() == 0) {
+        sameVolRename(mapOutputFile.getSpillIndexFile(0),
+          mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]));
+      } else {
+        indexCacheList.get(0).writeToFile(
+          mapOutputFile.getOutputIndexFileForWriteInVolume(filename[0]), conf);
+      }
+      return;
+    }
+
+    // read in paged indices
+    for (int i = indexCacheList.size(); i < numSpills; ++i) {
+      Path indexFileName = mapOutputFile.getSpillIndexFile(i);
+      indexCacheList.add(new TezSpillRecord(indexFileName, conf));
+    }
+
+    //make correction in the length to include the sequence file header
+    //lengths for each partition
+    finalOutFileSize += partitions * APPROX_HEADER_LENGTH;
+    finalIndexFileSize = partitions * MAP_OUTPUT_INDEX_RECORD_LENGTH;
+    Path finalOutputFile =
+        mapOutputFile.getOutputFileForWrite(finalOutFileSize);
+    Path finalIndexFile =
+        mapOutputFile.getOutputIndexFileForWrite(finalIndexFileSize);
+
+    //The output stream for the final single output file
+    FSDataOutputStream finalOut = rfs.create(finalOutputFile, true, 4096);
+
+    if (numSpills == 0) {
+      //create dummy files
+
+      TezSpillRecord sr = new TezSpillRecord(partitions);
+      try {
+        for (int i = 0; i < partitions; i++) {
+          long segmentStart = finalOut.getPos();
+          Writer writer =
+            new Writer(conf, finalOut, keyClass, valClass, codec, null);
+          writer.close();
+
+          TezIndexRecord rec =
+              new TezIndexRecord(
+                  segmentStart,
+                  writer.getRawLength(),
+                  writer.getCompressedLength());
+          sr.putIndex(rec, i);
+        }
+        sr.writeToFile(finalIndexFile, conf);
+      } finally {
+        finalOut.close();
+      }
+      return;
+    }
+    else {
+      TezMerger.considerFinalMergeForProgress();
+
+      final TezSpillRecord spillRec = new TezSpillRecord(partitions);
+      for (int parts = 0; parts < partitions; parts++) {
+        //create the segments to be merged
+        List<Segment> segmentList =
+          new ArrayList<Segment>(numSpills);
+        for(int i = 0; i < numSpills; i++) {
+          TezIndexRecord indexRecord = indexCacheList.get(i).getIndex(parts);
+
+          Segment s =
+            new Segment(conf, rfs, filename[i], indexRecord.getStartOffset(),
+                             indexRecord.getPartLength(), codec, true);
+          segmentList.add(i, s);
+
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("TaskIdentifier=" + taskIdentifier + " Partition=" + parts +
+                "Spill =" + i + "(" + indexRecord.getStartOffset() + "," +
+                indexRecord.getRawLength() + ", " +
+                indexRecord.getPartLength() + ")");
+          }
+        }
+
+        int mergeFactor =
+            this.conf.getInt(TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR,
+                TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
+        // sort the segments only if there are intermediate merges
+        boolean sortSegments = segmentList.size() > mergeFactor;
+        //merge
+        TezRawKeyValueIterator kvIter = TezMerger.merge(conf, rfs,
+                       keyClass, valClass, codec,
+                       segmentList, mergeFactor,
+                       new Path(taskIdentifier),
+                       (RawComparator)ConfigUtils.getIntermediateOutputKeyComparator(conf),
+                       nullProgressable, sortSegments,
+                       null, spilledRecordsCounter,
+                       null); // Not using any Progress in TezMerger. Should just work.
+
+        //write merged output to disk
+        long segmentStart = finalOut.getPos();
+        Writer writer =
+            new Writer(conf, finalOut, keyClass, valClass, codec,
+                spilledRecordsCounter);
+        if (combiner == null || numSpills < minSpillsForCombine) {
+          TezMerger.writeFile(kvIter, writer,
+              nullProgressable, conf);
+        } else {
+          runCombineProcessor(kvIter, writer);
+        }
+        writer.close();
+
+        // record offsets
+        final TezIndexRecord rec =
+            new TezIndexRecord(
+                segmentStart,
+                writer.getRawLength(),
+                writer.getCompressedLength());
+        spillRec.putIndex(rec, parts);
+      }
+      spillRec.writeToFile(finalIndexFile, conf);
+      finalOut.close();
+      for(int i = 0; i < numSpills; i++) {
+        rfs.delete(filename[i],true);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/InMemoryShuffleSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/InMemoryShuffleSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/InMemoryShuffleSorter.java
new file mode 100644
index 0000000..92ae916
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/dflt/InMemoryShuffleSorter.java
@@ -0,0 +1,126 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.common.sort.impl.dflt;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.IntBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+
+public class InMemoryShuffleSorter extends DefaultSorter {
+
+  private static final Log LOG = LogFactory.getLog(InMemoryShuffleSorter.class);
+  
+  static final int IFILE_EOF_LENGTH = 
+      2 * WritableUtils.getVIntSize(IFile.EOF_MARKER);
+  static final int IFILE_CHECKSUM_LENGTH = DataChecksum.Type.CRC32.size;
+  
+  private List<Integer> spillIndices = new ArrayList<Integer>();
+  private List<ShuffleHeader> shuffleHeaders = new ArrayList<ShuffleHeader>();
+
+  ShuffleHandler shuffleHandler = new ShuffleHandler(this);
+  
+  byte[] kvbuffer;
+  IntBuffer kvmeta;
+
+  @Override
+  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
+    super.initialize(outputContext, conf, numOutputs);
+    shuffleHandler.initialize(outputContext, conf);
+  }
+
+  @Override
+  protected void spill(int mstart, int mend) 
+      throws IOException, InterruptedException {
+    // Start the shuffleHandler
+    shuffleHandler.start();
+
+    // Don't spill!
+    
+    // Make a copy
+    this.kvbuffer = super.kvbuffer;
+    this.kvmeta = super.kvmeta;
+
+    // Just save spill-indices for serving later
+    int spindex = mstart;
+    for (int i = 0; i < partitions; ++i) {
+      spillIndices.add(spindex);
+      
+      int length = 0;
+      while (spindex < mend &&
+          kvmeta.get(offsetFor(spindex) + PARTITION) == i) {
+
+        final int kvoff = offsetFor(spindex);
+        int keyLen = 
+            kvmeta.get(kvoff + VALSTART) - kvmeta.get(kvoff + KEYSTART);
+        int valLen = getInMemVBytesLength(kvoff);
+        length += 
+            (keyLen + WritableUtils.getVIntSize(keyLen)) + 
+            (valLen + WritableUtils.getVIntSize(valLen));
+
+        ++spindex;
+      }
+      length += IFILE_EOF_LENGTH;
+      
+      shuffleHeaders.add( 
+          new ShuffleHeader(
+              outputContext.getUniqueIdentifier(), // TODO Verify that this is correct. 
+              length + IFILE_CHECKSUM_LENGTH, length, i)
+          );
+      LOG.info("shuffleHeader[" + i + "]:" +
+      		" rawLen=" + length + " partLen=" + (length + IFILE_CHECKSUM_LENGTH) + 
+          " spillIndex=" + spillIndices.get(i));
+    }
+    
+    LOG.info("Saved " + spillIndices.size() + " spill-indices and " + 
+        shuffleHeaders.size() + " shuffle headers");
+  }
+
+  @Override
+  public InputStream getSortedStream(int partition) {
+    return new SortBufferInputStream(this, partition);
+  }
+
+  @Override
+  public void close() throws IOException {
+    // FIXME
+    //shuffleHandler.stop();
+  }
+
+  @Override
+  public ShuffleHeader getShuffleHeader(int reduce) {
+    return shuffleHeaders.get(reduce);
+  }
+
+  public int getSpillIndex(int partition) {
+    return spillIndices.get(partition);
+  }
+
+}


[18/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 7a4dd13..0000000
--- a/tez-dag/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/b212ca1d/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
deleted file mode 100644
index 13c9088..0000000
--- a/tez-dag/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/b212ca1d/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezDependentTaskCompletionEvent.java
new file mode 100644
index 0000000..fd4c1ee
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java b/tez-dag/src/main/java/org/apache/tez/runtime/records/TezTaskDependencyCompletionEventsUpdate.java
new file mode 100644
index 0000000..ff4f267
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
index 6e42673..31513c5 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
@@ -70,7 +70,7 @@ import org.apache.tez.dag.history.avro.HistoryEventType;
 import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.engine.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 676e747..f2717be 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
@@ -82,7 +82,7 @@ import org.apache.tez.dag.records.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.runtime.api.impl.TaskSpec;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
index 3d00cb7..b524f6a 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
@@ -82,8 +82,8 @@ import org.apache.tez.dag.records.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.records.TezDependentTaskCompletionEvent;
-import org.apache.tez.engine.records.TezDependentTaskCompletionEvent.Status;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent.Status;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
index 1a07b5b..81715bd 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexScheduler.java
@@ -40,8 +40,8 @@ import org.apache.tez.dag.records.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.records.TezDependentTaskCompletionEvent;
 import org.apache.tez.mapreduce.hadoop.MRHelpers;
+import org.apache.tez.runtime.records.TezDependentTaskCompletionEvent;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 3a6e008..68ee532 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/TestContainerReuse.java
@@ -56,9 +56,9 @@ import org.apache.tez.dag.records.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.InputSpec;
-import org.apache.tez.engine.api.impl.OutputSpec;
-import org.apache.tez.engine.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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 c2457e1..fc89e82 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
@@ -65,9 +65,9 @@ import org.apache.tez.dag.records.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.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.TokenCache;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-engine/findbugs-exclude.xml b/tez-engine/findbugs-exclude.xml
deleted file mode 100644
index 5b11308..0000000
--- a/tez-engine/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/b212ca1d/tez-engine/pom.xml
----------------------------------------------------------------------
diff --git a/tez-engine/pom.xml b/tez-engine/pom.xml
deleted file mode 100644
index 498f2f2..0000000
--- a/tez-engine/pom.xml
+++ /dev/null
@@ -1,92 +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</artifactId>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.tez</groupId>
-      <artifactId>tez-api</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-yarn-common</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>com.google.inject</groupId>
-      <artifactId>guice</artifactId>
-    </dependency>
-    <dependency>
-     <groupId>com.google.protobuf</groupId>
-     <artifactId>protobuf-java</artifactId>
-    </dependency>
-  </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>
-                  <include>ShufflePayloads.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/b212ca1d/tez-engine/src/main/java/org/apache/hadoop/io/BufferUtils.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/hadoop/io/BufferUtils.java b/tez-engine/src/main/java/org/apache/hadoop/io/BufferUtils.java
deleted file mode 100644
index 16f7a8f..0000000
--- a/tez-engine/src/main/java/org/apache/hadoop/io/BufferUtils.java
+++ /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.
- */
-
-package org.apache.hadoop.io;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
-@InterfaceStability.Unstable
-public class BufferUtils {
-  public static int compare(DataInputBuffer buf1, DataInputBuffer buf2) {
-    byte[] b1 = buf1.getData();
-    byte[] b2 = buf2.getData();
-    int s1 = buf1.getPosition();
-    int s2 = buf2.getPosition();
-    int l1 = buf1.getLength();
-    int l2 = buf2.getLength();
-    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
-  }
-
-  public static int compare(DataOutputBuffer buf1, DataOutputBuffer buf2) {
-    byte[] b1 = buf1.getData();
-    byte[] b2 = buf2.getData();
-    int s1 = 0;
-    int s2 = 0;
-    int l1 = buf1.getLength();
-    int l2 = buf2.getLength();
-    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
-  }
-
-  public static int compare(DataInputBuffer buf1, DataOutputBuffer buf2) {
-    byte[] b1 = buf1.getData();
-    byte[] b2 = buf2.getData();
-    int s1 = buf1.getPosition();    
-    int s2 = 0;
-    int l1 = buf1.getLength();
-    int l2 = buf2.getLength();
-    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
-  }
-
-  public static int compare(DataOutputBuffer buf1, DataInputBuffer buf2) {
-    return compare(buf2, buf1);
-  }
-
-  public static void copy(DataInputBuffer src, DataOutputBuffer dst) 
-                              throws IOException {
-    byte[] b1 = src.getData();
-    int s1 = src.getPosition();    
-    int l1 = src.getLength();
-    dst.reset();
-    dst.write(b1, s1, l1 - s1);
-  }
-
-  public static void copy(DataOutputBuffer src, DataOutputBuffer dst) 
-                              throws IOException {
-    byte[] b1 = src.getData();
-    int s1 = 0;
-    int l1 = src.getLength();
-    dst.reset();
-    dst.write(b1, s1, l1);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/hadoop/io/HashComparator.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/hadoop/io/HashComparator.java b/tez-engine/src/main/java/org/apache/hadoop/io/HashComparator.java
deleted file mode 100644
index a372e01..0000000
--- a/tez-engine/src/main/java/org/apache/hadoop/io/HashComparator.java
+++ /dev/null
@@ -1,24 +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.hadoop.io;
-
-public interface HashComparator<KEY> {
-
-  int getHashCode(KEY key);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 8ea2909..0000000
--- a/tez-engine/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/b212ca1d/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
deleted file mode 100644
index df92bdc..0000000
--- a/tez-engine/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/b212ca1d/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
deleted file mode 100644
index e90f7fa..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/ContainerTask.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.common;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-import org.apache.tez.engine.api.impl.TaskSpec;
-
-public class ContainerTask implements Writable {
-
-  TaskSpec taskSpec;
-  boolean shouldDie;
-
-  public ContainerTask() {
-  }
-
-  public ContainerTask(TaskSpec taskSpec, boolean shouldDie) {
-    this.taskSpec = taskSpec;
-    this.shouldDie = shouldDie;
-  }
-
-  public TaskSpec getTaskSpec() {
-    return taskSpec;
-  }
-
-  public boolean shouldDie() {
-    return shouldDie;
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeBoolean(shouldDie);
-    if (taskSpec != null) {
-      out.writeBoolean(true);
-      taskSpec.write(out);
-    } else {
-      out.writeBoolean(false);
-    }
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    shouldDie = in.readBoolean();
-    boolean taskComing = in.readBoolean();
-    if (taskComing) {
-      taskSpec = new TaskSpec();
-      taskSpec.readFields(in);
-    }
-  }
-
-  @Override
-  public String toString() {
-    return "shouldDie: " + shouldDie + ", TaskSpec: "
-        + taskSpec;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 9e4129f..0000000
--- a/tez-engine/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.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 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.impl.TezHeartbeatRequest;
-import org.apache.tez.engine.api.impl.TezHeartbeatResponse;
-
-/** Protocol that task child process uses to contact its parent process.  The
- * parent is a daemon which which polls the central master for a new map or
- * reduce task and runs it as a child process.  All communication between child
- * and parent is via this protocol. */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-@ProtocolInfo(protocolName = "TezTaskUmbilicalProtocol", protocolVersion = 1)
-public interface TezTaskUmbilicalProtocol extends VersionedProtocol {
-
-  public static final long versionID = 19L;
-
-  ContainerTask getTask(ContainerContext containerContext) throws IOException;
-
-  boolean canCommit(TezTaskAttemptID taskid) throws IOException;
-
-  ProceedToCompletionResponse
-      proceedToCompletion(TezTaskAttemptID taskAttemptId) throws IOException;
-
-  /// Copies from TezUmbilical until complete re-factor is done
-  // TODONEWTEZ
-
-  public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
-      throws IOException, TezException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 0178b3a..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/KVReader.java
+++ /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.
- */
-
-package org.apache.tez.engine.api;
-
-import java.io.IOException;
-
-import org.apache.tez.engine.api.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/b212ca1d/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
deleted file mode 100644
index 970831b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/KVWriter.java
+++ /dev/null
@@ -1,40 +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.tez.engine.api.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/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/api/Partitioner.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/api/Partitioner.java b/tez-engine/src/main/java/org/apache/tez/engine/api/Partitioner.java
deleted file mode 100644
index ccf3cb8..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/Partitioner.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.engine.api;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.tez.common.TezJobConfig;
-
-/**
- * {@link Partitioner} is used by the TEZ framework to partition output
- * key/value pairs.
- * 
- * <b>Partitioner Initialization</b></p> The Partitioner class is picked up
- * using the TEZ_ENGINE_PARTITIONER_CLASS attribute in {@link TezJobConfig}
- * 
- * TODO NEWTEZ Change construction to first check for a Constructor with a bytep[] payload
- * 
- * Partitioners need to provide a single argument ({@link Configuration})
- * constructor or a 0 argument constructor. If both exist, preference is given
- * to the single argument constructor. This is primarily for MR support.
- * 
- * If using the configuration constructor, TEZ_ENGINE_NUM_EXPECTED_PARTITIONS
- * will be set in the configuration, to indicate the max number of expected
- * partitions.
- * 
- */
-public interface Partitioner {
-  
-  /**
-   * Get partition for given key/value.
-   * @param key key
-   * @param value value
-   * @param numPartitions number of partitions
-   * @return
-   */
-  int getPartition(Object key, Object value, int numPartitions);
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index a9f2c98..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.api.events;
-
-import org.apache.tez.engine.api.Event;
-
-public class TaskAttemptCompletedEvent extends Event {
-
-  public TaskAttemptCompletedEvent() {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index fc67472..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.api.events;
-
-import org.apache.tez.engine.api.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/b212ca1d/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
deleted file mode 100644
index c0d1ee6..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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.api.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/b212ca1d/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
deleted file mode 100644
index 64df7bb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 52fc10d..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index a9ef333..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 3a1d5d8..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 6527777..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 6841d72..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/api/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.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.Event;
-import org.apache.tez.engine.api.events.DataMovementEvent;
-import org.apache.tez.engine.api.events.InputFailedEvent;
-import org.apache.tez.engine.api.events.InputInformationEvent;
-import org.apache.tez.engine.api.events.InputReadErrorEvent;
-import org.apache.tez.engine.api.events.TaskAttemptCompletedEvent;
-import org.apache.tez.engine.api.events.TaskAttemptFailedEvent;
-import org.apache.tez.engine.api.events.TaskStatusUpdateEvent;
-import org.apache.tez.engine.api.events.EventProtos.DataMovementEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputFailedEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputInformationEventProto;
-import org.apache.tez.engine.api.events.EventProtos.InputReadErrorEventProto;
-import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptCompletedEventProto;
-import org.apache.tez.engine.api.events.SystemEventProtos.TaskAttemptFailedEventProto;
-
-import 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);
-    }
-  }
-
-}


[14/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index a6d1c5b..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/server/ShuffleHandler.java
+++ /dev/null
@@ -1,572 +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.shuffle.server;
-
-import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
-import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
-import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
-import static org.jboss.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED;
-import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.URL;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-
-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.io.DataInputByteBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.metrics2.MetricsSystem;
-import org.apache.hadoop.metrics2.annotation.Metric;
-import org.apache.hadoop.metrics2.annotation.Metrics;
-import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
-import org.apache.hadoop.metrics2.lib.MutableCounterInt;
-import org.apache.hadoop.metrics2.lib.MutableCounterLong;
-import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
-import org.apache.hadoop.security.ssl.SSLFactory;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-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.shuffle.common.ShuffleUtils;
-import org.jboss.netty.bootstrap.ServerBootstrap;
-import org.jboss.netty.buffer.ChannelBuffers;
-import org.jboss.netty.channel.Channel;
-import org.jboss.netty.channel.ChannelFactory;
-import org.jboss.netty.channel.ChannelFuture;
-import org.jboss.netty.channel.ChannelFutureListener;
-import org.jboss.netty.channel.ChannelHandlerContext;
-import org.jboss.netty.channel.ChannelPipeline;
-import org.jboss.netty.channel.ChannelPipelineFactory;
-import org.jboss.netty.channel.Channels;
-import org.jboss.netty.channel.ExceptionEvent;
-import org.jboss.netty.channel.MessageEvent;
-import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
-import org.jboss.netty.channel.group.ChannelGroup;
-import org.jboss.netty.channel.group.DefaultChannelGroup;
-import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
-import org.jboss.netty.handler.codec.frame.TooLongFrameException;
-import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
-import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
-import org.jboss.netty.handler.codec.http.HttpRequest;
-import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
-import org.jboss.netty.handler.codec.http.HttpResponse;
-import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
-import org.jboss.netty.handler.codec.http.HttpResponseStatus;
-import org.jboss.netty.handler.codec.http.QueryStringDecoder;
-import org.jboss.netty.handler.ssl.SslHandler;
-import org.jboss.netty.handler.stream.ChunkedStream;
-import org.jboss.netty.handler.stream.ChunkedWriteHandler;
-import org.jboss.netty.util.CharsetUtil;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-public class ShuffleHandler extends AuxiliaryService {
-
-  private static final Log LOG = LogFactory.getLog(ShuffleHandler.class);
-  
-  public static final String SHUFFLE_MANAGE_OS_CACHE = "mapreduce.shuffle.manage.os.cache";
-  public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
-
-  public static final String SHUFFLE_READAHEAD_BYTES = "mapreduce.shuffle.readahead.bytes";
-  public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
-
-  private int port;
-  private ChannelFactory selector;
-  private final ChannelGroup accepted = new DefaultChannelGroup();
-  private HttpPipelineFactory pipelineFact;
-  private int sslFileBufferSize;
-
-  public static final String MAPREDUCE_SHUFFLE_SERVICEID =
-      "mapreduce.shuffle";
-
-  private static final Map<String,String> userRsrc =
-    new ConcurrentHashMap<String,String>();
-  private static final JobTokenSecretManager secretManager =
-    new JobTokenSecretManager();
-  private SecretKey tokenSecret;
-
-  public static final String SHUFFLE_PORT_CONFIG_KEY = "mapreduce.shuffle.port";
-  public static final int DEFAULT_SHUFFLE_PORT = 8080;
-
-  public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
-    "mapreduce.shuffle.ssl.file.buffer.size";
-
-  public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
-
-  private ExternalSorter sorter;
-  
-  @Metrics(about="Shuffle output metrics", context="mapred")
-  static class ShuffleMetrics implements ChannelFutureListener {
-    @Metric("Shuffle output in bytes")
-        MutableCounterLong shuffleOutputBytes;
-    @Metric("# of failed shuffle outputs")
-        MutableCounterInt shuffleOutputsFailed;
-    @Metric("# of succeeeded shuffle outputs")
-        MutableCounterInt shuffleOutputsOK;
-    @Metric("# of current shuffle connections")
-        MutableGaugeInt shuffleConnections;
-
-    @Override
-    public void operationComplete(ChannelFuture future) throws Exception {
-      if (future.isSuccess()) {
-        shuffleOutputsOK.incr();
-      } else {
-        shuffleOutputsFailed.incr();
-      }
-      shuffleConnections.decr();
-    }
-  }
-
-  final ShuffleMetrics metrics;
-
-  ShuffleHandler(MetricsSystem ms) {
-    super("httpshuffle");
-    metrics = ms.register(new ShuffleMetrics());
-  }
-
-  public ShuffleHandler(ExternalSorter sorter) {
-    this(DefaultMetricsSystem.instance());
-    this.sorter = sorter;
-  }
-
-  /**
-   * Serialize the shuffle port into a ByteBuffer for use later on.
-   * @param port the port to be sent to the ApplciationMaster
-   * @return the serialized form of the port.
-   */
-  public static ByteBuffer serializeMetaData(int port) throws IOException {
-    //TODO these bytes should be versioned
-    DataOutputBuffer port_dob = new DataOutputBuffer();
-    port_dob.writeInt(port);
-    return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
-  }
-
-  /**
-   * A helper function to deserialize the metadata returned by ShuffleHandler.
-   * @param meta the metadata returned by the ShuffleHandler
-   * @return the port the Shuffle Handler is listening on to serve shuffle data.
-   */
-  public static int deserializeMetaData(ByteBuffer meta) throws IOException {
-    //TODO this should be returning a class not just an int
-    DataInputByteBuffer in = new DataInputByteBuffer();
-    in.reset(meta);
-    int port = in.readInt();
-    return port;
-  }
-
-  /**
-   * A helper function to serialize the JobTokenIdentifier to be sent to the
-   * ShuffleHandler as ServiceData.
-   * @param jobToken the job token to be used for authentication of
-   * shuffle data requests.
-   * @return the serialized version of the jobToken.
-   */
-  public static ByteBuffer serializeServiceData(Token<JobTokenIdentifier> jobToken) throws IOException {
-    //TODO these bytes should be versioned
-    DataOutputBuffer jobToken_dob = new DataOutputBuffer();
-    jobToken.write(jobToken_dob);
-    return ByteBuffer.wrap(jobToken_dob.getData(), 0, jobToken_dob.getLength());
-  }
-
-  static Token<JobTokenIdentifier> deserializeServiceData(ByteBuffer secret) throws IOException {
-    DataInputByteBuffer in = new DataInputByteBuffer();
-    in.reset(secret);
-    Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>();
-    jt.readFields(in);
-    return jt;
-  }
-
-  
-  @Override
-  public void initializeApplication(
-      ApplicationInitializationContext initAppContext) {
-    // TODO these bytes should be versioned
-    try {
-      String user = initAppContext.getUser();
-      ApplicationId appId = initAppContext.getApplicationId();
-      ByteBuffer secret = initAppContext.getApplicationDataForService();
-      Token<JobTokenIdentifier> jt = deserializeServiceData(secret);
-      // TODO: Once SHuffle is out of NM, this can use MR APIs
-      userRsrc.put(appId.toString(), user);
-      LOG.info("Added token for " + appId.toString());
-      secretManager.addTokenForJob(appId.toString(), jt);
-    } catch (IOException e) {
-      LOG.error("Error during initApp", e);
-      // TODO add API to AuxiliaryServices to report failures
-    }
-  }
-
-  @Override
-  public void stopApplication(ApplicationTerminationContext context) {
-    ApplicationId appId = context.getApplicationId();
-    secretManager.removeTokenForJob(appId.toString());
-    userRsrc.remove(appId.toString());
-  }
-
-  public void initialize(TezOutputContext outputContext, Configuration conf) throws IOException {
-    this.init(new Configuration(conf));
-    tokenSecret = ShuffleUtils.getJobTokenSecretFromTokenBytes(outputContext.getServiceConsumerMetaData(MAPREDUCE_SHUFFLE_SERVICEID));
-  }
-
-  @Override
-  public synchronized void serviceInit(Configuration conf) {
-    ThreadFactory bossFactory = new ThreadFactoryBuilder()
-      .setNameFormat("ShuffleHandler Netty Boss #%d")
-      .build();
-    ThreadFactory workerFactory = new ThreadFactoryBuilder()
-      .setNameFormat("ShuffleHandler Netty Worker #%d")
-      .build();
-    
-    selector = new NioServerSocketChannelFactory(
-        Executors.newCachedThreadPool(bossFactory),
-        Executors.newCachedThreadPool(workerFactory));    
-  }
-
-  // TODO change AbstractService to throw InterruptedException
-  @Override
-  public synchronized void serviceStart() {
-    Configuration conf = getConfig();
-    ServerBootstrap bootstrap = new ServerBootstrap(selector);
-    try {
-      pipelineFact = new HttpPipelineFactory(conf);
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-    bootstrap.setPipelineFactory(pipelineFact);
-    // Let OS pick the port
-    Channel ch = bootstrap.bind(new InetSocketAddress(0));
-    accepted.add(ch);
-    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
-    conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port));
-    pipelineFact.SHUFFLE.setPort(port);
-    LOG.info(getName() + " listening on port " + port);
-
-    sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
-                                    DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
-  }
-
-  @Override
-  public synchronized void serviceStop() {
-    accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
-    ServerBootstrap bootstrap = new ServerBootstrap(selector);
-    bootstrap.releaseExternalResources();
-    pipelineFact.destroy();
-  }
-
-  @Override
-  public synchronized ByteBuffer getMetaData() {
-    try {
-      return serializeMetaData(port); 
-    } catch (IOException e) {
-      LOG.error("Error during getMeta", e);
-      // TODO add API to AuxiliaryServices to report failures
-      return null;
-    }
-  }
-
-  class HttpPipelineFactory implements ChannelPipelineFactory {
-
-    final Shuffle SHUFFLE;
-    private SSLFactory sslFactory;
-
-    public HttpPipelineFactory(Configuration conf) throws Exception {
-      SHUFFLE = new Shuffle(conf);
-      if (conf.getBoolean(TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_SSL,
-              TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL)) {
-        sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
-        sslFactory.init();
-      }
-    }
-
-    public void destroy() {
-      if (sslFactory != null) {
-        sslFactory.destroy();
-      }
-    }
-
-    @Override
-    public ChannelPipeline getPipeline() throws Exception {
-      ChannelPipeline pipeline = Channels.pipeline();
-      if (sslFactory != null) {
-        pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
-      }
-      pipeline.addLast("decoder", new HttpRequestDecoder());
-      pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
-      pipeline.addLast("encoder", new HttpResponseEncoder());
-      pipeline.addLast("chunking", new ChunkedWriteHandler());
-      pipeline.addLast("shuffle", SHUFFLE);
-      return pipeline;
-      // TODO factor security manager into pipeline
-      // TODO factor out encode/decode to permit binary shuffle
-      // TODO factor out decode of index to permit alt. models
-    }
-
-  }
-
-  class Shuffle extends SimpleChannelUpstreamHandler {
-
-    private final Configuration conf;
-    private int port;
-
-    public Shuffle(Configuration conf) {
-      this.conf = conf;
-      this.port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
-    }
-    
-    public void setPort(int port) {
-      this.port = port;
-    }
-
-    private List<String> splitMaps(List<String> mapq) {
-      if (null == mapq) {
-        return null;
-      }
-      final List<String> ret = new ArrayList<String>();
-      for (String s : mapq) {
-        Collections.addAll(ret, s.split(","));
-      }
-      return ret;
-    }
-
-    @Override
-    public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
-        throws Exception {
-      HttpRequest request = (HttpRequest) evt.getMessage();
-      if (request.getMethod() != GET) {
-          sendError(ctx, METHOD_NOT_ALLOWED);
-          return;
-      }
-      // Check whether the shuffle version is compatible
-      if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(
-          request.getHeader(ShuffleHeader.HTTP_HEADER_NAME))
-          || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(
-              request.getHeader(ShuffleHeader.HTTP_HEADER_VERSION))) {
-        sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST);
-      }
-      final Map<String,List<String>> q =
-        new QueryStringDecoder(request.getUri()).getParameters();
-      final List<String> mapIds = splitMaps(q.get("map"));
-      final List<String> reduceQ = q.get("reduce");
-      final List<String> jobQ = q.get("job");
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("RECV: " + request.getUri() +
-            "\n  mapId: " + mapIds +
-            "\n  reduceId: " + reduceQ +
-            "\n  jobId: " + jobQ);
-      }
-
-      if (mapIds == null || reduceQ == null || jobQ == null) {
-        sendError(ctx, "Required param job, map and reduce", BAD_REQUEST);
-        return;
-      }
-      if (reduceQ.size() != 1 || jobQ.size() != 1) {
-        sendError(ctx, "Too many job/reduce parameters", BAD_REQUEST);
-        return;
-      }
-      int reduceId;
-      String jobId;
-      try {
-        reduceId = Integer.parseInt(reduceQ.get(0));
-        jobId = jobQ.get(0);
-      } catch (NumberFormatException e) {
-        sendError(ctx, "Bad reduce parameter", BAD_REQUEST);
-        return;
-      } catch (IllegalArgumentException e) {
-        sendError(ctx, "Bad job parameter", BAD_REQUEST);
-        return;
-      }
-
-      final String reqUri = request.getUri();
-      if (null == reqUri) {
-        // TODO? add upstream?
-        sendError(ctx, FORBIDDEN);
-        return;
-      }
-      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
-      try {
-        verifyRequest(jobId, ctx, request, response,
-            new URL("http", "", this.port, reqUri));
-      } catch (IOException e) {
-        LOG.warn("Shuffle failure ", e);
-        sendError(ctx, e.getMessage(), UNAUTHORIZED);
-        return;
-      }
-
-      Channel ch = evt.getChannel();
-      ch.write(response);
-      // TODO refactor the following into the pipeline
-      ChannelFuture lastMap = null;
-      for (String mapId : mapIds) {
-        try {
-          // TODO: Error handling - validate mapId via TezTaskAttemptId.forName
-          
-          // TODO NEWTEZ Fix this. TaskAttemptId is no longer valid. mapId validation will not work anymore.
-//          if (!mapId.equals(sorter.getTaskAttemptId().toString())) {
-//            String errorMessage =
-//                "Illegal shuffle request mapId: " + mapId
-//                    + " while actual mapId is " + sorter.getTaskAttemptId(); 
-//            LOG.warn(errorMessage);
-//            sendError(ctx, errorMessage, BAD_REQUEST);
-//            return;
-//          }
-
-          lastMap =
-            sendMapOutput(ctx, ch, userRsrc.get(jobId), jobId, mapId, reduceId);
-          if (null == lastMap) {
-            sendError(ctx, NOT_FOUND);
-            return;
-          }
-        } catch (IOException e) {
-          LOG.error("Shuffle error ", e);
-          sendError(ctx, e.getMessage(), INTERNAL_SERVER_ERROR);
-          return;
-        }
-      }
-      lastMap.addListener(metrics);
-      lastMap.addListener(ChannelFutureListener.CLOSE);
-    }
-
-    private void verifyRequest(String appid, ChannelHandlerContext ctx,
-        HttpRequest request, HttpResponse response, URL requestUri)
-        throws IOException {
-      if (null == tokenSecret) {
-        LOG.info("Request for unknown token " + appid);
-        throw new IOException("could not find jobid");
-      }
-      // string to encrypt
-      String enc_str = SecureShuffleUtils.buildMsgFrom(requestUri);
-      // hash from the fetcher
-      String urlHashStr =
-        request.getHeader(SecureShuffleUtils.HTTP_HEADER_URL_HASH);
-      if (urlHashStr == null) {
-        LOG.info("Missing header hash for " + appid);
-        throw new IOException("fetcher cannot be authenticated");
-      }
-      if (LOG.isDebugEnabled()) {
-        int len = urlHashStr.length();
-        LOG.debug("verifying request. enc_str=" + enc_str + "; hash=..." +
-            urlHashStr.substring(len-len/2, len-1));
-      }
-      // verify - throws exception
-      SecureShuffleUtils.verifyReply(urlHashStr, enc_str, tokenSecret);
-      // verification passed - encode the reply
-      String reply =
-        SecureShuffleUtils.generateHash(urlHashStr.getBytes(), tokenSecret);
-      response.setHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply);
-      addVersionToHeader(response);
-      if (LOG.isDebugEnabled()) {
-        int len = reply.length();
-        LOG.debug("Fetcher request verfied. enc_str=" + enc_str + ";reply=" +
-            reply.substring(len-len/2, len-1));
-      }
-    }
-
-    protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch,
-        String user, String jobId, String mapId, int reduce)
-        throws IOException {
-      final ShuffleHeader header = sorter.getShuffleHeader(reduce);
-      final DataOutputBuffer dob = new DataOutputBuffer();
-      header.write(dob);
-      ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
-
-      ChannelFuture writeFuture =
-          ch.write(
-              new ChunkedStream(
-                  sorter.getSortedStream(reduce), sslFileBufferSize
-                  )
-              );
-      metrics.shuffleConnections.incr();
-      metrics.shuffleOutputBytes.incr(header.getCompressedLength()); // optimistic
-      return writeFuture;
-    }
-
-    private void sendError(ChannelHandlerContext ctx,
-        HttpResponseStatus status) {
-      sendError(ctx, "", status);
-    }
-
-    private void sendError(ChannelHandlerContext ctx, String message,
-        HttpResponseStatus status) {
-      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
-      response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
-      addVersionToHeader(response);
-      response.setContent(
-        ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
-      // Close the connection as soon as the error message is sent.
-      ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
-    }
-    
-    private void addVersionToHeader(HttpResponse response) {
-      // Put shuffle version into http header
-      response.setHeader(ShuffleHeader.HTTP_HEADER_NAME,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
-      response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION,
-          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);      
-    }
-
-    @Override
-    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
-        throws Exception {
-      Channel ch = e.getChannel();
-      Throwable cause = e.getCause();
-      if (cause instanceof TooLongFrameException) {
-        sendError(ctx, BAD_REQUEST);
-        return;
-      }
-
-      LOG.error("Shuffle error: ", cause);
-      if (ch.isConnected()) {
-        LOG.error("Shuffle error " + e);
-        sendError(ctx, INTERNAL_SERVER_ERROR);
-      }
-    }
-
-  }
-
-  public int getPort() {
-    return port;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 5aa0ddf..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/ExternalSorter.java
+++ /dev/null
@@ -1,194 +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.impl;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-
-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.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
-import org.apache.hadoop.util.IndexedSorter;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.QuickSort;
-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.Partitioner;
-import org.apache.tez.engine.api.TezOutputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.TezEngineUtils;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.shuffle.impl.ShuffleHeader;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutput;
-import org.apache.tez.engine.hadoop.compat.NullProgressable;
-
-@SuppressWarnings({"unchecked", "rawtypes"})
-public abstract class ExternalSorter {
-
-  private static final Log LOG = LogFactory.getLog(ExternalSorter.class);
-
-  public abstract void close() throws IOException;
-
-  public abstract void flush() throws IOException;
-
-  public abstract void write(Object key, Object value) throws IOException;
-
-  protected Progressable nullProgressable = new NullProgressable();
-  protected TezOutputContext outputContext;
-  protected Combiner combiner;
-  protected Partitioner partitioner;
-  protected Configuration conf;
-  protected FileSystem rfs;
-  protected TezTaskOutput mapOutputFile;
-  protected int partitions;
-  protected Class keyClass;
-  protected Class valClass;
-  protected RawComparator comparator;
-  protected SerializationFactory serializationFactory;
-  protected Serializer keySerializer;
-  protected Serializer valSerializer;
-
-  protected IndexedSorter sorter;
-
-  // Compression for map-outputs
-  protected CompressionCodec codec;
-
-  // Counters
-  // TODO TEZ Rename all counter variables [Mapping of counter to MR for compatibility in the MR layer]
-  protected TezCounter mapOutputByteCounter;
-  protected TezCounter mapOutputRecordCounter;
-  protected TezCounter fileOutputByteCounter;
-  protected TezCounter spilledRecordsCounter;
-
-  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
-    this.outputContext = outputContext;
-    this.conf = conf;
-    this.partitions = numOutputs;
-
-    rfs = ((LocalFileSystem)FileSystem.getLocal(this.conf)).getRaw();
-
-    // sorter
-    sorter = ReflectionUtils.newInstance(this.conf.getClass(
-        TezJobConfig.TEZ_ENGINE_INTERNAL_SORTER_CLASS, QuickSort.class,
-        IndexedSorter.class), this.conf);
-
-    comparator = ConfigUtils.getIntermediateOutputKeyComparator(this.conf);
-
-    // k/v serialization
-    keyClass = ConfigUtils.getIntermediateOutputKeyClass(this.conf);
-    valClass = ConfigUtils.getIntermediateOutputValueClass(this.conf);
-    serializationFactory = new SerializationFactory(this.conf);
-    keySerializer = serializationFactory.getSerializer(keyClass);
-    valSerializer = serializationFactory.getSerializer(valClass);
-
-    //    counters
-    mapOutputByteCounter =
-        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_BYTES);
-    mapOutputRecordCounter =
-        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS);
-    fileOutputByteCounter =
-        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_MATERIALIZED_BYTES);
-    spilledRecordsCounter =
-        outputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
-    // compression
-    if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) {
-      Class<? extends CompressionCodec> codecClass =
-          ConfigUtils.getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, this.conf);
-    } else {
-      codec = null;
-    }
-
-    // Task outputs
-    mapOutputFile = TezEngineUtils.instantiateTaskOutputManager(conf, outputContext);
-    
-    LOG.info("Instantiating Partitioner: [" + conf.get(TezJobConfig.TEZ_ENGINE_PARTITIONER_CLASS) + "]");
-    this.conf.setInt(TezJobConfig.TEZ_ENGINE_NUM_EXPECTED_PARTITIONS, this.partitions);
-    this.partitioner = TezEngineUtils.instantiatePartitioner(this.conf);
-    this.combiner = TezEngineUtils.instantiateCombiner(this.conf, outputContext);
-  }
-
-  /**
-   * Exception indicating that the allocated sort buffer is insufficient to hold
-   * the current record.
-   */
-  @SuppressWarnings("serial")
-  public static class MapBufferTooSmallException extends IOException {
-    public MapBufferTooSmallException(String s) {
-      super(s);
-    }
-  }
-
-  @Private
-  public TezTaskOutput getMapOutput() {
-    return mapOutputFile;
-  }
-
-  protected void runCombineProcessor(TezRawKeyValueIterator kvIter,
-      Writer writer) throws IOException {
-    try {
-      combiner.combine(kvIter, writer);
-    } catch (InterruptedException e) {
-      throw new IOException(e);
-    }
-  }
-
-  /**
-   * Rename srcPath to dstPath on the same volume. This is the same as
-   * RawLocalFileSystem's rename method, except that it will not fall back to a
-   * copy, and it will create the target directory if it doesn't exist.
-   */
-  protected void sameVolRename(Path srcPath, Path dstPath) throws IOException {
-    RawLocalFileSystem rfs = (RawLocalFileSystem) this.rfs;
-    File src = rfs.pathToFile(srcPath);
-    File dst = rfs.pathToFile(dstPath);
-    if (!dst.getParentFile().exists()) {
-      if (!dst.getParentFile().mkdirs()) {
-        throw new IOException("Unable to rename " + src + " to " + dst
-            + ": couldn't create parent directory");
-      }
-    }
-
-    if (!src.renameTo(dst)) {
-      throw new IOException("Unable to rename " + src + " to " + dst);
-    }
-  }
-
-  public InputStream getSortedStream(int partition) {
-    throw new UnsupportedOperationException("getSortedStream isn't supported!");
-  }
-
-  public ShuffleHeader getShuffleHeader(int reduce) {
-    throw new UnsupportedOperationException("getShuffleHeader isn't supported!");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFile.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFile.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFile.java
deleted file mode 100644
index 7d10606..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFile.java
+++ /dev/null
@@ -1,559 +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.impl;
-
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-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.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.BufferUtils;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.io.compress.CodecPool;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionOutputStream;
-import org.apache.hadoop.io.compress.Compressor;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.io.serializer.SerializationFactory;
-import org.apache.hadoop.io.serializer.Serializer;
-import org.apache.tez.common.counters.TezCounter;
-
-/**
- * <code>IFile</code> is the simple <key-len, value-len, key, value> format
- * for the intermediate map-outputs in Map-Reduce.
- *
- * There is a <code>Writer</code> to write out map-outputs in this format and 
- * a <code>Reader</code> to read files of this format.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class IFile {
-  private static final Log LOG = LogFactory.getLog(IFile.class);
-  public static final int EOF_MARKER = -1; // End of File Marker
-  public static final int RLE_MARKER = -2; // Repeat same key marker
-  public static final DataInputBuffer REPEAT_KEY = new DataInputBuffer();
-    
-  /**
-   * <code>IFile.Writer</code> to write out intermediate map-outputs. 
-   */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  public static class Writer {
-    FSDataOutputStream out;
-    boolean ownOutputStream = false;
-    long start = 0;
-    FSDataOutputStream rawOut;
-    AtomicBoolean closed = new AtomicBoolean(false);
-    
-    CompressionOutputStream compressedOut;
-    Compressor compressor;
-    boolean compressOutput = false;
-    
-    long decompressedBytesWritten = 0;
-    long compressedBytesWritten = 0;
-
-    // Count records written to disk
-    private long numRecordsWritten = 0;
-    private final TezCounter writtenRecordsCounter;
-
-    IFileOutputStream checksumOut;
-
-    Class keyClass;
-    Class valueClass;
-    Serializer keySerializer;
-    Serializer valueSerializer;
-    
-    DataOutputBuffer buffer = new DataOutputBuffer();
-    DataOutputBuffer previous = new DataOutputBuffer();
-    
-    // de-dup keys or not
-    private boolean rle = false;
-
-    public Writer(Configuration conf, FileSystem fs, Path file, 
-                  Class keyClass, Class valueClass,
-                  CompressionCodec codec,
-                  TezCounter writesCounter) throws IOException {
-      this(conf, fs.create(file), keyClass, valueClass, codec,
-           writesCounter);
-      ownOutputStream = true;
-    }
-    
-    protected Writer(TezCounter writesCounter) {
-      writtenRecordsCounter = writesCounter;
-    }
-
-    public Writer(Configuration conf, FSDataOutputStream out, 
-        Class keyClass, Class valueClass,
-        CompressionCodec codec, TezCounter writesCounter)
-        throws IOException {
-      this.writtenRecordsCounter = writesCounter;
-      this.checksumOut = new IFileOutputStream(out);
-      this.rawOut = out;
-      this.start = this.rawOut.getPos();
-      if (codec != null) {
-        this.compressor = CodecPool.getCompressor(codec);
-        if (this.compressor != null) {
-          this.compressor.reset();
-          this.compressedOut = codec.createOutputStream(checksumOut, compressor);
-          this.out = new FSDataOutputStream(this.compressedOut,  null);
-          this.compressOutput = true;
-        } else {
-          LOG.warn("Could not obtain compressor from CodecPool");
-          this.out = new FSDataOutputStream(checksumOut,null);
-        }
-      } else {
-        this.out = new FSDataOutputStream(checksumOut,null);
-      }
-      
-      this.keyClass = keyClass;
-      this.valueClass = valueClass;
-
-      if (keyClass != null) {
-        SerializationFactory serializationFactory = 
-          new SerializationFactory(conf);
-        this.keySerializer = serializationFactory.getSerializer(keyClass);
-        this.keySerializer.open(buffer);
-        this.valueSerializer = serializationFactory.getSerializer(valueClass);
-        this.valueSerializer.open(buffer);
-      }
-    }
-
-    public Writer(Configuration conf, FileSystem fs, Path file) 
-    throws IOException {
-      this(conf, fs, file, null, null, null, null);
-    }
-
-    public void close() throws IOException {
-      if (closed.getAndSet(true)) {
-        throw new IOException("Writer was already closed earlier");
-      }
-
-      // When IFile writer is created by BackupStore, we do not have
-      // Key and Value classes set. So, check before closing the
-      // serializers
-      if (keyClass != null) {
-        keySerializer.close();
-        valueSerializer.close();
-      }
-
-      // Write EOF_MARKER for key/value length
-      WritableUtils.writeVInt(out, EOF_MARKER);
-      WritableUtils.writeVInt(out, EOF_MARKER);
-      decompressedBytesWritten += 2 * WritableUtils.getVIntSize(EOF_MARKER);
-      
-      //Flush the stream
-      out.flush();
-  
-      if (compressOutput) {
-        // Flush
-        compressedOut.finish();
-        compressedOut.resetState();
-      }
-      
-      // Close the underlying stream iff we own it...
-      if (ownOutputStream) {
-        out.close();
-      }
-      else {
-        // Write the checksum
-        checksumOut.finish();
-      }
-
-      compressedBytesWritten = rawOut.getPos() - start;
-
-      if (compressOutput) {
-        // Return back the compressor
-        CodecPool.returnCompressor(compressor);
-        compressor = null;
-      }
-
-      out = null;
-      if(writtenRecordsCounter != null) {
-        writtenRecordsCounter.increment(numRecordsWritten);
-      }
-    }
-
-    public void append(Object key, Object value) throws IOException {
-      if (key.getClass() != keyClass)
-        throw new IOException("wrong key class: "+ key.getClass()
-                              +" is not "+ keyClass);
-      if (value.getClass() != valueClass)
-        throw new IOException("wrong value class: "+ value.getClass()
-                              +" is not "+ valueClass);
-      
-      boolean sameKey = false;
-
-      // Append the 'key'
-      keySerializer.serialize(key);
-      int keyLength = buffer.getLength();
-      if (keyLength < 0) {
-        throw new IOException("Negative key-length not allowed: " + keyLength + 
-                              " for " + key);
-      }     
-      
-      if(keyLength == previous.getLength()) {
-        sameKey = (BufferUtils.compare(previous, buffer) == 0);       
-      }
-      
-      if(!sameKey) {
-        BufferUtils.copy(buffer, previous);
-      }
-
-      // Append the 'value'
-      valueSerializer.serialize(value);
-      int valueLength = buffer.getLength() - keyLength;
-      if (valueLength < 0) {
-        throw new IOException("Negative value-length not allowed: " + 
-                              valueLength + " for " + value);
-      }
-      
-      if(sameKey) {        
-        WritableUtils.writeVInt(out, RLE_MARKER);                   // Same key as previous
-        WritableUtils.writeVInt(out, valueLength);                  // value length
-        out.write(buffer.getData(), keyLength, buffer.getLength()); // only the value
-        // Update bytes written
-        decompressedBytesWritten += 0 + valueLength + 
-                                    WritableUtils.getVIntSize(RLE_MARKER) + 
-                                    WritableUtils.getVIntSize(valueLength);
-      } else {        
-        // Write the record out        
-        WritableUtils.writeVInt(out, keyLength);                  // key length
-        WritableUtils.writeVInt(out, valueLength);                // value length
-        out.write(buffer.getData(), 0, buffer.getLength());       // data
-        // Update bytes written
-        decompressedBytesWritten += keyLength + valueLength + 
-                                    WritableUtils.getVIntSize(keyLength) + 
-                                    WritableUtils.getVIntSize(valueLength);
-      }
-
-      // Reset
-      buffer.reset();
-      
-      
-      ++numRecordsWritten;
-    }
-    
-    public void append(DataInputBuffer key, DataInputBuffer value)
-    throws IOException {
-      int keyLength = key.getLength() - key.getPosition();
-      if (keyLength < 0) {
-        throw new IOException("Negative key-length not allowed: " + keyLength + 
-                              " for " + key);
-      }
-      
-      int valueLength = value.getLength() - value.getPosition();
-      if (valueLength < 0) {
-        throw new IOException("Negative value-length not allowed: " + 
-                              valueLength + " for " + value);
-      }
-      
-      boolean sameKey = false;
-      
-      if(rle && keyLength == previous.getLength()) {
-        sameKey = (keyLength != 0) && (BufferUtils.compare(previous, key) == 0);        
-      }
-      
-      if(rle && sameKey) {
-        WritableUtils.writeVInt(out, RLE_MARKER);
-        WritableUtils.writeVInt(out, valueLength);        
-        out.write(value.getData(), value.getPosition(), valueLength);
-
-        // Update bytes written
-        decompressedBytesWritten += 0 + valueLength
-            + WritableUtils.getVIntSize(RLE_MARKER)
-            + WritableUtils.getVIntSize(valueLength);
-      } else {
-        WritableUtils.writeVInt(out, keyLength);
-        WritableUtils.writeVInt(out, valueLength);
-        out.write(key.getData(), key.getPosition(), keyLength);
-        out.write(value.getData(), value.getPosition(), valueLength);
-
-        // Update bytes written
-        decompressedBytesWritten += keyLength + valueLength
-            + WritableUtils.getVIntSize(keyLength)
-            + WritableUtils.getVIntSize(valueLength);
-                
-        BufferUtils.copy(key, previous);        
-      }
-      ++numRecordsWritten;
-    }
-    
-    // Required for mark/reset
-    public DataOutputStream getOutputStream () {
-      return out;
-    }
-    
-    // Required for mark/reset
-    public void updateCountersForExternalAppend(long length) {
-      ++numRecordsWritten;
-      decompressedBytesWritten += length;
-    }
-    
-    public long getRawLength() {
-      return decompressedBytesWritten;
-    }
-    
-    public long getCompressedLength() {
-      return compressedBytesWritten;
-    }
-    
-    public void setRLE(boolean rle) {
-      this.rle = rle;
-      previous.reset();
-    }
-
-  }
-
-  /**
-   * <code>IFile.Reader</code> to read intermediate map-outputs. 
-   */
-  @InterfaceAudience.Private
-  @InterfaceStability.Unstable
-  public static class Reader {
-    
-    public enum KeyState {NO_KEY, NEW_KEY, SAME_KEY};
-    
-    private static final int DEFAULT_BUFFER_SIZE = 128*1024;
-
-    // Count records read from disk
-    private long numRecordsRead = 0;
-    private final TezCounter readRecordsCounter;
-
-    final InputStream in;        // Possibly decompressed stream that we read
-    Decompressor decompressor;
-    public long bytesRead = 0;
-    protected final long fileLength;
-    protected boolean eof = false;
-    final IFileInputStream checksumIn;
-    
-    protected byte[] buffer = null;
-    protected int bufferSize = DEFAULT_BUFFER_SIZE;
-    protected DataInputStream dataIn;
-
-    protected int recNo = 1;
-    protected int prevKeyLength;
-    protected int currentKeyLength;
-    protected int currentValueLength;
-    byte keyBytes[] = new byte[0];
-    
-    
-    /**
-     * Construct an IFile Reader.
-     * 
-     * @param conf Configuration File 
-     * @param fs  FileSystem
-     * @param file Path of the file to be opened. This file should have
-     *             checksum bytes for the data at the end of the file.
-     * @param codec codec
-     * @param readsCounter Counter for records read from disk
-     * @throws IOException
-     */
-    public Reader(Configuration conf, FileSystem fs, Path file,
-                  CompressionCodec codec,
-                  TezCounter readsCounter) throws IOException {
-      this(conf, fs.open(file), 
-           fs.getFileStatus(file).getLen(),
-           codec, readsCounter);
-    }
-
-    /**
-     * Construct an IFile Reader.
-     * 
-     * @param conf Configuration File 
-     * @param in   The input stream
-     * @param length Length of the data in the stream, including the checksum
-     *               bytes.
-     * @param codec codec
-     * @param readsCounter Counter for records read from disk
-     * @throws IOException
-     */
-    public Reader(Configuration conf, InputStream in, long length, 
-                  CompressionCodec codec,
-                  TezCounter readsCounter) throws IOException {
-      readRecordsCounter = readsCounter;
-      checksumIn = new IFileInputStream(in,length, conf);
-      if (codec != null) {
-        decompressor = CodecPool.getDecompressor(codec);
-        if (decompressor != null) {
-          this.in = codec.createInputStream(checksumIn, decompressor);
-        } else {
-          LOG.warn("Could not obtain decompressor from CodecPool");
-          this.in = checksumIn;
-        }
-      } else {
-        this.in = checksumIn;
-      }
-      this.dataIn = new DataInputStream(this.in);
-      this.fileLength = length;
-      
-      if (conf != null) {
-        bufferSize = conf.getInt("io.file.buffer.size", DEFAULT_BUFFER_SIZE);
-      }
-    }
-    
-    public long getLength() { 
-      return fileLength - checksumIn.getSize();
-    }
-    
-    public long getPosition() throws IOException {    
-      return checksumIn.getPosition(); 
-    }
-    
-    /**
-     * Read upto len bytes into buf starting at offset off.
-     * 
-     * @param buf buffer 
-     * @param off offset
-     * @param len length of buffer
-     * @return the no. of bytes read
-     * @throws IOException
-     */
-    private int readData(byte[] buf, int off, int len) throws IOException {
-      int bytesRead = 0;
-      while (bytesRead < len) {
-        int n = IOUtils.wrappedReadForCompressedData(in, buf, off + bytesRead,
-            len - bytesRead);
-        if (n < 0) {
-          return bytesRead;
-        }
-        bytesRead += n;
-      }
-      return len;
-    }
-    
-    protected boolean positionToNextRecord(DataInput dIn) throws IOException {
-      // Sanity check
-      if (eof) {
-        throw new EOFException("Completed reading " + bytesRead);
-      }
-      
-      // Read key and value lengths
-      prevKeyLength = currentKeyLength;
-      currentKeyLength = WritableUtils.readVInt(dIn);
-      currentValueLength = WritableUtils.readVInt(dIn);
-      bytesRead += WritableUtils.getVIntSize(currentKeyLength) +
-                   WritableUtils.getVIntSize(currentValueLength);
-      
-      // Check for EOF
-      if (currentKeyLength == EOF_MARKER && currentValueLength == EOF_MARKER) {
-        eof = true;
-        return false;
-      }      
-      
-      // Sanity check
-      if (currentKeyLength != RLE_MARKER && currentKeyLength < 0) {
-        throw new IOException("Rec# " + recNo + ": Negative key-length: " + 
-                              currentKeyLength);
-      }
-      if (currentValueLength < 0) {
-        throw new IOException("Rec# " + recNo + ": Negative value-length: " + 
-                              currentValueLength);
-      }
-            
-      return true;
-    }
-    
-    public boolean nextRawKey(DataInputBuffer key) throws IOException {
-      return readRawKey(key) != KeyState.NO_KEY;
-    }
-    
-    public KeyState readRawKey(DataInputBuffer key) throws IOException {
-      if (!positionToNextRecord(dataIn)) {
-        return KeyState.NO_KEY;
-      }
-      if(currentKeyLength == RLE_MARKER) {
-        currentKeyLength = prevKeyLength;
-        // no data to read
-        key.reset(keyBytes, currentKeyLength);
-        return KeyState.SAME_KEY;
-      }
-      if (keyBytes.length < currentKeyLength) {
-        keyBytes = new byte[currentKeyLength << 1];
-      }
-      int i = readData(keyBytes, 0, currentKeyLength);
-      if (i != currentKeyLength) {
-        throw new IOException ("Asked for " + currentKeyLength + " Got: " + i);
-      }
-      key.reset(keyBytes, currentKeyLength);
-      bytesRead += currentKeyLength;
-      return KeyState.NEW_KEY;
-    }
-    
-    public void nextRawValue(DataInputBuffer value) throws IOException {
-      final byte[] valBytes = 
-        ((value.getData().length < currentValueLength) || (value.getData() == keyBytes))
-        ? new byte[currentValueLength << 1]
-        : value.getData();
-      int i = readData(valBytes, 0, currentValueLength);
-      if (i != currentValueLength) {
-        throw new IOException ("Asked for " + currentValueLength + " Got: " + i);
-      }
-      value.reset(valBytes, currentValueLength);
-      
-      // Record the bytes read
-      bytesRead += currentValueLength;
-
-      ++recNo;
-      ++numRecordsRead;
-    }
-    
-    public void close() throws IOException {
-      // Close the underlying stream
-      in.close();
-      
-      // Release the buffer
-      dataIn = null;
-      buffer = null;
-      if(readRecordsCounter != null) {
-        readRecordsCounter.increment(numRecordsRead);
-      }
-
-      // Return the decompressor
-      if (decompressor != null) {
-        decompressor.reset();
-        CodecPool.returnDecompressor(decompressor);
-        decompressor = null;
-      }
-    }
-    
-    public void reset(int offset) {
-      return;
-    }
-
-    public void disableChecksumValidation() {
-      checksumIn.disableChecksumValidation();
-    }
-
-  }    
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileInputStream.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileInputStream.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileInputStream.java
deleted file mode 100644
index dfb69f1..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileInputStream.java
+++ /dev/null
@@ -1,276 +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.impl;
-
-import java.io.EOFException;
-import java.io.FileDescriptor;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-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.ChecksumException;
-import org.apache.hadoop.fs.HasFileDescriptor;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.ReadaheadPool;
-import org.apache.hadoop.io.ReadaheadPool.ReadaheadRequest;
-import org.apache.hadoop.util.DataChecksum;
-import org.apache.tez.common.TezJobConfig;
-/**
- * A checksum input stream, used for IFiles.
- * Used to validate the checksum of files created by {@link IFileOutputStream}. 
-*/
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class IFileInputStream extends InputStream {
-  
-  private final InputStream in; //The input stream to be verified for checksum.
-  private final FileDescriptor inFd; // the file descriptor, if it is known
-  private final long length; //The total length of the input file
-  private final long dataLength;
-  private DataChecksum sum;
-  private long currentOffset = 0;
-  private final byte b[] = new byte[1];
-  private byte csum[] = null;
-  private int checksumSize;
-  private byte[] buffer;
-  private int offset;
-
-  private ReadaheadRequest curReadahead = null;
-  private ReadaheadPool raPool = ReadaheadPool.getInstance();
-  private boolean readahead;
-  private int readaheadLength;
-
-  public static final Log LOG = LogFactory.getLog(IFileInputStream.class);
-
-  private boolean disableChecksumValidation = false;
-  
-  /**
-   * Create a checksum input stream that reads
-   * @param in The input stream to be verified for checksum.
-   * @param len The length of the input stream including checksum bytes.
-   */
-  public IFileInputStream(InputStream in, long len, Configuration conf) {
-    this.in = in;
-    this.inFd = getFileDescriptorIfAvail(in);
-    sum = DataChecksum.newDataChecksum(DataChecksum.Type.CRC32, 
-        Integer.MAX_VALUE);
-    checksumSize = sum.getChecksumSize();
-    buffer = new byte[4096];
-    offset = 0;
-    length = len;
-    dataLength = length - checksumSize;
-
-    conf = (conf != null) ? conf : new Configuration();
-    readahead = conf.getBoolean(TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_IFILE_READAHEAD);
-    readaheadLength = conf.getInt(TezJobConfig.TEZ_ENGINE_IFILE_READAHEAD_BYTES,
-        TezJobConfig.DEFAULT_TEZ_ENGINE_IFILE_READAHEAD_BYTES);
-
-    doReadahead();
-  }
-
-  private static FileDescriptor getFileDescriptorIfAvail(InputStream in) {
-    FileDescriptor fd = null;
-    try {
-      if (in instanceof HasFileDescriptor) {
-        fd = ((HasFileDescriptor)in).getFileDescriptor();
-      } else if (in instanceof FileInputStream) {
-        fd = ((FileInputStream)in).getFD();
-      }
-    } catch (IOException e) {
-      LOG.info("Unable to determine FileDescriptor", e);
-    }
-    return fd;
-  }
-
-  /**
-   * Close the input stream. Note that we need to read to the end of the
-   * stream to validate the checksum.
-   */
-  @Override
-  public void close() throws IOException {
-
-    if (curReadahead != null) {
-      curReadahead.cancel();
-    }
-    if (currentOffset < dataLength) {
-      byte[] t = new byte[Math.min((int)
-            (Integer.MAX_VALUE & (dataLength - currentOffset)), 32 * 1024)];
-      while (currentOffset < dataLength) {
-        int n = read(t, 0, t.length);
-        if (0 == n) {
-          throw new EOFException("Could not validate checksum");
-        }
-      }
-    }
-    in.close();
-  }
-  
-  @Override
-  public long skip(long n) throws IOException {
-   throw new IOException("Skip not supported for IFileInputStream");
-  }
-  
-  public long getPosition() {
-    return (currentOffset >= dataLength) ? dataLength : currentOffset;
-  }
-  
-  public long getSize() {
-    return checksumSize;
-  }
-
-  private void checksum(byte[] b, int off, int len) {
-    if(len >= buffer.length) {
-      sum.update(buffer, 0, offset);
-      offset = 0;
-      sum.update(b, off, len);
-      return;
-    }
-    final int remaining = buffer.length - offset;
-    if(len > remaining) {
-      sum.update(buffer, 0, offset);
-      offset = 0;
-    }
-    /* now we should have len < buffer.length */
-    System.arraycopy(b, off, buffer, offset, len);
-    offset += len;
-  }
-  
-  /**
-   * Read bytes from the stream.
-   * At EOF, checksum is validated, but the checksum
-   * bytes are not passed back in the buffer. 
-   */
-  public int read(byte[] b, int off, int len) throws IOException {
-
-    if (currentOffset >= dataLength) {
-      return -1;
-    }
-
-    doReadahead();
-
-    return doRead(b,off,len);
-  }
-
-  private void doReadahead() {
-    if (raPool != null && inFd != null && readahead) {
-      curReadahead = raPool.readaheadStream(
-          "ifile", inFd,
-          currentOffset, readaheadLength, dataLength,
-          curReadahead);
-    }
-  }
-
-  /**
-   * Read bytes from the stream.
-   * At EOF, checksum is validated and sent back
-   * as the last four bytes of the buffer. The caller should handle
-   * these bytes appropriately
-   */
-  public int readWithChecksum(byte[] b, int off, int len) throws IOException {
-
-    if (currentOffset == length) {
-      return -1;
-    }
-    else if (currentOffset >= dataLength) {
-      // If the previous read drained off all the data, then just return
-      // the checksum now. Note that checksum validation would have 
-      // happened in the earlier read
-      int lenToCopy = (int) (checksumSize - (currentOffset - dataLength));
-      if (len < lenToCopy) {
-        lenToCopy = len;
-      }
-      System.arraycopy(csum, (int) (currentOffset - dataLength), b, off, 
-          lenToCopy);
-      currentOffset += lenToCopy;
-      return lenToCopy;
-    }
-
-    int bytesRead = doRead(b,off,len);
-
-    if (currentOffset == dataLength) {
-      if (len >= bytesRead + checksumSize) {
-        System.arraycopy(csum, 0, b, off + bytesRead, checksumSize);
-        bytesRead += checksumSize;
-        currentOffset += checksumSize;
-      }
-    }
-    return bytesRead;
-  }
-
-  private int doRead(byte[]b, int off, int len) throws IOException {
-    
-    // If we are trying to read past the end of data, just read
-    // the left over data
-    if (currentOffset + len > dataLength) {
-      len = (int) dataLength - (int)currentOffset;
-    }
-    
-    int bytesRead = in.read(b, off, len);
-
-    if (bytesRead < 0) {
-      throw new ChecksumException("Checksum Error", 0);
-    }
-
-    checksum(b, off, bytesRead);
-
-    currentOffset += bytesRead;
-
-    if (disableChecksumValidation) {
-      return bytesRead;
-    }
-    
-    if (currentOffset == dataLength) {
-      // The last four bytes are checksum. Strip them and verify
-      sum.update(buffer, 0, offset);
-      csum = new byte[checksumSize];
-      IOUtils.readFully(in, csum, 0, checksumSize);
-      if (!sum.compare(csum, 0)) {
-        throw new ChecksumException("Checksum Error", 0);
-      }
-    }
-    return bytesRead;
-  }
-
-
-  @Override
-  public int read() throws IOException {    
-    b[0] = 0;
-    int l = read(b,0,1);
-    if (l < 0)  return l;
-    
-    // Upgrade the b[0] to an int so as not to misinterpret the
-    // first bit of the byte as a sign bit
-    int result = 0xFF & b[0];
-    return result;
-  }
-
-  public byte[] getChecksum() {
-    return csum;
-  }
-
-  void disableChecksumValidation() {
-    disableChecksumValidation = true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileOutputStream.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileOutputStream.java b/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileOutputStream.java
deleted file mode 100644
index 3b39900..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/sort/impl/IFileOutputStream.java
+++ /dev/null
@@ -1,129 +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.impl;
-
-import java.io.FilterOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.util.DataChecksum;
-/**
- * A Checksum output stream.
- * Checksum for the contents of the file is calculated and
- * appended to the end of the file on close of the stream.
- * Used for IFiles
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class IFileOutputStream extends FilterOutputStream {
-
-  /**
-   * The output stream to be checksummed.
-   */
-  private final DataChecksum sum;
-  private byte[] barray;
-  private byte[] buffer;
-  private int offset;
-  private boolean closed = false;
-  private boolean finished = false;
-
-  /**
-   * Create a checksum output stream that writes
-   * the bytes to the given stream.
-   * @param out
-   */
-  public IFileOutputStream(OutputStream out) {
-    super(out);
-    sum = DataChecksum.newDataChecksum(DataChecksum.Type.CRC32,
-        Integer.MAX_VALUE);
-    barray = new byte[sum.getChecksumSize()];
-    buffer = new byte[4096];
-    offset = 0;
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (closed) {
-      return;
-    }
-    closed = true;
-    finish();
-    out.close();
-  }
-
-  /**
-   * Finishes writing data to the output stream, by writing
-   * the checksum bytes to the end. The underlying stream is not closed.
-   * @throws IOException
-   */
-  public void finish() throws IOException {
-    if (finished) {
-      return;
-    }
-    finished = true;
-    sum.update(buffer, 0, offset);
-    sum.writeValue(barray, 0, false);
-    out.write (barray, 0, sum.getChecksumSize());
-    out.flush();
-  }
-
-  private void checksum(byte[] b, int off, int len) {
-    if(len >= buffer.length) {
-      sum.update(buffer, 0, offset);
-      offset = 0;
-      sum.update(b, off, len);
-      return;
-    }
-    final int remaining = buffer.length - offset;
-    if(len > remaining) {
-      sum.update(buffer, 0, offset);
-      offset = 0;
-    }
-    /*
-    // FIXME if needed re-enable this in debug mode
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("XXX checksum" +
-          " b=" + b + " off=" + off +
-          " buffer=" + " offset=" + offset +
-          " len=" + len);
-    }
-    */
-    /* now we should have len < buffer.length */
-    System.arraycopy(b, off, buffer, offset, len);
-    offset += len;
-  }
-
-  /**
-   * Write bytes to the stream.
-   */
-  @Override
-  public void write(byte[] b, int off, int len) throws IOException {
-    checksum(b, off, len);
-    out.write(b,off,len);
-  }
-
-  @Override
-  public void write(int b) throws IOException {
-    barray[0] = (byte) (b & 0xFF);
-    write(barray,0,1);
-  }
-
-}


[05/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Shuffle.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Shuffle.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Shuffle.java
new file mode 100644
index 0000000..8689d11
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Shuffle.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.runtime.library.common.shuffle.impl;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.crypto.SecretKey;
+
+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.LocalDirAllocator;
+import org.apache.hadoop.security.UserGroupInformation;
+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.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+
+import com.google.common.base.Preconditions;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Shuffle implements ExceptionReporter {
+  
+  private static final Log LOG = LogFactory.getLog(Shuffle.class);
+  private static final int PROGRESS_FREQUENCY = 2000;
+  
+  private final Configuration conf;
+  private final TezInputContext inputContext;
+  private final ShuffleClientMetrics metrics;
+
+  private final ShuffleInputEventHandler eventHandler;
+  private final ShuffleScheduler scheduler;
+  private final MergeManager merger;
+  private Throwable throwable = null;
+  private String throwingThreadName = null;
+  private final int numInputs;
+  private final AtomicInteger reduceStartId;
+  private final SecretKey jobTokenSecret;
+  private AtomicInteger reduceRange = new AtomicInteger(
+      TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE_DEFAULT);
+
+  private FutureTask<TezRawKeyValueIterator> runShuffleFuture;
+
+  public Shuffle(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
+    this.inputContext = inputContext;
+    this.conf = conf;
+    this.metrics = new ShuffleClientMetrics(inputContext.getDAGName(),
+        inputContext.getTaskVertexName(), inputContext.getTaskIndex(),
+        this.conf, UserGroupInformation.getCurrentUser().getShortUserName());
+            
+    this.numInputs = numInputs;
+    
+    this.jobTokenSecret = ShuffleUtils
+        .getJobTokenSecretFromTokenBytes(inputContext
+            .getServiceConsumerMetaData(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID));
+    
+    Combiner combiner = TezRuntimeUtils.instantiateCombiner(conf, inputContext);
+    
+    FileSystem localFS = FileSystem.getLocal(this.conf);
+    LocalDirAllocator localDirAllocator = 
+        new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+
+    // TODO TEZ Get rid of Map / Reduce references.
+    TezCounter shuffledMapsCounter = 
+        inputContext.getCounters().findCounter(TaskCounter.SHUFFLED_MAPS);
+    TezCounter reduceShuffleBytes =
+        inputContext.getCounters().findCounter(TaskCounter.REDUCE_SHUFFLE_BYTES);
+    TezCounter failedShuffleCounter =
+        inputContext.getCounters().findCounter(TaskCounter.FAILED_SHUFFLE);
+    TezCounter spilledRecordsCounter = 
+        inputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
+    TezCounter reduceCombineInputCounter =
+        inputContext.getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
+    TezCounter mergedMapOutputsCounter =
+        inputContext.getCounters().findCounter(TaskCounter.MERGED_MAP_OUTPUTS);
+    
+    reduceStartId = new AtomicInteger(inputContext.getTaskIndex());
+    LOG.info("Shuffle assigned reduce start id: " + reduceStartId.get()
+        + " with default reduce range: " + reduceRange.get());
+
+    scheduler = new ShuffleScheduler(
+          this.inputContext,
+          this.conf,
+          this.numInputs,
+          this,
+          shuffledMapsCounter,
+          reduceShuffleBytes,
+          failedShuffleCounter);
+    eventHandler= new ShuffleInputEventHandler(
+          inputContext,
+          this,
+          scheduler);
+    merger = new MergeManager(
+          this.conf,
+          localFS,
+          localDirAllocator,
+          inputContext,
+          combiner,
+          spilledRecordsCounter,
+          reduceCombineInputCounter,
+          mergedMapOutputsCounter,
+          this);
+  }
+
+  public void handleEvents(List<Event> events) {
+    eventHandler.handleEvents(events);
+  }
+  
+  /**
+   * Indicates whether the Shuffle and Merge processing is complete.
+   * @return false if not complete, true if complete or if an error occurred.
+   */
+  public boolean isInputReady() {
+    if (runShuffleFuture == null) {
+      return false;
+    }
+    return runShuffleFuture.isDone();
+    //return scheduler.isDone() && merger.isMergeComplete();
+  }
+
+  /**
+   * Waits for the Shuffle and Merge to complete, and returns an iterator over the input.
+   * @return an iterator over the fetched input.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public TezRawKeyValueIterator waitForInput() throws IOException, InterruptedException {
+    Preconditions.checkState(runShuffleFuture != null,
+        "waitForInput can only be called after run");
+    TezRawKeyValueIterator kvIter;
+    try {
+      kvIter = runShuffleFuture.get();
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof IOException) {
+        throw (IOException) cause;
+      } else if (cause instanceof InterruptedException) {
+        throw (InterruptedException) cause;
+      } else {
+        throw new TezUncheckedException(
+            "Unexpected exception type while running Shuffle and Merge", cause);
+      }
+    }
+    return kvIter;
+  }
+
+  public void run() {
+    RunShuffleCallable runShuffle = new RunShuffleCallable();
+    runShuffleFuture = new FutureTask<TezRawKeyValueIterator>(runShuffle);
+    new Thread(runShuffleFuture, "ShuffleMergeRunner").start();
+  }
+  
+  private class RunShuffleCallable implements Callable<TezRawKeyValueIterator> {
+    @Override
+    public TezRawKeyValueIterator call() throws IOException, InterruptedException {
+      // TODO NEWTEZ Limit # fetchers to number of inputs
+      final int numFetchers = 
+          conf.getInt(
+              TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES, 
+              TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
+      Fetcher[] fetchers = new Fetcher[numFetchers];
+      for (int i = 0; i < numFetchers; ++i) {
+        fetchers[i] = new Fetcher(conf, scheduler, merger, metrics, Shuffle.this, jobTokenSecret, inputContext);
+        fetchers[i].start();
+      }
+      
+      while (!scheduler.waitUntilDone(PROGRESS_FREQUENCY)) {
+        synchronized (this) {
+          if (throwable != null) {
+            throw new ShuffleError("error in shuffle in " + throwingThreadName,
+                                   throwable);
+          }
+        }
+      }
+      
+      // Stop the map-output fetcher threads
+      for (Fetcher fetcher : fetchers) {
+        fetcher.shutDown();
+      }
+      fetchers = null;
+      
+      // stop the scheduler
+      scheduler.close();
+
+
+      // Finish the on-going merges...
+      TezRawKeyValueIterator kvIter = null;
+      try {
+        kvIter = merger.close();
+      } catch (Throwable e) {
+        throw new ShuffleError("Error while doing final merge " , e);
+      }
+      
+      // Sanity check
+      synchronized (Shuffle.this) {
+        if (throwable != null) {
+          throw new ShuffleError("error in shuffle in " + throwingThreadName,
+                                 throwable);
+        }
+      }
+      return kvIter;
+    }
+  }
+  
+  public int getReduceStartId() {
+    return reduceStartId.get();
+  }
+  
+  public int getReduceRange() {
+    return reduceRange.get();
+  }
+  
+  public synchronized void reportException(Throwable t) {
+    if (throwable == null) {
+      throwable = t;
+      throwingThreadName = Thread.currentThread().getName();
+      // Notify the scheduler so that the reporting thread finds the 
+      // exception immediately.
+      synchronized (scheduler) {
+        scheduler.notifyAll();
+      }
+    }
+  }
+  
+  public static class ShuffleError extends IOException {
+    private static final long serialVersionUID = 5753909320586607881L;
+
+    ShuffleError(String msg, Throwable t) {
+      super(msg, t);
+    }
+  }
+
+  public void setPartitionRange(int range) {
+    if (range == reduceRange.get()) {
+      return;
+    }
+    if (reduceRange.compareAndSet(
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE_DEFAULT, range)) {
+      LOG.info("Reduce range set to: " + range);
+    } else {
+      TezUncheckedException e = 
+          new TezUncheckedException("Reduce range can be set only once.");
+      reportException(e);
+      throw e; 
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleClientMetrics.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleClientMetrics.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleClientMetrics.java
new file mode 100644
index 0000000..70de31f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleClientMetrics.java
@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.common.shuffle.impl;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.metrics.MetricsContext;
+import org.apache.hadoop.metrics.MetricsRecord;
+import org.apache.hadoop.metrics.MetricsUtil;
+import org.apache.hadoop.metrics.Updater;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+
+class ShuffleClientMetrics implements Updater {
+
+  private MetricsRecord shuffleMetrics = null;
+  private int numFailedFetches = 0;
+  private int numSuccessFetches = 0;
+  private long numBytes = 0;
+  private int numThreadsBusy = 0;
+  private final int numCopiers;
+  
+  ShuffleClientMetrics(String dagName, String vertexName, int taskIndex, Configuration conf, 
+      String user) {
+    this.numCopiers = 
+        conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
+
+    MetricsContext metricsContext = MetricsUtil.getContext(Constants.TEZ);
+    this.shuffleMetrics = 
+      MetricsUtil.createRecord(metricsContext, "shuffleInput");
+    this.shuffleMetrics.setTag("user", user);
+    this.shuffleMetrics.setTag("dagName", dagName);
+    this.shuffleMetrics.setTag("taskId", TezRuntimeUtils.getTaskIdentifier(vertexName, taskIndex));
+    this.shuffleMetrics.setTag("sessionId", 
+        conf.get(
+            TezJobConfig.TEZ_RUNTIME_METRICS_SESSION_ID, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_METRICS_SESSION_ID));
+    metricsContext.registerUpdater(this);
+  }
+  public synchronized void inputBytes(long numBytes) {
+    this.numBytes += numBytes;
+  }
+  public synchronized void failedFetch() {
+    ++numFailedFetches;
+  }
+  public synchronized void successFetch() {
+    ++numSuccessFetches;
+  }
+  public synchronized void threadBusy() {
+    ++numThreadsBusy;
+  }
+  public synchronized void threadFree() {
+    --numThreadsBusy;
+  }
+  public void doUpdates(MetricsContext unused) {
+    synchronized (this) {
+      shuffleMetrics.incrMetric("shuffle_input_bytes", numBytes);
+      shuffleMetrics.incrMetric("shuffle_failed_fetches", 
+                                numFailedFetches);
+      shuffleMetrics.incrMetric("shuffle_success_fetches", 
+                                numSuccessFetches);
+      if (numCopiers != 0) {
+        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent",
+            100*((float)numThreadsBusy/numCopiers));
+      } else {
+        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent", 0);
+      }
+      numBytes = 0;
+      numSuccessFetches = 0;
+      numFailedFetches = 0;
+    }
+    shuffleMetrics.update();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleHeader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleHeader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleHeader.java
new file mode 100644
index 0000000..327473e
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleHeader.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.runtime.library.common.shuffle.impl;
+
+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.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * Shuffle Header information that is sent by the TaskTracker and 
+ * deciphered by the Fetcher thread of Reduce task
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Stable
+public class ShuffleHeader implements Writable {
+  
+  /** Header info of the shuffle http request/response */
+  public static final String HTTP_HEADER_NAME = "name";
+  public static final String DEFAULT_HTTP_HEADER_NAME = "mapreduce";
+  public static final String HTTP_HEADER_VERSION = "version";
+  public static final String DEFAULT_HTTP_HEADER_VERSION = "1.0.0";
+
+  /**
+   * The longest possible length of task attempt id that we will accept.
+   */
+  private static final int MAX_ID_LENGTH = 1000;
+
+  String mapId;
+  long uncompressedLength;
+  long compressedLength;
+  int forReduce;
+  
+  public ShuffleHeader() { }
+  
+  public ShuffleHeader(String mapId, long compressedLength,
+      long uncompressedLength, int forReduce) {
+    this.mapId = mapId;
+    this.compressedLength = compressedLength;
+    this.uncompressedLength = uncompressedLength;
+    this.forReduce = forReduce;
+  }
+  
+  public String getMapId() {
+    return this.mapId;
+  }
+  
+  public int getPartition() {
+    return this.forReduce;
+  }
+  
+  public long getUncompressedLength() {
+    return uncompressedLength;
+  }
+
+  public long getCompressedLength() {
+    return compressedLength;
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    mapId = WritableUtils.readStringSafely(in, MAX_ID_LENGTH);
+    compressedLength = WritableUtils.readVLong(in);
+    uncompressedLength = WritableUtils.readVLong(in);
+    forReduce = WritableUtils.readVInt(in);
+  }
+
+  public void write(DataOutput out) throws IOException {
+    Text.writeString(out, mapId);
+    WritableUtils.writeVLong(out, compressedLength);
+    WritableUtils.writeVLong(out, uncompressedLength);
+    WritableUtils.writeVInt(out, forReduce);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandler.java
new file mode 100644
index 0000000..8b323b5
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandler.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common.shuffle.impl;
+
+import java.net.URI;
+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.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.api.events.InputInformationEvent;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.InputInformationEventPayloadProto;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+
+public class ShuffleInputEventHandler {
+  
+  private static final Log LOG = LogFactory.getLog(ShuffleInputEventHandler.class);
+
+  private final ShuffleScheduler scheduler;
+  private final TezInputContext inputContext;
+  private final Shuffle shuffle;
+
+  private int maxMapRuntime = 0;
+  private boolean shuffleRangeSet = false;
+  
+  public ShuffleInputEventHandler(TezInputContext inputContext,
+      Shuffle shuffle, ShuffleScheduler scheduler) {
+    this.inputContext = inputContext;
+    this.shuffle = shuffle;
+    this.scheduler = scheduler;
+  }
+
+  public void handleEvents(List<Event> events) {
+    for (Event event : events) {
+      handleEvent(event);
+    }
+  }
+  
+  
+  private void handleEvent(Event event) {
+    if (event instanceof InputInformationEvent) {
+      processInputInformationEvent((InputInformationEvent) event);
+    }
+    else if (event instanceof DataMovementEvent) {
+      processDataMovementEvent((DataMovementEvent) event);      
+    } else if (event instanceof InputFailedEvent) {
+      processTaskFailedEvent((InputFailedEvent) event);
+    }
+  }
+
+  private void processInputInformationEvent(InputInformationEvent iiEvent) {
+    InputInformationEventPayloadProto inputInfoPayload;
+    try {
+      inputInfoPayload = InputInformationEventPayloadProto.parseFrom(iiEvent.getUserPayload());
+    } catch (InvalidProtocolBufferException e) {
+      throw new TezUncheckedException("Unable to parse InputInformationEvent payload", e);
+    }
+    int partitionRange = inputInfoPayload.getPartitionRange();
+    shuffle.setPartitionRange(partitionRange);
+    this.shuffleRangeSet = true;
+  }
+
+  private void processDataMovementEvent(DataMovementEvent dmEvent) {
+    // FIXME TODO NEWTEZ
+    // Preconditions.checkState(shuffleRangeSet == true, "Shuffle Range must be set before a DataMovementEvent is processed");
+    DataMovementEventPayloadProto shufflePayload;
+    try {
+      shufflePayload = DataMovementEventPayloadProto.parseFrom(dmEvent.getUserPayload());
+    } catch (InvalidProtocolBufferException e) {
+      throw new TezUncheckedException("Unable to parse DataMovementEvent payload", e);
+    } 
+    int partitionId = dmEvent.getSourceIndex();
+    URI baseUri = getBaseURI(shufflePayload.getHost(), shufflePayload.getPort(), partitionId);
+
+    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dmEvent.getTargetIndex(), dmEvent.getVersion(), shufflePayload.getPathComponent());
+    scheduler.addKnownMapOutput(shufflePayload.getHost(), partitionId, baseUri.toString(), srcAttemptIdentifier);
+    
+    // TODO NEWTEZ See if this duration hack can be removed.
+    int duration = shufflePayload.getRunDuration();
+    if (duration > maxMapRuntime) {
+      maxMapRuntime = duration;
+      scheduler.informMaxMapRunTime(maxMapRuntime);
+    }
+  }
+  
+  private void processTaskFailedEvent(InputFailedEvent ifEvent) {
+    InputAttemptIdentifier taIdentifier = new InputAttemptIdentifier(ifEvent.getSourceIndex(), ifEvent.getVersion());
+    scheduler.obsoleteMapOutput(taIdentifier);
+    LOG.info("Obsoleting output of src-task: " + taIdentifier);
+  }
+
+  // TODO NEWTEZ Handle encrypted shuffle
+  private URI getBaseURI(String host, int port, int partitionId) {
+    StringBuilder sb = new StringBuilder("http://");
+    sb.append(host);
+    sb.append(":");
+    sb.append(String.valueOf(port));
+    sb.append("/");
+    
+    sb.append("mapOutput?job=");
+    // Required to use the existing ShuffleHandler
+    sb.append(inputContext.getApplicationId().toString().replace("application", "job"));
+    
+    sb.append("&reduce=");
+    sb.append(partitionId);
+    sb.append("&map=");
+    URI u = URI.create(sb.toString());
+    return u;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleScheduler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleScheduler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleScheduler.java
new file mode 100644
index 0000000..a682a09
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleScheduler.java
@@ -0,0 +1,521 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.common.shuffle.impl;
+
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.lang.mutable.MutableInt;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+
+import com.google.common.collect.Lists;
+
+class ShuffleScheduler {
+  static ThreadLocal<Long> shuffleStart = new ThreadLocal<Long>() {
+    protected Long initialValue() {
+      return 0L;
+    }
+  };
+
+  private static final Log LOG = LogFactory.getLog(ShuffleScheduler.class);
+  private static final int MAX_MAPS_AT_ONCE = 20;
+  private static final long INITIAL_PENALTY = 10000;
+  private static final float PENALTY_GROWTH_RATE = 1.3f;
+  
+  // TODO NEWTEZ May need to be a string if attempting to fetch from multiple inputs.
+  private final Map<Integer, MutableInt> finishedMaps;
+  private final int numInputs;
+  private int remainingMaps;
+  private Map<InputAttemptIdentifier, MapHost> mapLocations = new HashMap<InputAttemptIdentifier, MapHost>();
+  //TODO NEWTEZ Clean this and other maps at some point
+  private ConcurrentMap<String, InputAttemptIdentifier> pathToIdentifierMap = new ConcurrentHashMap<String, InputAttemptIdentifier>(); 
+  private Set<MapHost> pendingHosts = new HashSet<MapHost>();
+  private Set<InputAttemptIdentifier> obsoleteMaps = new HashSet<InputAttemptIdentifier>();
+  
+  private final Random random = new Random(System.currentTimeMillis());
+  private final DelayQueue<Penalty> penalties = new DelayQueue<Penalty>();
+  private final Referee referee = new Referee();
+  private final Map<InputAttemptIdentifier, IntWritable> failureCounts =
+    new HashMap<InputAttemptIdentifier,IntWritable>(); 
+  private final Map<String,IntWritable> hostFailures = 
+    new HashMap<String,IntWritable>();
+  private final TezInputContext inputContext;
+  private final Shuffle shuffle;
+  private final int abortFailureLimit;
+  private final TezCounter shuffledMapsCounter;
+  private final TezCounter reduceShuffleBytes;
+  private final TezCounter failedShuffleCounter;
+  
+  private final long startTime;
+  private long lastProgressTime;
+  
+  private int maxMapRuntime = 0;
+  private int maxFailedUniqueFetches = 5;
+  private int maxFetchFailuresBeforeReporting;
+  
+  private long totalBytesShuffledTillNow = 0;
+  private DecimalFormat  mbpsFormat = new DecimalFormat("0.00");
+
+  private boolean reportReadErrorImmediately = true;
+  
+  public ShuffleScheduler(TezInputContext inputContext,
+                          Configuration conf,
+                          int tasksInDegree,
+                          Shuffle shuffle,
+                          TezCounter shuffledMapsCounter,
+                          TezCounter reduceShuffleBytes,
+                          TezCounter failedShuffleCounter) {
+    this.inputContext = inputContext;
+    this.numInputs = tasksInDegree;
+    abortFailureLimit = Math.max(30, tasksInDegree / 10);
+    remainingMaps = tasksInDegree;
+  //TODO NEWTEZ May need to be a string or a more usable construct if attempting to fetch from multiple inputs. Define a taskId / taskAttemptId pair
+    finishedMaps = new HashMap<Integer, MutableInt>(remainingMaps);
+    this.shuffle = shuffle;
+    this.shuffledMapsCounter = shuffledMapsCounter;
+    this.reduceShuffleBytes = reduceShuffleBytes;
+    this.failedShuffleCounter = failedShuffleCounter;
+    this.startTime = System.currentTimeMillis();
+    this.lastProgressTime = startTime;
+    referee.start();
+    this.maxFailedUniqueFetches = Math.min(tasksInDegree,
+        this.maxFailedUniqueFetches);
+    this.maxFetchFailuresBeforeReporting = 
+        conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT);
+    this.reportReadErrorImmediately = 
+        conf.getBoolean(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR);
+  }
+
+  public synchronized void copySucceeded(InputAttemptIdentifier srcAttemptIdentifier, 
+                                         MapHost host,
+                                         long bytes,
+                                         long milis,
+                                         MapOutput output
+                                         ) throws IOException {
+    String taskIdentifier = TezRuntimeUtils.getTaskAttemptIdentifier(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(), srcAttemptIdentifier.getAttemptNumber());
+    failureCounts.remove(taskIdentifier);
+    hostFailures.remove(host.getHostName());
+    
+    if (!isFinishedTaskTrue(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex())) {
+      output.commit();
+      if(incrementTaskCopyAndCheckCompletion(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex())) {
+        shuffledMapsCounter.increment(1);
+        if (--remainingMaps == 0) {
+          notifyAll();
+        }
+      }
+
+      // update the status
+      lastProgressTime = System.currentTimeMillis();
+      totalBytesShuffledTillNow += bytes;
+      logProgress();
+      reduceShuffleBytes.increment(bytes);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("src task: "
+            + TezRuntimeUtils.getTaskAttemptIdentifier(
+                inputContext.getSourceVertexName(), srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
+                srcAttemptIdentifier.getAttemptNumber()) + " done");
+      }
+    }
+    // TODO NEWTEZ Should this be releasing the output, if not committed ? Possible memory leak in case of speculation.
+  }
+
+  private void logProgress() {
+    float mbs = (float) totalBytesShuffledTillNow / (1024 * 1024);
+    int mapsDone = numInputs - remainingMaps;
+    long secsSinceStart = (System.currentTimeMillis() - startTime) / 1000 + 1;
+
+    float transferRate = mbs / secsSinceStart;
+    LOG.info("copy(" + mapsDone + " of " + numInputs + " at "
+        + mbpsFormat.format(transferRate) + " MB/s)");
+  }
+
+  public synchronized void copyFailed(InputAttemptIdentifier srcAttempt,
+                                      MapHost host,
+                                      boolean readError) {
+    host.penalize();
+    int failures = 1;
+    if (failureCounts.containsKey(srcAttempt)) {
+      IntWritable x = failureCounts.get(srcAttempt);
+      x.set(x.get() + 1);
+      failures = x.get();
+    } else {
+      failureCounts.put(srcAttempt, new IntWritable(1));      
+    }
+    String hostname = host.getHostName();
+    if (hostFailures.containsKey(hostname)) {
+      IntWritable x = hostFailures.get(hostname);
+      x.set(x.get() + 1);
+    } else {
+      hostFailures.put(hostname, new IntWritable(1));
+    }
+    if (failures >= abortFailureLimit) {
+      try {
+        throw new IOException(failures
+            + " failures downloading "
+            + TezRuntimeUtils.getTaskAttemptIdentifier(
+                inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
+                srcAttempt.getAttemptNumber()));
+      } catch (IOException ie) {
+        shuffle.reportException(ie);
+      }
+    }
+    
+    checkAndInformJobTracker(failures, srcAttempt, readError);
+
+    checkReducerHealth();
+    
+    long delay = (long) (INITIAL_PENALTY *
+        Math.pow(PENALTY_GROWTH_RATE, failures));
+    
+    penalties.add(new Penalty(host, delay));
+    
+    failedShuffleCounter.increment(1);
+  }
+  
+  // Notify the JobTracker  
+  // after every read error, if 'reportReadErrorImmediately' is true or
+  // after every 'maxFetchFailuresBeforeReporting' failures
+  private void checkAndInformJobTracker(
+      int failures, InputAttemptIdentifier srcAttempt, boolean readError) {
+    if ((reportReadErrorImmediately && readError)
+        || ((failures % maxFetchFailuresBeforeReporting) == 0)) {
+      LOG.info("Reporting fetch failure for "
+          + TezRuntimeUtils.getTaskAttemptIdentifier(
+              inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
+              srcAttempt.getAttemptNumber()) + " to jobtracker.");
+
+      List<Event> failedEvents = Lists.newArrayListWithCapacity(1);
+      failedEvents.add(new InputReadErrorEvent("Fetch failure for "
+          + TezRuntimeUtils.getTaskAttemptIdentifier(
+              inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
+              srcAttempt.getAttemptNumber()) + " to jobtracker.", srcAttempt.getInputIdentifier()
+          .getSrcTaskIndex(), srcAttempt.getAttemptNumber()));
+
+      inputContext.sendEvents(failedEvents);      
+      //status.addFailedDependency(mapId);
+    }
+  }
+    
+  private void checkReducerHealth() {
+    final float MAX_ALLOWED_FAILED_FETCH_ATTEMPT_PERCENT = 0.5f;
+    final float MIN_REQUIRED_PROGRESS_PERCENT = 0.5f;
+    final float MAX_ALLOWED_STALL_TIME_PERCENT = 0.5f;
+
+    long totalFailures = failedShuffleCounter.getValue();
+    int doneMaps = numInputs - remainingMaps;
+    
+    boolean reducerHealthy =
+      (((float)totalFailures / (totalFailures + doneMaps))
+          < MAX_ALLOWED_FAILED_FETCH_ATTEMPT_PERCENT);
+    
+    // check if the reducer has progressed enough
+    boolean reducerProgressedEnough =
+      (((float)doneMaps / numInputs)
+          >= MIN_REQUIRED_PROGRESS_PERCENT);
+
+    // check if the reducer is stalled for a long time
+    // duration for which the reducer is stalled
+    int stallDuration =
+      (int)(System.currentTimeMillis() - lastProgressTime);
+    
+    // duration for which the reducer ran with progress
+    int shuffleProgressDuration =
+      (int)(lastProgressTime - startTime);
+
+    // min time the reducer should run without getting killed
+    int minShuffleRunDuration =
+      (shuffleProgressDuration > maxMapRuntime)
+      ? shuffleProgressDuration
+          : maxMapRuntime;
+    
+    boolean reducerStalled =
+      (((float)stallDuration / minShuffleRunDuration)
+          >= MAX_ALLOWED_STALL_TIME_PERCENT);
+
+    // kill if not healthy and has insufficient progress
+    if ((failureCounts.size() >= maxFailedUniqueFetches ||
+        failureCounts.size() == (numInputs - doneMaps))
+        && !reducerHealthy
+        && (!reducerProgressedEnough || reducerStalled)) {
+      LOG.fatal("Shuffle failed with too many fetch failures " +
+      "and insufficient progress!");
+      String errorMsg = "Exceeded MAX_FAILED_UNIQUE_FETCHES; bailing-out.";
+      shuffle.reportException(new IOException(errorMsg));
+    }
+
+  }
+  
+  public synchronized void tipFailed(int srcTaskIndex) {
+    if (!isFinishedTaskTrue(srcTaskIndex)) {
+      setFinishedTaskTrue(srcTaskIndex);
+      if (--remainingMaps == 0) {
+        notifyAll();
+      }
+      logProgress();
+    }
+  }
+  
+  public synchronized void addKnownMapOutput(String hostName,
+                                             int partitionId,
+                                             String hostUrl,
+                                             InputAttemptIdentifier srcAttempt) {
+    String identifier = MapHost.createIdentifier(hostName, partitionId);
+    MapHost host = mapLocations.get(identifier);
+    if (host == null) {
+      host = new MapHost(partitionId, hostName, hostUrl);
+      assert identifier.equals(host.getIdentifier());
+      mapLocations.put(srcAttempt, host);
+    }
+    host.addKnownMap(srcAttempt);
+    pathToIdentifierMap.put(srcAttempt.getPathComponent(), srcAttempt);
+
+    // Mark the host as pending
+    if (host.getState() == MapHost.State.PENDING) {
+      pendingHosts.add(host);
+      notifyAll();
+    }
+  }
+  
+  public synchronized void obsoleteMapOutput(InputAttemptIdentifier srcAttempt) {
+    // The incoming srcAttempt does not contain a path component.
+    obsoleteMaps.add(srcAttempt);
+  }
+  
+  public synchronized void putBackKnownMapOutput(MapHost host,
+                                                 InputAttemptIdentifier srcAttempt) {
+    host.addKnownMap(srcAttempt);
+  }
+
+  public synchronized MapHost getHost() throws InterruptedException {
+      while(pendingHosts.isEmpty()) {
+        wait();
+      }
+      
+      MapHost host = null;
+      Iterator<MapHost> iter = pendingHosts.iterator();
+      int numToPick = random.nextInt(pendingHosts.size());
+      for (int i=0; i <= numToPick; ++i) {
+        host = iter.next();
+      }
+      
+      pendingHosts.remove(host);     
+      host.markBusy();
+      
+      LOG.info("Assigning " + host + " with " + host.getNumKnownMapOutputs() + 
+               " to " + Thread.currentThread().getName());
+      shuffleStart.set(System.currentTimeMillis());
+      
+      return host;
+  }
+  
+  public InputAttemptIdentifier getIdentifierForPathComponent(String pathComponent) {
+    return pathToIdentifierMap.get(pathComponent);
+  }
+  
+  public synchronized List<InputAttemptIdentifier> getMapsForHost(MapHost host) {
+    List<InputAttemptIdentifier> list = host.getAndClearKnownMaps();
+    Iterator<InputAttemptIdentifier> itr = list.iterator();
+    List<InputAttemptIdentifier> result = new ArrayList<InputAttemptIdentifier>();
+    int includedMaps = 0;
+    int totalSize = list.size();
+    // find the maps that we still need, up to the limit
+    while (itr.hasNext()) {
+      InputAttemptIdentifier id = itr.next();
+      if (!obsoleteMaps.contains(id) && !isFinishedTaskTrue(id.getInputIdentifier().getSrcTaskIndex())) {
+        result.add(id);
+        if (++includedMaps >= MAX_MAPS_AT_ONCE) {
+          break;
+        }
+      }
+    }
+    // put back the maps left after the limit
+    while (itr.hasNext()) {
+      InputAttemptIdentifier id = itr.next();
+      if (!obsoleteMaps.contains(id) && !isFinishedTaskTrue(id.getInputIdentifier().getSrcTaskIndex())) {
+        host.addKnownMap(id);
+      }
+    }
+    LOG.info("assigned " + includedMaps + " of " + totalSize + " to " +
+             host + " to " + Thread.currentThread().getName());
+    return result;
+  }
+
+  public synchronized void freeHost(MapHost host) {
+    if (host.getState() != MapHost.State.PENALIZED) {
+      if (host.markAvailable() == MapHost.State.PENDING) {
+        pendingHosts.add(host);
+        notifyAll();
+      }
+    }
+    LOG.info(host + " freed by " + Thread.currentThread().getName() + " in " + 
+             (System.currentTimeMillis()-shuffleStart.get()) + "s");
+  }
+    
+  public synchronized void resetKnownMaps() {
+    mapLocations.clear();
+    obsoleteMaps.clear();
+    pendingHosts.clear();
+    pathToIdentifierMap.clear();
+  }
+
+  /**
+   * Utility method to check if the Shuffle data fetch is complete.
+   * @return
+   */
+  public synchronized boolean isDone() {
+    return remainingMaps == 0;
+  }
+
+  /**
+   * Wait until the shuffle finishes or until the timeout.
+   * @param millis maximum wait time
+   * @return true if the shuffle is done
+   * @throws InterruptedException
+   */
+  public synchronized boolean waitUntilDone(int millis
+                                            ) throws InterruptedException {
+    if (remainingMaps > 0) {
+      wait(millis);
+      return remainingMaps == 0;
+    }
+    return true;
+  }
+  
+  /**
+   * A structure that records the penalty for a host.
+   */
+  private static class Penalty implements Delayed {
+    MapHost host;
+    private long endTime;
+    
+    Penalty(MapHost host, long delay) {
+      this.host = host;
+      this.endTime = System.currentTimeMillis() + delay;
+    }
+
+    public long getDelay(TimeUnit unit) {
+      long remainingTime = endTime - System.currentTimeMillis();
+      return unit.convert(remainingTime, TimeUnit.MILLISECONDS);
+    }
+
+    public int compareTo(Delayed o) {
+      long other = ((Penalty) o).endTime;
+      return endTime == other ? 0 : (endTime < other ? -1 : 1);
+    }
+    
+  }
+  
+  /**
+   * A thread that takes hosts off of the penalty list when the timer expires.
+   */
+  private class Referee extends Thread {
+    public Referee() {
+      setName("ShufflePenaltyReferee");
+      setDaemon(true);
+    }
+
+    public void run() {
+      try {
+        while (true) {
+          // take the first host that has an expired penalty
+          MapHost host = penalties.take().host;
+          synchronized (ShuffleScheduler.this) {
+            if (host.markAvailable() == MapHost.State.PENDING) {
+              pendingHosts.add(host);
+              ShuffleScheduler.this.notifyAll();
+            }
+          }
+        }
+      } catch (InterruptedException ie) {
+        return;
+      } catch (Throwable t) {
+        shuffle.reportException(t);
+      }
+    }
+  }
+  
+  public void close() throws InterruptedException {
+    referee.interrupt();
+    referee.join();
+  }
+
+  public synchronized void informMaxMapRunTime(int duration) {
+    if (duration > maxMapRuntime) {
+      maxMapRuntime = duration;
+    }
+  }
+  
+  void setFinishedTaskTrue(int srcTaskIndex) {
+    synchronized(finishedMaps) {
+      finishedMaps.put(srcTaskIndex, new MutableInt(shuffle.getReduceRange()));
+    }
+  }
+  
+  boolean incrementTaskCopyAndCheckCompletion(int srcTaskIndex) {
+    synchronized(finishedMaps) {
+      MutableInt result = finishedMaps.get(srcTaskIndex);
+      if(result == null) {
+        result = new MutableInt(0);
+        finishedMaps.put(srcTaskIndex, result);
+      }
+      result.increment();
+      return isFinishedTaskTrue(srcTaskIndex);
+    }
+  }
+  
+  boolean isFinishedTaskTrue(int srcTaskIndex) {
+    synchronized (finishedMaps) {
+      MutableInt result = finishedMaps.get(srcTaskIndex);
+      if(result == null) {
+        return false;
+      }
+      if (result.intValue() == shuffle.getReduceRange()) {
+        return true;
+      }
+      return false;      
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/server/ShuffleHandler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/server/ShuffleHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/server/ShuffleHandler.java
new file mode 100644
index 0000000..9a206c6
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/server/ShuffleHandler.java
@@ -0,0 +1,572 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.common.shuffle.server;
+
+import static org.jboss.netty.buffer.ChannelBuffers.wrappedBuffer;
+import static org.jboss.netty.handler.codec.http.HttpHeaders.Names.CONTENT_TYPE;
+import static org.jboss.netty.handler.codec.http.HttpMethod.GET;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.FORBIDDEN;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.INTERNAL_SERVER_ERROR;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.METHOD_NOT_ALLOWED;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.NOT_FOUND;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.jboss.netty.handler.codec.http.HttpResponseStatus.UNAUTHORIZED;
+import static org.jboss.netty.handler.codec.http.HttpVersion.HTTP_1_1;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+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.io.DataInputByteBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.annotation.Metric;
+import org.apache.hadoop.metrics2.annotation.Metrics;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.lib.MutableCounterInt;
+import org.apache.hadoop.metrics2.lib.MutableCounterLong;
+import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.security.ssl.SSLFactory;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+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.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.common.security.JobTokenIdentifier;
+import org.apache.tez.runtime.library.common.security.JobTokenSecretManager;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.ExternalSorter;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+import org.jboss.netty.bootstrap.ServerBootstrap;
+import org.jboss.netty.buffer.ChannelBuffers;
+import org.jboss.netty.channel.Channel;
+import org.jboss.netty.channel.ChannelFactory;
+import org.jboss.netty.channel.ChannelFuture;
+import org.jboss.netty.channel.ChannelFutureListener;
+import org.jboss.netty.channel.ChannelHandlerContext;
+import org.jboss.netty.channel.ChannelPipeline;
+import org.jboss.netty.channel.ChannelPipelineFactory;
+import org.jboss.netty.channel.Channels;
+import org.jboss.netty.channel.ExceptionEvent;
+import org.jboss.netty.channel.MessageEvent;
+import org.jboss.netty.channel.SimpleChannelUpstreamHandler;
+import org.jboss.netty.channel.group.ChannelGroup;
+import org.jboss.netty.channel.group.DefaultChannelGroup;
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory;
+import org.jboss.netty.handler.codec.frame.TooLongFrameException;
+import org.jboss.netty.handler.codec.http.DefaultHttpResponse;
+import org.jboss.netty.handler.codec.http.HttpChunkAggregator;
+import org.jboss.netty.handler.codec.http.HttpRequest;
+import org.jboss.netty.handler.codec.http.HttpRequestDecoder;
+import org.jboss.netty.handler.codec.http.HttpResponse;
+import org.jboss.netty.handler.codec.http.HttpResponseEncoder;
+import org.jboss.netty.handler.codec.http.HttpResponseStatus;
+import org.jboss.netty.handler.codec.http.QueryStringDecoder;
+import org.jboss.netty.handler.ssl.SslHandler;
+import org.jboss.netty.handler.stream.ChunkedStream;
+import org.jboss.netty.handler.stream.ChunkedWriteHandler;
+import org.jboss.netty.util.CharsetUtil;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public class ShuffleHandler extends AuxiliaryService {
+
+  private static final Log LOG = LogFactory.getLog(ShuffleHandler.class);
+  
+  public static final String SHUFFLE_MANAGE_OS_CACHE = "mapreduce.shuffle.manage.os.cache";
+  public static final boolean DEFAULT_SHUFFLE_MANAGE_OS_CACHE = true;
+
+  public static final String SHUFFLE_READAHEAD_BYTES = "mapreduce.shuffle.readahead.bytes";
+  public static final int DEFAULT_SHUFFLE_READAHEAD_BYTES = 4 * 1024 * 1024;
+
+  private int port;
+  private ChannelFactory selector;
+  private final ChannelGroup accepted = new DefaultChannelGroup();
+  private HttpPipelineFactory pipelineFact;
+  private int sslFileBufferSize;
+
+  public static final String MAPREDUCE_SHUFFLE_SERVICEID =
+      "mapreduce.shuffle";
+
+  private static final Map<String,String> userRsrc =
+    new ConcurrentHashMap<String,String>();
+  private static final JobTokenSecretManager secretManager =
+    new JobTokenSecretManager();
+  private SecretKey tokenSecret;
+
+  public static final String SHUFFLE_PORT_CONFIG_KEY = "mapreduce.shuffle.port";
+  public static final int DEFAULT_SHUFFLE_PORT = 8080;
+
+  public static final String SUFFLE_SSL_FILE_BUFFER_SIZE_KEY =
+    "mapreduce.shuffle.ssl.file.buffer.size";
+
+  public static final int DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE = 60 * 1024;
+
+  private ExternalSorter sorter;
+  
+  @Metrics(about="Shuffle output metrics", context="mapred")
+  static class ShuffleMetrics implements ChannelFutureListener {
+    @Metric("Shuffle output in bytes")
+        MutableCounterLong shuffleOutputBytes;
+    @Metric("# of failed shuffle outputs")
+        MutableCounterInt shuffleOutputsFailed;
+    @Metric("# of succeeeded shuffle outputs")
+        MutableCounterInt shuffleOutputsOK;
+    @Metric("# of current shuffle connections")
+        MutableGaugeInt shuffleConnections;
+
+    @Override
+    public void operationComplete(ChannelFuture future) throws Exception {
+      if (future.isSuccess()) {
+        shuffleOutputsOK.incr();
+      } else {
+        shuffleOutputsFailed.incr();
+      }
+      shuffleConnections.decr();
+    }
+  }
+
+  final ShuffleMetrics metrics;
+
+  ShuffleHandler(MetricsSystem ms) {
+    super("httpshuffle");
+    metrics = ms.register(new ShuffleMetrics());
+  }
+
+  public ShuffleHandler(ExternalSorter sorter) {
+    this(DefaultMetricsSystem.instance());
+    this.sorter = sorter;
+  }
+
+  /**
+   * Serialize the shuffle port into a ByteBuffer for use later on.
+   * @param port the port to be sent to the ApplciationMaster
+   * @return the serialized form of the port.
+   */
+  public static ByteBuffer serializeMetaData(int port) throws IOException {
+    //TODO these bytes should be versioned
+    DataOutputBuffer port_dob = new DataOutputBuffer();
+    port_dob.writeInt(port);
+    return ByteBuffer.wrap(port_dob.getData(), 0, port_dob.getLength());
+  }
+
+  /**
+   * A helper function to deserialize the metadata returned by ShuffleHandler.
+   * @param meta the metadata returned by the ShuffleHandler
+   * @return the port the Shuffle Handler is listening on to serve shuffle data.
+   */
+  public static int deserializeMetaData(ByteBuffer meta) throws IOException {
+    //TODO this should be returning a class not just an int
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    in.reset(meta);
+    int port = in.readInt();
+    return port;
+  }
+
+  /**
+   * A helper function to serialize the JobTokenIdentifier to be sent to the
+   * ShuffleHandler as ServiceData.
+   * @param jobToken the job token to be used for authentication of
+   * shuffle data requests.
+   * @return the serialized version of the jobToken.
+   */
+  public static ByteBuffer serializeServiceData(Token<JobTokenIdentifier> jobToken) throws IOException {
+    //TODO these bytes should be versioned
+    DataOutputBuffer jobToken_dob = new DataOutputBuffer();
+    jobToken.write(jobToken_dob);
+    return ByteBuffer.wrap(jobToken_dob.getData(), 0, jobToken_dob.getLength());
+  }
+
+  static Token<JobTokenIdentifier> deserializeServiceData(ByteBuffer secret) throws IOException {
+    DataInputByteBuffer in = new DataInputByteBuffer();
+    in.reset(secret);
+    Token<JobTokenIdentifier> jt = new Token<JobTokenIdentifier>();
+    jt.readFields(in);
+    return jt;
+  }
+
+  
+  @Override
+  public void initializeApplication(
+      ApplicationInitializationContext initAppContext) {
+    // TODO these bytes should be versioned
+    try {
+      String user = initAppContext.getUser();
+      ApplicationId appId = initAppContext.getApplicationId();
+      ByteBuffer secret = initAppContext.getApplicationDataForService();
+      Token<JobTokenIdentifier> jt = deserializeServiceData(secret);
+      // TODO: Once SHuffle is out of NM, this can use MR APIs
+      userRsrc.put(appId.toString(), user);
+      LOG.info("Added token for " + appId.toString());
+      secretManager.addTokenForJob(appId.toString(), jt);
+    } catch (IOException e) {
+      LOG.error("Error during initApp", e);
+      // TODO add API to AuxiliaryServices to report failures
+    }
+  }
+
+  @Override
+  public void stopApplication(ApplicationTerminationContext context) {
+    ApplicationId appId = context.getApplicationId();
+    secretManager.removeTokenForJob(appId.toString());
+    userRsrc.remove(appId.toString());
+  }
+
+  public void initialize(TezOutputContext outputContext, Configuration conf) throws IOException {
+    this.init(new Configuration(conf));
+    tokenSecret = ShuffleUtils.getJobTokenSecretFromTokenBytes(outputContext.getServiceConsumerMetaData(MAPREDUCE_SHUFFLE_SERVICEID));
+  }
+
+  @Override
+  public synchronized void serviceInit(Configuration conf) {
+    ThreadFactory bossFactory = new ThreadFactoryBuilder()
+      .setNameFormat("ShuffleHandler Netty Boss #%d")
+      .build();
+    ThreadFactory workerFactory = new ThreadFactoryBuilder()
+      .setNameFormat("ShuffleHandler Netty Worker #%d")
+      .build();
+    
+    selector = new NioServerSocketChannelFactory(
+        Executors.newCachedThreadPool(bossFactory),
+        Executors.newCachedThreadPool(workerFactory));    
+  }
+
+  // TODO change AbstractService to throw InterruptedException
+  @Override
+  public synchronized void serviceStart() {
+    Configuration conf = getConfig();
+    ServerBootstrap bootstrap = new ServerBootstrap(selector);
+    try {
+      pipelineFact = new HttpPipelineFactory(conf);
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+    bootstrap.setPipelineFactory(pipelineFact);
+    // Let OS pick the port
+    Channel ch = bootstrap.bind(new InetSocketAddress(0));
+    accepted.add(ch);
+    port = ((InetSocketAddress)ch.getLocalAddress()).getPort();
+    conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port));
+    pipelineFact.SHUFFLE.setPort(port);
+    LOG.info(getName() + " listening on port " + port);
+
+    sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
+                                    DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
+  }
+
+  @Override
+  public synchronized void serviceStop() {
+    accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
+    ServerBootstrap bootstrap = new ServerBootstrap(selector);
+    bootstrap.releaseExternalResources();
+    pipelineFact.destroy();
+  }
+
+  @Override
+  public synchronized ByteBuffer getMetaData() {
+    try {
+      return serializeMetaData(port); 
+    } catch (IOException e) {
+      LOG.error("Error during getMeta", e);
+      // TODO add API to AuxiliaryServices to report failures
+      return null;
+    }
+  }
+
+  class HttpPipelineFactory implements ChannelPipelineFactory {
+
+    final Shuffle SHUFFLE;
+    private SSLFactory sslFactory;
+
+    public HttpPipelineFactory(Configuration conf) throws Exception {
+      SHUFFLE = new Shuffle(conf);
+      if (conf.getBoolean(TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL,
+              TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL)) {
+        sslFactory = new SSLFactory(SSLFactory.Mode.SERVER, conf);
+        sslFactory.init();
+      }
+    }
+
+    public void destroy() {
+      if (sslFactory != null) {
+        sslFactory.destroy();
+      }
+    }
+
+    @Override
+    public ChannelPipeline getPipeline() throws Exception {
+      ChannelPipeline pipeline = Channels.pipeline();
+      if (sslFactory != null) {
+        pipeline.addLast("ssl", new SslHandler(sslFactory.createSSLEngine()));
+      }
+      pipeline.addLast("decoder", new HttpRequestDecoder());
+      pipeline.addLast("aggregator", new HttpChunkAggregator(1 << 16));
+      pipeline.addLast("encoder", new HttpResponseEncoder());
+      pipeline.addLast("chunking", new ChunkedWriteHandler());
+      pipeline.addLast("shuffle", SHUFFLE);
+      return pipeline;
+      // TODO factor security manager into pipeline
+      // TODO factor out encode/decode to permit binary shuffle
+      // TODO factor out decode of index to permit alt. models
+    }
+
+  }
+
+  class Shuffle extends SimpleChannelUpstreamHandler {
+
+    private final Configuration conf;
+    private int port;
+
+    public Shuffle(Configuration conf) {
+      this.conf = conf;
+      this.port = conf.getInt(SHUFFLE_PORT_CONFIG_KEY, DEFAULT_SHUFFLE_PORT);
+    }
+    
+    public void setPort(int port) {
+      this.port = port;
+    }
+
+    private List<String> splitMaps(List<String> mapq) {
+      if (null == mapq) {
+        return null;
+      }
+      final List<String> ret = new ArrayList<String>();
+      for (String s : mapq) {
+        Collections.addAll(ret, s.split(","));
+      }
+      return ret;
+    }
+
+    @Override
+    public void messageReceived(ChannelHandlerContext ctx, MessageEvent evt)
+        throws Exception {
+      HttpRequest request = (HttpRequest) evt.getMessage();
+      if (request.getMethod() != GET) {
+          sendError(ctx, METHOD_NOT_ALLOWED);
+          return;
+      }
+      // Check whether the shuffle version is compatible
+      if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(
+          request.getHeader(ShuffleHeader.HTTP_HEADER_NAME))
+          || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(
+              request.getHeader(ShuffleHeader.HTTP_HEADER_VERSION))) {
+        sendError(ctx, "Incompatible shuffle request version", BAD_REQUEST);
+      }
+      final Map<String,List<String>> q =
+        new QueryStringDecoder(request.getUri()).getParameters();
+      final List<String> mapIds = splitMaps(q.get("map"));
+      final List<String> reduceQ = q.get("reduce");
+      final List<String> jobQ = q.get("job");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RECV: " + request.getUri() +
+            "\n  mapId: " + mapIds +
+            "\n  reduceId: " + reduceQ +
+            "\n  jobId: " + jobQ);
+      }
+
+      if (mapIds == null || reduceQ == null || jobQ == null) {
+        sendError(ctx, "Required param job, map and reduce", BAD_REQUEST);
+        return;
+      }
+      if (reduceQ.size() != 1 || jobQ.size() != 1) {
+        sendError(ctx, "Too many job/reduce parameters", BAD_REQUEST);
+        return;
+      }
+      int reduceId;
+      String jobId;
+      try {
+        reduceId = Integer.parseInt(reduceQ.get(0));
+        jobId = jobQ.get(0);
+      } catch (NumberFormatException e) {
+        sendError(ctx, "Bad reduce parameter", BAD_REQUEST);
+        return;
+      } catch (IllegalArgumentException e) {
+        sendError(ctx, "Bad job parameter", BAD_REQUEST);
+        return;
+      }
+
+      final String reqUri = request.getUri();
+      if (null == reqUri) {
+        // TODO? add upstream?
+        sendError(ctx, FORBIDDEN);
+        return;
+      }
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, OK);
+      try {
+        verifyRequest(jobId, ctx, request, response,
+            new URL("http", "", this.port, reqUri));
+      } catch (IOException e) {
+        LOG.warn("Shuffle failure ", e);
+        sendError(ctx, e.getMessage(), UNAUTHORIZED);
+        return;
+      }
+
+      Channel ch = evt.getChannel();
+      ch.write(response);
+      // TODO refactor the following into the pipeline
+      ChannelFuture lastMap = null;
+      for (String mapId : mapIds) {
+        try {
+          // TODO: Error handling - validate mapId via TezTaskAttemptId.forName
+          
+          // TODO NEWTEZ Fix this. TaskAttemptId is no longer valid. mapId validation will not work anymore.
+//          if (!mapId.equals(sorter.getTaskAttemptId().toString())) {
+//            String errorMessage =
+//                "Illegal shuffle request mapId: " + mapId
+//                    + " while actual mapId is " + sorter.getTaskAttemptId(); 
+//            LOG.warn(errorMessage);
+//            sendError(ctx, errorMessage, BAD_REQUEST);
+//            return;
+//          }
+
+          lastMap =
+            sendMapOutput(ctx, ch, userRsrc.get(jobId), jobId, mapId, reduceId);
+          if (null == lastMap) {
+            sendError(ctx, NOT_FOUND);
+            return;
+          }
+        } catch (IOException e) {
+          LOG.error("Shuffle error ", e);
+          sendError(ctx, e.getMessage(), INTERNAL_SERVER_ERROR);
+          return;
+        }
+      }
+      lastMap.addListener(metrics);
+      lastMap.addListener(ChannelFutureListener.CLOSE);
+    }
+
+    private void verifyRequest(String appid, ChannelHandlerContext ctx,
+        HttpRequest request, HttpResponse response, URL requestUri)
+        throws IOException {
+      if (null == tokenSecret) {
+        LOG.info("Request for unknown token " + appid);
+        throw new IOException("could not find jobid");
+      }
+      // string to encrypt
+      String enc_str = SecureShuffleUtils.buildMsgFrom(requestUri);
+      // hash from the fetcher
+      String urlHashStr =
+        request.getHeader(SecureShuffleUtils.HTTP_HEADER_URL_HASH);
+      if (urlHashStr == null) {
+        LOG.info("Missing header hash for " + appid);
+        throw new IOException("fetcher cannot be authenticated");
+      }
+      if (LOG.isDebugEnabled()) {
+        int len = urlHashStr.length();
+        LOG.debug("verifying request. enc_str=" + enc_str + "; hash=..." +
+            urlHashStr.substring(len-len/2, len-1));
+      }
+      // verify - throws exception
+      SecureShuffleUtils.verifyReply(urlHashStr, enc_str, tokenSecret);
+      // verification passed - encode the reply
+      String reply =
+        SecureShuffleUtils.generateHash(urlHashStr.getBytes(), tokenSecret);
+      response.setHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply);
+      addVersionToHeader(response);
+      if (LOG.isDebugEnabled()) {
+        int len = reply.length();
+        LOG.debug("Fetcher request verfied. enc_str=" + enc_str + ";reply=" +
+            reply.substring(len-len/2, len-1));
+      }
+    }
+
+    protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch,
+        String user, String jobId, String mapId, int reduce)
+        throws IOException {
+      final ShuffleHeader header = sorter.getShuffleHeader(reduce);
+      final DataOutputBuffer dob = new DataOutputBuffer();
+      header.write(dob);
+      ch.write(wrappedBuffer(dob.getData(), 0, dob.getLength()));
+
+      ChannelFuture writeFuture =
+          ch.write(
+              new ChunkedStream(
+                  sorter.getSortedStream(reduce), sslFileBufferSize
+                  )
+              );
+      metrics.shuffleConnections.incr();
+      metrics.shuffleOutputBytes.incr(header.getCompressedLength()); // optimistic
+      return writeFuture;
+    }
+
+    private void sendError(ChannelHandlerContext ctx,
+        HttpResponseStatus status) {
+      sendError(ctx, "", status);
+    }
+
+    private void sendError(ChannelHandlerContext ctx, String message,
+        HttpResponseStatus status) {
+      HttpResponse response = new DefaultHttpResponse(HTTP_1_1, status);
+      response.setHeader(CONTENT_TYPE, "text/plain; charset=UTF-8");
+      addVersionToHeader(response);
+      response.setContent(
+        ChannelBuffers.copiedBuffer(message, CharsetUtil.UTF_8));
+      // Close the connection as soon as the error message is sent.
+      ctx.getChannel().write(response).addListener(ChannelFutureListener.CLOSE);
+    }
+    
+    private void addVersionToHeader(HttpResponse response) {
+      // Put shuffle version into http header
+      response.setHeader(ShuffleHeader.HTTP_HEADER_NAME,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+      response.setHeader(ShuffleHeader.HTTP_HEADER_VERSION,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);      
+    }
+
+    @Override
+    public void exceptionCaught(ChannelHandlerContext ctx, ExceptionEvent e)
+        throws Exception {
+      Channel ch = e.getChannel();
+      Throwable cause = e.getCause();
+      if (cause instanceof TooLongFrameException) {
+        sendError(ctx, BAD_REQUEST);
+        return;
+      }
+
+      LOG.error("Shuffle error: ", cause);
+      if (ch.isConnected()) {
+        LOG.error("Shuffle error " + e);
+        sendError(ctx, INTERNAL_SERVER_ERROR);
+      }
+    }
+
+  }
+
+  public int getPort() {
+    return port;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
new file mode 100644
index 0000000..c362d98
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/sort/impl/ExternalSorter.java
@@ -0,0 +1,194 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.tez.runtime.library.common.sort.impl;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+
+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.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.io.serializer.SerializationFactory;
+import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.hadoop.util.IndexedSorter;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.QuickSort;
+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.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.library.api.Partitioner;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleHeader;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutput;
+import org.apache.tez.runtime.library.hadoop.compat.NullProgressable;
+
+@SuppressWarnings({"unchecked", "rawtypes"})
+public abstract class ExternalSorter {
+
+  private static final Log LOG = LogFactory.getLog(ExternalSorter.class);
+
+  public abstract void close() throws IOException;
+
+  public abstract void flush() throws IOException;
+
+  public abstract void write(Object key, Object value) throws IOException;
+
+  protected Progressable nullProgressable = new NullProgressable();
+  protected TezOutputContext outputContext;
+  protected Combiner combiner;
+  protected Partitioner partitioner;
+  protected Configuration conf;
+  protected FileSystem rfs;
+  protected TezTaskOutput mapOutputFile;
+  protected int partitions;
+  protected Class keyClass;
+  protected Class valClass;
+  protected RawComparator comparator;
+  protected SerializationFactory serializationFactory;
+  protected Serializer keySerializer;
+  protected Serializer valSerializer;
+
+  protected IndexedSorter sorter;
+
+  // Compression for map-outputs
+  protected CompressionCodec codec;
+
+  // Counters
+  // TODO TEZ Rename all counter variables [Mapping of counter to MR for compatibility in the MR layer]
+  protected TezCounter mapOutputByteCounter;
+  protected TezCounter mapOutputRecordCounter;
+  protected TezCounter fileOutputByteCounter;
+  protected TezCounter spilledRecordsCounter;
+
+  public void initialize(TezOutputContext outputContext, Configuration conf, int numOutputs) throws IOException {
+    this.outputContext = outputContext;
+    this.conf = conf;
+    this.partitions = numOutputs;
+
+    rfs = ((LocalFileSystem)FileSystem.getLocal(this.conf)).getRaw();
+
+    // sorter
+    sorter = ReflectionUtils.newInstance(this.conf.getClass(
+        TezJobConfig.TEZ_RUNTIME_INTERNAL_SORTER_CLASS, QuickSort.class,
+        IndexedSorter.class), this.conf);
+
+    comparator = ConfigUtils.getIntermediateOutputKeyComparator(this.conf);
+
+    // k/v serialization
+    keyClass = ConfigUtils.getIntermediateOutputKeyClass(this.conf);
+    valClass = ConfigUtils.getIntermediateOutputValueClass(this.conf);
+    serializationFactory = new SerializationFactory(this.conf);
+    keySerializer = serializationFactory.getSerializer(keyClass);
+    valSerializer = serializationFactory.getSerializer(valClass);
+
+    //    counters
+    mapOutputByteCounter =
+        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_BYTES);
+    mapOutputRecordCounter =
+        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS);
+    fileOutputByteCounter =
+        outputContext.getCounters().findCounter(TaskCounter.MAP_OUTPUT_MATERIALIZED_BYTES);
+    spilledRecordsCounter =
+        outputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
+    // compression
+    if (ConfigUtils.shouldCompressIntermediateOutput(this.conf)) {
+      Class<? extends CompressionCodec> codecClass =
+          ConfigUtils.getIntermediateOutputCompressorClass(this.conf, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, this.conf);
+    } else {
+      codec = null;
+    }
+
+    // Task outputs
+    mapOutputFile = TezRuntimeUtils.instantiateTaskOutputManager(conf, outputContext);
+    
+    LOG.info("Instantiating Partitioner: [" + conf.get(TezJobConfig.TEZ_RUNTIME_PARTITIONER_CLASS) + "]");
+    this.conf.setInt(TezJobConfig.TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS, this.partitions);
+    this.partitioner = TezRuntimeUtils.instantiatePartitioner(this.conf);
+    this.combiner = TezRuntimeUtils.instantiateCombiner(this.conf, outputContext);
+  }
+
+  /**
+   * Exception indicating that the allocated sort buffer is insufficient to hold
+   * the current record.
+   */
+  @SuppressWarnings("serial")
+  public static class MapBufferTooSmallException extends IOException {
+    public MapBufferTooSmallException(String s) {
+      super(s);
+    }
+  }
+
+  @Private
+  public TezTaskOutput getMapOutput() {
+    return mapOutputFile;
+  }
+
+  protected void runCombineProcessor(TezRawKeyValueIterator kvIter,
+      Writer writer) throws IOException {
+    try {
+      combiner.combine(kvIter, writer);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Rename srcPath to dstPath on the same volume. This is the same as
+   * RawLocalFileSystem's rename method, except that it will not fall back to a
+   * copy, and it will create the target directory if it doesn't exist.
+   */
+  protected void sameVolRename(Path srcPath, Path dstPath) throws IOException {
+    RawLocalFileSystem rfs = (RawLocalFileSystem) this.rfs;
+    File src = rfs.pathToFile(srcPath);
+    File dst = rfs.pathToFile(dstPath);
+    if (!dst.getParentFile().exists()) {
+      if (!dst.getParentFile().mkdirs()) {
+        throw new IOException("Unable to rename " + src + " to " + dst
+            + ": couldn't create parent directory");
+      }
+    }
+
+    if (!src.renameTo(dst)) {
+      throw new IOException("Unable to rename " + src + " to " + dst);
+    }
+  }
+
+  public InputStream getSortedStream(int partition) {
+    throw new UnsupportedOperationException("getSortedStream isn't supported!");
+  }
+
+  public ShuffleHeader getShuffleHeader(int reduce) {
+    throw new UnsupportedOperationException("getShuffleHeader isn't supported!");
+  }
+}


[20/20] git commit: Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2:
  - tez-engine-library for user-visible Input/Output/Processor implementations
  - tez-engine-internals for framework internals


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

Branch: refs/heads/TEZ-398
Commit: b212ca1d2a087e80329511236022bf297919b960
Parents: 5d86b93
Author: Hitesh Shah <hi...@apache.org>
Authored: Tue Sep 24 15:42:53 2013 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Tue Sep 24 15:42:53 2013 -0700

----------------------------------------------------------------------
 pom.xml                                         |   10 +-
 .../org/apache/tez/common/TezJobConfig.java     |  213 ++--
 .../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 -
 .../java/org/apache/tez/runtime/api/Event.java  |   28 +
 .../java/org/apache/tez/runtime/api/Input.java  |   71 ++
 .../tez/runtime/api/LogicalIOProcessor.java     |   43 +
 .../apache/tez/runtime/api/LogicalInput.java    |   37 +
 .../apache/tez/runtime/api/LogicalOutput.java   |   36 +
 .../java/org/apache/tez/runtime/api/Output.java |   71 ++
 .../org/apache/tez/runtime/api/Processor.java   |   55 +
 .../java/org/apache/tez/runtime/api/Reader.java |   26 +
 .../apache/tez/runtime/api/TezInputContext.java |   32 +
 .../tez/runtime/api/TezOutputContext.java       |   33 +
 .../tez/runtime/api/TezProcessorContext.java    |   41 +
 .../apache/tez/runtime/api/TezTaskContext.java  |  130 ++
 .../java/org/apache/tez/runtime/api/Writer.java |   26 +
 .../runtime/api/events/DataMovementEvent.java   |  109 ++
 .../runtime/api/events/InputFailedEvent.java    |   89 ++
 .../api/events/InputInformationEvent.java       |   41 +
 .../runtime/api/events/InputReadErrorEvent.java |   65 +
 .../common/objectregistry/ObjectLifeCycle.java  |   37 +
 .../common/objectregistry/ObjectRegistry.java   |   56 +
 .../objectregistry/ObjectRegistryFactory.java   |   32 +
 tez-api/src/main/proto/Events.proto             |    2 +-
 tez-dag/pom.xml                                 |    2 +-
 .../apache/hadoop/mapred/YarnTezDagChild.java   |   36 +-
 .../org/apache/tez/dag/app/DAGAppMaster.java    |    2 +-
 .../dag/app/TaskAttemptListenerImpTezDag.java   |    8 +-
 .../org/apache/tez/dag/app/dag/EdgeManager.java |    6 +-
 .../java/org/apache/tez/dag/app/dag/Task.java   |    2 +-
 .../java/org/apache/tez/dag/app/dag/Vertex.java |    6 +-
 .../apache/tez/dag/app/dag/VertexScheduler.java |    2 +-
 .../dag/event/TaskAttemptEventStatusUpdate.java |    2 +-
 .../dag/app/dag/event/TaskEventAddTezEvent.java |    2 +-
 .../app/dag/event/VertexEventRouteEvent.java    |    2 +-
 .../VertexEventSourceTaskAttemptCompleted.java  |    2 +-
 .../event/VertexEventTaskAttemptCompleted.java  |    2 +-
 .../dag/app/dag/impl/BroadcastEdgeManager.java  |    6 +-
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |    6 +-
 .../org/apache/tez/dag/app/dag/impl/Edge.java   |   16 +-
 .../dag/impl/ImmediateStartVertexScheduler.java |    2 +-
 .../dag/app/dag/impl/OneToOneEdgeManager.java   |    6 +-
 .../app/dag/impl/ScatterGatherEdgeManager.java  |    6 +-
 .../dag/app/dag/impl/ShuffleVertexManager.java  |   12 +-
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |    4 +-
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   |    4 +-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |   18 +-
 .../app/rm/AMSchedulerEventTALaunchRequest.java |    2 +-
 .../rm/container/AMContainerEventAssignTA.java  |    2 +-
 .../app/rm/container/AMContainerHelpers.java    |    8 +-
 .../dag/app/rm/container/AMContainerImpl.java   |    2 +-
 .../dag/app/rm/container/AMContainerTask.java   |    2 +-
 .../apache/tez/dag/utils/TezEngineChildJVM.java |  122 --
 .../tez/dag/utils/TezRuntimeChildJVM.java       |  122 ++
 .../TezDependentTaskCompletionEvent.java        |  228 ----
 ...TezTaskDependencyCompletionEventsUpdate.java |   64 -
 .../TezDependentTaskCompletionEvent.java        |  228 ++++
 ...TezTaskDependencyCompletionEventsUpdate.java |   64 +
 .../tez/dag/app/dag/impl/TestDAGImpl.java       |    2 +-
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |    2 +-
 .../tez/dag/app/dag/impl/TestVertexImpl.java    |    4 +-
 .../dag/app/dag/impl/TestVertexScheduler.java   |    2 +-
 .../tez/dag/app/rm/TestContainerReuse.java      |    6 +-
 .../dag/app/rm/container/TestAMContainer.java   |    6 +-
 tez-engine/findbugs-exclude.xml                 |   16 -
 tez-engine/pom.xml                              |   92 --
 .../java/org/apache/hadoop/io/BufferUtils.java  |   81 --
 .../org/apache/hadoop/io/HashComparator.java    |   24 -
 .../java/org/apache/tez/common/Constants.java   |   57 -
 .../org/apache/tez/common/ContainerContext.java |   64 -
 .../org/apache/tez/common/ContainerTask.java    |   74 --
 .../tez/common/TezTaskUmbilicalProtocol.java    |   57 -
 .../org/apache/tez/engine/api/KVReader.java     |   81 --
 .../org/apache/tez/engine/api/KVWriter.java     |   40 -
 .../org/apache/tez/engine/api/Partitioner.java  |   52 -
 .../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  |  138 ---
 .../broadcast/input/BroadcastKVReader.java      |  225 ----
 .../BroadcastShuffleInputEventHandler.java      |   88 --
 .../input/BroadcastShuffleManager.java          |  489 --------
 .../broadcast/output/FileBasedKVWriter.java     |  125 --
 .../apache/tez/engine/common/ConfigUtils.java   |  148 ---
 .../engine/common/InputAttemptIdentifier.java   |   95 --
 .../tez/engine/common/InputIdentifier.java      |   56 -
 .../tez/engine/common/TezEngineUtils.java       |  153 ---
 .../tez/engine/common/ValuesIterator.java       |  194 ---
 .../apache/tez/engine/common/YARNMaster.java    |   57 -
 .../tez/engine/common/combine/Combiner.java     |   42 -
 .../common/localshuffle/LocalShuffle.java       |  120 --
 .../objectregistry/ObjectRegistryImpl.java      |   65 -
 .../objectregistry/ObjectRegistryModule.java    |   43 -
 .../common/security/JobTokenIdentifier.java     |   98 --
 .../common/security/JobTokenSecretManager.java  |  137 ---
 .../common/security/JobTokenSelector.java       |   53 -
 .../tez/engine/common/security/Master.java      |   57 -
 .../common/security/SecureShuffleUtils.java     |  142 ---
 .../tez/engine/common/security/TokenCache.java  |  205 ----
 .../common/shuffle/impl/ExceptionReporter.java  |   25 -
 .../tez/engine/common/shuffle/impl/Fetcher.java |  624 ----------
 .../common/shuffle/impl/InMemoryReader.java     |  156 ---
 .../common/shuffle/impl/InMemoryWriter.java     |  100 --
 .../tez/engine/common/shuffle/impl/MapHost.java |  124 --
 .../engine/common/shuffle/impl/MapOutput.java   |  227 ----
 .../common/shuffle/impl/MergeManager.java       |  782 ------------
 .../engine/common/shuffle/impl/MergeThread.java |  108 --
 .../tez/engine/common/shuffle/impl/Shuffle.java |  278 -----
 .../shuffle/impl/ShuffleClientMetrics.java      |   91 --
 .../common/shuffle/impl/ShuffleHeader.java      |   94 --
 .../shuffle/impl/ShuffleInputEventHandler.java  |  134 ---
 .../common/shuffle/impl/ShuffleScheduler.java   |  521 --------
 .../common/shuffle/server/ShuffleHandler.java   |  572 ---------
 .../engine/common/sort/impl/ExternalSorter.java |  194 ---
 .../tez/engine/common/sort/impl/IFile.java      |  559 ---------
 .../common/sort/impl/IFileInputStream.java      |  276 -----
 .../common/sort/impl/IFileOutputStream.java     |  129 --
 .../common/sort/impl/PipelinedSorter.java       |  932 ---------------
 .../engine/common/sort/impl/TezIndexRecord.java |   45 -
 .../tez/engine/common/sort/impl/TezMerger.java  |  798 -------------
 .../sort/impl/TezRawKeyValueIterator.java       |   70 --
 .../engine/common/sort/impl/TezSpillRecord.java |  146 ---
 .../common/sort/impl/dflt/DefaultSorter.java    | 1108 ------------------
 .../sort/impl/dflt/InMemoryShuffleSorter.java   |  126 --
 .../sort/impl/dflt/SortBufferInputStream.java   |  271 -----
 .../engine/common/task/impl/ValuesIterator.java |  149 ---
 .../local/output/TezLocalTaskOutputFiles.java   |  249 ----
 .../common/task/local/output/TezTaskOutput.java |  165 ---
 .../task/local/output/TezTaskOutputFiles.java   |  246 ----
 .../engine/hadoop/compat/NullProgressable.java  |   33 -
 .../tez/engine/lib/input/LocalMergedInput.java  |   52 -
 .../engine/lib/input/ShuffledMergedInput.java   |  179 ---
 .../lib/input/ShuffledMergedInputLegacy.java    |   30 -
 .../lib/input/ShuffledUnorderedKVInput.java     |   76 --
 .../engine/lib/output/InMemorySortedOutput.java |   81 --
 .../lib/output/LocalOnFileSorterOutput.java     |   63 -
 .../engine/lib/output/OnFileSortedOutput.java   |  123 --
 .../lib/output/OnFileUnorderedKVOutput.java     |   98 --
 .../LogicalIOProcessorRuntimeTask.java          |  475 --------
 .../tez/engine/newruntime/RuntimeTask.java      |  106 --
 .../tez/engine/newruntime/RuntimeUtils.java     |   62 -
 .../engine/shuffle/common/DiskFetchedInput.java |  111 --
 .../tez/engine/shuffle/common/FetchResult.java  |   70 --
 .../tez/engine/shuffle/common/FetchedInput.java |  144 ---
 .../shuffle/common/FetchedInputAllocator.java   |   31 -
 .../shuffle/common/FetchedInputCallback.java    |   29 -
 .../tez/engine/shuffle/common/Fetcher.java      |  608 ----------
 .../engine/shuffle/common/FetcherCallback.java  |   31 -
 .../tez/engine/shuffle/common/InputHost.java    |   90 --
 .../shuffle/common/MemoryFetchedInput.java      |   89 --
 .../tez/engine/shuffle/common/ShuffleUtils.java |   81 --
 tez-engine/src/main/proto/Events.proto          |   28 -
 tez-engine/src/main/proto/ShufflePayloads.proto |   33 -
 .../objectregistry/TestObjectRegistry.java      |   56 -
 .../tez/mapreduce/examples/MRRSleepJob.java     |   10 +-
 .../mapreduce/examples/OrderedWordCount.java    |    4 +-
 .../apache/tez/mapreduce/TestMRRJobsDAGApi.java |    4 +-
 tez-mapreduce/pom.xml                           |    7 +-
 .../apache/hadoop/mapred/LocalJobRunnerTez.java |    2 +-
 .../tez/mapreduce/combine/MRCombiner.java       |   16 +-
 .../tez/mapreduce/hadoop/DeprecatedKeys.java    |   97 +-
 .../apache/tez/mapreduce/hadoop/MRHelpers.java  |    6 +-
 .../hadoop/MultiStageMRConfToTezTranslator.java |   12 +-
 .../hadoop/MultiStageMRConfigUtil.java          |    2 +-
 .../tez/mapreduce/hadoop/mapred/MRReporter.java |    4 +-
 .../hadoop/mapreduce/MapContextImpl.java        |    2 +-
 .../mapreduce/TaskAttemptContextImpl.java       |    2 +-
 .../mapreduce/TaskInputOutputContextImpl.java   |    2 +-
 .../org/apache/tez/mapreduce/input/MRInput.java |    9 +-
 .../apache/tez/mapreduce/output/MROutput.java   |    8 +-
 .../tez/mapreduce/partition/MRPartitioner.java  |    8 +-
 .../apache/tez/mapreduce/processor/MRTask.java  |   17 +-
 .../tez/mapreduce/processor/MRTaskReporter.java |    8 +-
 .../mapreduce/processor/map/MapProcessor.java   |   16 +-
 .../processor/reduce/ReduceProcessor.java       |   22 +-
 .../org/apache/tez/mapreduce/TestUmbilical.java |    6 +-
 .../hadoop/TestConfigTranslationMRToTez.java    |    2 +-
 .../mapreduce/hadoop/TestDeprecatedKeys.java    |   18 +-
 .../tez/mapreduce/processor/MapUtils.java       |   12 +-
 .../processor/map/TestMapProcessor.java         |   24 +-
 .../processor/reduce/TestReduceProcessor.java   |   24 +-
 tez-runtime-internals/findbugs-exclude.xml      |   16 +
 tez-runtime-internals/pom.xml                   |   95 ++
 .../org/apache/tez/common/ContainerContext.java |   64 +
 .../org/apache/tez/common/ContainerTask.java    |   74 ++
 .../tez/common/TezTaskUmbilicalProtocol.java    |   57 +
 .../runtime/LogicalIOProcessorRuntimeTask.java  |  475 ++++++++
 .../org/apache/tez/runtime/RuntimeTask.java     |  106 ++
 .../org/apache/tez/runtime/RuntimeUtils.java    |   62 +
 .../api/events/TaskAttemptCompletedEvent.java   |   28 +
 .../api/events/TaskAttemptFailedEvent.java      |   35 +
 .../api/events/TaskStatusUpdateEvent.java       |   70 ++
 .../tez/runtime/api/impl/EventMetaData.java     |  152 +++
 .../apache/tez/runtime/api/impl/EventType.java  |   29 +
 .../apache/tez/runtime/api/impl/InputSpec.java  |   88 ++
 .../apache/tez/runtime/api/impl/OutputSpec.java |   87 ++
 .../apache/tez/runtime/api/impl/TaskSpec.java   |  146 +++
 .../apache/tez/runtime/api/impl/TezEvent.java   |  248 ++++
 .../runtime/api/impl/TezHeartbeatRequest.java   |  137 +++
 .../runtime/api/impl/TezHeartbeatResponse.java  |  105 ++
 .../runtime/api/impl/TezInputContextImpl.java   |   84 ++
 .../runtime/api/impl/TezOutputContextImpl.java  |   85 ++
 .../api/impl/TezProcessorContextImpl.java       |   86 ++
 .../runtime/api/impl/TezTaskContextImpl.java    |  145 +++
 .../tez/runtime/api/impl/TezUmbilical.java      |   36 +
 .../objectregistry/ObjectRegistryImpl.java      |   68 ++
 .../objectregistry/ObjectRegistryModule.java    |   46 +
 .../src/main/proto/Events.proto                 |   28 +
 .../objectregistry/TestObjectRegistry.java      |   60 +
 tez-runtime-library/pom.xml                     |   79 ++
 .../java/org/apache/hadoop/io/BufferUtils.java  |   81 ++
 .../org/apache/hadoop/io/HashComparator.java    |   24 +
 .../tez/runtime/library/api/KVReader.java       |   81 ++
 .../tez/runtime/library/api/KVWriter.java       |   40 +
 .../tez/runtime/library/api/Partitioner.java    |   52 +
 .../broadcast/input/BroadcastInputManager.java  |  138 +++
 .../broadcast/input/BroadcastKVReader.java      |  225 ++++
 .../BroadcastShuffleInputEventHandler.java      |   88 ++
 .../input/BroadcastShuffleManager.java          |  489 ++++++++
 .../broadcast/output/FileBasedKVWriter.java     |  125 ++
 .../tez/runtime/library/common/ConfigUtils.java |  148 +++
 .../tez/runtime/library/common/Constants.java   |   61 +
 .../library/common/InputAttemptIdentifier.java  |   95 ++
 .../runtime/library/common/InputIdentifier.java |   56 +
 .../runtime/library/common/TezRuntimeUtils.java |  152 +++
 .../runtime/library/common/ValuesIterator.java  |  194 +++
 .../tez/runtime/library/common/YARNMaster.java  |   57 +
 .../library/common/combine/Combiner.java        |   42 +
 .../common/localshuffle/LocalShuffle.java       |  120 ++
 .../common/security/JobTokenIdentifier.java     |   98 ++
 .../common/security/JobTokenSecretManager.java  |  137 +++
 .../common/security/JobTokenSelector.java       |   53 +
 .../runtime/library/common/security/Master.java |   57 +
 .../common/security/SecureShuffleUtils.java     |  142 +++
 .../library/common/security/TokenCache.java     |  205 ++++
 .../common/shuffle/impl/ExceptionReporter.java  |   25 +
 .../library/common/shuffle/impl/Fetcher.java    |  624 ++++++++++
 .../common/shuffle/impl/InMemoryReader.java     |  156 +++
 .../common/shuffle/impl/InMemoryWriter.java     |  100 ++
 .../library/common/shuffle/impl/MapHost.java    |  124 ++
 .../library/common/shuffle/impl/MapOutput.java  |  227 ++++
 .../common/shuffle/impl/MergeManager.java       |  782 ++++++++++++
 .../common/shuffle/impl/MergeThread.java        |  108 ++
 .../library/common/shuffle/impl/Shuffle.java    |  278 +++++
 .../shuffle/impl/ShuffleClientMetrics.java      |   91 ++
 .../common/shuffle/impl/ShuffleHeader.java      |   94 ++
 .../shuffle/impl/ShuffleInputEventHandler.java  |  134 +++
 .../common/shuffle/impl/ShuffleScheduler.java   |  521 ++++++++
 .../common/shuffle/server/ShuffleHandler.java   |  572 +++++++++
 .../common/sort/impl/ExternalSorter.java        |  194 +++
 .../runtime/library/common/sort/impl/IFile.java |  559 +++++++++
 .../common/sort/impl/IFileInputStream.java      |  276 +++++
 .../common/sort/impl/IFileOutputStream.java     |  129 ++
 .../common/sort/impl/PipelinedSorter.java       |  932 +++++++++++++++
 .../common/sort/impl/TezIndexRecord.java        |   45 +
 .../library/common/sort/impl/TezMerger.java     |  798 +++++++++++++
 .../sort/impl/TezRawKeyValueIterator.java       |   70 ++
 .../common/sort/impl/TezSpillRecord.java        |  146 +++
 .../common/sort/impl/dflt/DefaultSorter.java    | 1108 ++++++++++++++++++
 .../sort/impl/dflt/InMemoryShuffleSorter.java   |  126 ++
 .../sort/impl/dflt/SortBufferInputStream.java   |  271 +++++
 .../common/task/impl/ValuesIterator.java        |  149 +++
 .../local/output/TezLocalTaskOutputFiles.java   |  249 ++++
 .../common/task/local/output/TezTaskOutput.java |  165 +++
 .../task/local/output/TezTaskOutputFiles.java   |  246 ++++
 .../library/hadoop/compat/NullProgressable.java |   33 +
 .../runtime/library/input/LocalMergedInput.java |   52 +
 .../library/input/ShuffledMergedInput.java      |  179 +++
 .../input/ShuffledMergedInputLegacy.java        |   30 +
 .../library/input/ShuffledUnorderedKVInput.java |   76 ++
 .../library/output/InMemorySortedOutput.java    |   81 ++
 .../library/output/LocalOnFileSorterOutput.java |   63 +
 .../library/output/OnFileSortedOutput.java      |  123 ++
 .../library/output/OnFileUnorderedKVOutput.java |   98 ++
 .../shuffle/common/DiskFetchedInput.java        |  111 ++
 .../library/shuffle/common/FetchResult.java     |   70 ++
 .../library/shuffle/common/FetchedInput.java    |  144 +++
 .../shuffle/common/FetchedInputAllocator.java   |   31 +
 .../shuffle/common/FetchedInputCallback.java    |   29 +
 .../runtime/library/shuffle/common/Fetcher.java |  608 ++++++++++
 .../library/shuffle/common/FetcherCallback.java |   31 +
 .../library/shuffle/common/InputHost.java       |   90 ++
 .../shuffle/common/MemoryFetchedInput.java      |   89 ++
 .../library/shuffle/common/ShuffleUtils.java    |   81 ++
 .../src/main/proto/ShufflePayloads.proto        |   33 +
 .../org/apache/tez/mapreduce/YARNRunner.java    |    4 +-
 318 files changed, 18962 insertions(+), 18856 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 63f17eb..ce10f6d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -100,7 +100,12 @@
       </dependency>
       <dependency>
         <groupId>org.apache.tez</groupId>
-        <artifactId>tez-engine</artifactId>
+        <artifactId>tez-runtime-internals</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.tez</groupId>
+        <artifactId>tez-runtime-library</artifactId>
         <version>${project.version}</version>
       </dependency>
       <dependency>
@@ -247,7 +252,8 @@
   <modules>
     <module>tez-api</module>
     <module>tez-common</module>
-    <module>tez-engine</module>
+    <module>tez-runtime-library</module>
+    <module>tez-runtime-internals</module>
     <module>tez-yarn-client</module>
     <module>tez-mapreduce</module>
     <module>tez-mapreduce-examples</module>

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
index 2c4b911..ace87ca 100644
--- a/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
+++ b/tez-api/src/main/java/org/apache/tez/common/TezJobConfig.java
@@ -42,16 +42,16 @@ public class TezJobConfig {
   /**
    * Configuration key to enable/disable IFile readahead.
    */
-  public static final String TEZ_ENGINE_IFILE_READAHEAD =
-      "tez.engine.ifile.readahead";
-  public static final boolean DEFAULT_TEZ_ENGINE_IFILE_READAHEAD = true;
+  public static final String TEZ_RUNTIME_IFILE_READAHEAD =
+      "tez.runtime.ifile.readahead";
+  public static final boolean DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD = true;
 
   /**
    * Configuration key to set the IFile readahead length in bytes.
    */
-  public static final String TEZ_ENGINE_IFILE_READAHEAD_BYTES =
-      "tez.engine.ifile.readahead.bytes";
-  public static final int DEFAULT_TEZ_ENGINE_IFILE_READAHEAD_BYTES =
+  public static final String TEZ_RUNTIME_IFILE_READAHEAD_BYTES =
+      "tez.runtime.ifile.readahead.bytes";
+  public static final int DEFAULT_TEZ_RUNTIME_IFILE_READAHEAD_BYTES =
       4 * 1024 * 1024;
 
   /**
@@ -62,102 +62,103 @@ public class TezJobConfig {
   public static final long DEFAULT_RECORDS_BEFORE_PROGRESS = 10000; 
 
   /**
-   * List of directories avialble to the engine. 
+   * List of directories avialble to the Runtime. 
    */
   @Private
-  public static final String LOCAL_DIRS = "tez.engine.local.dirs";
+  public static final String LOCAL_DIRS = "tez.runtime.local.dirs";
   public static final String DEFAULT_LOCAL_DIRS = "/tmp";
 
   /**
    * One local dir for the speicfic job.
    */
-  public static final String JOB_LOCAL_DIR = "tez.engine.job.local.dir";
+  public static final String JOB_LOCAL_DIR = "tez.runtime.job.local.dir";
   
   /**
    * The directory which contains the localized files for this task.
    */
   @Private
-  public static final String TASK_LOCAL_RESOURCE_DIR = "tez.engine.task-local-resource.dir";
+  public static final String TASK_LOCAL_RESOURCE_DIR = "tez.runtime.task-local-resource.dir";
   public static final String DEFAULT_TASK_LOCAL_RESOURCE_DIR = "/tmp";
   
-  public static final String TEZ_TASK_WORKING_DIR = "tez.engine.task.working.dir";
+  public static final String TEZ_TASK_WORKING_DIR = "tez.runtime.task.working.dir";
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_IO_SORT_FACTOR = 
-      "tez.engine.io.sort.factor";
-  public static final int DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR = 100;
+  public static final String TEZ_RUNTIME_IO_SORT_FACTOR = 
+      "tez.runtime.io.sort.factor";
+  public static final int DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR = 100;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SORT_SPILL_PERCENT = 
-      "tez.engine.sort.spill.percent";
-  public static float DEFAULT_TEZ_ENGINE_SORT_SPILL_PERCENT = 0.8f; 
+  public static final String TEZ_RUNTIME_SORT_SPILL_PERCENT = 
+      "tez.runtime.sort.spill.percent";
+  public static float DEFAULT_TEZ_RUNTIME_SORT_SPILL_PERCENT = 0.8f; 
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_IO_SORT_MB = "tez.engine.io.sort.mb";
-  public static final int DEFAULT_TEZ_ENGINE_IO_SORT_MB = 100;
+  public static final String TEZ_RUNTIME_IO_SORT_MB = "tez.runtime.io.sort.mb";
+  public static final int DEFAULT_TEZ_RUNTIME_IO_SORT_MB = 100;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
-      "tez.engine.index.cache.memory.limit.bytes";
-  public static final int DEFAULT_TEZ_ENGINE_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
+  public static final String TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
+      "tez.runtime.index.cache.memory.limit.bytes";
+  public static final int DEFAULT_TEZ_RUNTIME_INDEX_CACHE_MEMORY_LIMIT_BYTES = 
       1024 * 1024;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_COMBINE_MIN_SPILLS = 
-      "tez.engine.combine.min.spills";
-  public static final int  DEFAULT_TEZ_ENGINE_COMBINE_MIN_SPILLS = 3;
+  public static final String TEZ_RUNTIME_COMBINE_MIN_SPILLS = 
+      "tez.runtime.combine.min.spills";
+  public static final int  DEFAULT_TEZ_RUNTIME_COMBINE_MIN_SPILLS = 3;
   
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SORT_THREADS = 
-	      "tez.engine.sort.threads";
-  public static final int DEFAULT_TEZ_ENGINE_SORT_THREADS = 1;
+  public static final String TEZ_RUNTIME_SORT_THREADS = 
+	      "tez.runtime.sort.threads";
+  public static final int DEFAULT_TEZ_RUNTIME_SORT_THREADS = 1;
 
   /**
-   * Specifies a partitioner class, which is used in Tez engine components like OnFileSortedOutput
+   * Specifies a partitioner class, which is used in Tez Runtime components
+   * like OnFileSortedOutput
    */
-  public static final String TEZ_ENGINE_PARTITIONER_CLASS = "tez.engine.partitioner.class";
+  public static final String TEZ_RUNTIME_PARTITIONER_CLASS = "tez.runtime.partitioner.class";
   
   /**
    * Specifies a combiner class (primarily for Shuffle)
    */
-  public static final String TEZ_ENGINE_COMBINER_CLASS = "tez.engine.combiner.class";
+  public static final String TEZ_RUNTIME_COMBINER_CLASS = "tez.runtime.combiner.class";
   
-  public static final String TEZ_ENGINE_NUM_EXPECTED_PARTITIONS = "tez.engine.num.expected.partitions";
+  public static final String TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS = "tez.runtime.num.expected.partitions";
   
   /**
    * 
    */
-  public static final String COUNTERS_MAX_KEY = "tez.engine.job.counters.max";
+  public static final String COUNTERS_MAX_KEY = "tez.runtime.job.counters.max";
   public static final int COUNTERS_MAX_DEFAULT = 120;
 
   /**
    * 
    */
-  public static final String COUNTER_GROUP_NAME_MAX_KEY = "tez.engine.job.counters.group.name.max";
+  public static final String COUNTER_GROUP_NAME_MAX_KEY = "tez.runtime.job.counters.group.name.max";
   public static final int COUNTER_GROUP_NAME_MAX_DEFAULT = 128;
 
   /**
    * 
    */
-  public static final String COUNTER_NAME_MAX_KEY = "tez.engine.job.counters.counter.name.max";
+  public static final String COUNTER_NAME_MAX_KEY = "tez.runtime.job.counters.counter.name.max";
   public static final int COUNTER_NAME_MAX_DEFAULT = 64;
 
   /**
    * 
    */
-  public static final String COUNTER_GROUPS_MAX_KEY = "tez.engine.job.counters.groups.max";
+  public static final String COUNTER_GROUPS_MAX_KEY = "tez.runtime.job.counters.groups.max";
   public static final int COUNTER_GROUPS_MAX_DEFAULT = 50;
 
   
@@ -166,145 +167,145 @@ public class TezJobConfig {
    * in-memory shuffle should be used.
    */
   @Private
-  public static final String TEZ_ENGINE_SHUFFLE_USE_IN_MEMORY =
-      "tez.engine.shuffle.use.in-memory";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_USE_IN_MEMORY = false;
+  public static final String TEZ_RUNTIME_SHUFFLE_USE_IN_MEMORY =
+      "tez.runtime.shuffle.use.in-memory";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_USE_IN_MEMORY = false;
 
   // TODO NEWTEZ Remove these config parameters. Will be part of an event.
   @Private
-  public static final String TEZ_ENGINE_SHUFFLE_PARTITION_RANGE = 
-      "tez.engine.shuffle.partition-range";
-  public static int TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT = 1;
+  public static final String TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE = 
+      "tez.runtime.shuffle.partition-range";
+  public static int TEZ_RUNTIME_SHUFFLE_PARTITION_RANGE_DEFAULT = 1;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES = 
-      "tez.engine.shuffle.parallel.copies";
-  public static final int DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES = 20;
+  public static final String TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES = 
+      "tez.runtime.shuffle.parallel.copies";
+  public static final int DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES = 20;
 
   /**
    * TODO Is this user configurable.
    */
-  public static final String TEZ_ENGINE_METRICS_SESSION_ID = 
-      "tez.engine.metrics.session.id";
-  public static final String DEFAULT_TEZ_ENGINE_METRICS_SESSION_ID = "";
+  public static final String TEZ_RUNTIME_METRICS_SESSION_ID = 
+      "tez.runtime.metrics.session.id";
+  public static final String DEFAULT_TEZ_RUNTIME_METRICS_SESSION_ID = "";
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_FETCH_FAILURES = 
-      "tez.engine.shuffle.fetch.failures.limit";
-  public final static int DEFAULT_TEZ_ENGINE_SHUFFLE_FETCH_FAILURES_LIMIT = 10;
+  public static final String TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES = 
+      "tez.runtime.shuffle.fetch.failures.limit";
+  public final static int DEFAULT_TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT = 10;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR = 
-      "tez.engine.shuffle.notify.readerror";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR = true;
+  public static final String TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR = 
+      "tez.runtime.shuffle.notify.readerror";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_NOTIFY_READERROR = true;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_CONNECT_TIMEOUT = 
-      "tez.engine.shuffle.connect.timeout";
-  public static final int DEFAULT_TEZ_ENGINE_SHUFFLE_STALLED_COPY_TIMEOUT = 
+  public static final String TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT = 
+      "tez.runtime.shuffle.connect.timeout";
+  public static final int DEFAULT_TEZ_RUNTIME_SHUFFLE_STALLED_COPY_TIMEOUT = 
       3 * 60 * 1000;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_READ_TIMEOUT = "tez.engine.shuffle.read.timeout";
-  public final static int DEFAULT_TEZ_ENGINE_SHUFFLE_READ_TIMEOUT = 
+  public static final String TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT = "tez.runtime.shuffle.read.timeout";
+  public final static int DEFAULT_TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT = 
       3 * 60 * 1000;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_ENABLE_SSL = 
-      "tez.engine.shuffle.ssl.enable";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_SSL = false;
+  public static final String TEZ_RUNTIME_SHUFFLE_ENABLE_SSL = 
+      "tez.runtime.shuffle.ssl.enable";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL = false;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT = 
-      "tez.engine.shuffle.input.buffer.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT =
+  public static final String TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT = 
+      "tez.runtime.shuffle.input.buffer.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT =
       0.90f;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT = 
-      "tez.engine.shuffle.memory.limit.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT = 
+  public static final String TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT = 
+      "tez.runtime.shuffle.memory.limit.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT = 
       0.25f;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_MERGE_PERCENT = 
-      "tez.engine.shuffle.merge.percent";
-  public static final float DEFAULT_TEZ_ENGINE_SHUFFLE_MERGE_PERCENT = 0.90f;
+  public static final String TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT = 
+      "tez.runtime.shuffle.merge.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT = 0.90f;
   
   /**
    * TODO TEZAM3 default value ?
    */
-  public static final String TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS = 
-      "tez.engine.shuffle.memory-to-memory.segments";
+  public static final String TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS = 
+      "tez.runtime.shuffle.memory-to-memory.segments";
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
-      "tez.engine.shuffle.memory-to-memory.enable";
-  public static final boolean DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM = 
+  public static final String TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM = 
+      "tez.runtime.shuffle.memory-to-memory.enable";
+  public static final boolean DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM = 
       false;
 
   /**
    * 
    */
-  public static final String TEZ_ENGINE_INPUT_BUFFER_PERCENT = 
-      "tez.engine.task.input.buffer.percent";
-  public static final float DEFAULT_TEZ_ENGINE_INPUT_BUFFER_PERCENT = 0.0f;
+  public static final String TEZ_RUNTIME_INPUT_BUFFER_PERCENT = 
+      "tez.runtime.task.input.buffer.percent";
+  public static final float DEFAULT_TEZ_RUNTIME_INPUT_BUFFER_PERCENT = 0.0f;
 
   // TODO Rename. 
-  public static final String TEZ_ENGINE_GROUP_COMPARATOR_CLASS = 
-      "tez.engine.group.comparator.class";
+  public static final String TEZ_RUNTIME_GROUP_COMPARATOR_CLASS = 
+      "tez.runtime.group.comparator.class";
   
   // TODO Better name.
-  public static final String TEZ_ENGINE_INTERNAL_SORTER_CLASS = 
-      "tez.engine.internal.sorter.class";
+  public static final String TEZ_RUNTIME_INTERNAL_SORTER_CLASS = 
+      "tez.runtime.internal.sorter.class";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-output.key.comparator.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-input.key.comparator.class";
-
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_KEY_CLASS = 
-      "tez.engine.intermediate-output.key.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_CLASS = 
-      "tez.engine.intermediate-input.key.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_COMPARATOR_CLASS = 
+      "tez.runtime.intermediate-output.key.comparator.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_COMPARATOR_CLASS = 
+      "tez.runtime.intermediate-input.key.comparator.class";
+
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_KEY_CLASS = 
+      "tez.runtime.intermediate-output.key.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_CLASS = 
+      "tez.runtime.intermediate-input.key.class";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_VALUE_CLASS = 
-      "tez.engine.intermediate-output.value.class";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_VALUE_CLASS = 
-      "tez.engine.intermediate-input.value.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_VALUE_CLASS = 
+      "tez.runtime.intermediate-output.value.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_VALUE_CLASS = 
+      "tez.runtime.intermediate-input.value.class";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS = 
-      "tez.engine.intermediate-output.should-compress";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_IS_COMPRESSED = 
-      "tez.engine.intermdiate-input.is-compressed";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_SHOULD_COMPRESS = 
+      "tez.runtime.intermediate-output.should-compress";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_IS_COMPRESSED = 
+      "tez.runtime.intermdiate-input.is-compressed";
   
-  public static final String TEZ_ENGINE_INTERMEDIATE_OUTPUT_COMPRESS_CODEC = 
-      "tez.engine.intermediate-output.compress.codec";
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_COMPRESS_CODEC = 
-      "tez.engine.intermediate-input.compress.codec";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_OUTPUT_COMPRESS_CODEC = 
+      "tez.runtime.intermediate-output.compress.codec";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_COMPRESS_CODEC = 
+      "tez.runtime.intermediate-input.compress.codec";
 
-  public static final String TEZ_ENGINE_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS = 
-      "tez.engine.intermediate-input.key.secondary.comparator.class";
+  public static final String TEZ_RUNTIME_INTERMEDIATE_INPUT_KEY_SECONDARY_COMPARATOR_CLASS = 
+      "tez.runtime.intermediate-input.key.secondary.comparator.class";
   
   // TODO This should be in DAGConfiguration
   /* config for tracking the local file where all the credentials for the job

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 80da655..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index e333075..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 90be09e..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 4a47ccf..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 4626fbd..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index ec679ed..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 05e6b84..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/Processor.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.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/b212ca1d/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
deleted file mode 100644
index 502c5f2..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index ddf1ff8..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 791a0f0..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 2bbbe81..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 706e646..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index c9503a3..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 3f35555..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 57de09b..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 3656d45..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index fa49b79..0000000
--- a/tez-api/src/main/java/org/apache/tez/engine/api/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.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/b212ca1d/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
deleted file mode 100644
index 7099299..0000000
--- a/tez-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/b212ca1d/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
deleted file mode 100644
index a27903d..0000000
--- a/tez-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/b212ca1d/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
deleted file mode 100644
index 94352b3..0000000
--- a/tez-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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Event.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Event.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Event.java
new file mode 100644
index 0000000..6f3d667
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/Input.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/Input.java b/tez-api/src/main/java/org/apache/tez/runtime/api/Input.java
new file mode 100644
index 0000000..a15c106
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalIOProcessor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalIOProcessor.java b/tez-api/src/main/java/org/apache/tez/runtime/api/LogicalIOProcessor.java
new file mode 100644
index 0000000..9372c70
--- /dev/null
+++ b/tez-api/src/main/java/org/apache/tez/runtime/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.runtime.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;
+
+}


[15/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index b2a0b54..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MapOutput.java
+++ /dev/null
@@ -1,227 +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.shuffle.impl;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.Comparator;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BoundedByteArrayOutputStream;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-
-
-class MapOutput {
-  private static final Log LOG = LogFactory.getLog(MapOutput.class);
-  private static AtomicInteger ID = new AtomicInteger(0);
-  
-  public static enum Type {
-    WAIT,
-    MEMORY,
-    DISK
-  }
-  
-  private InputAttemptIdentifier attemptIdentifier;
-  private final int id;
-  
-  private final MergeManager merger;
-  
-  private final long size;
-  
-  private final byte[] memory;
-  private BoundedByteArrayOutputStream byteStream;
-  
-  private final FileSystem localFS;
-  private final Path tmpOutputPath;
-  private final Path outputPath;
-  private final OutputStream disk; 
-  
-  private final Type type;
-  
-  private final boolean primaryMapOutput;
-  
-  MapOutput(InputAttemptIdentifier attemptIdentifier, MergeManager merger, long size, 
-            Configuration conf, LocalDirAllocator localDirAllocator,
-            int fetcher, boolean primaryMapOutput, 
-            TezTaskOutputFiles mapOutputFile)
-         throws IOException {
-    this.id = ID.incrementAndGet();
-    this.attemptIdentifier = attemptIdentifier;
-    this.merger = merger;
-
-    type = Type.DISK;
-
-    memory = null;
-    byteStream = null;
-
-    this.size = size;
-    
-    this.localFS = FileSystem.getLocal(conf);
-    outputPath =
-      mapOutputFile.getInputFileForWrite(this.attemptIdentifier.getInputIdentifier().getSrcTaskIndex(), size);
-    tmpOutputPath = outputPath.suffix(String.valueOf(fetcher));
-
-    disk = localFS.create(tmpOutputPath);
-    
-    this.primaryMapOutput = primaryMapOutput;
-  }
-  
-  MapOutput(InputAttemptIdentifier attemptIdentifier, MergeManager merger, int size, 
-            boolean primaryMapOutput) {
-    this.id = ID.incrementAndGet();
-    this.attemptIdentifier = attemptIdentifier;
-    this.merger = merger;
-
-    type = Type.MEMORY;
-    byteStream = new BoundedByteArrayOutputStream(size);
-    memory = byteStream.getBuffer();
-
-    this.size = size;
-    
-    localFS = null;
-    disk = null;
-    outputPath = null;
-    tmpOutputPath = null;
-    
-    this.primaryMapOutput = primaryMapOutput;
-  }
-
-  public MapOutput(InputAttemptIdentifier attemptIdentifier) {
-    this.id = ID.incrementAndGet();
-    this.attemptIdentifier = attemptIdentifier;
-
-    type = Type.WAIT;
-    merger = null;
-    memory = null;
-    byteStream = null;
-    
-    size = -1;
-    
-    localFS = null;
-    disk = null;
-    outputPath = null;
-    tmpOutputPath = null;
-
-    this.primaryMapOutput = false;
-}
-  
-  public boolean isPrimaryMapOutput() {
-    return primaryMapOutput;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof MapOutput) {
-      return id == ((MapOutput)obj).id;
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return id;
-  }
-
-  public Path getOutputPath() {
-    return outputPath;
-  }
-
-  public byte[] getMemory() {
-    return memory;
-  }
-
-  public BoundedByteArrayOutputStream getArrayStream() {
-    return byteStream;
-  }
-  
-  public OutputStream getDisk() {
-    return disk;
-  }
-
-  public InputAttemptIdentifier getAttemptIdentifier() {
-    return this.attemptIdentifier;
-  }
-
-  public Type getType() {
-    return type;
-  }
-
-  public long getSize() {
-    return size;
-  }
-
-  public void commit() throws IOException {
-    if (type == Type.MEMORY) {
-      merger.closeInMemoryFile(this);
-    } else if (type == Type.DISK) {
-      localFS.rename(tmpOutputPath, outputPath);
-      merger.closeOnDiskFile(outputPath);
-    } else {
-      throw new IOException("Cannot commit MapOutput of type WAIT!");
-    }
-  }
-  
-  public void abort() {
-    if (type == Type.MEMORY) {
-      merger.unreserve(memory.length);
-    } else if (type == Type.DISK) {
-      try {
-        localFS.delete(tmpOutputPath, false);
-      } catch (IOException ie) {
-        LOG.info("failure to clean up " + tmpOutputPath, ie);
-      }
-    } else {
-      throw new IllegalArgumentException
-                   ("Cannot commit MapOutput with of type WAIT!");
-    }
-  }
-  
-  public String toString() {
-    return "MapOutput( AttemptIdentifier: " + attemptIdentifier + ", Type: " + type + ")";
-  }
-  
-  public static class MapOutputComparator 
-  implements Comparator<MapOutput> {
-    public int compare(MapOutput o1, MapOutput o2) {
-      if (o1.id == o2.id) { 
-        return 0;
-      }
-      
-      if (o1.size < o2.size) {
-        return -1;
-      } else if (o1.size > o2.size) {
-        return 1;
-      }
-      
-      if (o1.id < o2.id) {
-        return -1;
-      } else {
-        return 1;
-      
-      }
-    }
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index b8792fb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeManager.java
+++ /dev/null
@@ -1,782 +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.shuffle.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Set;
-import java.util.TreeSet;
-
-
-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.classification.InterfaceAudience.Private;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ChecksumFileSystem;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalDirAllocator;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.DataInputBuffer;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.DefaultCodec;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.common.counters.TezCounter;
-import org.apache.tez.engine.api.TezInputContext;
-import org.apache.tez.engine.common.ConfigUtils;
-import org.apache.tez.engine.common.InputAttemptIdentifier;
-import org.apache.tez.engine.common.combine.Combiner;
-import org.apache.tez.engine.common.sort.impl.IFile;
-import org.apache.tez.engine.common.sort.impl.TezMerger;
-import org.apache.tez.engine.common.sort.impl.TezRawKeyValueIterator;
-import org.apache.tez.engine.common.sort.impl.IFile.Writer;
-import org.apache.tez.engine.common.sort.impl.TezMerger.Segment;
-import org.apache.tez.engine.common.task.local.output.TezTaskOutputFiles;
-import org.apache.tez.engine.hadoop.compat.NullProgressable;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-@SuppressWarnings(value={"rawtypes"})
-public class MergeManager {
-  
-  private static final Log LOG = LogFactory.getLog(MergeManager.class);
-
-  private final Configuration conf;
-  private final FileSystem localFS;
-  private final FileSystem rfs;
-  private final LocalDirAllocator localDirAllocator;
-  
-  private final  TezTaskOutputFiles mapOutputFile;
-  private final Progressable nullProgressable = new NullProgressable();
-  private final Combiner combiner;  
-  
-  Set<MapOutput> inMemoryMergedMapOutputs = 
-    new TreeSet<MapOutput>(new MapOutput.MapOutputComparator());
-  private final IntermediateMemoryToMemoryMerger memToMemMerger;
-
-  Set<MapOutput> inMemoryMapOutputs = 
-    new TreeSet<MapOutput>(new MapOutput.MapOutputComparator());
-  private final InMemoryMerger inMemoryMerger;
-  
-  Set<Path> onDiskMapOutputs = new TreeSet<Path>();
-  private final OnDiskMerger onDiskMerger;
-  
-  private final long memoryLimit;
-  private long usedMemory;
-  private long commitMemory;
-  private final long maxSingleShuffleLimit;
-  
-  private final int memToMemMergeOutputsThreshold; 
-  private final long mergeThreshold;
-  
-  private final int ioSortFactor;
-
-  private final ExceptionReporter exceptionReporter;
-  
-  private final TezInputContext inputContext;
-
-  private final TezCounter spilledRecordsCounter;
-
-  private final TezCounter reduceCombineInputCounter;
-
-  private final TezCounter mergedMapOutputsCounter;
-  
-  private final CompressionCodec codec;
-  
-  private volatile boolean finalMergeComplete = false;
-
-  public MergeManager(Configuration conf, 
-                      FileSystem localFS,
-                      LocalDirAllocator localDirAllocator,  
-                      TezInputContext inputContext,
-                      Combiner combiner,
-                      TezCounter spilledRecordsCounter,
-                      TezCounter reduceCombineInputCounter,
-                      TezCounter mergedMapOutputsCounter,
-                      ExceptionReporter exceptionReporter) {
-    this.inputContext = inputContext;
-    this.conf = conf;
-    this.localDirAllocator = localDirAllocator;
-    this.exceptionReporter = exceptionReporter;
-    
-    this.combiner = combiner;
-
-    this.reduceCombineInputCounter = reduceCombineInputCounter;
-    this.spilledRecordsCounter = spilledRecordsCounter;
-    this.mergedMapOutputsCounter = mergedMapOutputsCounter;
-    this.mapOutputFile = new TezTaskOutputFiles(conf, inputContext.getUniqueIdentifier());
-    
-    this.localFS = localFS;
-    this.rfs = ((LocalFileSystem)localFS).getRaw();
-
-    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
-      Class<? extends CompressionCodec> codecClass =
-          ConfigUtils.getIntermediateInputCompressorClass(conf, DefaultCodec.class);
-      codec = ReflectionUtils.newInstance(codecClass, conf);
-    } else {
-      codec = null;
-    }
-
-    final float maxInMemCopyUse =
-      conf.getFloat(
-          TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT, 
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT);
-    if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0) {
-      throw new IllegalArgumentException("Invalid value for " +
-          TezJobConfig.TEZ_ENGINE_SHUFFLE_INPUT_BUFFER_PERCENT + ": " +
-          maxInMemCopyUse);
-    }
-
-    // Allow unit tests to fix Runtime memory
-    this.memoryLimit = 
-      (long)(conf.getLong(Constants.TEZ_ENGINE_TASK_MEMORY,
-          Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE))
-        * maxInMemCopyUse);
- 
-    this.ioSortFactor = 
-        conf.getInt(
-            TezJobConfig.TEZ_ENGINE_IO_SORT_FACTOR, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_IO_SORT_FACTOR);
-
-    final float singleShuffleMemoryLimitPercent =
-        conf.getFloat(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT,
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT);
-    if (singleShuffleMemoryLimitPercent <= 0.0f
-        || singleShuffleMemoryLimitPercent > 1.0f) {
-      throw new IllegalArgumentException("Invalid value for "
-          + TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
-          + singleShuffleMemoryLimitPercent);
-    }
-
-    this.maxSingleShuffleLimit = 
-      (long)(memoryLimit * singleShuffleMemoryLimitPercent);
-    this.memToMemMergeOutputsThreshold = 
-            conf.getInt(
-                TezJobConfig.TEZ_ENGINE_SHUFFLE_MEMTOMEM_SEGMENTS, 
-                ioSortFactor);
-    this.mergeThreshold = 
-        (long)(this.memoryLimit * 
-               conf.getFloat(
-                   TezJobConfig.TEZ_ENGINE_SHUFFLE_MERGE_PERCENT, 
-                   TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_MERGE_PERCENT));
-    LOG.info("MergerManager: memoryLimit=" + memoryLimit + ", " +
-             "maxSingleShuffleLimit=" + maxSingleShuffleLimit + ", " +
-             "mergeThreshold=" + mergeThreshold + ", " + 
-             "ioSortFactor=" + ioSortFactor + ", " +
-             "memToMemMergeOutputsThreshold=" + memToMemMergeOutputsThreshold);
-
-    if (this.maxSingleShuffleLimit >= this.mergeThreshold) {
-      throw new RuntimeException("Invlaid configuration: "
-          + "maxSingleShuffleLimit should be less than mergeThreshold"
-          + "maxSingleShuffleLimit: " + this.maxSingleShuffleLimit
-          + "mergeThreshold: " + this.mergeThreshold);
-    }
-
-    boolean allowMemToMemMerge = 
-      conf.getBoolean(
-          TezJobConfig.TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM, 
-          TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_ENABLE_MEMTOMEM);
-    if (allowMemToMemMerge) {
-      this.memToMemMerger = 
-        new IntermediateMemoryToMemoryMerger(this,
-                                             memToMemMergeOutputsThreshold);
-      this.memToMemMerger.start();
-    } else {
-      this.memToMemMerger = null;
-    }
-    
-    this.inMemoryMerger = new InMemoryMerger(this);
-    this.inMemoryMerger.start();
-    
-    this.onDiskMerger = new OnDiskMerger(this);
-    this.onDiskMerger.start();
-  }
-
-  public void waitForInMemoryMerge() throws InterruptedException {
-    inMemoryMerger.waitForMerge();
-  }
-  
-  private boolean canShuffleToMemory(long requestedSize) {
-    return (requestedSize < maxSingleShuffleLimit); 
-  }
-
-  final private MapOutput stallShuffle = new MapOutput(null);
-
-  public synchronized MapOutput reserve(InputAttemptIdentifier srcAttemptIdentifier, 
-                                             long requestedSize,
-                                             int fetcher
-                                             ) throws IOException {
-    if (!canShuffleToMemory(requestedSize)) {
-      LOG.info(srcAttemptIdentifier + ": Shuffling to disk since " + requestedSize + 
-               " is greater than maxSingleShuffleLimit (" + 
-               maxSingleShuffleLimit + ")");
-      return new MapOutput(srcAttemptIdentifier, this, requestedSize, conf, 
-                                localDirAllocator, fetcher, true,
-                                mapOutputFile);
-    }
-    
-    // Stall shuffle if we are above the memory limit
-
-    // It is possible that all threads could just be stalling and not make
-    // progress at all. This could happen when:
-    //
-    // requested size is causing the used memory to go above limit &&
-    // requested size < singleShuffleLimit &&
-    // current used size < mergeThreshold (merge will not get triggered)
-    //
-    // To avoid this from happening, we allow exactly one thread to go past
-    // the memory limit. We check (usedMemory > memoryLimit) and not
-    // (usedMemory + requestedSize > memoryLimit). When this thread is done
-    // fetching, this will automatically trigger a merge thereby unlocking
-    // all the stalled threads
-    
-    if (usedMemory > memoryLimit) {
-      LOG.debug(srcAttemptIdentifier + ": Stalling shuffle since usedMemory (" + usedMemory
-          + ") is greater than memoryLimit (" + memoryLimit + ")." + 
-          " CommitMemory is (" + commitMemory + ")"); 
-      return stallShuffle;
-    }
-    
-    // Allow the in-memory shuffle to progress
-    LOG.debug(srcAttemptIdentifier + ": Proceeding with shuffle since usedMemory ("
-        + usedMemory + ") is lesser than memoryLimit (" + memoryLimit + ")."
-        + "CommitMemory is (" + commitMemory + ")"); 
-    return unconditionalReserve(srcAttemptIdentifier, requestedSize, true);
-  }
-  
-  /**
-   * Unconditional Reserve is used by the Memory-to-Memory thread
-   * @return
-   */
-  private synchronized MapOutput unconditionalReserve(
-      InputAttemptIdentifier srcAttemptIdentifier, long requestedSize, boolean primaryMapOutput) {
-    usedMemory += requestedSize;
-    return new MapOutput(srcAttemptIdentifier, this, (int)requestedSize, 
-        primaryMapOutput);
-  }
-  
-  synchronized void unreserve(long size) {
-    commitMemory -= size;
-    usedMemory -= size;
-  }
-
-  public synchronized void closeInMemoryFile(MapOutput mapOutput) { 
-    inMemoryMapOutputs.add(mapOutput);
-    LOG.info("closeInMemoryFile -> map-output of size: " + mapOutput.getSize()
-        + ", inMemoryMapOutputs.size() -> " + inMemoryMapOutputs.size()
-        + ", commitMemory -> " + commitMemory + ", usedMemory ->" + usedMemory);
-
-    commitMemory+= mapOutput.getSize();
-
-    synchronized (inMemoryMerger) {
-      // Can hang if mergeThreshold is really low.
-      if (!inMemoryMerger.isInProgress() && commitMemory >= mergeThreshold) {
-        LOG.info("Starting inMemoryMerger's merge since commitMemory=" +
-            commitMemory + " > mergeThreshold=" + mergeThreshold + 
-            ". Current usedMemory=" + usedMemory);
-        inMemoryMapOutputs.addAll(inMemoryMergedMapOutputs);
-        inMemoryMergedMapOutputs.clear();
-        inMemoryMerger.startMerge(inMemoryMapOutputs);
-      } 
-    }
-    
-    if (memToMemMerger != null) {
-      synchronized (memToMemMerger) {
-        if (!memToMemMerger.isInProgress() && 
-            inMemoryMapOutputs.size() >= memToMemMergeOutputsThreshold) {
-          memToMemMerger.startMerge(inMemoryMapOutputs);
-        }
-      }
-    }
-  }
-  
-  
-  public synchronized void closeInMemoryMergedFile(MapOutput mapOutput) {
-    inMemoryMergedMapOutputs.add(mapOutput);
-    LOG.info("closeInMemoryMergedFile -> size: " + mapOutput.getSize() + 
-             ", inMemoryMergedMapOutputs.size() -> " + 
-             inMemoryMergedMapOutputs.size());
-  }
-  
-  public synchronized void closeOnDiskFile(Path file) {
-    onDiskMapOutputs.add(file);
-    
-    synchronized (onDiskMerger) {
-      if (!onDiskMerger.isInProgress() && 
-          onDiskMapOutputs.size() >= (2 * ioSortFactor - 1)) {
-        onDiskMerger.startMerge(onDiskMapOutputs);
-      }
-    }
-  }
-
-  /**
-   * Should <b>only</b> be used after the Shuffle phaze is complete, otherwise can
-   * return an invalid state since a merge may not be in progress dur to
-   * inadequate inputs
-   * 
-   * @return true if the merge process is complete, otherwise false
-   */
-  @Private
-  public boolean isMergeComplete() {
-    return finalMergeComplete;
-  }
-  
-  public TezRawKeyValueIterator close() throws Throwable {
-    // Wait for on-going merges to complete
-    if (memToMemMerger != null) { 
-      memToMemMerger.close();
-    }
-    inMemoryMerger.close();
-    onDiskMerger.close();
-    
-    List<MapOutput> memory = 
-      new ArrayList<MapOutput>(inMemoryMergedMapOutputs);
-    memory.addAll(inMemoryMapOutputs);
-    List<Path> disk = new ArrayList<Path>(onDiskMapOutputs);
-    TezRawKeyValueIterator kvIter = finalMerge(conf, rfs, memory, disk);
-    this.finalMergeComplete = true;
-    return kvIter;
-  }
-   
-  void runCombineProcessor(TezRawKeyValueIterator kvIter, Writer writer)
-      throws IOException, InterruptedException {
-    combiner.combine(kvIter, writer);
-  }
-
-  private class IntermediateMemoryToMemoryMerger 
-  extends MergeThread<MapOutput> {
-    
-    public IntermediateMemoryToMemoryMerger(MergeManager manager, 
-                                            int mergeFactor) {
-      super(manager, mergeFactor, exceptionReporter);
-      setName("InMemoryMerger - Thread to do in-memory merge of in-memory " +
-      		    "shuffled map-outputs");
-      setDaemon(true);
-    }
-
-    @Override
-    public void merge(List<MapOutput> inputs) throws IOException {
-      if (inputs == null || inputs.size() == 0) {
-        return;
-      }
-
-      InputAttemptIdentifier dummyMapId = inputs.get(0).getAttemptIdentifier(); 
-      List<Segment> inMemorySegments = new ArrayList<Segment>();
-      long mergeOutputSize = 
-        createInMemorySegments(inputs, inMemorySegments, 0);
-      int noInMemorySegments = inMemorySegments.size();
-      
-      MapOutput mergedMapOutputs = 
-        unconditionalReserve(dummyMapId, mergeOutputSize, false);
-      
-      Writer writer = 
-        new InMemoryWriter(mergedMapOutputs.getArrayStream());
-      
-      LOG.info("Initiating Memory-to-Memory merge with " + noInMemorySegments +
-               " segments of total-size: " + mergeOutputSize);
-
-      TezRawKeyValueIterator rIter = 
-        TezMerger.merge(conf, rfs,
-                       ConfigUtils.getIntermediateInputKeyClass(conf),
-                       ConfigUtils.getIntermediateInputValueClass(conf),
-                       inMemorySegments, inMemorySegments.size(),
-                       new Path(inputContext.getUniqueIdentifier()),
-                       (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf),
-                       nullProgressable, null, null, null);
-      TezMerger.writeFile(rIter, writer, nullProgressable, conf);
-      writer.close();
-
-      LOG.info(inputContext.getUniqueIdentifier() +  
-               " Memory-to-Memory merge of the " + noInMemorySegments +
-               " files in-memory complete.");
-
-      // Note the output of the merge
-      closeInMemoryMergedFile(mergedMapOutputs);
-    }
-  }
-  
-  private class InMemoryMerger extends MergeThread<MapOutput> {
-    
-    public InMemoryMerger(MergeManager manager) {
-      super(manager, Integer.MAX_VALUE, exceptionReporter);
-      setName
-      ("InMemoryMerger - Thread to merge in-memory shuffled map-outputs");
-      setDaemon(true);
-    }
-    
-    @Override
-    public void merge(List<MapOutput> inputs) throws IOException, InterruptedException {
-      if (inputs == null || inputs.size() == 0) {
-        return;
-      }
-      
-      //name this output file same as the name of the first file that is 
-      //there in the current list of inmem files (this is guaranteed to
-      //be absent on the disk currently. So we don't overwrite a prev. 
-      //created spill). Also we need to create the output file now since
-      //it is not guaranteed that this file will be present after merge
-      //is called (we delete empty files as soon as we see them
-      //in the merge method)
-
-      //figure out the mapId 
-      InputAttemptIdentifier srcTaskIdentifier = inputs.get(0).getAttemptIdentifier();
-
-      List<Segment> inMemorySegments = new ArrayList<Segment>();
-      long mergeOutputSize = 
-        createInMemorySegments(inputs, inMemorySegments,0);
-      int noInMemorySegments = inMemorySegments.size();
-
-      Path outputPath = mapOutputFile.getInputFileForWrite(
-          srcTaskIdentifier.getInputIdentifier().getSrcTaskIndex(),
-          mergeOutputSize).suffix(Constants.MERGED_OUTPUT_PREFIX);
-
-      Writer writer = null;
-      try {
-        writer =
-            new Writer(conf, rfs, outputPath,
-                (Class)ConfigUtils.getIntermediateInputKeyClass(conf),
-                (Class)ConfigUtils.getIntermediateInputValueClass(conf),
-                codec, null);
-
-        TezRawKeyValueIterator rIter = null;
-        LOG.info("Initiating in-memory merge with " + noInMemorySegments + 
-            " segments...");
-
-        rIter = TezMerger.merge(conf, rfs,
-            (Class)ConfigUtils.getIntermediateInputKeyClass(conf),
-            (Class)ConfigUtils.getIntermediateInputValueClass(conf),
-            inMemorySegments, inMemorySegments.size(),
-            new Path(inputContext.getUniqueIdentifier()),
-            (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf),
-            nullProgressable, spilledRecordsCounter, null, null);
-
-        if (null == combiner) {
-          TezMerger.writeFile(rIter, writer, nullProgressable, conf);
-        } else {
-          runCombineProcessor(rIter, writer);
-        }
-        writer.close();
-        writer = null;
-
-        LOG.info(inputContext.getUniqueIdentifier() +  
-            " Merge of the " + noInMemorySegments +
-            " files in-memory complete." +
-            " Local file is " + outputPath + " of size " + 
-            localFS.getFileStatus(outputPath).getLen());
-      } catch (IOException e) { 
-        //make sure that we delete the ondisk file that we created 
-        //earlier when we invoked cloneFileAttributes
-        localFS.delete(outputPath, true);
-        throw e;
-      } finally {
-        if (writer != null) {
-          writer.close();
-        }
-      }
-
-      // Note the output of the merge
-      closeOnDiskFile(outputPath);
-    }
-
-  }
-  
-  private class OnDiskMerger extends MergeThread<Path> {
-    
-    public OnDiskMerger(MergeManager manager) {
-      super(manager, Integer.MAX_VALUE, exceptionReporter);
-      setName("OnDiskMerger - Thread to merge on-disk map-outputs");
-      setDaemon(true);
-    }
-    
-    @Override
-    public void merge(List<Path> inputs) throws IOException {
-      // sanity check
-      if (inputs == null || inputs.isEmpty()) {
-        LOG.info("No ondisk files to merge...");
-        return;
-      }
-      
-      long approxOutputSize = 0;
-      int bytesPerSum = 
-        conf.getInt("io.bytes.per.checksum", 512);
-      
-      LOG.info("OnDiskMerger: We have  " + inputs.size() + 
-               " map outputs on disk. Triggering merge...");
-      
-      // 1. Prepare the list of files to be merged. 
-      for (Path file : inputs) {
-        approxOutputSize += localFS.getFileStatus(file).getLen();
-      }
-
-      // add the checksum length
-      approxOutputSize += 
-        ChecksumFileSystem.getChecksumLength(approxOutputSize, bytesPerSum);
-
-      // 2. Start the on-disk merge process
-      Path outputPath = 
-        localDirAllocator.getLocalPathForWrite(inputs.get(0).toString(), 
-            approxOutputSize, conf).suffix(Constants.MERGED_OUTPUT_PREFIX);
-      Writer writer = 
-        new Writer(conf, rfs, outputPath, 
-                        (Class)ConfigUtils.getIntermediateInputKeyClass(conf), 
-                        (Class)ConfigUtils.getIntermediateInputValueClass(conf),
-                        codec, null);
-      TezRawKeyValueIterator iter  = null;
-      Path tmpDir = new Path(inputContext.getUniqueIdentifier());
-      try {
-        iter = TezMerger.merge(conf, rfs,
-                            (Class)ConfigUtils.getIntermediateInputKeyClass(conf), 
-                            (Class)ConfigUtils.getIntermediateInputValueClass(conf),
-                            codec, inputs.toArray(new Path[inputs.size()]), 
-                            true, ioSortFactor, tmpDir, 
-                            (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf), 
-                            nullProgressable, spilledRecordsCounter, null, 
-                            mergedMapOutputsCounter, null);
-
-        TezMerger.writeFile(iter, writer, nullProgressable, conf);
-        writer.close();
-      } catch (IOException e) {
-        localFS.delete(outputPath, true);
-        throw e;
-      }
-
-      closeOnDiskFile(outputPath);
-
-      LOG.info(inputContext.getUniqueIdentifier() +
-          " Finished merging " + inputs.size() + 
-          " map output files on disk of total-size " + 
-          approxOutputSize + "." + 
-          " Local output file is " + outputPath + " of size " +
-          localFS.getFileStatus(outputPath).getLen());
-    }
-  }
-  
-  private long createInMemorySegments(List<MapOutput> inMemoryMapOutputs,
-                                      List<Segment> inMemorySegments, 
-                                      long leaveBytes
-                                      ) throws IOException {
-    long totalSize = 0L;
-    // We could use fullSize could come from the RamManager, but files can be
-    // closed but not yet present in inMemoryMapOutputs
-    long fullSize = 0L;
-    for (MapOutput mo : inMemoryMapOutputs) {
-      fullSize += mo.getMemory().length;
-    }
-    while(fullSize > leaveBytes) {
-      MapOutput mo = inMemoryMapOutputs.remove(0);
-      byte[] data = mo.getMemory();
-      long size = data.length;
-      totalSize += size;
-      fullSize -= size;
-      IFile.Reader reader = new InMemoryReader(MergeManager.this, 
-                                                   mo.getAttemptIdentifier(),
-                                                   data, 0, (int)size);
-      inMemorySegments.add(new Segment(reader, true, 
-                                            (mo.isPrimaryMapOutput() ? 
-                                            mergedMapOutputsCounter : null)));
-    }
-    return totalSize;
-  }
-
-  class RawKVIteratorReader extends IFile.Reader {
-
-    private final TezRawKeyValueIterator kvIter;
-
-    public RawKVIteratorReader(TezRawKeyValueIterator kvIter, long size)
-        throws IOException {
-      super(null, null, size, null, spilledRecordsCounter);
-      this.kvIter = kvIter;
-    }
-    public boolean nextRawKey(DataInputBuffer key) throws IOException {
-      if (kvIter.next()) {
-        final DataInputBuffer kb = kvIter.getKey();
-        final int kp = kb.getPosition();
-        final int klen = kb.getLength() - kp;
-        key.reset(kb.getData(), kp, klen);
-        bytesRead += klen;
-        return true;
-      }
-      return false;
-    }
-    public void nextRawValue(DataInputBuffer value) throws IOException {
-      final DataInputBuffer vb = kvIter.getValue();
-      final int vp = vb.getPosition();
-      final int vlen = vb.getLength() - vp;
-      value.reset(vb.getData(), vp, vlen);
-      bytesRead += vlen;
-    }
-    public long getPosition() throws IOException {
-      return bytesRead;
-    }
-
-    public void close() throws IOException {
-      kvIter.close();
-    }
-  }
-
-  private TezRawKeyValueIterator finalMerge(Configuration job, FileSystem fs,
-                                       List<MapOutput> inMemoryMapOutputs,
-                                       List<Path> onDiskMapOutputs
-                                       ) throws IOException {
-    LOG.info("finalMerge called with " + 
-             inMemoryMapOutputs.size() + " in-memory map-outputs and " + 
-             onDiskMapOutputs.size() + " on-disk map-outputs");
-    
-    final float maxRedPer =
-      job.getFloat(
-          TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT,
-          TezJobConfig.DEFAULT_TEZ_ENGINE_INPUT_BUFFER_PERCENT);
-    if (maxRedPer > 1.0 || maxRedPer < 0.0) {
-      throw new IOException(TezJobConfig.TEZ_ENGINE_INPUT_BUFFER_PERCENT +
-                            maxRedPer);
-    }
-    int maxInMemReduce = (int)Math.min(
-        Runtime.getRuntime().maxMemory() * maxRedPer, Integer.MAX_VALUE);
-    
-
-    // merge config params
-    Class keyClass = (Class)ConfigUtils.getIntermediateInputKeyClass(job);
-    Class valueClass = (Class)ConfigUtils.getIntermediateInputValueClass(job);
-    final Path tmpDir = new Path(inputContext.getUniqueIdentifier());
-    final RawComparator comparator =
-      (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(job);
-
-    // segments required to vacate memory
-    List<Segment> memDiskSegments = new ArrayList<Segment>();
-    long inMemToDiskBytes = 0;
-    boolean mergePhaseFinished = false;
-    if (inMemoryMapOutputs.size() > 0) {
-      int srcTaskId = inMemoryMapOutputs.get(0).getAttemptIdentifier().getInputIdentifier().getSrcTaskIndex();
-      inMemToDiskBytes = createInMemorySegments(inMemoryMapOutputs, 
-                                                memDiskSegments,
-                                                maxInMemReduce);
-      final int numMemDiskSegments = memDiskSegments.size();
-      if (numMemDiskSegments > 0 &&
-            ioSortFactor > onDiskMapOutputs.size()) {
-        
-        // If we reach here, it implies that we have less than io.sort.factor
-        // disk segments and this will be incremented by 1 (result of the 
-        // memory segments merge). Since this total would still be 
-        // <= io.sort.factor, we will not do any more intermediate merges,
-        // the merge of all these disk segments would be directly fed to the
-        // reduce method
-        
-        mergePhaseFinished = true;
-        // must spill to disk, but can't retain in-mem for intermediate merge
-        final Path outputPath = 
-          mapOutputFile.getInputFileForWrite(srcTaskId,
-                                             inMemToDiskBytes).suffix(
-                                                 Constants.MERGED_OUTPUT_PREFIX);
-        final TezRawKeyValueIterator rIter = TezMerger.merge(job, fs,
-            keyClass, valueClass, memDiskSegments, numMemDiskSegments,
-            tmpDir, comparator, nullProgressable, spilledRecordsCounter, null, null);
-        final Writer writer = new Writer(job, fs, outputPath,
-            keyClass, valueClass, codec, null);
-        try {
-          TezMerger.writeFile(rIter, writer, nullProgressable, job);
-          // add to list of final disk outputs.
-          onDiskMapOutputs.add(outputPath);
-        } catch (IOException e) {
-          if (null != outputPath) {
-            try {
-              fs.delete(outputPath, true);
-            } catch (IOException ie) {
-              // NOTHING
-            }
-          }
-          throw e;
-        } finally {
-          if (null != writer) {
-            writer.close();
-          }
-        }
-        LOG.info("Merged " + numMemDiskSegments + " segments, " +
-                 inMemToDiskBytes + " bytes to disk to satisfy " +
-                 "reduce memory limit");
-        inMemToDiskBytes = 0;
-        memDiskSegments.clear();
-      } else if (inMemToDiskBytes != 0) {
-        LOG.info("Keeping " + numMemDiskSegments + " segments, " +
-                 inMemToDiskBytes + " bytes in memory for " +
-                 "intermediate, on-disk merge");
-      }
-    }
-
-    // segments on disk
-    List<Segment> diskSegments = new ArrayList<Segment>();
-    long onDiskBytes = inMemToDiskBytes;
-    Path[] onDisk = onDiskMapOutputs.toArray(new Path[onDiskMapOutputs.size()]);
-    for (Path file : onDisk) {
-      onDiskBytes += fs.getFileStatus(file).getLen();
-      LOG.debug("Disk file: " + file + " Length is " + 
-          fs.getFileStatus(file).getLen());
-      diskSegments.add(new Segment(job, fs, file, codec, false,
-                                         (file.toString().endsWith(
-                                             Constants.MERGED_OUTPUT_PREFIX) ?
-                                          null : mergedMapOutputsCounter)
-                                        ));
-    }
-    LOG.info("Merging " + onDisk.length + " files, " +
-             onDiskBytes + " bytes from disk");
-    Collections.sort(diskSegments, new Comparator<Segment>() {
-      public int compare(Segment o1, Segment o2) {
-        if (o1.getLength() == o2.getLength()) {
-          return 0;
-        }
-        return o1.getLength() < o2.getLength() ? -1 : 1;
-      }
-    });
-
-    // build final list of segments from merged backed by disk + in-mem
-    List<Segment> finalSegments = new ArrayList<Segment>();
-    long inMemBytes = createInMemorySegments(inMemoryMapOutputs, 
-                                             finalSegments, 0);
-    LOG.info("Merging " + finalSegments.size() + " segments, " +
-             inMemBytes + " bytes from memory into reduce");
-    if (0 != onDiskBytes) {
-      final int numInMemSegments = memDiskSegments.size();
-      diskSegments.addAll(0, memDiskSegments);
-      memDiskSegments.clear();
-      TezRawKeyValueIterator diskMerge = TezMerger.merge(
-          job, fs, keyClass, valueClass, diskSegments,
-          ioSortFactor, numInMemSegments, tmpDir, comparator,
-          nullProgressable, false, spilledRecordsCounter, null, null);
-      diskSegments.clear();
-      if (0 == finalSegments.size()) {
-        return diskMerge;
-      }
-      finalSegments.add(new Segment(
-            new RawKVIteratorReader(diskMerge, onDiskBytes), true));
-    }
-    return TezMerger.merge(job, fs, keyClass, valueClass,
-                 finalSegments, finalSegments.size(), tmpDir,
-                 comparator, nullProgressable, spilledRecordsCounter, null,
-                 null);
-  
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeThread.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeThread.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeThread.java
deleted file mode 100644
index bab882e..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/MergeThread.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.engine.common.shuffle.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-abstract class MergeThread<T> extends Thread {
-  
-  private static final Log LOG = LogFactory.getLog(MergeThread.class);
-
-  private volatile boolean inProgress = false;
-  private List<T> inputs = new ArrayList<T>();
-  protected final MergeManager manager;
-  private final ExceptionReporter reporter;
-  private boolean closed = false;
-  private final int mergeFactor;
-  
-  public MergeThread(MergeManager manager, int mergeFactor,
-                     ExceptionReporter reporter) {
-    this.manager = manager;
-    this.mergeFactor = mergeFactor;
-    this.reporter = reporter;
-  }
-  
-  public synchronized void close() throws InterruptedException {
-    closed = true;
-    waitForMerge();
-    interrupt();
-  }
-
-  public synchronized boolean isInProgress() {
-    return inProgress;
-  }
-  
-  public synchronized void startMerge(Set<T> inputs) {
-    if (!closed) {
-      inProgress = true;
-      this.inputs = new ArrayList<T>();
-      Iterator<T> iter=inputs.iterator();
-      for (int ctr = 0; iter.hasNext() && ctr < mergeFactor; ++ctr) {
-        this.inputs.add(iter.next());
-        iter.remove();
-      }
-      LOG.info(getName() + ": Starting merge with " + this.inputs.size() + 
-               " segments, while ignoring " + inputs.size() + " segments");
-      notifyAll();
-    }
-  }
-
-  public synchronized void waitForMerge() throws InterruptedException {
-    while (inProgress) {
-      wait();
-    }
-  }
-
-  public void run() {
-    while (true) {
-      try {
-        // Wait for notification to start the merge...
-        synchronized (this) {
-          while (!inProgress) {
-            wait();
-          }
-        }
-
-        // Merge
-        merge(inputs);
-      } catch (InterruptedException ie) {
-        return;
-      } catch(Throwable t) {
-        reporter.reportException(t);
-        return;
-      } finally {
-        synchronized (this) {
-          // Clear inputs
-          inputs = null;
-          inProgress = false;        
-          notifyAll();
-        }
-      }
-    }
-  }
-
-  public abstract void merge(List<T> inputs) 
-      throws IOException, InterruptedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index 15332a1..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/Shuffle.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.engine.common.shuffle.impl;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.FutureTask;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.crypto.SecretKey;
-
-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.LocalDirAllocator;
-import org.apache.hadoop.security.UserGroupInformation;
-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.shuffle.common.ShuffleUtils;
-
-import com.google.common.base.Preconditions;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class Shuffle implements ExceptionReporter {
-  
-  private static final Log LOG = LogFactory.getLog(Shuffle.class);
-  private static final int PROGRESS_FREQUENCY = 2000;
-  
-  private final Configuration conf;
-  private final TezInputContext inputContext;
-  private final ShuffleClientMetrics metrics;
-
-  private final ShuffleInputEventHandler eventHandler;
-  private final ShuffleScheduler scheduler;
-  private final MergeManager merger;
-  private Throwable throwable = null;
-  private String throwingThreadName = null;
-  private final int numInputs;
-  private final AtomicInteger reduceStartId;
-  private final SecretKey jobTokenSecret;
-  private AtomicInteger reduceRange = new AtomicInteger(
-      TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT);
-
-  private FutureTask<TezRawKeyValueIterator> runShuffleFuture;
-
-  public Shuffle(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
-    this.inputContext = inputContext;
-    this.conf = conf;
-    this.metrics = new ShuffleClientMetrics(inputContext.getDAGName(),
-        inputContext.getTaskVertexName(), inputContext.getTaskIndex(),
-        this.conf, UserGroupInformation.getCurrentUser().getShortUserName());
-            
-    this.numInputs = numInputs;
-    
-    this.jobTokenSecret = ShuffleUtils
-        .getJobTokenSecretFromTokenBytes(inputContext
-            .getServiceConsumerMetaData(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID));
-    
-    Combiner combiner = TezEngineUtils.instantiateCombiner(conf, inputContext);
-    
-    FileSystem localFS = FileSystem.getLocal(this.conf);
-    LocalDirAllocator localDirAllocator = 
-        new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
-
-    // TODO TEZ Get rid of Map / Reduce references.
-    TezCounter shuffledMapsCounter = 
-        inputContext.getCounters().findCounter(TaskCounter.SHUFFLED_MAPS);
-    TezCounter reduceShuffleBytes =
-        inputContext.getCounters().findCounter(TaskCounter.REDUCE_SHUFFLE_BYTES);
-    TezCounter failedShuffleCounter =
-        inputContext.getCounters().findCounter(TaskCounter.FAILED_SHUFFLE);
-    TezCounter spilledRecordsCounter = 
-        inputContext.getCounters().findCounter(TaskCounter.SPILLED_RECORDS);
-    TezCounter reduceCombineInputCounter =
-        inputContext.getCounters().findCounter(TaskCounter.COMBINE_INPUT_RECORDS);
-    TezCounter mergedMapOutputsCounter =
-        inputContext.getCounters().findCounter(TaskCounter.MERGED_MAP_OUTPUTS);
-    
-    reduceStartId = new AtomicInteger(inputContext.getTaskIndex());
-    LOG.info("Shuffle assigned reduce start id: " + reduceStartId.get()
-        + " with default reduce range: " + reduceRange.get());
-
-    scheduler = new ShuffleScheduler(
-          this.inputContext,
-          this.conf,
-          this.numInputs,
-          this,
-          shuffledMapsCounter,
-          reduceShuffleBytes,
-          failedShuffleCounter);
-    eventHandler= new ShuffleInputEventHandler(
-          inputContext,
-          this,
-          scheduler);
-    merger = new MergeManager(
-          this.conf,
-          localFS,
-          localDirAllocator,
-          inputContext,
-          combiner,
-          spilledRecordsCounter,
-          reduceCombineInputCounter,
-          mergedMapOutputsCounter,
-          this);
-  }
-
-  public void handleEvents(List<Event> events) {
-    eventHandler.handleEvents(events);
-  }
-  
-  /**
-   * Indicates whether the Shuffle and Merge processing is complete.
-   * @return false if not complete, true if complete or if an error occurred.
-   */
-  public boolean isInputReady() {
-    if (runShuffleFuture == null) {
-      return false;
-    }
-    return runShuffleFuture.isDone();
-    //return scheduler.isDone() && merger.isMergeComplete();
-  }
-
-  /**
-   * Waits for the Shuffle and Merge to complete, and returns an iterator over the input.
-   * @return an iterator over the fetched input.
-   * @throws IOException
-   * @throws InterruptedException
-   */
-  public TezRawKeyValueIterator waitForInput() throws IOException, InterruptedException {
-    Preconditions.checkState(runShuffleFuture != null,
-        "waitForInput can only be called after run");
-    TezRawKeyValueIterator kvIter;
-    try {
-      kvIter = runShuffleFuture.get();
-    } catch (ExecutionException e) {
-      Throwable cause = e.getCause();
-      if (cause instanceof IOException) {
-        throw (IOException) cause;
-      } else if (cause instanceof InterruptedException) {
-        throw (InterruptedException) cause;
-      } else {
-        throw new TezUncheckedException(
-            "Unexpected exception type while running Shuffle and Merge", cause);
-      }
-    }
-    return kvIter;
-  }
-
-  public void run() {
-    RunShuffleCallable runShuffle = new RunShuffleCallable();
-    runShuffleFuture = new FutureTask<TezRawKeyValueIterator>(runShuffle);
-    new Thread(runShuffleFuture, "ShuffleMergeRunner").start();
-  }
-  
-  private class RunShuffleCallable implements Callable<TezRawKeyValueIterator> {
-    @Override
-    public TezRawKeyValueIterator call() throws IOException, InterruptedException {
-      // TODO NEWTEZ Limit # fetchers to number of inputs
-      final int numFetchers = 
-          conf.getInt(
-              TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES, 
-              TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
-      Fetcher[] fetchers = new Fetcher[numFetchers];
-      for (int i = 0; i < numFetchers; ++i) {
-        fetchers[i] = new Fetcher(conf, scheduler, merger, metrics, Shuffle.this, jobTokenSecret, inputContext);
-        fetchers[i].start();
-      }
-      
-      while (!scheduler.waitUntilDone(PROGRESS_FREQUENCY)) {
-        synchronized (this) {
-          if (throwable != null) {
-            throw new ShuffleError("error in shuffle in " + throwingThreadName,
-                                   throwable);
-          }
-        }
-      }
-      
-      // Stop the map-output fetcher threads
-      for (Fetcher fetcher : fetchers) {
-        fetcher.shutDown();
-      }
-      fetchers = null;
-      
-      // stop the scheduler
-      scheduler.close();
-
-
-      // Finish the on-going merges...
-      TezRawKeyValueIterator kvIter = null;
-      try {
-        kvIter = merger.close();
-      } catch (Throwable e) {
-        throw new ShuffleError("Error while doing final merge " , e);
-      }
-      
-      // Sanity check
-      synchronized (Shuffle.this) {
-        if (throwable != null) {
-          throw new ShuffleError("error in shuffle in " + throwingThreadName,
-                                 throwable);
-        }
-      }
-      return kvIter;
-    }
-  }
-  
-  public int getReduceStartId() {
-    return reduceStartId.get();
-  }
-  
-  public int getReduceRange() {
-    return reduceRange.get();
-  }
-  
-  public synchronized void reportException(Throwable t) {
-    if (throwable == null) {
-      throwable = t;
-      throwingThreadName = Thread.currentThread().getName();
-      // Notify the scheduler so that the reporting thread finds the 
-      // exception immediately.
-      synchronized (scheduler) {
-        scheduler.notifyAll();
-      }
-    }
-  }
-  
-  public static class ShuffleError extends IOException {
-    private static final long serialVersionUID = 5753909320586607881L;
-
-    ShuffleError(String msg, Throwable t) {
-      super(msg, t);
-    }
-  }
-
-  public void setPartitionRange(int range) {
-    if (range == reduceRange.get()) {
-      return;
-    }
-    if (reduceRange.compareAndSet(
-        TezJobConfig.TEZ_ENGINE_SHUFFLE_PARTITION_RANGE_DEFAULT, range)) {
-      LOG.info("Reduce range set to: " + range);
-    } else {
-      TezUncheckedException e = 
-          new TezUncheckedException("Reduce range can be set only once.");
-      reportException(e);
-      throw e; 
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleClientMetrics.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleClientMetrics.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleClientMetrics.java
deleted file mode 100644
index 850dbeb..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleClientMetrics.java
+++ /dev/null
@@ -1,91 +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.shuffle.impl;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.metrics.MetricsContext;
-import org.apache.hadoop.metrics.MetricsRecord;
-import org.apache.hadoop.metrics.MetricsUtil;
-import org.apache.hadoop.metrics.Updater;
-import org.apache.tez.common.Constants;
-import org.apache.tez.common.TezJobConfig;
-import org.apache.tez.engine.common.TezEngineUtils;
-
-class ShuffleClientMetrics implements Updater {
-
-  private MetricsRecord shuffleMetrics = null;
-  private int numFailedFetches = 0;
-  private int numSuccessFetches = 0;
-  private long numBytes = 0;
-  private int numThreadsBusy = 0;
-  private final int numCopiers;
-  
-  ShuffleClientMetrics(String dagName, String vertexName, int taskIndex, Configuration conf, 
-      String user) {
-    this.numCopiers = 
-        conf.getInt(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_PARALLEL_COPIES);
-
-    MetricsContext metricsContext = MetricsUtil.getContext(Constants.TEZ);
-    this.shuffleMetrics = 
-      MetricsUtil.createRecord(metricsContext, "shuffleInput");
-    this.shuffleMetrics.setTag("user", user);
-    this.shuffleMetrics.setTag("dagName", dagName);
-    this.shuffleMetrics.setTag("taskId", TezEngineUtils.getTaskIdentifier(vertexName, taskIndex));
-    this.shuffleMetrics.setTag("sessionId", 
-        conf.get(
-            TezJobConfig.TEZ_ENGINE_METRICS_SESSION_ID, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_METRICS_SESSION_ID));
-    metricsContext.registerUpdater(this);
-  }
-  public synchronized void inputBytes(long numBytes) {
-    this.numBytes += numBytes;
-  }
-  public synchronized void failedFetch() {
-    ++numFailedFetches;
-  }
-  public synchronized void successFetch() {
-    ++numSuccessFetches;
-  }
-  public synchronized void threadBusy() {
-    ++numThreadsBusy;
-  }
-  public synchronized void threadFree() {
-    --numThreadsBusy;
-  }
-  public void doUpdates(MetricsContext unused) {
-    synchronized (this) {
-      shuffleMetrics.incrMetric("shuffle_input_bytes", numBytes);
-      shuffleMetrics.incrMetric("shuffle_failed_fetches", 
-                                numFailedFetches);
-      shuffleMetrics.incrMetric("shuffle_success_fetches", 
-                                numSuccessFetches);
-      if (numCopiers != 0) {
-        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent",
-            100*((float)numThreadsBusy/numCopiers));
-      } else {
-        shuffleMetrics.setMetric("shuffle_fetchers_busy_percent", 0);
-      }
-      numBytes = 0;
-      numSuccessFetches = 0;
-      numFailedFetches = 0;
-    }
-    shuffleMetrics.update();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleHeader.java
----------------------------------------------------------------------
diff --git a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleHeader.java b/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleHeader.java
deleted file mode 100644
index a918ef1..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleHeader.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.engine.common.shuffle.impl;
-
-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.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-
-/**
- * Shuffle Header information that is sent by the TaskTracker and 
- * deciphered by the Fetcher thread of Reduce task
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Stable
-public class ShuffleHeader implements Writable {
-  
-  /** Header info of the shuffle http request/response */
-  public static final String HTTP_HEADER_NAME = "name";
-  public static final String DEFAULT_HTTP_HEADER_NAME = "mapreduce";
-  public static final String HTTP_HEADER_VERSION = "version";
-  public static final String DEFAULT_HTTP_HEADER_VERSION = "1.0.0";
-
-  /**
-   * The longest possible length of task attempt id that we will accept.
-   */
-  private static final int MAX_ID_LENGTH = 1000;
-
-  String mapId;
-  long uncompressedLength;
-  long compressedLength;
-  int forReduce;
-  
-  public ShuffleHeader() { }
-  
-  public ShuffleHeader(String mapId, long compressedLength,
-      long uncompressedLength, int forReduce) {
-    this.mapId = mapId;
-    this.compressedLength = compressedLength;
-    this.uncompressedLength = uncompressedLength;
-    this.forReduce = forReduce;
-  }
-  
-  public String getMapId() {
-    return this.mapId;
-  }
-  
-  public int getPartition() {
-    return this.forReduce;
-  }
-  
-  public long getUncompressedLength() {
-    return uncompressedLength;
-  }
-
-  public long getCompressedLength() {
-    return compressedLength;
-  }
-
-  public void readFields(DataInput in) throws IOException {
-    mapId = WritableUtils.readStringSafely(in, MAX_ID_LENGTH);
-    compressedLength = WritableUtils.readVLong(in);
-    uncompressedLength = WritableUtils.readVLong(in);
-    forReduce = WritableUtils.readVInt(in);
-  }
-
-  public void write(DataOutput out) throws IOException {
-    Text.writeString(out, mapId);
-    WritableUtils.writeVLong(out, compressedLength);
-    WritableUtils.writeVLong(out, uncompressedLength);
-    WritableUtils.writeVInt(out, forReduce);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index a8e5fe4..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleInputEventHandler.java
+++ /dev/null
@@ -1,134 +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.shuffle.impl;
-
-import java.net.URI;
-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 com.google.common.base.Preconditions;
-import com.google.protobuf.InvalidProtocolBufferException;
-
-public class ShuffleInputEventHandler {
-  
-  private static final Log LOG = LogFactory.getLog(ShuffleInputEventHandler.class);
-
-  private final ShuffleScheduler scheduler;
-  private final TezInputContext inputContext;
-  private final Shuffle shuffle;
-
-  private int maxMapRuntime = 0;
-  private boolean shuffleRangeSet = false;
-  
-  public ShuffleInputEventHandler(TezInputContext inputContext,
-      Shuffle shuffle, ShuffleScheduler scheduler) {
-    this.inputContext = inputContext;
-    this.shuffle = shuffle;
-    this.scheduler = scheduler;
-  }
-
-  public void handleEvents(List<Event> events) {
-    for (Event event : events) {
-      handleEvent(event);
-    }
-  }
-  
-  
-  private void handleEvent(Event event) {
-    if (event instanceof InputInformationEvent) {
-      processInputInformationEvent((InputInformationEvent) event);
-    }
-    else if (event instanceof DataMovementEvent) {
-      processDataMovementEvent((DataMovementEvent) event);      
-    } else if (event instanceof InputFailedEvent) {
-      processTaskFailedEvent((InputFailedEvent) event);
-    }
-  }
-
-  private void processInputInformationEvent(InputInformationEvent iiEvent) {
-    InputInformationEventPayloadProto inputInfoPayload;
-    try {
-      inputInfoPayload = InputInformationEventPayloadProto.parseFrom(iiEvent.getUserPayload());
-    } catch (InvalidProtocolBufferException e) {
-      throw new TezUncheckedException("Unable to parse InputInformationEvent payload", e);
-    }
-    int partitionRange = inputInfoPayload.getPartitionRange();
-    shuffle.setPartitionRange(partitionRange);
-    this.shuffleRangeSet = true;
-  }
-
-  private void processDataMovementEvent(DataMovementEvent dmEvent) {
-    // FIXME TODO NEWTEZ
-    // Preconditions.checkState(shuffleRangeSet == true, "Shuffle Range must be set before a DataMovementEvent is processed");
-    DataMovementEventPayloadProto shufflePayload;
-    try {
-      shufflePayload = DataMovementEventPayloadProto.parseFrom(dmEvent.getUserPayload());
-    } catch (InvalidProtocolBufferException e) {
-      throw new TezUncheckedException("Unable to parse DataMovementEvent payload", e);
-    } 
-    int partitionId = dmEvent.getSourceIndex();
-    URI baseUri = getBaseURI(shufflePayload.getHost(), shufflePayload.getPort(), partitionId);
-
-    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dmEvent.getTargetIndex(), dmEvent.getVersion(), shufflePayload.getPathComponent());
-    scheduler.addKnownMapOutput(shufflePayload.getHost(), partitionId, baseUri.toString(), srcAttemptIdentifier);
-    
-    // TODO NEWTEZ See if this duration hack can be removed.
-    int duration = shufflePayload.getRunDuration();
-    if (duration > maxMapRuntime) {
-      maxMapRuntime = duration;
-      scheduler.informMaxMapRunTime(maxMapRuntime);
-    }
-  }
-  
-  private void processTaskFailedEvent(InputFailedEvent ifEvent) {
-    InputAttemptIdentifier taIdentifier = new InputAttemptIdentifier(ifEvent.getSourceIndex(), ifEvent.getVersion());
-    scheduler.obsoleteMapOutput(taIdentifier);
-    LOG.info("Obsoleting output of src-task: " + taIdentifier);
-  }
-
-  // TODO NEWTEZ Handle encrypted shuffle
-  private URI getBaseURI(String host, int port, int partitionId) {
-    StringBuilder sb = new StringBuilder("http://");
-    sb.append(host);
-    sb.append(":");
-    sb.append(String.valueOf(port));
-    sb.append("/");
-    
-    sb.append("mapOutput?job=");
-    // Required to use the existing ShuffleHandler
-    sb.append(inputContext.getApplicationId().toString().replace("application", "job"));
-    
-    sb.append("&reduce=");
-    sb.append(partitionId);
-    sb.append("&map=");
-    URI u = URI.create(sb.toString());
-    return u;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/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
deleted file mode 100644
index be75668..0000000
--- a/tez-engine/src/main/java/org/apache/tez/engine/common/shuffle/impl/ShuffleScheduler.java
+++ /dev/null
@@ -1,521 +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.shuffle.impl;
-
-import java.io.IOException;
-import java.text.DecimalFormat;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.DelayQueue;
-import java.util.concurrent.Delayed;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.lang.mutable.MutableInt;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-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 com.google.common.collect.Lists;
-
-class ShuffleScheduler {
-  static ThreadLocal<Long> shuffleStart = new ThreadLocal<Long>() {
-    protected Long initialValue() {
-      return 0L;
-    }
-  };
-
-  private static final Log LOG = LogFactory.getLog(ShuffleScheduler.class);
-  private static final int MAX_MAPS_AT_ONCE = 20;
-  private static final long INITIAL_PENALTY = 10000;
-  private static final float PENALTY_GROWTH_RATE = 1.3f;
-  
-  // TODO NEWTEZ May need to be a string if attempting to fetch from multiple inputs.
-  private final Map<Integer, MutableInt> finishedMaps;
-  private final int numInputs;
-  private int remainingMaps;
-  private Map<InputAttemptIdentifier, MapHost> mapLocations = new HashMap<InputAttemptIdentifier, MapHost>();
-  //TODO NEWTEZ Clean this and other maps at some point
-  private ConcurrentMap<String, InputAttemptIdentifier> pathToIdentifierMap = new ConcurrentHashMap<String, InputAttemptIdentifier>(); 
-  private Set<MapHost> pendingHosts = new HashSet<MapHost>();
-  private Set<InputAttemptIdentifier> obsoleteMaps = new HashSet<InputAttemptIdentifier>();
-  
-  private final Random random = new Random(System.currentTimeMillis());
-  private final DelayQueue<Penalty> penalties = new DelayQueue<Penalty>();
-  private final Referee referee = new Referee();
-  private final Map<InputAttemptIdentifier, IntWritable> failureCounts =
-    new HashMap<InputAttemptIdentifier,IntWritable>(); 
-  private final Map<String,IntWritable> hostFailures = 
-    new HashMap<String,IntWritable>();
-  private final TezInputContext inputContext;
-  private final Shuffle shuffle;
-  private final int abortFailureLimit;
-  private final TezCounter shuffledMapsCounter;
-  private final TezCounter reduceShuffleBytes;
-  private final TezCounter failedShuffleCounter;
-  
-  private final long startTime;
-  private long lastProgressTime;
-  
-  private int maxMapRuntime = 0;
-  private int maxFailedUniqueFetches = 5;
-  private int maxFetchFailuresBeforeReporting;
-  
-  private long totalBytesShuffledTillNow = 0;
-  private DecimalFormat  mbpsFormat = new DecimalFormat("0.00");
-
-  private boolean reportReadErrorImmediately = true;
-  
-  public ShuffleScheduler(TezInputContext inputContext,
-                          Configuration conf,
-                          int tasksInDegree,
-                          Shuffle shuffle,
-                          TezCounter shuffledMapsCounter,
-                          TezCounter reduceShuffleBytes,
-                          TezCounter failedShuffleCounter) {
-    this.inputContext = inputContext;
-    this.numInputs = tasksInDegree;
-    abortFailureLimit = Math.max(30, tasksInDegree / 10);
-    remainingMaps = tasksInDegree;
-  //TODO NEWTEZ May need to be a string or a more usable construct if attempting to fetch from multiple inputs. Define a taskId / taskAttemptId pair
-    finishedMaps = new HashMap<Integer, MutableInt>(remainingMaps);
-    this.shuffle = shuffle;
-    this.shuffledMapsCounter = shuffledMapsCounter;
-    this.reduceShuffleBytes = reduceShuffleBytes;
-    this.failedShuffleCounter = failedShuffleCounter;
-    this.startTime = System.currentTimeMillis();
-    this.lastProgressTime = startTime;
-    referee.start();
-    this.maxFailedUniqueFetches = Math.min(tasksInDegree,
-        this.maxFailedUniqueFetches);
-    this.maxFetchFailuresBeforeReporting = 
-        conf.getInt(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_FETCH_FAILURES, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_FETCH_FAILURES_LIMIT);
-    this.reportReadErrorImmediately = 
-        conf.getBoolean(
-            TezJobConfig.TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR, 
-            TezJobConfig.DEFAULT_TEZ_ENGINE_SHUFFLE_NOTIFY_READERROR);
-  }
-
-  public synchronized void copySucceeded(InputAttemptIdentifier srcAttemptIdentifier, 
-                                         MapHost host,
-                                         long bytes,
-                                         long milis,
-                                         MapOutput output
-                                         ) throws IOException {
-    String taskIdentifier = TezEngineUtils.getTaskAttemptIdentifier(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(), srcAttemptIdentifier.getAttemptNumber());
-    failureCounts.remove(taskIdentifier);
-    hostFailures.remove(host.getHostName());
-    
-    if (!isFinishedTaskTrue(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex())) {
-      output.commit();
-      if(incrementTaskCopyAndCheckCompletion(srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex())) {
-        shuffledMapsCounter.increment(1);
-        if (--remainingMaps == 0) {
-          notifyAll();
-        }
-      }
-
-      // update the status
-      lastProgressTime = System.currentTimeMillis();
-      totalBytesShuffledTillNow += bytes;
-      logProgress();
-      reduceShuffleBytes.increment(bytes);
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("src task: "
-            + TezEngineUtils.getTaskAttemptIdentifier(
-                inputContext.getSourceVertexName(), srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
-                srcAttemptIdentifier.getAttemptNumber()) + " done");
-      }
-    }
-    // TODO NEWTEZ Should this be releasing the output, if not committed ? Possible memory leak in case of speculation.
-  }
-
-  private void logProgress() {
-    float mbs = (float) totalBytesShuffledTillNow / (1024 * 1024);
-    int mapsDone = numInputs - remainingMaps;
-    long secsSinceStart = (System.currentTimeMillis() - startTime) / 1000 + 1;
-
-    float transferRate = mbs / secsSinceStart;
-    LOG.info("copy(" + mapsDone + " of " + numInputs + " at "
-        + mbpsFormat.format(transferRate) + " MB/s)");
-  }
-
-  public synchronized void copyFailed(InputAttemptIdentifier srcAttempt,
-                                      MapHost host,
-                                      boolean readError) {
-    host.penalize();
-    int failures = 1;
-    if (failureCounts.containsKey(srcAttempt)) {
-      IntWritable x = failureCounts.get(srcAttempt);
-      x.set(x.get() + 1);
-      failures = x.get();
-    } else {
-      failureCounts.put(srcAttempt, new IntWritable(1));      
-    }
-    String hostname = host.getHostName();
-    if (hostFailures.containsKey(hostname)) {
-      IntWritable x = hostFailures.get(hostname);
-      x.set(x.get() + 1);
-    } else {
-      hostFailures.put(hostname, new IntWritable(1));
-    }
-    if (failures >= abortFailureLimit) {
-      try {
-        throw new IOException(failures
-            + " failures downloading "
-            + TezEngineUtils.getTaskAttemptIdentifier(
-                inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
-                srcAttempt.getAttemptNumber()));
-      } catch (IOException ie) {
-        shuffle.reportException(ie);
-      }
-    }
-    
-    checkAndInformJobTracker(failures, srcAttempt, readError);
-
-    checkReducerHealth();
-    
-    long delay = (long) (INITIAL_PENALTY *
-        Math.pow(PENALTY_GROWTH_RATE, failures));
-    
-    penalties.add(new Penalty(host, delay));
-    
-    failedShuffleCounter.increment(1);
-  }
-  
-  // Notify the JobTracker  
-  // after every read error, if 'reportReadErrorImmediately' is true or
-  // after every 'maxFetchFailuresBeforeReporting' failures
-  private void checkAndInformJobTracker(
-      int failures, InputAttemptIdentifier srcAttempt, boolean readError) {
-    if ((reportReadErrorImmediately && readError)
-        || ((failures % maxFetchFailuresBeforeReporting) == 0)) {
-      LOG.info("Reporting fetch failure for "
-          + TezEngineUtils.getTaskAttemptIdentifier(
-              inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
-              srcAttempt.getAttemptNumber()) + " to jobtracker.");
-
-      List<Event> failedEvents = Lists.newArrayListWithCapacity(1);
-      failedEvents.add(new InputReadErrorEvent("Fetch failure for "
-          + TezEngineUtils.getTaskAttemptIdentifier(
-              inputContext.getSourceVertexName(), srcAttempt.getInputIdentifier().getSrcTaskIndex(),
-              srcAttempt.getAttemptNumber()) + " to jobtracker.", srcAttempt.getInputIdentifier()
-          .getSrcTaskIndex(), srcAttempt.getAttemptNumber()));
-
-      inputContext.sendEvents(failedEvents);      
-      //status.addFailedDependency(mapId);
-    }
-  }
-    
-  private void checkReducerHealth() {
-    final float MAX_ALLOWED_FAILED_FETCH_ATTEMPT_PERCENT = 0.5f;
-    final float MIN_REQUIRED_PROGRESS_PERCENT = 0.5f;
-    final float MAX_ALLOWED_STALL_TIME_PERCENT = 0.5f;
-
-    long totalFailures = failedShuffleCounter.getValue();
-    int doneMaps = numInputs - remainingMaps;
-    
-    boolean reducerHealthy =
-      (((float)totalFailures / (totalFailures + doneMaps))
-          < MAX_ALLOWED_FAILED_FETCH_ATTEMPT_PERCENT);
-    
-    // check if the reducer has progressed enough
-    boolean reducerProgressedEnough =
-      (((float)doneMaps / numInputs)
-          >= MIN_REQUIRED_PROGRESS_PERCENT);
-
-    // check if the reducer is stalled for a long time
-    // duration for which the reducer is stalled
-    int stallDuration =
-      (int)(System.currentTimeMillis() - lastProgressTime);
-    
-    // duration for which the reducer ran with progress
-    int shuffleProgressDuration =
-      (int)(lastProgressTime - startTime);
-
-    // min time the reducer should run without getting killed
-    int minShuffleRunDuration =
-      (shuffleProgressDuration > maxMapRuntime)
-      ? shuffleProgressDuration
-          : maxMapRuntime;
-    
-    boolean reducerStalled =
-      (((float)stallDuration / minShuffleRunDuration)
-          >= MAX_ALLOWED_STALL_TIME_PERCENT);
-
-    // kill if not healthy and has insufficient progress
-    if ((failureCounts.size() >= maxFailedUniqueFetches ||
-        failureCounts.size() == (numInputs - doneMaps))
-        && !reducerHealthy
-        && (!reducerProgressedEnough || reducerStalled)) {
-      LOG.fatal("Shuffle failed with too many fetch failures " +
-      "and insufficient progress!");
-      String errorMsg = "Exceeded MAX_FAILED_UNIQUE_FETCHES; bailing-out.";
-      shuffle.reportException(new IOException(errorMsg));
-    }
-
-  }
-  
-  public synchronized void tipFailed(int srcTaskIndex) {
-    if (!isFinishedTaskTrue(srcTaskIndex)) {
-      setFinishedTaskTrue(srcTaskIndex);
-      if (--remainingMaps == 0) {
-        notifyAll();
-      }
-      logProgress();
-    }
-  }
-  
-  public synchronized void addKnownMapOutput(String hostName,
-                                             int partitionId,
-                                             String hostUrl,
-                                             InputAttemptIdentifier srcAttempt) {
-    String identifier = MapHost.createIdentifier(hostName, partitionId);
-    MapHost host = mapLocations.get(identifier);
-    if (host == null) {
-      host = new MapHost(partitionId, hostName, hostUrl);
-      assert identifier.equals(host.getIdentifier());
-      mapLocations.put(srcAttempt, host);
-    }
-    host.addKnownMap(srcAttempt);
-    pathToIdentifierMap.put(srcAttempt.getPathComponent(), srcAttempt);
-
-    // Mark the host as pending
-    if (host.getState() == MapHost.State.PENDING) {
-      pendingHosts.add(host);
-      notifyAll();
-    }
-  }
-  
-  public synchronized void obsoleteMapOutput(InputAttemptIdentifier srcAttempt) {
-    // The incoming srcAttempt does not contain a path component.
-    obsoleteMaps.add(srcAttempt);
-  }
-  
-  public synchronized void putBackKnownMapOutput(MapHost host,
-                                                 InputAttemptIdentifier srcAttempt) {
-    host.addKnownMap(srcAttempt);
-  }
-
-  public synchronized MapHost getHost() throws InterruptedException {
-      while(pendingHosts.isEmpty()) {
-        wait();
-      }
-      
-      MapHost host = null;
-      Iterator<MapHost> iter = pendingHosts.iterator();
-      int numToPick = random.nextInt(pendingHosts.size());
-      for (int i=0; i <= numToPick; ++i) {
-        host = iter.next();
-      }
-      
-      pendingHosts.remove(host);     
-      host.markBusy();
-      
-      LOG.info("Assigning " + host + " with " + host.getNumKnownMapOutputs() + 
-               " to " + Thread.currentThread().getName());
-      shuffleStart.set(System.currentTimeMillis());
-      
-      return host;
-  }
-  
-  public InputAttemptIdentifier getIdentifierForPathComponent(String pathComponent) {
-    return pathToIdentifierMap.get(pathComponent);
-  }
-  
-  public synchronized List<InputAttemptIdentifier> getMapsForHost(MapHost host) {
-    List<InputAttemptIdentifier> list = host.getAndClearKnownMaps();
-    Iterator<InputAttemptIdentifier> itr = list.iterator();
-    List<InputAttemptIdentifier> result = new ArrayList<InputAttemptIdentifier>();
-    int includedMaps = 0;
-    int totalSize = list.size();
-    // find the maps that we still need, up to the limit
-    while (itr.hasNext()) {
-      InputAttemptIdentifier id = itr.next();
-      if (!obsoleteMaps.contains(id) && !isFinishedTaskTrue(id.getInputIdentifier().getSrcTaskIndex())) {
-        result.add(id);
-        if (++includedMaps >= MAX_MAPS_AT_ONCE) {
-          break;
-        }
-      }
-    }
-    // put back the maps left after the limit
-    while (itr.hasNext()) {
-      InputAttemptIdentifier id = itr.next();
-      if (!obsoleteMaps.contains(id) && !isFinishedTaskTrue(id.getInputIdentifier().getSrcTaskIndex())) {
-        host.addKnownMap(id);
-      }
-    }
-    LOG.info("assigned " + includedMaps + " of " + totalSize + " to " +
-             host + " to " + Thread.currentThread().getName());
-    return result;
-  }
-
-  public synchronized void freeHost(MapHost host) {
-    if (host.getState() != MapHost.State.PENALIZED) {
-      if (host.markAvailable() == MapHost.State.PENDING) {
-        pendingHosts.add(host);
-        notifyAll();
-      }
-    }
-    LOG.info(host + " freed by " + Thread.currentThread().getName() + " in " + 
-             (System.currentTimeMillis()-shuffleStart.get()) + "s");
-  }
-    
-  public synchronized void resetKnownMaps() {
-    mapLocations.clear();
-    obsoleteMaps.clear();
-    pendingHosts.clear();
-    pathToIdentifierMap.clear();
-  }
-
-  /**
-   * Utility method to check if the Shuffle data fetch is complete.
-   * @return
-   */
-  public synchronized boolean isDone() {
-    return remainingMaps == 0;
-  }
-
-  /**
-   * Wait until the shuffle finishes or until the timeout.
-   * @param millis maximum wait time
-   * @return true if the shuffle is done
-   * @throws InterruptedException
-   */
-  public synchronized boolean waitUntilDone(int millis
-                                            ) throws InterruptedException {
-    if (remainingMaps > 0) {
-      wait(millis);
-      return remainingMaps == 0;
-    }
-    return true;
-  }
-  
-  /**
-   * A structure that records the penalty for a host.
-   */
-  private static class Penalty implements Delayed {
-    MapHost host;
-    private long endTime;
-    
-    Penalty(MapHost host, long delay) {
-      this.host = host;
-      this.endTime = System.currentTimeMillis() + delay;
-    }
-
-    public long getDelay(TimeUnit unit) {
-      long remainingTime = endTime - System.currentTimeMillis();
-      return unit.convert(remainingTime, TimeUnit.MILLISECONDS);
-    }
-
-    public int compareTo(Delayed o) {
-      long other = ((Penalty) o).endTime;
-      return endTime == other ? 0 : (endTime < other ? -1 : 1);
-    }
-    
-  }
-  
-  /**
-   * A thread that takes hosts off of the penalty list when the timer expires.
-   */
-  private class Referee extends Thread {
-    public Referee() {
-      setName("ShufflePenaltyReferee");
-      setDaemon(true);
-    }
-
-    public void run() {
-      try {
-        while (true) {
-          // take the first host that has an expired penalty
-          MapHost host = penalties.take().host;
-          synchronized (ShuffleScheduler.this) {
-            if (host.markAvailable() == MapHost.State.PENDING) {
-              pendingHosts.add(host);
-              ShuffleScheduler.this.notifyAll();
-            }
-          }
-        }
-      } catch (InterruptedException ie) {
-        return;
-      } catch (Throwable t) {
-        shuffle.reportException(t);
-      }
-    }
-  }
-  
-  public void close() throws InterruptedException {
-    referee.interrupt();
-    referee.join();
-  }
-
-  public synchronized void informMaxMapRunTime(int duration) {
-    if (duration > maxMapRuntime) {
-      maxMapRuntime = duration;
-    }
-  }
-  
-  void setFinishedTaskTrue(int srcTaskIndex) {
-    synchronized(finishedMaps) {
-      finishedMaps.put(srcTaskIndex, new MutableInt(shuffle.getReduceRange()));
-    }
-  }
-  
-  boolean incrementTaskCopyAndCheckCompletion(int srcTaskIndex) {
-    synchronized(finishedMaps) {
-      MutableInt result = finishedMaps.get(srcTaskIndex);
-      if(result == null) {
-        result = new MutableInt(0);
-        finishedMaps.put(srcTaskIndex, result);
-      }
-      result.increment();
-      return isFinishedTaskTrue(srcTaskIndex);
-    }
-  }
-  
-  boolean isFinishedTaskTrue(int srcTaskIndex) {
-    synchronized (finishedMaps) {
-      MutableInt result = finishedMaps.get(srcTaskIndex);
-      if(result == null) {
-        return false;
-      }
-      if (result.intValue() == shuffle.getReduceRange()) {
-        return true;
-      }
-      return false;      
-    }
-  }
-}


[08/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatResponse.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezHeartbeatResponse.java
new file mode 100644
index 0000000..10699ac
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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)
+        + " }";
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezInputContextImpl.java
new file mode 100644
index 0000000..9169895
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+
+public class TezInputContextImpl extends TezTaskContextImpl
+    implements TezInputContext {
+
+  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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezOutputContextImpl.java
new file mode 100644
index 0000000..fd4c3a3
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezOutputContext;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+
+public class TezOutputContextImpl extends TezTaskContextImpl
+    implements TezOutputContext {
+
+  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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezProcessorContextImpl.java
new file mode 100644
index 0000000..e73baf4
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezProcessorContext;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+
+public class TezProcessorContextImpl extends TezTaskContextImpl
+  implements TezProcessorContext {
+
+  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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezTaskContextImpl.java
new file mode 100644
index 0000000..ee9e96d
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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.runtime.RuntimeTask;
+import org.apache.tez.runtime.api.TezTaskContext;
+
+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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezUmbilical.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezUmbilical.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezUmbilical.java
new file mode 100644
index 0000000..addccda
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/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.runtime.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/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryImpl.java
new file mode 100644
index 0000000..a47526b
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryImpl.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.common.objectregistry;
+
+import java.util.AbstractMap;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+
+import com.google.inject.Singleton;
+
+@Singleton
+public class ObjectRegistryImpl implements ObjectRegistry {
+
+  private Map<String, Map.Entry<Object, ObjectLifeCycle>> objectCache =
+      new HashMap<String, Map.Entry<Object, ObjectLifeCycle>>();
+
+  @Override
+  public synchronized Object add(ObjectLifeCycle lifeCycle,
+      String key, Object value) {
+    Map.Entry<Object, ObjectLifeCycle> oldEntry =
+        objectCache.put(key,
+            new AbstractMap.SimpleImmutableEntry<Object, ObjectLifeCycle>(
+                value, lifeCycle));
+    return oldEntry != null ? oldEntry.getKey() : null;
+  }
+
+  @Override
+  public synchronized Object get(String key) {
+    Map.Entry<Object, ObjectLifeCycle> entry =
+        objectCache.get(key);
+    return entry != null ? entry.getKey() : null;
+  }
+
+  @Override
+  public synchronized boolean delete(String key) {
+    return (null != objectCache.remove(key));
+  }
+
+  public synchronized void clearCache(ObjectLifeCycle lifeCycle) {
+    for (Entry<String, Entry<Object, ObjectLifeCycle>> entry :
+      objectCache.entrySet()) {
+      if (entry.getValue().getValue().equals(lifeCycle)) {
+        objectCache.remove(entry.getKey());
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryModule.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryModule.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryModule.java
new file mode 100644
index 0000000..97ccf7c
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/common/objectregistry/ObjectRegistryModule.java
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.common.objectregistry;
+
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.inject.AbstractModule;
+
+public class ObjectRegistryModule extends AbstractModule {
+
+  private final ObjectRegistry objectRegistry;
+
+  public ObjectRegistryModule(ObjectRegistry objectRegistry) {
+    this.objectRegistry = objectRegistry;
+  }
+
+  @VisibleForTesting
+  public ObjectRegistryModule() {
+    objectRegistry = new ObjectRegistryImpl();
+  }
+
+  @Override
+  protected void configure() {
+    bind(ObjectRegistry.class).toInstance(this.objectRegistry);
+    requestStaticInjection(ObjectRegistryFactory.class);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/main/proto/Events.proto
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/proto/Events.proto b/tez-runtime-internals/src/main/proto/Events.proto
new file mode 100644
index 0000000..558a2b3
--- /dev/null
+++ b/tez-runtime-internals/src/main/proto/Events.proto
@@ -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.
+ */
+
+option java_package = "org.apache.tez.runtime.internals.api.events";
+option java_outer_classname = "SystemEventProtos";
+option java_generate_equals_and_hash = true;
+
+message TaskAttemptFailedEventProto {
+  optional string diagnostics = 1;
+}
+
+message TaskAttemptCompletedEventProto {
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-internals/src/test/java/org/apache/tez/runtime/common/objectregistry/TestObjectRegistry.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/common/objectregistry/TestObjectRegistry.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/common/objectregistry/TestObjectRegistry.java
new file mode 100644
index 0000000..35192e7
--- /dev/null
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/common/objectregistry/TestObjectRegistry.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.common.objectregistry;
+
+import org.apache.tez.runtime.common.objectregistry.ObjectLifeCycle;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistry;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryFactory;
+import org.apache.tez.runtime.common.objectregistry.ObjectRegistryModule;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+
+public class TestObjectRegistry {
+
+  @SuppressWarnings("unused")
+  @Before
+  public void setup() {
+    Injector injector = Guice.createInjector(new ObjectRegistryModule());
+  }
+
+  @Test
+  public void testBasicCRUD() {
+    ObjectRegistry objectRegistry =
+        ObjectRegistryFactory.getObjectRegistry();
+    Assert.assertNotNull(objectRegistry);
+
+    Assert.assertNull(objectRegistry.get("foo"));
+    Assert.assertFalse(objectRegistry.delete("foo"));
+    Integer one = new Integer(1);
+    Integer two_1 = new Integer(2);
+    Integer two_2 = new Integer(3);
+    Assert.assertNull(objectRegistry.add(ObjectLifeCycle.DAG, "one", one));
+    Assert.assertEquals(one, objectRegistry.get("one"));
+    Assert.assertNull(objectRegistry.add(ObjectLifeCycle.DAG, "two", two_1));
+    Assert.assertNotNull(objectRegistry.add(ObjectLifeCycle.SESSION, "two", two_2));
+    Assert.assertNotEquals(two_1, objectRegistry.get("two"));
+    Assert.assertEquals(two_2, objectRegistry.get("two"));
+    Assert.assertTrue(objectRegistry.delete("one"));
+    Assert.assertFalse(objectRegistry.delete("one"));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/pom.xml
----------------------------------------------------------------------
diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml
new file mode 100644
index 0000000..dcdabe1
--- /dev/null
+++ b/tez-runtime-library/pom.xml
@@ -0,0 +1,79 @@
+<?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-runtime-library</artifactId>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-common</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.tez</groupId>
+      <artifactId>tez-api</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>ShufflePayloads.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/b212ca1d/tez-runtime-library/src/main/java/org/apache/hadoop/io/BufferUtils.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/hadoop/io/BufferUtils.java b/tez-runtime-library/src/main/java/org/apache/hadoop/io/BufferUtils.java
new file mode 100644
index 0000000..16f7a8f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/hadoop/io/BufferUtils.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.hadoop.io;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
+@InterfaceStability.Unstable
+public class BufferUtils {
+  public static int compare(DataInputBuffer buf1, DataInputBuffer buf2) {
+    byte[] b1 = buf1.getData();
+    byte[] b2 = buf2.getData();
+    int s1 = buf1.getPosition();
+    int s2 = buf2.getPosition();
+    int l1 = buf1.getLength();
+    int l2 = buf2.getLength();
+    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
+  }
+
+  public static int compare(DataOutputBuffer buf1, DataOutputBuffer buf2) {
+    byte[] b1 = buf1.getData();
+    byte[] b2 = buf2.getData();
+    int s1 = 0;
+    int s2 = 0;
+    int l1 = buf1.getLength();
+    int l2 = buf2.getLength();
+    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
+  }
+
+  public static int compare(DataInputBuffer buf1, DataOutputBuffer buf2) {
+    byte[] b1 = buf1.getData();
+    byte[] b2 = buf2.getData();
+    int s1 = buf1.getPosition();    
+    int s2 = 0;
+    int l1 = buf1.getLength();
+    int l2 = buf2.getLength();
+    return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2);
+  }
+
+  public static int compare(DataOutputBuffer buf1, DataInputBuffer buf2) {
+    return compare(buf2, buf1);
+  }
+
+  public static void copy(DataInputBuffer src, DataOutputBuffer dst) 
+                              throws IOException {
+    byte[] b1 = src.getData();
+    int s1 = src.getPosition();    
+    int l1 = src.getLength();
+    dst.reset();
+    dst.write(b1, s1, l1 - s1);
+  }
+
+  public static void copy(DataOutputBuffer src, DataOutputBuffer dst) 
+                              throws IOException {
+    byte[] b1 = src.getData();
+    int s1 = 0;
+    int l1 = src.getLength();
+    dst.reset();
+    dst.write(b1, s1, l1);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/hadoop/io/HashComparator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/hadoop/io/HashComparator.java b/tez-runtime-library/src/main/java/org/apache/hadoop/io/HashComparator.java
new file mode 100644
index 0000000..a372e01
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/hadoop/io/HashComparator.java
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS 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.io;
+
+public interface HashComparator<KEY> {
+
+  int getHashCode(KEY key);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVReader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVReader.java
new file mode 100644
index 0000000..9c6b380
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/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.runtime.library.api;
+
+import java.io.IOException;
+
+import org.apache.tez.runtime.api.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;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVWriter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/KVWriter.java
new file mode 100644
index 0000000..ff952ed
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/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.runtime.library.api;
+
+import java.io.IOException;
+
+import org.apache.tez.runtime.api.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;
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/Partitioner.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/Partitioner.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/Partitioner.java
new file mode 100644
index 0000000..680c9b8
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/Partitioner.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.runtime.library.api;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.TezJobConfig;
+
+/**
+ * {@link Partitioner} is used by the TEZ framework to partition output
+ * key/value pairs.
+ * 
+ * <b>Partitioner Initialization</b></p> The Partitioner class is picked up
+ * using the TEZ_RUNTIME_PARTITIONER_CLASS attribute in {@link TezJobConfig}
+ * 
+ * TODO NEWTEZ Change construction to first check for a Constructor with a bytep[] payload
+ * 
+ * Partitioners need to provide a single argument ({@link Configuration})
+ * constructor or a 0 argument constructor. If both exist, preference is given
+ * to the single argument constructor. This is primarily for MR support.
+ * 
+ * If using the configuration constructor, TEZ_RUNTIME_NUM_EXPECTED_PARTITIONS
+ * will be set in the configuration, to indicate the max number of expected
+ * partitions.
+ * 
+ */
+public interface Partitioner {
+  
+  /**
+   * Get partition for given key/value.
+   * @param key key
+   * @param value value
+   * @param numPartitions number of partitions
+   * @return
+   */
+  int getPartition(Object key, Object value, int numPartitions);
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastInputManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastInputManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastInputManager.java
new file mode 100644
index 0000000..cda52da
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastInputManager.java
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.broadcast.input;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
+import org.apache.tez.runtime.library.shuffle.common.DiskFetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInputAllocator;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInputCallback;
+import org.apache.tez.runtime.library.shuffle.common.MemoryFetchedInput;
+
+public class BroadcastInputManager implements FetchedInputAllocator,
+    FetchedInputCallback {
+
+  private final Configuration conf;
+
+  private final TezTaskOutputFiles fileNameAllocator;
+  private final LocalDirAllocator localDirAllocator;
+
+  // Configuration parameters
+  private final long memoryLimit;
+  private final long maxSingleShuffleLimit;
+
+  private long usedMemory = 0;
+
+  public BroadcastInputManager(TezInputContext inputContext, Configuration conf) {
+    this.conf = conf;
+
+    this.fileNameAllocator = new TezTaskOutputFiles(conf,
+        inputContext.getUniqueIdentifier());
+    this.localDirAllocator = new LocalDirAllocator(TezJobConfig.LOCAL_DIRS);
+
+    // Setup configuration
+    final float maxInMemCopyUse = conf.getFloat(
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT);
+    if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0) {
+      throw new IllegalArgumentException("Invalid value for "
+          + TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT + ": "
+          + maxInMemCopyUse);
+    }
+
+    // Allow unit tests to fix Runtime memory
+    this.memoryLimit = (long) (conf.getLong(Constants.TEZ_RUNTIME_TASK_MEMORY,
+        Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE)) * maxInMemCopyUse);
+
+    final float singleShuffleMemoryLimitPercent = conf.getFloat(
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT);
+    if (singleShuffleMemoryLimitPercent <= 0.0f
+        || singleShuffleMemoryLimitPercent > 1.0f) {
+      throw new IllegalArgumentException("Invalid value for "
+          + TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
+          + singleShuffleMemoryLimitPercent);
+    }
+
+    this.maxSingleShuffleLimit = (long) (memoryLimit * singleShuffleMemoryLimitPercent);
+  }
+
+  @Override
+  public synchronized FetchedInput allocate(long size,
+      InputAttemptIdentifier inputAttemptIdentifier) throws IOException {
+    if (size > maxSingleShuffleLimit
+        || this.usedMemory + size > this.memoryLimit) {
+      return new DiskFetchedInput(size, inputAttemptIdentifier, this, conf,
+          localDirAllocator, fileNameAllocator);
+    } else {
+      this.usedMemory += size;
+      return new MemoryFetchedInput(size, inputAttemptIdentifier, this);
+    }
+  }
+
+  @Override
+  public void fetchComplete(FetchedInput fetchedInput) {
+    switch (fetchedInput.getType()) {
+    // Not tracking anything here.
+    case DISK:
+    case MEMORY:
+      break;
+    default:
+      throw new TezUncheckedException("InputType: " + fetchedInput.getType()
+          + " not expected for Broadcast fetch");
+    }
+  }
+
+  @Override
+  public void fetchFailed(FetchedInput fetchedInput) {
+    cleanup(fetchedInput);
+  }
+
+  @Override
+  public void freeResources(FetchedInput fetchedInput) {
+    cleanup(fetchedInput);
+  }
+
+  private void cleanup(FetchedInput fetchedInput) {
+    switch (fetchedInput.getType()) {
+    case DISK:
+      break;
+    case MEMORY:
+      unreserve(fetchedInput.getSize());
+      break;
+    default:
+      throw new TezUncheckedException("InputType: " + fetchedInput.getType()
+          + " not expected for Broadcast fetch");
+    }
+  }
+
+  private synchronized void unreserve(long size) {
+    this.usedMemory -= size;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastKVReader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastKVReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastKVReader.java
new file mode 100644
index 0000000..16e9645
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastKVReader.java
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.broadcast.input;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+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.compress.CompressionCodec;
+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.runtime.library.api.KVReader;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.InMemoryReader;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.MemoryFetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
+
+public class BroadcastKVReader<K, V> implements KVReader {
+
+  private static final Log LOG = LogFactory.getLog(BroadcastKVReader.class);
+  
+  private final BroadcastShuffleManager shuffleManager;
+  private final Configuration conf;
+  private final CompressionCodec codec;
+  
+  private final Class<K> keyClass;
+  private final Class<V> valClass;
+  private final Deserializer<K> keyDeserializer;
+  private final Deserializer<V> valDeserializer;
+  private final DataInputBuffer keyIn;
+  private final DataInputBuffer valIn;
+
+  private final SimpleValueIterator valueIterator;
+  private final SimpleIterable valueIterable;
+  
+  private K key;
+  private V value;
+  
+  private FetchedInput currentFetchedInput;
+  private IFile.Reader currentReader;
+  
+  
+  public BroadcastKVReader(BroadcastShuffleManager shuffleManager,
+      Configuration conf) {
+    this.shuffleManager = shuffleManager;
+    this.conf = conf;
+
+    if (ConfigUtils.isIntermediateInputCompressed(this.conf)) {
+      Class<? extends CompressionCodec> codecClass = ConfigUtils
+          .getIntermediateInputCompressorClass(conf, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, conf);
+    } else {
+      codec = null;
+    }
+
+    this.keyClass = ConfigUtils.getIntermediateInputKeyClass(conf);
+    this.valClass = ConfigUtils.getIntermediateInputKeyClass(conf);
+
+    this.keyIn = new DataInputBuffer();
+    this.valIn = new DataInputBuffer();
+
+    SerializationFactory serializationFactory = new SerializationFactory(conf);
+
+    this.keyDeserializer = serializationFactory.getDeserializer(keyClass); 
+    this.valDeserializer = serializationFactory.getDeserializer(valClass);
+    
+    this.valueIterator = new SimpleValueIterator();
+    this.valueIterable = new SimpleIterable(this.valueIterator);
+  }
+
+  // TODO NEWTEZ Maybe add an interface to check whether next will block.
+  
+  /**
+   * 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
+   */
+  @Override  
+  public boolean next() throws IOException {
+    if (readNextFromCurrentReader()) {
+      return true;
+    } else {
+      boolean nextInputExists = moveToNextInput();
+      while (nextInputExists) {
+        if(readNextFromCurrentReader()) {
+          return true;
+        }
+        nextInputExists = moveToNextInput();
+      }
+      return false;
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public KVRecord getCurrentKV() throws IOException {
+    this.valueIterator.setValue(value);
+    return new KVRecord((Object)key, (Iterable<Object>)this.valueIterable);
+  }
+
+  /**
+   * Tries reading the next key and value from the current reader.
+   * @return true if the current reader has more records
+   * @throws IOException
+   */
+  private boolean readNextFromCurrentReader() throws IOException {
+    // Initial reader.
+    if (this.currentReader == null) {
+      return false;
+    } else {
+      boolean hasMore = this.currentReader.nextRawKey(keyIn);
+      if (hasMore) {
+        this.currentReader.nextRawValue(valIn);
+        this.key = keyDeserializer.deserialize(this.key);
+        this.value = valDeserializer.deserialize(this.value);
+        return true;
+      }
+      return false;
+    }
+  }
+  
+  /**
+   * Moves to the next available input. This method may block if the input is not ready yet.
+   * Also takes care of closing the previous input.
+   * 
+   * @return true if the next input exists, false otherwise
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private boolean moveToNextInput() throws IOException {
+    if (currentReader != null) { // Close the current reader.
+      currentReader.close();
+      currentFetchedInput.free();
+    }
+    try {
+      currentFetchedInput = shuffleManager.getNextInput();
+    } catch (InterruptedException e) {
+      LOG.warn("Interrupted while waiting for next available input", e);
+      throw new IOException(e);
+    }
+    if (currentFetchedInput == null) {
+      return false; // No more inputs
+    } else {
+      currentReader = openIFileReader(currentFetchedInput);
+      return true;
+    }
+  }
+
+  public IFile.Reader openIFileReader(FetchedInput fetchedInput)
+      throws IOException {
+    if (fetchedInput.getType() == Type.MEMORY) {
+      MemoryFetchedInput mfi = (MemoryFetchedInput) fetchedInput;
+
+      return new InMemoryReader(null, mfi.getInputAttemptIdentifier(),
+          mfi.getBytes(), 0, (int) mfi.getSize());
+    } else {
+      return new IFile.Reader(conf, fetchedInput.getInputStream(),
+          fetchedInput.getSize(), codec, null);
+    }
+  }
+
+  
+  
+  // TODO NEWTEZ Move this into a common class. Also used in MRInput
+  private class SimpleValueIterator implements Iterator<V> {
+
+    private V value;
+
+    public void setValue(V value) {
+      this.value = value;
+    }
+
+    public boolean hasNext() {
+      return value != null;
+    }
+
+    public V next() {
+      V value = this.value;
+      this.value = null;
+      return value;
+    }
+
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  private class SimpleIterable implements Iterable<V> {
+    private final Iterator<V> iterator;
+    public SimpleIterable(Iterator<V> iterator) {
+      this.iterator = iterator;
+    }
+
+    @Override
+    public Iterator<V> iterator() {
+      return iterator;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleInputEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleInputEventHandler.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleInputEventHandler.java
new file mode 100644
index 0000000..c64379a
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleInputEventHandler.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.runtime.library.broadcast.input;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.shuffle.impl.ShuffleInputEventHandler;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
+
+public class BroadcastShuffleInputEventHandler {
+
+  private static final Log LOG = LogFactory.getLog(ShuffleInputEventHandler.class);
+  
+  private final BroadcastShuffleManager shuffleManager;
+  
+  public BroadcastShuffleInputEventHandler(TezInputContext inputContext, BroadcastShuffleManager shuffleManager) {
+    this.shuffleManager = shuffleManager;
+  }
+
+  public void handleEvents(List<Event> events) {
+    for (Event event : events) {
+      handleEvent(event);
+    }
+  }
+  
+  private void handleEvent(Event event) {
+    if (event instanceof DataMovementEvent) {
+      processDataMovementEvent((DataMovementEvent)event);
+    } else if (event instanceof InputFailedEvent) {
+      processInputFailedEvent((InputFailedEvent)event);
+    } else {
+      throw new TezUncheckedException("Unexpected event type: " + event.getClass().getName());
+    }
+  }
+  
+  
+  private void processDataMovementEvent(DataMovementEvent dme) {
+    Preconditions.checkArgument(dme.getSourceIndex() == 0,
+        "Unexpected srcIndex: " + dme.getSourceIndex()
+            + " on DataMovementEvent. Can only be 0");
+    DataMovementEventPayloadProto shufflePayload;
+    try {
+      shufflePayload = DataMovementEventPayloadProto.parseFrom(dme.getUserPayload());
+    } catch (InvalidProtocolBufferException e) {
+      throw new TezUncheckedException("Unable to parse DataMovementEvent payload", e);
+    }
+    if (shufflePayload.getOutputGenerated()) {
+      InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dme.getTargetIndex(), dme.getVersion(), shufflePayload.getPathComponent());
+      shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(), srcAttemptIdentifier, 0);
+    } else {
+      shuffleManager.addCompletedInputWithNoData(new InputAttemptIdentifier(dme.getTargetIndex(), dme.getVersion()));
+    }
+  }
+  
+  private void processInputFailedEvent(InputFailedEvent ife) {
+    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(ife.getTargetIndex(), ife.getVersion());
+    shuffleManager.obsoleteKnownInput(srcAttemptIdentifier);
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleManager.java
new file mode 100644
index 0000000..2a5c22f
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/broadcast/input/BroadcastShuffleManager.java
@@ -0,0 +1,489 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.broadcast.input;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+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.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+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.runtime.api.Event;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.api.events.InputReadErrorEvent;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.InputIdentifier;
+import org.apache.tez.runtime.library.common.TezRuntimeUtils;
+import org.apache.tez.runtime.library.common.shuffle.server.ShuffleHandler;
+import org.apache.tez.runtime.library.shuffle.common.FetchResult;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInputAllocator;
+import org.apache.tez.runtime.library.shuffle.common.Fetcher;
+import org.apache.tez.runtime.library.shuffle.common.FetcherCallback;
+import org.apache.tez.runtime.library.shuffle.common.InputHost;
+import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
+import org.apache.tez.runtime.library.shuffle.common.Fetcher.FetcherBuilder;
+
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+public class BroadcastShuffleManager implements FetcherCallback {
+
+  private static final Log LOG = LogFactory.getLog(BroadcastShuffleManager.class);
+  
+  private TezInputContext inputContext;
+  private int numInputs;
+  private Configuration conf;
+  
+  private final BroadcastShuffleInputEventHandler inputEventHandler;
+  private final FetchedInputAllocator inputManager;
+  
+  private final ExecutorService fetcherRawExecutor;
+  private final ListeningExecutorService fetcherExecutor;
+
+  private final BlockingQueue<FetchedInput> completedInputs;
+  private final Set<InputIdentifier> completedInputSet;
+  private final Set<InputIdentifier> pendingInputs;
+  private final ConcurrentMap<String, InputHost> knownSrcHosts;
+  private final Set<InputHost> pendingHosts;
+  private final Set<InputAttemptIdentifier> obsoletedInputs;
+  
+  private final AtomicInteger numCompletedInputs = new AtomicInteger(0);
+  
+  private final long startTime;
+  private long lastProgressTime;
+  
+  private FutureTask<Void> runShuffleFuture;
+  
+  // Required to be held when manipulating pendingHosts
+  private ReentrantLock lock = new ReentrantLock();
+  private Condition wakeLoop = lock.newCondition();
+  
+  private final int numFetchers;
+  private final AtomicInteger numRunningFetchers = new AtomicInteger(0);
+  
+  // Parameters required by Fetchers
+  private final SecretKey shuffleSecret;
+  private final int connectionTimeout;
+  private final int readTimeout;
+  private final CompressionCodec codec;
+  private final Decompressor decompressor;
+  
+  private final FetchFutureCallback fetchFutureCallback = new FetchFutureCallback();
+  
+  private volatile Throwable shuffleError;
+  
+  // TODO NEWTEZ Add counters.
+  
+  public BroadcastShuffleManager(TezInputContext inputContext, Configuration conf, int numInputs) throws IOException {
+    this.inputContext = inputContext;
+    this.conf = conf;
+    this.numInputs = numInputs;
+    
+    this.inputEventHandler = new BroadcastShuffleInputEventHandler(inputContext, this);
+    this.inputManager = new BroadcastInputManager(inputContext, conf);
+
+    pendingInputs = Collections.newSetFromMap(new ConcurrentHashMap<InputIdentifier, Boolean>(numInputs));
+    completedInputSet = Collections.newSetFromMap(new ConcurrentHashMap<InputIdentifier, Boolean>(numInputs));
+    completedInputs = new LinkedBlockingQueue<FetchedInput>(numInputs);
+    knownSrcHosts = new ConcurrentHashMap<String, InputHost>();
+    pendingHosts = Collections.newSetFromMap(new ConcurrentHashMap<InputHost, Boolean>());
+    obsoletedInputs = Collections.newSetFromMap(new ConcurrentHashMap<InputAttemptIdentifier, Boolean>());
+    
+    int maxConfiguredFetchers = 
+        conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_PARALLEL_COPIES);
+    
+    this.numFetchers = Math.min(maxConfiguredFetchers, numInputs);
+    
+    this.fetcherRawExecutor = Executors.newFixedThreadPool(numFetchers,
+        new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Fetcher #%d")
+            .build());
+    this.fetcherExecutor = MoreExecutors.listeningDecorator(fetcherRawExecutor);
+    
+    this.startTime = System.currentTimeMillis();
+    this.lastProgressTime = startTime;
+    
+    this.shuffleSecret = ShuffleUtils
+        .getJobTokenSecretFromTokenBytes(inputContext
+            .getServiceConsumerMetaData(ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID));
+    
+    this.connectionTimeout = conf.getInt(
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_STALLED_COPY_TIMEOUT);
+    this.readTimeout = conf.getInt(
+        TezJobConfig.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT,
+        TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT);
+    
+    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
+      Class<? extends CompressionCodec> codecClass = ConfigUtils
+          .getIntermediateInputCompressorClass(conf, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, conf);
+      decompressor = CodecPool.getDecompressor(codec);
+    } else {
+      codec = null;
+      decompressor = null;
+    }
+  }
+  
+  public void run() {
+    RunBroadcastShuffleCallable callable = new RunBroadcastShuffleCallable();
+    runShuffleFuture = new FutureTask<Void>(callable);
+    new Thread(runShuffleFuture, "ShuffleRunner");
+  }
+  
+  private class RunBroadcastShuffleCallable implements Callable<Void> {
+
+    @Override
+    public Void call() throws Exception {
+      while (numCompletedInputs.get() < numInputs) {
+        if (numRunningFetchers.get() >= numFetchers || pendingHosts.size() == 0) {
+          synchronized(lock) {
+            wakeLoop.await();
+          }
+          if (shuffleError != null) {
+            // InputContext has already been informed of a fatal error.
+            // Initiate shutdown.
+            break;
+          }
+          
+          if (numCompletedInputs.get() < numInputs) {
+            synchronized (lock) {
+              int numFetchersToRun = Math.min(pendingHosts.size(), numFetchers - numRunningFetchers.get());
+              int count = 0;
+              for (Iterator<InputHost> inputHostIter = pendingHosts.iterator() ; inputHostIter.hasNext() ; ) {
+                InputHost inputHost = inputHostIter.next();
+                inputHostIter.remove();
+                if (inputHost.getNumPendingInputs() > 0) {
+                  Fetcher fetcher = constructFetcherForHost(inputHost);
+                  numRunningFetchers.incrementAndGet();
+                  ListenableFuture<FetchResult> future = fetcherExecutor
+                      .submit(fetcher);
+                  Futures.addCallback(future, fetchFutureCallback);
+                  if (++count >= numFetchersToRun) {
+                    break;
+                  }
+                }
+              }
+            }
+          }
+        }
+      }
+      // TODO NEWTEZ Maybe clean up inputs.
+      if (!fetcherExecutor.isShutdown()) {
+        fetcherExecutor.shutdownNow();
+      }
+      return null;
+    }
+  }
+  
+  private Fetcher constructFetcherForHost(InputHost inputHost) {
+    FetcherBuilder fetcherBuilder = new FetcherBuilder(
+        BroadcastShuffleManager.this, inputManager,
+        inputContext.getApplicationId(), shuffleSecret, conf);
+    fetcherBuilder.setConnectionParameters(connectionTimeout, readTimeout);
+    fetcherBuilder.setCompressionParameters(codec, decompressor);
+
+    // Remove obsolete inputs from the list being given to the fetcher. Also
+    // remove from the obsolete list.
+    List<InputAttemptIdentifier> pendingInputsForHost = inputHost
+        .clearAndGetPendingInputs();
+    for (Iterator<InputAttemptIdentifier> inputIter = pendingInputsForHost
+        .iterator(); inputIter.hasNext();) {
+      InputAttemptIdentifier input = inputIter.next();
+      // Avoid adding attempts which have already completed.
+      if (completedInputSet.contains(input.getInputIdentifier())) {
+        inputIter.remove();
+      }
+      // Avoid adding attempts which have been marked as OBSOLETE 
+      if (obsoletedInputs.contains(input)) {
+        inputIter.remove();
+        obsoletedInputs.remove(input);
+      }
+    }
+    // TODO NEWTEZ Maybe limit the number of inputs being given to a single
+    // fetcher, especially in the case where #hosts < #fetchers
+    fetcherBuilder.assignWork(inputHost.getHost(), inputHost.getPort(), 0,
+        inputHost.clearAndGetPendingInputs());
+    return fetcherBuilder.build();
+  }
+  
+  /////////////////// Methods for InputEventHandler
+  
+  public void addKnownInput(String hostName, int port,
+      InputAttemptIdentifier srcAttemptIdentifier, int partition) {
+    InputHost host = knownSrcHosts.get(hostName);
+    if (host == null) {
+      host = new InputHost(hostName, port, inputContext.getApplicationId());
+      InputHost old = knownSrcHosts.putIfAbsent(hostName, host);
+      if (old != null) {
+        host = old;
+      }
+    }
+    host.addKnownInput(srcAttemptIdentifier);
+    synchronized(lock) {
+      pendingHosts.add(host);
+      wakeLoop.signal();
+    }
+  }
+
+  public void addCompletedInputWithNoData(
+      InputAttemptIdentifier srcAttemptIdentifier) {
+    InputIdentifier inputIdentifier = srcAttemptIdentifier.getInputIdentifier();
+    LOG.info("No input data exists for SrcTask: " + inputIdentifier + ". Marking as complete.");
+    if (pendingInputs.remove(inputIdentifier)) {
+      completedInputSet.add(inputIdentifier);
+      completedInputs.add(new NullFetchedInput(srcAttemptIdentifier));
+      numCompletedInputs.incrementAndGet();
+    }
+
+    // Awake the loop to check for termination.
+    synchronized (lock) {
+      wakeLoop.signal();
+    } 
+  }
+
+  public synchronized void obsoleteKnownInput(InputAttemptIdentifier srcAttemptIdentifier) {
+    obsoletedInputs.add(srcAttemptIdentifier);
+    // TODO NEWTEZ Maybe inform the fetcher about this. For now, this is used during the initial fetch list construction.
+  }
+  
+  
+  public void handleEvents(List<Event> events) {
+    inputEventHandler.handleEvents(events);
+  }
+
+  /////////////////// End of Methods for InputEventHandler
+  /////////////////// Methods from FetcherCallbackHandler
+  
+  @Override
+  public void fetchSucceeded(String host,
+      InputAttemptIdentifier srcAttemptIdentifier, FetchedInput fetchedInput, long fetchedBytes,
+      long copyDuration) throws IOException {
+    InputIdentifier inputIdentifier = srcAttemptIdentifier.getInputIdentifier();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Complete fetch for attempt: " + srcAttemptIdentifier + " to " + fetchedInput.getType());
+    }
+    
+    // Count irrespective of whether this is a copy of an already fetched input
+    synchronized(lock) {
+      lastProgressTime = System.currentTimeMillis();
+    }
+    
+    boolean committed = false;
+    if (!completedInputSet.contains(inputIdentifier)) {
+      synchronized (completedInputSet) {
+        if (!completedInputSet.contains(inputIdentifier)) {
+          fetchedInput.commit();
+          committed = true;
+          pendingInputs.remove(inputIdentifier);
+          completedInputSet.add(inputIdentifier);
+          completedInputs.add(fetchedInput);
+          numCompletedInputs.incrementAndGet();
+        }
+      }
+    }
+    if (!committed) {
+      fetchedInput.abort(); // If this fails, the fetcher may attempt another abort.
+    } else {
+      synchronized(lock) {
+        // Signal the wakeLoop to check for termination.
+        wakeLoop.signal();
+      }
+    }
+    // TODO NEWTEZ Maybe inform fetchers, in case they have an alternate attempt of the same task in their queue.
+  }
+
+  @Override
+  public void fetchFailed(String host,
+      InputAttemptIdentifier srcAttemptIdentifier, boolean connectFailed) {
+    // TODO NEWTEZ. Implement logic to report fetch failures after a threshold.
+    // For now, reporting immediately.
+    InputReadErrorEvent readError = new InputReadErrorEvent(
+        "Fetch failure while fetching from "
+            + TezRuntimeUtils.getTaskAttemptIdentifier(
+                inputContext.getSourceVertexName(),
+                srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
+                srcAttemptIdentifier.getAttemptNumber()),
+        srcAttemptIdentifier.getInputIdentifier().getSrcTaskIndex(),
+        srcAttemptIdentifier.getAttemptNumber());
+    
+    List<Event> failedEvents = Lists.newArrayListWithCapacity(1);
+    failedEvents.add(readError);
+    inputContext.sendEvents(failedEvents);
+  }
+  /////////////////// End of Methods from FetcherCallbackHandler
+
+  public void shutdown() throws InterruptedException {
+    if (this.fetcherExecutor != null && !this.fetcherExecutor.isShutdown()) {
+      this.fetcherExecutor.shutdown();
+      this.fetcherExecutor.awaitTermination(2000l, TimeUnit.MILLISECONDS);
+      if (!this.fetcherExecutor.isShutdown()) {
+        this.fetcherExecutor.shutdownNow();
+      }
+    }
+  }
+  
+  /////////////////// Methods for walking the available inputs
+  
+  /**
+   * @return true if there is another input ready for consumption.
+   */
+  public boolean newInputAvailable() {
+    FetchedInput head = completedInputs.peek();
+    if (head == null || head instanceof NullFetchedInput) {
+      return false;
+    } else {
+      return true;
+    }
+  }
+
+  /**
+   * @return true if all of the required inputs have been fetched.
+   */
+  public boolean allInputsFetched() {
+    return numCompletedInputs.get() == numInputs;
+  }
+
+  /**
+   * @return the next available input, or null if there are no available inputs.
+   *         This method will block if there are currently no available inputs,
+   *         but more may become available.
+   */
+  public FetchedInput getNextInput() throws InterruptedException {
+    FetchedInput input = null;
+    do {
+      input = completedInputs.peek();
+      if (input == null) {
+        if (allInputsFetched()) {
+          break;
+        } else {
+          input = completedInputs.take(); // block
+        }
+      } else {
+        input = completedInputs.poll();
+      }
+    } while (input instanceof NullFetchedInput);
+    return input;
+  }
+
+  /////////////////// End of methods for walking the available inputs
+  
+  
+  /**
+   * Fake input that is added to the completed input list in case an input does not have any data.
+   *
+   */
+  private class NullFetchedInput extends FetchedInput {
+
+    public NullFetchedInput(InputAttemptIdentifier inputAttemptIdentifier) {
+      super(Type.MEMORY, -1, inputAttemptIdentifier, null);
+    }
+
+    @Override
+    public OutputStream getOutputStream() throws IOException {
+      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
+    }
+
+    @Override
+    public InputStream getInputStream() throws IOException {
+      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
+    }
+
+    @Override
+    public void commit() throws IOException {
+      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
+    }
+
+    @Override
+    public void abort() throws IOException {
+      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
+    }
+
+    @Override
+    public void free() {
+      throw new UnsupportedOperationException("Not supported for NullFetchedInput");
+    }
+  }
+  
+  
+  private class FetchFutureCallback implements FutureCallback<FetchResult> {
+
+    private void doBookKeepingForFetcherComplete() {
+      numRunningFetchers.decrementAndGet();
+      synchronized(lock) {
+        wakeLoop.signal();
+      }
+    }
+    
+    @Override
+    public void onSuccess(FetchResult result) {
+      Iterable<InputAttemptIdentifier> pendingInputs = result.getPendingInputs();
+      if (pendingInputs != null && pendingInputs.iterator().hasNext()) {
+        InputHost inputHost = knownSrcHosts.get(result.getHost());
+        assert inputHost != null;
+        for (InputAttemptIdentifier input : pendingInputs) {
+          inputHost.addKnownInput(input);
+        }
+        pendingHosts.add(inputHost);
+      }
+      doBookKeepingForFetcherComplete();
+    }
+
+    @Override
+    public void onFailure(Throwable t) {
+      LOG.error("Fetcher failed with error: " + t);
+      shuffleError = t;
+      inputContext.fatalError(t, "Fetched failed");
+      doBookKeepingForFetcherComplete();
+    }
+  }
+}


[06/20] Rename tez-engine-api to tez-runtime-api and tez-engine is split into 2: - tez-engine-library for user-visible Input/Output/Processor implementations - tez-engine-internals for framework internals

Posted by hi...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
new file mode 100644
index 0000000..f5d1802
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/Fetcher.java
@@ -0,0 +1,624 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.common.shuffle.impl;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.HttpURLConnection;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLConnection;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import javax.crypto.SecretKey;
+import javax.net.ssl.HttpsURLConnection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.DefaultCodec;
+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.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.security.SecureShuffleUtils;
+import org.apache.tez.runtime.library.common.shuffle.impl.MapOutput.Type;
+import org.apache.tez.runtime.library.common.sort.impl.IFileInputStream;
+
+import com.google.common.annotations.VisibleForTesting;
+
+class Fetcher extends Thread {
+  
+  private static final Log LOG = LogFactory.getLog(Fetcher.class);
+  
+  /** Basic/unit connection timeout (in milliseconds) */
+  private final static int UNIT_CONNECT_TIMEOUT = 60 * 1000;
+
+  private static enum ShuffleErrors{IO_ERROR, WRONG_LENGTH, BAD_ID, WRONG_MAP,
+                                    CONNECTION, WRONG_REDUCE}
+  
+  private final static String SHUFFLE_ERR_GRP_NAME = "Shuffle Errors";
+  private final TezCounter connectionErrs;
+  private final TezCounter ioErrs;
+  private final TezCounter wrongLengthErrs;
+  private final TezCounter badIdErrs;
+  private final TezCounter wrongMapErrs;
+  private final TezCounter wrongReduceErrs;
+  private final MergeManager merger;
+  private final ShuffleScheduler scheduler;
+  private final ShuffleClientMetrics metrics;
+  private final Shuffle shuffle;
+  private final int id;
+  private static int nextId = 0;
+  
+  private final int connectionTimeout;
+  private final int readTimeout;
+  
+  // Decompression of map-outputs
+  private final CompressionCodec codec;
+  private final Decompressor decompressor;
+  private final SecretKey jobTokenSecret;
+
+  private volatile boolean stopped = false;
+
+  private Configuration job;
+
+  private static boolean sslShuffle;
+  private static SSLFactory sslFactory;
+
+  public Fetcher(Configuration job, 
+      ShuffleScheduler scheduler, MergeManager merger,
+      ShuffleClientMetrics metrics,
+      Shuffle shuffle, SecretKey jobTokenSecret, TezInputContext inputContext) throws IOException {
+    this.job = job;
+    this.scheduler = scheduler;
+    this.merger = merger;
+    this.metrics = metrics;
+    this.shuffle = shuffle;
+    this.id = ++nextId;
+    this.jobTokenSecret = jobTokenSecret;
+    ioErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.IO_ERROR.toString());
+    wrongLengthErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.WRONG_LENGTH.toString());
+    badIdErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.BAD_ID.toString());
+    wrongMapErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.WRONG_MAP.toString());
+    connectionErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.CONNECTION.toString());
+    wrongReduceErrs = inputContext.getCounters().findCounter(SHUFFLE_ERR_GRP_NAME,
+        ShuffleErrors.WRONG_REDUCE.toString());
+
+    if (ConfigUtils.isIntermediateInputCompressed(job)) {
+      Class<? extends CompressionCodec> codecClass =
+          ConfigUtils.getIntermediateInputCompressorClass(job, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, job);
+      decompressor = CodecPool.getDecompressor(codec);
+    } else {
+      codec = null;
+      decompressor = null;
+    }
+
+    this.connectionTimeout = 
+        job.getInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT,
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_STALLED_COPY_TIMEOUT);
+    this.readTimeout = 
+        job.getInt(TezJobConfig.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT);
+
+    setName("fetcher#" + id);
+    setDaemon(true);
+
+    synchronized (Fetcher.class) {
+      sslShuffle = job.getBoolean(TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_SSL,
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_SSL);
+      if (sslShuffle && sslFactory == null) {
+        sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, job);
+        try {
+          sslFactory.init();
+        } catch (Exception ex) {
+          sslFactory.destroy();
+          throw new RuntimeException(ex);
+        }
+      }
+    }
+  }
+  
+  public void run() {
+    try {
+      while (!stopped && !Thread.currentThread().isInterrupted()) {
+        MapHost host = null;
+        try {
+          // If merge is on, block
+          merger.waitForInMemoryMerge();
+
+          // Get a host to shuffle from
+          host = scheduler.getHost();
+          metrics.threadBusy();
+
+          // Shuffle
+          copyFromHost(host);
+        } finally {
+          if (host != null) {
+            scheduler.freeHost(host);
+            metrics.threadFree();            
+          }
+        }
+      }
+    } catch (InterruptedException ie) {
+      return;
+    } catch (Throwable t) {
+      shuffle.reportException(t);
+    }
+  }
+
+  public void shutDown() throws InterruptedException {
+    this.stopped = true;
+    interrupt();
+    try {
+      join(5000);
+    } catch (InterruptedException ie) {
+      LOG.warn("Got interrupt while joining " + getName(), ie);
+    }
+    if (sslFactory != null) {
+      sslFactory.destroy();
+    }
+  }
+
+  @VisibleForTesting
+  protected HttpURLConnection openConnection(URL url) throws IOException {
+    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    if (sslShuffle) {
+      HttpsURLConnection httpsConn = (HttpsURLConnection) conn;
+      try {
+        httpsConn.setSSLSocketFactory(sslFactory.createSSLSocketFactory());
+      } catch (GeneralSecurityException ex) {
+        throw new IOException(ex);
+      }
+      httpsConn.setHostnameVerifier(sslFactory.getHostnameVerifier());
+    }
+    return conn;
+  }
+  
+  /**
+   * The crux of the matter...
+   * 
+   * @param host {@link MapHost} from which we need to  
+   *              shuffle available map-outputs.
+   */
+  @VisibleForTesting
+  protected void copyFromHost(MapHost host) throws IOException {
+    // Get completed maps on 'host'
+    List<InputAttemptIdentifier> srcAttempts = scheduler.getMapsForHost(host);
+    
+    // Sanity check to catch hosts with only 'OBSOLETE' maps, 
+    // especially at the tail of large jobs
+    if (srcAttempts.size() == 0) {
+      return;
+    }
+    
+    if(LOG.isDebugEnabled()) {
+      LOG.debug("Fetcher " + id + " going to fetch from " + host + " for: "
+        + srcAttempts);
+    }
+    
+    // List of maps to be fetched yet
+    Set<InputAttemptIdentifier> remaining = new HashSet<InputAttemptIdentifier>(srcAttempts);
+    
+    // Construct the url and connect
+    DataInputStream input;
+    boolean connectSucceeded = false;
+    
+    try {
+      URL url = getMapOutputURL(host, srcAttempts);
+      HttpURLConnection connection = openConnection(url);
+      
+      // generate hash of the url
+      String msgToEncode = SecureShuffleUtils.buildMsgFrom(url);
+      String encHash = SecureShuffleUtils.hashFromString(msgToEncode, jobTokenSecret);
+      
+      // put url hash into http header
+      connection.addRequestProperty(
+          SecureShuffleUtils.HTTP_HEADER_URL_HASH, encHash);
+      // set the read timeout
+      connection.setReadTimeout(readTimeout);
+      // put shuffle version into http header
+      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_NAME,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_NAME);
+      connection.addRequestProperty(ShuffleHeader.HTTP_HEADER_VERSION,
+          ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION);
+      connect(connection, connectionTimeout);
+      connectSucceeded = true;
+      input = new DataInputStream(connection.getInputStream());
+
+      // Validate response code
+      int rc = connection.getResponseCode();
+      if (rc != HttpURLConnection.HTTP_OK) {
+        throw new IOException(
+            "Got invalid response code " + rc + " from " + url +
+            ": " + connection.getResponseMessage());
+      }
+      // get the shuffle version
+      if (!ShuffleHeader.DEFAULT_HTTP_HEADER_NAME.equals(
+          connection.getHeaderField(ShuffleHeader.HTTP_HEADER_NAME))
+          || !ShuffleHeader.DEFAULT_HTTP_HEADER_VERSION.equals(
+              connection.getHeaderField(ShuffleHeader.HTTP_HEADER_VERSION))) {
+        throw new IOException("Incompatible shuffle response version");
+      }
+      // get the replyHash which is HMac of the encHash we sent to the server
+      String replyHash = connection.getHeaderField(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH);
+      if(replyHash==null) {
+        throw new IOException("security validation of TT Map output failed");
+      }
+      LOG.debug("url="+msgToEncode+";encHash="+encHash+";replyHash="+replyHash);
+      // verify that replyHash is HMac of encHash
+      SecureShuffleUtils.verifyReply(replyHash, encHash, jobTokenSecret);
+      LOG.info("for url="+msgToEncode+" sent hash and receievd reply");
+    } catch (IOException ie) {
+      ioErrs.increment(1);
+      LOG.warn("Failed to connect to " + host + " with " + remaining.size() + 
+               " map outputs", ie);
+
+      // If connect did not succeed, just mark all the maps as failed,
+      // indirectly penalizing the host
+      if (!connectSucceeded) {
+        for(InputAttemptIdentifier left: remaining) {
+          scheduler.copyFailed(left, host, connectSucceeded);
+        }
+      } else {
+        // If we got a read error at this stage, it implies there was a problem
+        // with the first map, typically lost map. So, penalize only that map
+        // and add the rest
+        InputAttemptIdentifier firstMap = srcAttempts.get(0);
+        scheduler.copyFailed(firstMap, host, connectSucceeded);
+      }
+      
+      // Add back all the remaining maps, WITHOUT marking them as failed
+      for(InputAttemptIdentifier left: remaining) {
+        // TODO Should the first one be skipped ?
+        scheduler.putBackKnownMapOutput(host, left);
+      }
+      
+      return;
+    }
+    
+    try {
+      // Loop through available map-outputs and fetch them
+      // On any error, faildTasks is not null and we exit
+      // after putting back the remaining maps to the 
+      // yet_to_be_fetched list and marking the failed tasks.
+      InputAttemptIdentifier[] failedTasks = null;
+      while (!remaining.isEmpty() && failedTasks == null) {
+        failedTasks = copyMapOutput(host, input, remaining);
+      }
+      
+      if(failedTasks != null && failedTasks.length > 0) {
+        LOG.warn("copyMapOutput failed for tasks "+Arrays.toString(failedTasks));
+        for(InputAttemptIdentifier left: failedTasks) {
+          scheduler.copyFailed(left, host, true);
+        }
+      }
+      
+      IOUtils.cleanup(LOG, input);
+      
+      // Sanity check
+      if (failedTasks == null && !remaining.isEmpty()) {
+        throw new IOException("server didn't return all expected map outputs: "
+            + remaining.size() + " left.");
+      }
+    } finally {
+      for (InputAttemptIdentifier left : remaining) {
+        scheduler.putBackKnownMapOutput(host, left);
+      }
+    }
+  }
+  
+  private static InputAttemptIdentifier[] EMPTY_ATTEMPT_ID_ARRAY = new InputAttemptIdentifier[0];
+  
+  private InputAttemptIdentifier[] copyMapOutput(MapHost host,
+                                DataInputStream input,
+                                Set<InputAttemptIdentifier> remaining) {
+    MapOutput mapOutput = null;
+    InputAttemptIdentifier srcAttemptId = null;
+    long decompressedLength = -1;
+    long compressedLength = -1;
+    
+    try {
+      long startTime = System.currentTimeMillis();
+      int forReduce = -1;
+      //Read the shuffle header
+      try {
+        ShuffleHeader header = new ShuffleHeader();
+        header.readFields(input);
+        String pathComponent = header.mapId;
+        srcAttemptId = scheduler.getIdentifierForPathComponent(pathComponent);
+        compressedLength = header.compressedLength;
+        decompressedLength = header.uncompressedLength;
+        forReduce = header.forReduce;
+      } catch (IllegalArgumentException e) {
+        badIdErrs.increment(1);
+        LOG.warn("Invalid map id ", e);
+        //Don't know which one was bad, so consider all of them as bad
+        return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
+      }
+
+ 
+      // Do some basic sanity verification
+      if (!verifySanity(compressedLength, decompressedLength, forReduce,
+          remaining, srcAttemptId)) {
+        return new InputAttemptIdentifier[] {srcAttemptId};
+      }
+      
+      if(LOG.isDebugEnabled()) {
+        LOG.debug("header: " + srcAttemptId + ", len: " + compressedLength + 
+            ", decomp len: " + decompressedLength);
+      }
+      
+      // Get the location for the map output - either in-memory or on-disk
+      mapOutput = merger.reserve(srcAttemptId, decompressedLength, id);
+      
+      // Check if we can shuffle *now* ...
+      if (mapOutput.getType() == Type.WAIT) {
+        LOG.info("fetcher#" + id + " - MergerManager returned Status.WAIT ...");
+        //Not an error but wait to process data.
+        return EMPTY_ATTEMPT_ID_ARRAY;
+      } 
+      
+      // Go!
+      LOG.info("fetcher#" + id + " about to shuffle output of map " + 
+               mapOutput.getAttemptIdentifier() + " decomp: " +
+               decompressedLength + " len: " + compressedLength + " to " +
+               mapOutput.getType());
+      if (mapOutput.getType() == Type.MEMORY) {
+        shuffleToMemory(host, mapOutput, input, 
+                        (int) decompressedLength, (int) compressedLength);
+      } else {
+        shuffleToDisk(host, mapOutput, input, compressedLength);
+      }
+      
+      // Inform the shuffle scheduler
+      long endTime = System.currentTimeMillis();
+      scheduler.copySucceeded(srcAttemptId, host, compressedLength, 
+                              endTime - startTime, mapOutput);
+      // Note successful shuffle
+      remaining.remove(srcAttemptId);
+      metrics.successFetch();
+      return null;
+    } catch (IOException ioe) {
+      ioErrs.increment(1);
+      if (srcAttemptId == null || mapOutput == null) {
+        LOG.info("fetcher#" + id + " failed to read map header" + 
+                 srcAttemptId + " decomp: " + 
+                 decompressedLength + ", " + compressedLength, ioe);
+        if(srcAttemptId == null) {
+          return remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
+        } else {
+          return new InputAttemptIdentifier[] {srcAttemptId};
+        }
+      }
+      
+      LOG.warn("Failed to shuffle output of " + srcAttemptId + 
+               " from " + host.getHostName(), ioe); 
+
+      // Inform the shuffle-scheduler
+      mapOutput.abort();
+      metrics.failedFetch();
+      return new InputAttemptIdentifier[] {srcAttemptId};
+    }
+
+  }
+  
+  /**
+   * Do some basic verification on the input received -- Being defensive
+   * @param compressedLength
+   * @param decompressedLength
+   * @param forReduce
+   * @param remaining
+   * @param mapId
+   * @return true/false, based on if the verification succeeded or not
+   */
+  private boolean verifySanity(long compressedLength, long decompressedLength,
+      int forReduce, Set<InputAttemptIdentifier> remaining, InputAttemptIdentifier srcAttemptId) {
+    if (compressedLength < 0 || decompressedLength < 0) {
+      wrongLengthErrs.increment(1);
+      LOG.warn(getName() + " invalid lengths in map output header: id: " +
+          srcAttemptId + " len: " + compressedLength + ", decomp len: " + 
+               decompressedLength);
+      return false;
+    }
+    
+    int reduceStartId = shuffle.getReduceStartId();
+    int reduceRange = shuffle.getReduceRange();
+    if (forReduce < reduceStartId || forReduce >= reduceStartId+reduceRange) {
+      wrongReduceErrs.increment(1);
+      LOG.warn(getName() + " data for the wrong reduce map: " +
+               srcAttemptId + " len: " + compressedLength + " decomp len: " +
+               decompressedLength + " for reduce " + forReduce);
+      return false;
+    }
+
+    // Sanity check
+    if (!remaining.contains(srcAttemptId)) {
+      wrongMapErrs.increment(1);
+      LOG.warn("Invalid map-output! Received output for " + srcAttemptId);
+      return false;
+    }
+    
+    return true;
+  }
+
+  /**
+   * Create the map-output-url. This will contain all the map ids
+   * separated by commas
+   * @param host
+   * @param maps
+   * @return
+   * @throws MalformedURLException
+   */
+  private URL getMapOutputURL(MapHost host, List<InputAttemptIdentifier> srcAttempts
+                              )  throws MalformedURLException {
+    // Get the base url
+    StringBuffer url = new StringBuffer(host.getBaseUrl());
+    
+    boolean first = true;
+    for (InputAttemptIdentifier mapId : srcAttempts) {
+      if (!first) {
+        url.append(",");
+      }
+      url.append(mapId.getPathComponent());
+      first = false;
+    }
+   
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("MapOutput URL for " + host + " -> " + url.toString());
+    }
+    return new URL(url.toString());
+  }
+  
+  /** 
+   * The connection establishment is attempted multiple times and is given up 
+   * only on the last failure. Instead of connecting with a timeout of 
+   * X, we try connecting with a timeout of x < X but multiple times. 
+   */
+  private void connect(URLConnection connection, int connectionTimeout)
+  throws IOException {
+    int unit = 0;
+    if (connectionTimeout < 0) {
+      throw new IOException("Invalid timeout "
+                            + "[timeout = " + connectionTimeout + " ms]");
+    } else if (connectionTimeout > 0) {
+      unit = Math.min(UNIT_CONNECT_TIMEOUT, connectionTimeout);
+    }
+    // set the connect timeout to the unit-connect-timeout
+    connection.setConnectTimeout(unit);
+    while (true) {
+      try {
+        connection.connect();
+        break;
+      } catch (IOException ioe) {
+        // update the total remaining connect-timeout
+        connectionTimeout -= unit;
+
+        // throw an exception if we have waited for timeout amount of time
+        // note that the updated value if timeout is used here
+        if (connectionTimeout == 0) {
+          throw ioe;
+        }
+
+        // reset the connect timeout for the last try
+        if (connectionTimeout < unit) {
+          unit = connectionTimeout;
+          // reset the connect time out for the final connect
+          connection.setConnectTimeout(unit);
+        }
+      }
+    }
+  }
+
+  private void shuffleToMemory(MapHost host, MapOutput mapOutput, 
+                               InputStream input, 
+                               int decompressedLength, 
+                               int compressedLength) throws IOException {    
+    IFileInputStream checksumIn = 
+      new IFileInputStream(input, compressedLength, job);
+
+    input = checksumIn;       
+  
+    // Are map-outputs compressed?
+    if (codec != null) {
+      decompressor.reset();
+      input = codec.createInputStream(input, decompressor);
+    }
+  
+    // Copy map-output into an in-memory buffer
+    byte[] shuffleData = mapOutput.getMemory();
+    
+    try {
+      IOUtils.readFully(input, shuffleData, 0, shuffleData.length);
+      metrics.inputBytes(shuffleData.length);
+      LOG.info("Read " + shuffleData.length + " bytes from map-output for " +
+               mapOutput.getAttemptIdentifier());
+    } catch (IOException ioe) {      
+      // Close the streams
+      IOUtils.cleanup(LOG, input);
+
+      // Re-throw
+      throw ioe;
+    }
+
+  }
+  
+  private void shuffleToDisk(MapHost host, MapOutput mapOutput, 
+                             InputStream input, 
+                             long compressedLength) 
+  throws IOException {
+    // Copy data to local-disk
+    OutputStream output = mapOutput.getDisk();
+    long bytesLeft = compressedLength;
+    try {
+      final int BYTES_TO_READ = 64 * 1024;
+      byte[] buf = new byte[BYTES_TO_READ];
+      while (bytesLeft > 0) {
+        int n = input.read(buf, 0, (int) Math.min(bytesLeft, BYTES_TO_READ));
+        if (n < 0) {
+          throw new IOException("read past end of stream reading " + 
+                                mapOutput.getAttemptIdentifier());
+        }
+        output.write(buf, 0, n);
+        bytesLeft -= n;
+        metrics.inputBytes(n);
+      }
+
+      LOG.info("Read " + (compressedLength - bytesLeft) + 
+               " bytes from map-output for " +
+               mapOutput.getAttemptIdentifier());
+
+      output.close();
+    } catch (IOException ioe) {
+      // Close the streams
+      IOUtils.cleanup(LOG, input, output);
+
+      // Re-throw
+      throw ioe;
+    }
+
+    // Sanity check
+    if (bytesLeft != 0) {
+      throw new IOException("Incomplete map output received for " +
+                            mapOutput.getAttemptIdentifier() + " from " +
+                            host.getHostName() + " (" + 
+                            bytesLeft + " bytes missing of " + 
+                            compressedLength + ")"
+      );
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryReader.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryReader.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryReader.java
new file mode 100644
index 0000000..ae95268
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryReader.java
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.runtime.library.common.shuffle.impl;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Reader;
+
+/**
+ * <code>IFile.InMemoryReader</code> to read map-outputs present in-memory.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InMemoryReader extends Reader {
+  private final InputAttemptIdentifier taskAttemptId;
+  private final MergeManager merger;
+  DataInputBuffer memDataIn = new DataInputBuffer();
+  private int start;
+  private int length;
+  private int prevKeyPos;
+
+  public InMemoryReader(MergeManager merger, InputAttemptIdentifier taskAttemptId,
+                        byte[] data, int start, int length)
+  throws IOException {
+    super(null, null, length - start, null, null);
+    this.merger = merger;
+    this.taskAttemptId = taskAttemptId;
+
+    buffer = data;
+    bufferSize = (int)fileLength;
+    memDataIn.reset(buffer, start, length);
+    this.start = start;
+    this.length = length;
+  }
+
+  @Override
+  public void reset(int offset) {
+    memDataIn.reset(buffer, start + offset, length);
+    bytesRead = offset;
+    eof = false;
+  }
+
+  @Override
+  public long getPosition() throws IOException {
+    // InMemoryReader does not initialize streams like Reader, so in.getPos()
+    // would not work. Instead, return the number of uncompressed bytes read,
+    // which will be correct since in-memory data is not compressed.
+    return bytesRead;
+  }
+  
+  @Override
+  public long getLength() { 
+    return fileLength;
+  }
+  
+  private void dumpOnError() {
+    File dumpFile = new File("../output/" + taskAttemptId + ".dump");
+    System.err.println("Dumping corrupt map-output of " + taskAttemptId + 
+                       " to " + dumpFile.getAbsolutePath());
+    try {
+      FileOutputStream fos = new FileOutputStream(dumpFile);
+      fos.write(buffer, 0, bufferSize);
+      fos.close();
+    } catch (IOException ioe) {
+      System.err.println("Failed to dump map-output of " + taskAttemptId);
+    }
+  }
+  
+  public KeyState readRawKey(DataInputBuffer key) throws IOException {
+    try {
+      if (!positionToNextRecord(memDataIn)) {
+        return KeyState.NO_KEY;
+      }
+      // Setup the key
+      int pos = memDataIn.getPosition();
+      byte[] data = memDataIn.getData();      
+      if(currentKeyLength == IFile.RLE_MARKER) {
+        key.reset(data, prevKeyPos, prevKeyLength);
+        currentKeyLength = prevKeyLength;
+        return KeyState.SAME_KEY;
+      }      
+      key.reset(data, pos, currentKeyLength);
+      prevKeyPos = pos;
+      // Position for the next value
+      long skipped = memDataIn.skip(currentKeyLength);
+      if (skipped != currentKeyLength) {
+        throw new IOException("Rec# " + recNo + 
+            ": Failed to skip past key of length: " + 
+            currentKeyLength);
+      }
+
+      // Record the byte
+      bytesRead += currentKeyLength;
+      return KeyState.NEW_KEY;
+    } catch (IOException ioe) {
+      dumpOnError();
+      throw ioe;
+    }
+  }
+  
+  public void nextRawValue(DataInputBuffer value) throws IOException {
+    try {
+      int pos = memDataIn.getPosition();
+      byte[] data = memDataIn.getData();
+      value.reset(data, pos, currentValueLength);
+
+      // Position for the next record
+      long skipped = memDataIn.skip(currentValueLength);
+      if (skipped != currentValueLength) {
+        throw new IOException("Rec# " + recNo + 
+            ": Failed to skip past value of length: " + 
+            currentValueLength);
+      }
+      // Record the byte
+      bytesRead += currentValueLength;
+
+      ++recNo;
+    } catch (IOException ioe) {
+      dumpOnError();
+      throw ioe;
+    }
+  }
+    
+  public void close() {
+    // Release
+    dataIn = null;
+    buffer = null;
+      // Inform the MergeManager
+    if (merger != null) {
+      merger.unreserve(bufferSize);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryWriter.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryWriter.java
new file mode 100644
index 0000000..f81b28e
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/InMemoryWriter.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.runtime.library.common.shuffle.impl;
+
+import java.io.DataOutputStream;
+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.io.BoundedByteArrayOutputStream;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.IFileOutputStream;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class InMemoryWriter extends Writer {
+  private static final Log LOG = LogFactory.getLog(InMemoryWriter.class);
+
+  private DataOutputStream out;
+
+  public InMemoryWriter(BoundedByteArrayOutputStream arrayStream) {
+    super(null);
+    this.out =
+      new DataOutputStream(new IFileOutputStream(arrayStream));
+  }
+
+  public void append(Object key, Object value) throws IOException {
+    throw new UnsupportedOperationException
+    ("InMemoryWriter.append(K key, V value");
+  }
+
+  public void append(DataInputBuffer key, DataInputBuffer value)
+  throws IOException {
+    int keyLength = key.getLength() - key.getPosition();
+    if (keyLength < 0) {
+      throw new IOException("Negative key-length not allowed: " + keyLength +
+                            " for " + key);
+    }
+
+    boolean sameKey = (key == IFile.REPEAT_KEY);
+
+    int valueLength = value.getLength() - value.getPosition();
+    if (valueLength < 0) {
+      throw new IOException("Negative value-length not allowed: " +
+                            valueLength + " for " + value);
+    }
+
+    if(sameKey) {
+      WritableUtils.writeVInt(out, IFile.RLE_MARKER);
+      WritableUtils.writeVInt(out, valueLength);
+      out.write(value.getData(), value.getPosition(), valueLength);
+    } else {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("InMemWriter.append" +
+            " key.data=" + key.getData() +
+            " key.pos=" + key.getPosition() +
+            " key.len=" +key.getLength() +
+            " val.data=" + value.getData() +
+            " val.pos=" + value.getPosition() +
+            " val.len=" + value.getLength());
+      }
+      WritableUtils.writeVInt(out, keyLength);
+      WritableUtils.writeVInt(out, valueLength);
+      out.write(key.getData(), key.getPosition(), keyLength);
+      out.write(value.getData(), value.getPosition(), valueLength);
+    }
+
+  }
+
+  public void close() throws IOException {
+    // Write EOF_MARKER for key/value length
+    WritableUtils.writeVInt(out, IFile.EOF_MARKER);
+    WritableUtils.writeVInt(out, IFile.EOF_MARKER);
+
+    // Close the stream
+    out.close();
+    out = null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapHost.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapHost.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapHost.java
new file mode 100644
index 0000000..b8be657
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapHost.java
@@ -0,0 +1,124 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.common.shuffle.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+
+@Private
+class MapHost {
+  
+  public static enum State {
+    IDLE,               // No map outputs available
+    BUSY,               // Map outputs are being fetched
+    PENDING,            // Known map outputs which need to be fetched
+    PENALIZED           // Host penalized due to shuffle failures
+  }
+  
+  private State state = State.IDLE;
+  private final String hostName;
+  private final int partitionId;
+  private final String baseUrl;
+  private final String identifier;
+  // Tracks attempt IDs
+  private List<InputAttemptIdentifier> maps = new ArrayList<InputAttemptIdentifier>();
+  
+  public MapHost(int partitionId, String hostName, String baseUrl) {
+    this.partitionId = partitionId;
+    this.hostName = hostName;
+    this.baseUrl = baseUrl;
+    this.identifier = createIdentifier(hostName, partitionId);
+  }
+  
+  public static String createIdentifier(String hostName, int partitionId) {
+    return hostName + ":" + Integer.toString(partitionId);
+  }
+  
+  public String getIdentifier() {
+    return identifier;
+  }
+  
+  public int getPartitionId() {
+    return partitionId;
+  }
+
+  public State getState() {
+    return state;
+  }
+
+  public String getHostName() {
+    return hostName;
+  }
+
+  public String getBaseUrl() {
+    return baseUrl;
+  }
+
+  public synchronized void addKnownMap(InputAttemptIdentifier srcAttempt) {
+    maps.add(srcAttempt);
+    if (state == State.IDLE) {
+      state = State.PENDING;
+    }
+  }
+
+  public synchronized List<InputAttemptIdentifier> getAndClearKnownMaps() {
+    List<InputAttemptIdentifier> currentKnownMaps = maps;
+    maps = new ArrayList<InputAttemptIdentifier>();
+    return currentKnownMaps;
+  }
+  
+  public synchronized void markBusy() {
+    state = State.BUSY;
+  }
+  
+  public synchronized void markPenalized() {
+    state = State.PENALIZED;
+  }
+  
+  public synchronized int getNumKnownMapOutputs() {
+    return maps.size();
+  }
+
+  /**
+   * Called when the node is done with its penalty or done copying.
+   * @return the host's new state
+   */
+  public synchronized State markAvailable() {
+    if (maps.isEmpty()) {
+      state = State.IDLE;
+    } else {
+      state = State.PENDING;
+    }
+    return state;
+  }
+  
+  @Override
+  public String toString() {
+    return hostName;
+  }
+  
+  /**
+   * Mark the host as penalized
+   */
+  public synchronized void penalize() {
+    state = State.PENALIZED;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapOutput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapOutput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapOutput.java
new file mode 100644
index 0000000..9f673a0
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MapOutput.java
@@ -0,0 +1,227 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.common.shuffle.impl;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Comparator;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BoundedByteArrayOutputStream;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
+
+
+class MapOutput {
+  private static final Log LOG = LogFactory.getLog(MapOutput.class);
+  private static AtomicInteger ID = new AtomicInteger(0);
+  
+  public static enum Type {
+    WAIT,
+    MEMORY,
+    DISK
+  }
+  
+  private InputAttemptIdentifier attemptIdentifier;
+  private final int id;
+  
+  private final MergeManager merger;
+  
+  private final long size;
+  
+  private final byte[] memory;
+  private BoundedByteArrayOutputStream byteStream;
+  
+  private final FileSystem localFS;
+  private final Path tmpOutputPath;
+  private final Path outputPath;
+  private final OutputStream disk; 
+  
+  private final Type type;
+  
+  private final boolean primaryMapOutput;
+  
+  MapOutput(InputAttemptIdentifier attemptIdentifier, MergeManager merger, long size, 
+            Configuration conf, LocalDirAllocator localDirAllocator,
+            int fetcher, boolean primaryMapOutput, 
+            TezTaskOutputFiles mapOutputFile)
+         throws IOException {
+    this.id = ID.incrementAndGet();
+    this.attemptIdentifier = attemptIdentifier;
+    this.merger = merger;
+
+    type = Type.DISK;
+
+    memory = null;
+    byteStream = null;
+
+    this.size = size;
+    
+    this.localFS = FileSystem.getLocal(conf);
+    outputPath =
+      mapOutputFile.getInputFileForWrite(this.attemptIdentifier.getInputIdentifier().getSrcTaskIndex(), size);
+    tmpOutputPath = outputPath.suffix(String.valueOf(fetcher));
+
+    disk = localFS.create(tmpOutputPath);
+    
+    this.primaryMapOutput = primaryMapOutput;
+  }
+  
+  MapOutput(InputAttemptIdentifier attemptIdentifier, MergeManager merger, int size, 
+            boolean primaryMapOutput) {
+    this.id = ID.incrementAndGet();
+    this.attemptIdentifier = attemptIdentifier;
+    this.merger = merger;
+
+    type = Type.MEMORY;
+    byteStream = new BoundedByteArrayOutputStream(size);
+    memory = byteStream.getBuffer();
+
+    this.size = size;
+    
+    localFS = null;
+    disk = null;
+    outputPath = null;
+    tmpOutputPath = null;
+    
+    this.primaryMapOutput = primaryMapOutput;
+  }
+
+  public MapOutput(InputAttemptIdentifier attemptIdentifier) {
+    this.id = ID.incrementAndGet();
+    this.attemptIdentifier = attemptIdentifier;
+
+    type = Type.WAIT;
+    merger = null;
+    memory = null;
+    byteStream = null;
+    
+    size = -1;
+    
+    localFS = null;
+    disk = null;
+    outputPath = null;
+    tmpOutputPath = null;
+
+    this.primaryMapOutput = false;
+}
+  
+  public boolean isPrimaryMapOutput() {
+    return primaryMapOutput;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof MapOutput) {
+      return id == ((MapOutput)obj).id;
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return id;
+  }
+
+  public Path getOutputPath() {
+    return outputPath;
+  }
+
+  public byte[] getMemory() {
+    return memory;
+  }
+
+  public BoundedByteArrayOutputStream getArrayStream() {
+    return byteStream;
+  }
+  
+  public OutputStream getDisk() {
+    return disk;
+  }
+
+  public InputAttemptIdentifier getAttemptIdentifier() {
+    return this.attemptIdentifier;
+  }
+
+  public Type getType() {
+    return type;
+  }
+
+  public long getSize() {
+    return size;
+  }
+
+  public void commit() throws IOException {
+    if (type == Type.MEMORY) {
+      merger.closeInMemoryFile(this);
+    } else if (type == Type.DISK) {
+      localFS.rename(tmpOutputPath, outputPath);
+      merger.closeOnDiskFile(outputPath);
+    } else {
+      throw new IOException("Cannot commit MapOutput of type WAIT!");
+    }
+  }
+  
+  public void abort() {
+    if (type == Type.MEMORY) {
+      merger.unreserve(memory.length);
+    } else if (type == Type.DISK) {
+      try {
+        localFS.delete(tmpOutputPath, false);
+      } catch (IOException ie) {
+        LOG.info("failure to clean up " + tmpOutputPath, ie);
+      }
+    } else {
+      throw new IllegalArgumentException
+                   ("Cannot commit MapOutput with of type WAIT!");
+    }
+  }
+  
+  public String toString() {
+    return "MapOutput( AttemptIdentifier: " + attemptIdentifier + ", Type: " + type + ")";
+  }
+  
+  public static class MapOutputComparator 
+  implements Comparator<MapOutput> {
+    public int compare(MapOutput o1, MapOutput o2) {
+      if (o1.id == o2.id) { 
+        return 0;
+      }
+      
+      if (o1.size < o2.size) {
+        return -1;
+      } else if (o1.size > o2.size) {
+        return 1;
+      }
+      
+      if (o1.id < o2.id) {
+        return -1;
+      } else {
+        return 1;
+      
+      }
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeManager.java
new file mode 100644
index 0000000..0abe530
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeManager.java
@@ -0,0 +1,782 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.runtime.library.common.shuffle.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+
+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.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ChecksumFileSystem;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.tez.common.TezJobConfig;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.runtime.api.TezInputContext;
+import org.apache.tez.runtime.library.common.ConfigUtils;
+import org.apache.tez.runtime.library.common.Constants;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.combine.Combiner;
+import org.apache.tez.runtime.library.common.sort.impl.IFile;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger;
+import org.apache.tez.runtime.library.common.sort.impl.TezRawKeyValueIterator;
+import org.apache.tez.runtime.library.common.sort.impl.IFile.Writer;
+import org.apache.tez.runtime.library.common.sort.impl.TezMerger.Segment;
+import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
+import org.apache.tez.runtime.library.hadoop.compat.NullProgressable;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+@SuppressWarnings(value={"rawtypes"})
+public class MergeManager {
+  
+  private static final Log LOG = LogFactory.getLog(MergeManager.class);
+
+  private final Configuration conf;
+  private final FileSystem localFS;
+  private final FileSystem rfs;
+  private final LocalDirAllocator localDirAllocator;
+  
+  private final  TezTaskOutputFiles mapOutputFile;
+  private final Progressable nullProgressable = new NullProgressable();
+  private final Combiner combiner;  
+  
+  Set<MapOutput> inMemoryMergedMapOutputs = 
+    new TreeSet<MapOutput>(new MapOutput.MapOutputComparator());
+  private final IntermediateMemoryToMemoryMerger memToMemMerger;
+
+  Set<MapOutput> inMemoryMapOutputs = 
+    new TreeSet<MapOutput>(new MapOutput.MapOutputComparator());
+  private final InMemoryMerger inMemoryMerger;
+  
+  Set<Path> onDiskMapOutputs = new TreeSet<Path>();
+  private final OnDiskMerger onDiskMerger;
+  
+  private final long memoryLimit;
+  private long usedMemory;
+  private long commitMemory;
+  private final long maxSingleShuffleLimit;
+  
+  private final int memToMemMergeOutputsThreshold; 
+  private final long mergeThreshold;
+  
+  private final int ioSortFactor;
+
+  private final ExceptionReporter exceptionReporter;
+  
+  private final TezInputContext inputContext;
+
+  private final TezCounter spilledRecordsCounter;
+
+  private final TezCounter reduceCombineInputCounter;
+
+  private final TezCounter mergedMapOutputsCounter;
+  
+  private final CompressionCodec codec;
+  
+  private volatile boolean finalMergeComplete = false;
+
+  public MergeManager(Configuration conf, 
+                      FileSystem localFS,
+                      LocalDirAllocator localDirAllocator,  
+                      TezInputContext inputContext,
+                      Combiner combiner,
+                      TezCounter spilledRecordsCounter,
+                      TezCounter reduceCombineInputCounter,
+                      TezCounter mergedMapOutputsCounter,
+                      ExceptionReporter exceptionReporter) {
+    this.inputContext = inputContext;
+    this.conf = conf;
+    this.localDirAllocator = localDirAllocator;
+    this.exceptionReporter = exceptionReporter;
+    
+    this.combiner = combiner;
+
+    this.reduceCombineInputCounter = reduceCombineInputCounter;
+    this.spilledRecordsCounter = spilledRecordsCounter;
+    this.mergedMapOutputsCounter = mergedMapOutputsCounter;
+    this.mapOutputFile = new TezTaskOutputFiles(conf, inputContext.getUniqueIdentifier());
+    
+    this.localFS = localFS;
+    this.rfs = ((LocalFileSystem)localFS).getRaw();
+
+    if (ConfigUtils.isIntermediateInputCompressed(conf)) {
+      Class<? extends CompressionCodec> codecClass =
+          ConfigUtils.getIntermediateInputCompressorClass(conf, DefaultCodec.class);
+      codec = ReflectionUtils.newInstance(codecClass, conf);
+    } else {
+      codec = null;
+    }
+
+    final float maxInMemCopyUse =
+      conf.getFloat(
+          TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT, 
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT);
+    if (maxInMemCopyUse > 1.0 || maxInMemCopyUse < 0.0) {
+      throw new IllegalArgumentException("Invalid value for " +
+          TezJobConfig.TEZ_RUNTIME_SHUFFLE_INPUT_BUFFER_PERCENT + ": " +
+          maxInMemCopyUse);
+    }
+
+    // Allow unit tests to fix Runtime memory
+    this.memoryLimit = 
+      (long)(conf.getLong(Constants.TEZ_RUNTIME_TASK_MEMORY,
+          Math.min(Runtime.getRuntime().maxMemory(), Integer.MAX_VALUE))
+        * maxInMemCopyUse);
+ 
+    this.ioSortFactor = 
+        conf.getInt(
+            TezJobConfig.TEZ_RUNTIME_IO_SORT_FACTOR, 
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_IO_SORT_FACTOR);
+
+    final float singleShuffleMemoryLimitPercent =
+        conf.getFloat(
+            TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT,
+            TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT);
+    if (singleShuffleMemoryLimitPercent <= 0.0f
+        || singleShuffleMemoryLimitPercent > 1.0f) {
+      throw new IllegalArgumentException("Invalid value for "
+          + TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT + ": "
+          + singleShuffleMemoryLimitPercent);
+    }
+
+    this.maxSingleShuffleLimit = 
+      (long)(memoryLimit * singleShuffleMemoryLimitPercent);
+    this.memToMemMergeOutputsThreshold = 
+            conf.getInt(
+                TezJobConfig.TEZ_RUNTIME_SHUFFLE_MEMTOMEM_SEGMENTS, 
+                ioSortFactor);
+    this.mergeThreshold = 
+        (long)(this.memoryLimit * 
+               conf.getFloat(
+                   TezJobConfig.TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT, 
+                   TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_MERGE_PERCENT));
+    LOG.info("MergerManager: memoryLimit=" + memoryLimit + ", " +
+             "maxSingleShuffleLimit=" + maxSingleShuffleLimit + ", " +
+             "mergeThreshold=" + mergeThreshold + ", " + 
+             "ioSortFactor=" + ioSortFactor + ", " +
+             "memToMemMergeOutputsThreshold=" + memToMemMergeOutputsThreshold);
+
+    if (this.maxSingleShuffleLimit >= this.mergeThreshold) {
+      throw new RuntimeException("Invlaid configuration: "
+          + "maxSingleShuffleLimit should be less than mergeThreshold"
+          + "maxSingleShuffleLimit: " + this.maxSingleShuffleLimit
+          + "mergeThreshold: " + this.mergeThreshold);
+    }
+
+    boolean allowMemToMemMerge = 
+      conf.getBoolean(
+          TezJobConfig.TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM, 
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_SHUFFLE_ENABLE_MEMTOMEM);
+    if (allowMemToMemMerge) {
+      this.memToMemMerger = 
+        new IntermediateMemoryToMemoryMerger(this,
+                                             memToMemMergeOutputsThreshold);
+      this.memToMemMerger.start();
+    } else {
+      this.memToMemMerger = null;
+    }
+    
+    this.inMemoryMerger = new InMemoryMerger(this);
+    this.inMemoryMerger.start();
+    
+    this.onDiskMerger = new OnDiskMerger(this);
+    this.onDiskMerger.start();
+  }
+
+  public void waitForInMemoryMerge() throws InterruptedException {
+    inMemoryMerger.waitForMerge();
+  }
+  
+  private boolean canShuffleToMemory(long requestedSize) {
+    return (requestedSize < maxSingleShuffleLimit); 
+  }
+
+  final private MapOutput stallShuffle = new MapOutput(null);
+
+  public synchronized MapOutput reserve(InputAttemptIdentifier srcAttemptIdentifier, 
+                                             long requestedSize,
+                                             int fetcher
+                                             ) throws IOException {
+    if (!canShuffleToMemory(requestedSize)) {
+      LOG.info(srcAttemptIdentifier + ": Shuffling to disk since " + requestedSize + 
+               " is greater than maxSingleShuffleLimit (" + 
+               maxSingleShuffleLimit + ")");
+      return new MapOutput(srcAttemptIdentifier, this, requestedSize, conf, 
+                                localDirAllocator, fetcher, true,
+                                mapOutputFile);
+    }
+    
+    // Stall shuffle if we are above the memory limit
+
+    // It is possible that all threads could just be stalling and not make
+    // progress at all. This could happen when:
+    //
+    // requested size is causing the used memory to go above limit &&
+    // requested size < singleShuffleLimit &&
+    // current used size < mergeThreshold (merge will not get triggered)
+    //
+    // To avoid this from happening, we allow exactly one thread to go past
+    // the memory limit. We check (usedMemory > memoryLimit) and not
+    // (usedMemory + requestedSize > memoryLimit). When this thread is done
+    // fetching, this will automatically trigger a merge thereby unlocking
+    // all the stalled threads
+    
+    if (usedMemory > memoryLimit) {
+      LOG.debug(srcAttemptIdentifier + ": Stalling shuffle since usedMemory (" + usedMemory
+          + ") is greater than memoryLimit (" + memoryLimit + ")." + 
+          " CommitMemory is (" + commitMemory + ")"); 
+      return stallShuffle;
+    }
+    
+    // Allow the in-memory shuffle to progress
+    LOG.debug(srcAttemptIdentifier + ": Proceeding with shuffle since usedMemory ("
+        + usedMemory + ") is lesser than memoryLimit (" + memoryLimit + ")."
+        + "CommitMemory is (" + commitMemory + ")"); 
+    return unconditionalReserve(srcAttemptIdentifier, requestedSize, true);
+  }
+  
+  /**
+   * Unconditional Reserve is used by the Memory-to-Memory thread
+   * @return
+   */
+  private synchronized MapOutput unconditionalReserve(
+      InputAttemptIdentifier srcAttemptIdentifier, long requestedSize, boolean primaryMapOutput) {
+    usedMemory += requestedSize;
+    return new MapOutput(srcAttemptIdentifier, this, (int)requestedSize, 
+        primaryMapOutput);
+  }
+  
+  synchronized void unreserve(long size) {
+    commitMemory -= size;
+    usedMemory -= size;
+  }
+
+  public synchronized void closeInMemoryFile(MapOutput mapOutput) { 
+    inMemoryMapOutputs.add(mapOutput);
+    LOG.info("closeInMemoryFile -> map-output of size: " + mapOutput.getSize()
+        + ", inMemoryMapOutputs.size() -> " + inMemoryMapOutputs.size()
+        + ", commitMemory -> " + commitMemory + ", usedMemory ->" + usedMemory);
+
+    commitMemory+= mapOutput.getSize();
+
+    synchronized (inMemoryMerger) {
+      // Can hang if mergeThreshold is really low.
+      if (!inMemoryMerger.isInProgress() && commitMemory >= mergeThreshold) {
+        LOG.info("Starting inMemoryMerger's merge since commitMemory=" +
+            commitMemory + " > mergeThreshold=" + mergeThreshold + 
+            ". Current usedMemory=" + usedMemory);
+        inMemoryMapOutputs.addAll(inMemoryMergedMapOutputs);
+        inMemoryMergedMapOutputs.clear();
+        inMemoryMerger.startMerge(inMemoryMapOutputs);
+      } 
+    }
+    
+    if (memToMemMerger != null) {
+      synchronized (memToMemMerger) {
+        if (!memToMemMerger.isInProgress() && 
+            inMemoryMapOutputs.size() >= memToMemMergeOutputsThreshold) {
+          memToMemMerger.startMerge(inMemoryMapOutputs);
+        }
+      }
+    }
+  }
+  
+  
+  public synchronized void closeInMemoryMergedFile(MapOutput mapOutput) {
+    inMemoryMergedMapOutputs.add(mapOutput);
+    LOG.info("closeInMemoryMergedFile -> size: " + mapOutput.getSize() + 
+             ", inMemoryMergedMapOutputs.size() -> " + 
+             inMemoryMergedMapOutputs.size());
+  }
+  
+  public synchronized void closeOnDiskFile(Path file) {
+    onDiskMapOutputs.add(file);
+    
+    synchronized (onDiskMerger) {
+      if (!onDiskMerger.isInProgress() && 
+          onDiskMapOutputs.size() >= (2 * ioSortFactor - 1)) {
+        onDiskMerger.startMerge(onDiskMapOutputs);
+      }
+    }
+  }
+
+  /**
+   * Should <b>only</b> be used after the Shuffle phaze is complete, otherwise can
+   * return an invalid state since a merge may not be in progress dur to
+   * inadequate inputs
+   * 
+   * @return true if the merge process is complete, otherwise false
+   */
+  @Private
+  public boolean isMergeComplete() {
+    return finalMergeComplete;
+  }
+  
+  public TezRawKeyValueIterator close() throws Throwable {
+    // Wait for on-going merges to complete
+    if (memToMemMerger != null) { 
+      memToMemMerger.close();
+    }
+    inMemoryMerger.close();
+    onDiskMerger.close();
+    
+    List<MapOutput> memory = 
+      new ArrayList<MapOutput>(inMemoryMergedMapOutputs);
+    memory.addAll(inMemoryMapOutputs);
+    List<Path> disk = new ArrayList<Path>(onDiskMapOutputs);
+    TezRawKeyValueIterator kvIter = finalMerge(conf, rfs, memory, disk);
+    this.finalMergeComplete = true;
+    return kvIter;
+  }
+   
+  void runCombineProcessor(TezRawKeyValueIterator kvIter, Writer writer)
+      throws IOException, InterruptedException {
+    combiner.combine(kvIter, writer);
+  }
+
+  private class IntermediateMemoryToMemoryMerger 
+  extends MergeThread<MapOutput> {
+    
+    public IntermediateMemoryToMemoryMerger(MergeManager manager, 
+                                            int mergeFactor) {
+      super(manager, mergeFactor, exceptionReporter);
+      setName("InMemoryMerger - Thread to do in-memory merge of in-memory " +
+      		    "shuffled map-outputs");
+      setDaemon(true);
+    }
+
+    @Override
+    public void merge(List<MapOutput> inputs) throws IOException {
+      if (inputs == null || inputs.size() == 0) {
+        return;
+      }
+
+      InputAttemptIdentifier dummyMapId = inputs.get(0).getAttemptIdentifier(); 
+      List<Segment> inMemorySegments = new ArrayList<Segment>();
+      long mergeOutputSize = 
+        createInMemorySegments(inputs, inMemorySegments, 0);
+      int noInMemorySegments = inMemorySegments.size();
+      
+      MapOutput mergedMapOutputs = 
+        unconditionalReserve(dummyMapId, mergeOutputSize, false);
+      
+      Writer writer = 
+        new InMemoryWriter(mergedMapOutputs.getArrayStream());
+      
+      LOG.info("Initiating Memory-to-Memory merge with " + noInMemorySegments +
+               " segments of total-size: " + mergeOutputSize);
+
+      TezRawKeyValueIterator rIter = 
+        TezMerger.merge(conf, rfs,
+                       ConfigUtils.getIntermediateInputKeyClass(conf),
+                       ConfigUtils.getIntermediateInputValueClass(conf),
+                       inMemorySegments, inMemorySegments.size(),
+                       new Path(inputContext.getUniqueIdentifier()),
+                       (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf),
+                       nullProgressable, null, null, null);
+      TezMerger.writeFile(rIter, writer, nullProgressable, conf);
+      writer.close();
+
+      LOG.info(inputContext.getUniqueIdentifier() +  
+               " Memory-to-Memory merge of the " + noInMemorySegments +
+               " files in-memory complete.");
+
+      // Note the output of the merge
+      closeInMemoryMergedFile(mergedMapOutputs);
+    }
+  }
+  
+  private class InMemoryMerger extends MergeThread<MapOutput> {
+    
+    public InMemoryMerger(MergeManager manager) {
+      super(manager, Integer.MAX_VALUE, exceptionReporter);
+      setName
+      ("InMemoryMerger - Thread to merge in-memory shuffled map-outputs");
+      setDaemon(true);
+    }
+    
+    @Override
+    public void merge(List<MapOutput> inputs) throws IOException, InterruptedException {
+      if (inputs == null || inputs.size() == 0) {
+        return;
+      }
+      
+      //name this output file same as the name of the first file that is 
+      //there in the current list of inmem files (this is guaranteed to
+      //be absent on the disk currently. So we don't overwrite a prev. 
+      //created spill). Also we need to create the output file now since
+      //it is not guaranteed that this file will be present after merge
+      //is called (we delete empty files as soon as we see them
+      //in the merge method)
+
+      //figure out the mapId 
+      InputAttemptIdentifier srcTaskIdentifier = inputs.get(0).getAttemptIdentifier();
+
+      List<Segment> inMemorySegments = new ArrayList<Segment>();
+      long mergeOutputSize = 
+        createInMemorySegments(inputs, inMemorySegments,0);
+      int noInMemorySegments = inMemorySegments.size();
+
+      Path outputPath = mapOutputFile.getInputFileForWrite(
+          srcTaskIdentifier.getInputIdentifier().getSrcTaskIndex(),
+          mergeOutputSize).suffix(Constants.MERGED_OUTPUT_PREFIX);
+
+      Writer writer = null;
+      try {
+        writer =
+            new Writer(conf, rfs, outputPath,
+                (Class)ConfigUtils.getIntermediateInputKeyClass(conf),
+                (Class)ConfigUtils.getIntermediateInputValueClass(conf),
+                codec, null);
+
+        TezRawKeyValueIterator rIter = null;
+        LOG.info("Initiating in-memory merge with " + noInMemorySegments + 
+            " segments...");
+
+        rIter = TezMerger.merge(conf, rfs,
+            (Class)ConfigUtils.getIntermediateInputKeyClass(conf),
+            (Class)ConfigUtils.getIntermediateInputValueClass(conf),
+            inMemorySegments, inMemorySegments.size(),
+            new Path(inputContext.getUniqueIdentifier()),
+            (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf),
+            nullProgressable, spilledRecordsCounter, null, null);
+
+        if (null == combiner) {
+          TezMerger.writeFile(rIter, writer, nullProgressable, conf);
+        } else {
+          runCombineProcessor(rIter, writer);
+        }
+        writer.close();
+        writer = null;
+
+        LOG.info(inputContext.getUniqueIdentifier() +  
+            " Merge of the " + noInMemorySegments +
+            " files in-memory complete." +
+            " Local file is " + outputPath + " of size " + 
+            localFS.getFileStatus(outputPath).getLen());
+      } catch (IOException e) { 
+        //make sure that we delete the ondisk file that we created 
+        //earlier when we invoked cloneFileAttributes
+        localFS.delete(outputPath, true);
+        throw e;
+      } finally {
+        if (writer != null) {
+          writer.close();
+        }
+      }
+
+      // Note the output of the merge
+      closeOnDiskFile(outputPath);
+    }
+
+  }
+  
+  private class OnDiskMerger extends MergeThread<Path> {
+    
+    public OnDiskMerger(MergeManager manager) {
+      super(manager, Integer.MAX_VALUE, exceptionReporter);
+      setName("OnDiskMerger - Thread to merge on-disk map-outputs");
+      setDaemon(true);
+    }
+    
+    @Override
+    public void merge(List<Path> inputs) throws IOException {
+      // sanity check
+      if (inputs == null || inputs.isEmpty()) {
+        LOG.info("No ondisk files to merge...");
+        return;
+      }
+      
+      long approxOutputSize = 0;
+      int bytesPerSum = 
+        conf.getInt("io.bytes.per.checksum", 512);
+      
+      LOG.info("OnDiskMerger: We have  " + inputs.size() + 
+               " map outputs on disk. Triggering merge...");
+      
+      // 1. Prepare the list of files to be merged. 
+      for (Path file : inputs) {
+        approxOutputSize += localFS.getFileStatus(file).getLen();
+      }
+
+      // add the checksum length
+      approxOutputSize += 
+        ChecksumFileSystem.getChecksumLength(approxOutputSize, bytesPerSum);
+
+      // 2. Start the on-disk merge process
+      Path outputPath = 
+        localDirAllocator.getLocalPathForWrite(inputs.get(0).toString(), 
+            approxOutputSize, conf).suffix(Constants.MERGED_OUTPUT_PREFIX);
+      Writer writer = 
+        new Writer(conf, rfs, outputPath, 
+                        (Class)ConfigUtils.getIntermediateInputKeyClass(conf), 
+                        (Class)ConfigUtils.getIntermediateInputValueClass(conf),
+                        codec, null);
+      TezRawKeyValueIterator iter  = null;
+      Path tmpDir = new Path(inputContext.getUniqueIdentifier());
+      try {
+        iter = TezMerger.merge(conf, rfs,
+                            (Class)ConfigUtils.getIntermediateInputKeyClass(conf), 
+                            (Class)ConfigUtils.getIntermediateInputValueClass(conf),
+                            codec, inputs.toArray(new Path[inputs.size()]), 
+                            true, ioSortFactor, tmpDir, 
+                            (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(conf), 
+                            nullProgressable, spilledRecordsCounter, null, 
+                            mergedMapOutputsCounter, null);
+
+        TezMerger.writeFile(iter, writer, nullProgressable, conf);
+        writer.close();
+      } catch (IOException e) {
+        localFS.delete(outputPath, true);
+        throw e;
+      }
+
+      closeOnDiskFile(outputPath);
+
+      LOG.info(inputContext.getUniqueIdentifier() +
+          " Finished merging " + inputs.size() + 
+          " map output files on disk of total-size " + 
+          approxOutputSize + "." + 
+          " Local output file is " + outputPath + " of size " +
+          localFS.getFileStatus(outputPath).getLen());
+    }
+  }
+  
+  private long createInMemorySegments(List<MapOutput> inMemoryMapOutputs,
+                                      List<Segment> inMemorySegments, 
+                                      long leaveBytes
+                                      ) throws IOException {
+    long totalSize = 0L;
+    // We could use fullSize could come from the RamManager, but files can be
+    // closed but not yet present in inMemoryMapOutputs
+    long fullSize = 0L;
+    for (MapOutput mo : inMemoryMapOutputs) {
+      fullSize += mo.getMemory().length;
+    }
+    while(fullSize > leaveBytes) {
+      MapOutput mo = inMemoryMapOutputs.remove(0);
+      byte[] data = mo.getMemory();
+      long size = data.length;
+      totalSize += size;
+      fullSize -= size;
+      IFile.Reader reader = new InMemoryReader(MergeManager.this, 
+                                                   mo.getAttemptIdentifier(),
+                                                   data, 0, (int)size);
+      inMemorySegments.add(new Segment(reader, true, 
+                                            (mo.isPrimaryMapOutput() ? 
+                                            mergedMapOutputsCounter : null)));
+    }
+    return totalSize;
+  }
+
+  class RawKVIteratorReader extends IFile.Reader {
+
+    private final TezRawKeyValueIterator kvIter;
+
+    public RawKVIteratorReader(TezRawKeyValueIterator kvIter, long size)
+        throws IOException {
+      super(null, null, size, null, spilledRecordsCounter);
+      this.kvIter = kvIter;
+    }
+    public boolean nextRawKey(DataInputBuffer key) throws IOException {
+      if (kvIter.next()) {
+        final DataInputBuffer kb = kvIter.getKey();
+        final int kp = kb.getPosition();
+        final int klen = kb.getLength() - kp;
+        key.reset(kb.getData(), kp, klen);
+        bytesRead += klen;
+        return true;
+      }
+      return false;
+    }
+    public void nextRawValue(DataInputBuffer value) throws IOException {
+      final DataInputBuffer vb = kvIter.getValue();
+      final int vp = vb.getPosition();
+      final int vlen = vb.getLength() - vp;
+      value.reset(vb.getData(), vp, vlen);
+      bytesRead += vlen;
+    }
+    public long getPosition() throws IOException {
+      return bytesRead;
+    }
+
+    public void close() throws IOException {
+      kvIter.close();
+    }
+  }
+
+  private TezRawKeyValueIterator finalMerge(Configuration job, FileSystem fs,
+                                       List<MapOutput> inMemoryMapOutputs,
+                                       List<Path> onDiskMapOutputs
+                                       ) throws IOException {
+    LOG.info("finalMerge called with " + 
+             inMemoryMapOutputs.size() + " in-memory map-outputs and " + 
+             onDiskMapOutputs.size() + " on-disk map-outputs");
+    
+    final float maxRedPer =
+      job.getFloat(
+          TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT,
+          TezJobConfig.DEFAULT_TEZ_RUNTIME_INPUT_BUFFER_PERCENT);
+    if (maxRedPer > 1.0 || maxRedPer < 0.0) {
+      throw new IOException(TezJobConfig.TEZ_RUNTIME_INPUT_BUFFER_PERCENT +
+                            maxRedPer);
+    }
+    int maxInMemReduce = (int)Math.min(
+        Runtime.getRuntime().maxMemory() * maxRedPer, Integer.MAX_VALUE);
+    
+
+    // merge config params
+    Class keyClass = (Class)ConfigUtils.getIntermediateInputKeyClass(job);
+    Class valueClass = (Class)ConfigUtils.getIntermediateInputValueClass(job);
+    final Path tmpDir = new Path(inputContext.getUniqueIdentifier());
+    final RawComparator comparator =
+      (RawComparator)ConfigUtils.getIntermediateInputKeyComparator(job);
+
+    // segments required to vacate memory
+    List<Segment> memDiskSegments = new ArrayList<Segment>();
+    long inMemToDiskBytes = 0;
+    boolean mergePhaseFinished = false;
+    if (inMemoryMapOutputs.size() > 0) {
+      int srcTaskId = inMemoryMapOutputs.get(0).getAttemptIdentifier().getInputIdentifier().getSrcTaskIndex();
+      inMemToDiskBytes = createInMemorySegments(inMemoryMapOutputs, 
+                                                memDiskSegments,
+                                                maxInMemReduce);
+      final int numMemDiskSegments = memDiskSegments.size();
+      if (numMemDiskSegments > 0 &&
+            ioSortFactor > onDiskMapOutputs.size()) {
+        
+        // If we reach here, it implies that we have less than io.sort.factor
+        // disk segments and this will be incremented by 1 (result of the 
+        // memory segments merge). Since this total would still be 
+        // <= io.sort.factor, we will not do any more intermediate merges,
+        // the merge of all these disk segments would be directly fed to the
+        // reduce method
+        
+        mergePhaseFinished = true;
+        // must spill to disk, but can't retain in-mem for intermediate merge
+        final Path outputPath = 
+          mapOutputFile.getInputFileForWrite(srcTaskId,
+                                             inMemToDiskBytes).suffix(
+                                                 Constants.MERGED_OUTPUT_PREFIX);
+        final TezRawKeyValueIterator rIter = TezMerger.merge(job, fs,
+            keyClass, valueClass, memDiskSegments, numMemDiskSegments,
+            tmpDir, comparator, nullProgressable, spilledRecordsCounter, null, null);
+        final Writer writer = new Writer(job, fs, outputPath,
+            keyClass, valueClass, codec, null);
+        try {
+          TezMerger.writeFile(rIter, writer, nullProgressable, job);
+          // add to list of final disk outputs.
+          onDiskMapOutputs.add(outputPath);
+        } catch (IOException e) {
+          if (null != outputPath) {
+            try {
+              fs.delete(outputPath, true);
+            } catch (IOException ie) {
+              // NOTHING
+            }
+          }
+          throw e;
+        } finally {
+          if (null != writer) {
+            writer.close();
+          }
+        }
+        LOG.info("Merged " + numMemDiskSegments + " segments, " +
+                 inMemToDiskBytes + " bytes to disk to satisfy " +
+                 "reduce memory limit");
+        inMemToDiskBytes = 0;
+        memDiskSegments.clear();
+      } else if (inMemToDiskBytes != 0) {
+        LOG.info("Keeping " + numMemDiskSegments + " segments, " +
+                 inMemToDiskBytes + " bytes in memory for " +
+                 "intermediate, on-disk merge");
+      }
+    }
+
+    // segments on disk
+    List<Segment> diskSegments = new ArrayList<Segment>();
+    long onDiskBytes = inMemToDiskBytes;
+    Path[] onDisk = onDiskMapOutputs.toArray(new Path[onDiskMapOutputs.size()]);
+    for (Path file : onDisk) {
+      onDiskBytes += fs.getFileStatus(file).getLen();
+      LOG.debug("Disk file: " + file + " Length is " + 
+          fs.getFileStatus(file).getLen());
+      diskSegments.add(new Segment(job, fs, file, codec, false,
+                                         (file.toString().endsWith(
+                                             Constants.MERGED_OUTPUT_PREFIX) ?
+                                          null : mergedMapOutputsCounter)
+                                        ));
+    }
+    LOG.info("Merging " + onDisk.length + " files, " +
+             onDiskBytes + " bytes from disk");
+    Collections.sort(diskSegments, new Comparator<Segment>() {
+      public int compare(Segment o1, Segment o2) {
+        if (o1.getLength() == o2.getLength()) {
+          return 0;
+        }
+        return o1.getLength() < o2.getLength() ? -1 : 1;
+      }
+    });
+
+    // build final list of segments from merged backed by disk + in-mem
+    List<Segment> finalSegments = new ArrayList<Segment>();
+    long inMemBytes = createInMemorySegments(inMemoryMapOutputs, 
+                                             finalSegments, 0);
+    LOG.info("Merging " + finalSegments.size() + " segments, " +
+             inMemBytes + " bytes from memory into reduce");
+    if (0 != onDiskBytes) {
+      final int numInMemSegments = memDiskSegments.size();
+      diskSegments.addAll(0, memDiskSegments);
+      memDiskSegments.clear();
+      TezRawKeyValueIterator diskMerge = TezMerger.merge(
+          job, fs, keyClass, valueClass, diskSegments,
+          ioSortFactor, numInMemSegments, tmpDir, comparator,
+          nullProgressable, false, spilledRecordsCounter, null, null);
+      diskSegments.clear();
+      if (0 == finalSegments.size()) {
+        return diskMerge;
+      }
+      finalSegments.add(new Segment(
+            new RawKVIteratorReader(diskMerge, onDiskBytes), true));
+    }
+    return TezMerger.merge(job, fs, keyClass, valueClass,
+                 finalSegments, finalSegments.size(), tmpDir,
+                 comparator, nullProgressable, spilledRecordsCounter, null,
+                 null);
+  
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/b212ca1d/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeThread.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeThread.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeThread.java
new file mode 100644
index 0000000..d8a7722
--- /dev/null
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/MergeThread.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.runtime.library.common.shuffle.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+abstract class MergeThread<T> extends Thread {
+  
+  private static final Log LOG = LogFactory.getLog(MergeThread.class);
+
+  private volatile boolean inProgress = false;
+  private List<T> inputs = new ArrayList<T>();
+  protected final MergeManager manager;
+  private final ExceptionReporter reporter;
+  private boolean closed = false;
+  private final int mergeFactor;
+  
+  public MergeThread(MergeManager manager, int mergeFactor,
+                     ExceptionReporter reporter) {
+    this.manager = manager;
+    this.mergeFactor = mergeFactor;
+    this.reporter = reporter;
+  }
+  
+  public synchronized void close() throws InterruptedException {
+    closed = true;
+    waitForMerge();
+    interrupt();
+  }
+
+  public synchronized boolean isInProgress() {
+    return inProgress;
+  }
+  
+  public synchronized void startMerge(Set<T> inputs) {
+    if (!closed) {
+      inProgress = true;
+      this.inputs = new ArrayList<T>();
+      Iterator<T> iter=inputs.iterator();
+      for (int ctr = 0; iter.hasNext() && ctr < mergeFactor; ++ctr) {
+        this.inputs.add(iter.next());
+        iter.remove();
+      }
+      LOG.info(getName() + ": Starting merge with " + this.inputs.size() + 
+               " segments, while ignoring " + inputs.size() + " segments");
+      notifyAll();
+    }
+  }
+
+  public synchronized void waitForMerge() throws InterruptedException {
+    while (inProgress) {
+      wait();
+    }
+  }
+
+  public void run() {
+    while (true) {
+      try {
+        // Wait for notification to start the merge...
+        synchronized (this) {
+          while (!inProgress) {
+            wait();
+          }
+        }
+
+        // Merge
+        merge(inputs);
+      } catch (InterruptedException ie) {
+        return;
+      } catch(Throwable t) {
+        reporter.reportException(t);
+        return;
+      } finally {
+        synchronized (this) {
+          // Clear inputs
+          inputs = null;
+          inProgress = false;        
+          notifyAll();
+        }
+      }
+    }
+  }
+
+  public abstract void merge(List<T> inputs) 
+      throws IOException, InterruptedException;
+}