You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by ss...@apache.org on 2014/05/19 20:17:20 UTC

git commit: TEZ-1135. Fix ShuffledUnorderedKVInput handling of empty partitions. (sseth)

Repository: incubator-tez
Updated Branches:
  refs/heads/master 031b2deb0 -> e5a825b68


TEZ-1135. Fix ShuffledUnorderedKVInput handling of empty partitions.
(sseth)


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

Branch: refs/heads/master
Commit: e5a825b683b8642bd8288717b385dfceecc15780
Parents: 031b2de
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon May 19 11:16:40 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon May 19 11:16:40 2014 -0700

----------------------------------------------------------------------
 .../library/common/InputAttemptIdentifier.java  |   2 +-
 .../impl/ShuffleInputEventHandlerImpl.java      |  24 +--
 .../impl/TestShuffleInputEventHandlerImpl.java  | 159 +++++++++++++++++++
 3 files changed, 172 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/e5a825b6/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
index 4c9d525..7c8a23b 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
@@ -43,7 +43,7 @@ public class InputAttemptIdentifier {
     this.pathComponent = pathComponent;
     if (pathComponent != null && !pathComponent.startsWith(PATH_PREFIX)) {
       throw new TezUncheckedException(
-          "Path component must start with: " + PATH_PREFIX + this);
+          "Path component must start with: " + PATH_PREFIX + " " + this);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/e5a825b6/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java
index da4cb71..628298a 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java
@@ -112,20 +112,20 @@ public class ShuffleInputEventHandlerImpl implements ShuffleEventHandler {
         shuffleManager.addCompletedInputWithNoData(srcAttemptIdentifier);
         return;
       }
+    } 
+    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dme.getTargetIndex(),
+        dme.getVersion(), shufflePayload.getPathComponent());
+    if (shufflePayload.hasData()) {
+      DataProto dataProto = shufflePayload.getData();
+      FetchedInput fetchedInput = inputAllocator.allocate(dataProto.getRawLength(),
+          dataProto.getCompressedLength(), srcAttemptIdentifier);
+      moveDataToFetchedInput(dataProto, fetchedInput, hostIdentifier);
+      shuffleManager.addCompletedInputWithData(srcAttemptIdentifier, fetchedInput);
     } else {
-      InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dme.getTargetIndex(),
-          dme.getVersion(), shufflePayload.getPathComponent());
-      if (shufflePayload.hasData()) {
-        DataProto dataProto = shufflePayload.getData();
-        FetchedInput fetchedInput = inputAllocator.allocate(dataProto.getRawLength(),
-            dataProto.getCompressedLength(), srcAttemptIdentifier);
-        moveDataToFetchedInput(dataProto, fetchedInput, hostIdentifier);
-        shuffleManager.addCompletedInputWithData(srcAttemptIdentifier, fetchedInput);
-      } else {
-        shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(),
-            srcAttemptIdentifier, srcIndex);
-      }
+      shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(),
+          srcAttemptIdentifier, srcIndex);
     }
