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/07/24 01:33:57 UTC

git commit: TEZ-318. Additional cleanup of user payload handling. (hitesh)

Updated Branches:
  refs/heads/master 4e7629c23 -> 699e168e4


TEZ-318. Additional cleanup of user payload handling. (hitesh)


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

Branch: refs/heads/master
Commit: 699e168e4ae060259383240ea1f65cafa6a9843a
Parents: 4e7629c
Author: Hitesh Shah <hi...@apache.org>
Authored: Tue Jul 23 16:33:29 2013 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Tue Jul 23 16:33:29 2013 -0700

----------------------------------------------------------------------
 .../apache/tez/dag/app/dag/impl/VertexImpl.java | 28 +-------------------
 1 file changed, 1 insertion(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/699e168e/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 eacf0e2..3936843 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
@@ -35,7 +35,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 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.mapred.MRVertexOutputCommitter;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
@@ -105,10 +104,8 @@ import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.engine.common.security.JobTokenIdentifier;
 import org.apache.tez.engine.records.TezDependentTaskCompletionEvent;
-import org.apache.tez.mapreduce.hadoop.MRHelpers;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ByteString;
 
 
 /** Implementation of Vertex interface. Maintains the state machines of Vertex.
@@ -339,7 +336,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
 
   private final String vertexName;
   private final ProcessorDescriptor processorDescriptor;
-  private final byte[] userPayload;
 
   // For committer
   private final VertexContext vertexContext;
@@ -404,9 +400,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
     this.javaOpts = vertexPlan.getTaskConfig().hasJavaOpts() ? vertexPlan
         .getTaskConfig().getJavaOpts() : null;
 
-    this.userPayload = initializeUserPayload();
     this.vertexContext = new VertexContext(getDAGId(),
-        userPayload, this.vertexId,
+        this.processorDescriptor.getUserPayload(), this.vertexId,
         getApplicationAttemptId());
 
     // This "this leak" is okay because the retained pointer is in an
@@ -1444,25 +1439,4 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
     return this.vertexScheduler;
   }
 
-  private byte[] initializeUserPayload() {
-    for (VertexPlan vertexPlan : getDAG().getJobPlan().getVertexList()) {
-      if (vertexPlan.getName().equals(vertexName)) {
-        if (!vertexPlan.getProcessorDescriptor().hasUserPayload()) {
-          return null;
-        } else {
-          // Needs to be a ByteBuffer which allows toArray. PB returns a
-          // readOnlyBuffer
-          ByteString byteString = vertexPlan.getProcessorDescriptor()
-              .getUserPayload();
-          byte[] b = byteString.toByteArray();
-          return b;
-        }
-      }
-    }
-    return null;
-  }
-
-  public byte[] getUserPayload() {
-    return userPayload;
-  }
 }