+
   }
   
   private void moveDataToFetchedInput(DataProto dataProto,

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/e5a825b6/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/shuffle/common/impl/TestShuffleInputEventHandlerImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/shuffle/common/impl/TestShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/shuffle/common/impl/TestShuffleInputEventHandlerImpl.java
new file mode 100644
index 0000000..e95c30d
--- /dev/null
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/shuffle/common/impl/TestShuffleInputEventHandlerImpl.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.impl;
+
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.tez.common.TezUtils;
+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.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInputAllocator;
+import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
+import org.junit.Test;
+
+import com.google.protobuf.ByteString;
+
+public class TestShuffleInputEventHandlerImpl {
+
+  private static final String HOST = "localhost";
+  private static final int PORT = 8080;
+  private static final String PATH_COMPONENT = "attempttmp";
+
+  @Test
+  public void testSimple() throws IOException {
+    TezInputContext inputContext = mock(TezInputContext.class);
+    ShuffleManager shuffleManager = mock(ShuffleManager.class);
+    FetchedInputAllocator inputAllocator = mock(FetchedInputAllocator.class);
+
+    ShuffleInputEventHandlerImpl handler = new ShuffleInputEventHandlerImpl(inputContext,
+        shuffleManager, inputAllocator, null, false, 0);
+
+    int taskIndex = 1;
+    Event dme = createDataMovementEvent(0, taskIndex, null);
+
+    List<Event> eventList = new LinkedList<Event>();
+    eventList.add(dme);
+    handler.handleEvents(eventList);
+
+    InputAttemptIdentifier expectedIdentifier = new InputAttemptIdentifier(taskIndex, 0,
+        PATH_COMPONENT);
+
+    verify(shuffleManager).addKnownInput(eq(HOST), eq(PORT), eq(expectedIdentifier), eq(0));
+  }
+
+  @Test
+  public void testCurrentPartitionEmpty() throws IOException {
+    TezInputContext inputContext = mock(TezInputContext.class);
+    ShuffleManager shuffleManager = mock(ShuffleManager.class);
+    FetchedInputAllocator inputAllocator = mock(FetchedInputAllocator.class);
+
+    ShuffleInputEventHandlerImpl handler = new ShuffleInputEventHandlerImpl(inputContext,
+        shuffleManager, inputAllocator, null, false, 0);
+
+    int taskIndex = 1;
+    Event dme = createDataMovementEvent(0, taskIndex, createEmptyPartitionByteString(0));
+
+    List<Event> eventList = new LinkedList<Event>();
+    eventList.add(dme);
+    handler.handleEvents(eventList);
+
+    InputAttemptIdentifier expectedIdentifier = new InputAttemptIdentifier(taskIndex, 0);
+
+    verify(shuffleManager).addCompletedInputWithNoData(eq(expectedIdentifier));
+  }
+
+  @Test
+  public void testOtherPartitionEmpty() throws IOException {
+    TezInputContext inputContext = mock(TezInputContext.class);
+    ShuffleManager shuffleManager = mock(ShuffleManager.class);
+    FetchedInputAllocator inputAllocator = mock(FetchedInputAllocator.class);
+
+    ShuffleInputEventHandlerImpl handler = new ShuffleInputEventHandlerImpl(inputContext,
+        shuffleManager, inputAllocator, null, false, 0);
+
+    int taskIndex = 1;
+    Event dme = createDataMovementEvent(0, taskIndex, createEmptyPartitionByteString(1));
+    List<Event> eventList = new LinkedList<Event>();
+    eventList.add(dme);
+    handler.handleEvents(eventList);
+
+    InputAttemptIdentifier expectedIdentifier = new InputAttemptIdentifier(taskIndex, 0, PATH_COMPONENT);
+
+    verify(shuffleManager).addKnownInput(eq(HOST), eq(PORT), eq(expectedIdentifier), eq(0));
+  }
+
+  @Test
+  public void testMultipleEvents1() throws IOException {
+    TezInputContext inputContext = mock(TezInputContext.class);
+    ShuffleManager shuffleManager = mock(ShuffleManager.class);
+    FetchedInputAllocator inputAllocator = mock(FetchedInputAllocator.class);
+
+    ShuffleInputEventHandlerImpl handler = new ShuffleInputEventHandlerImpl(inputContext,
+        shuffleManager, inputAllocator, null, false, 0);
+
+    int taskIndex1 = 1;
+    Event dme1 = createDataMovementEvent(0, taskIndex1, createEmptyPartitionByteString(0));
+    int taskIndex2 = 2;
+    Event dme2 = createDataMovementEvent(0, taskIndex2, null);
+    
+    List<Event> eventList = new LinkedList<Event>();
+    eventList.add(dme1);
+    eventList.add(dme2);
+    handler.handleEvents(eventList);
+
+    InputAttemptIdentifier expectedIdentifier1 = new InputAttemptIdentifier(taskIndex1, 0);
+    InputAttemptIdentifier expectedIdentifier2 = new InputAttemptIdentifier(taskIndex2, 0, PATH_COMPONENT);
+
+    verify(shuffleManager).addCompletedInputWithNoData(eq(expectedIdentifier1));
+    verify(shuffleManager).addKnownInput(eq(HOST), eq(PORT), eq(expectedIdentifier2), eq(0));
+  }
+  
+  private Event createDataMovementEvent(int srcIndex, int targetIndex,
+      ByteString emptyPartitionByteString) {
+    DataMovementEventPayloadProto.Builder builder = DataMovementEventPayloadProto.newBuilder();
+    builder.setHost(HOST);
+    builder.setPort(PORT);
+    builder.setPathComponent("attempttmp");
+    if (emptyPartitionByteString != null) {
+      builder.setEmptyPartitions(emptyPartitionByteString);
+    }
+    Event dme = new DataMovementEvent(srcIndex, targetIndex, 0, builder.build().toByteArray());
+    return dme;
+  }
+
+  private ByteString createEmptyPartitionByteString(int... emptyPartitions) throws IOException {
+    BitSet bitSet = new BitSet();
+    for (int i : emptyPartitions) {
+      bitSet.set(i);
+    }
+    ByteString emptyPartitionsBytesString = TezUtils.compressByteArrayToByteString(TezUtils
+        .toByteArray(bitSet));
+    return emptyPartitionsBytesString;
+  }
+
+}